diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp index e60d9e6b637a941..da1919586b82332 100644 --- a/be/src/agent/agent_server.cpp +++ b/be/src/agent/agent_server.cpp @@ -39,6 +39,7 @@ AgentServer::AgentServer(ExecEnv* exec_env, const TMasterInfo& master_info) : _exec_env(exec_env), _master_info(master_info), _topic_subscriber(new TopicSubscriber()) { + // TODO(yingchun): move these code into a init() function for (auto& path : exec_env->store_paths()) { try { string dpp_download_path_str = path.path + DPP_PREFIX; @@ -99,22 +100,23 @@ 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. +// TODO(yingchun): means extractly once? void AgentServer::submit_tasks(TAgentResult& agent_result, const vector& tasks) { Status ret_st; // TODO check master_info here if it is the same with that of heartbeat rpc - if (_master_info.network_address.hostname == "" || _master_info.network_address.port == 0) { + if (_master_info.network_address.hostname.empty() || _master_info.network_address.port == 0) { Status ret_st = Status::Cancelled("Have not get FE Master heartbeat yet"); ret_st.to_thrift(&agent_result.status); return; } - for (auto task : tasks) { + for (const auto& task : tasks) { VLOG_RPC << "submit one task: " << apache::thrift::ThriftDebugString(task).c_str(); TTaskType::type task_type = task.task_type; int64_t signature = task.signature; -#define HANDLE_TYPE(t_task_type, work_pool, req_member) \ +#define HANDLE_TYPE(t_task_type, work_pool, req_member) \ case t_task_type: \ if (task.__isset.req_member) { \ work_pool->submit_task(task); \ @@ -231,7 +233,7 @@ void AgentServer::release_snapshot(TAgentResult& t_agent_result, const std::stri Status ret_st; OLAPStatus err_code = SnapshotManager::instance()->release_snapshot(snapshot_path); if (err_code != OLAP_SUCCESS) { - LOG(WARNING) << "failt to release_snapshot. snapshot_path: " << snapshot_path + LOG(WARNING) << "failed to release_snapshot. snapshot_path: " << snapshot_path << ", err_code: " << err_code; ret_st = Status::RuntimeError(strings::Substitute( "fail to release_snapshot. err_code=$0", err_code)); @@ -266,6 +268,7 @@ void AgentServer::get_etl_status(TMiniLoadEtlStatusResult& t_agent_result, Status status = _exec_env->etl_job_mgr()->get_job_state(request.mini_load_id, &t_agent_result); if (!status.ok()) { LOG(WARNING) << "fail to get job state. [id=" << request.mini_load_id << "]"; + return; } VLOG_RPC << "success to get job state. [id=" << request.mini_load_id << ", status=" @@ -289,4 +292,4 @@ void AgentServer::delete_etl_files(TAgentResult& t_agent_result, status.to_thrift(&t_agent_result.status); } -} // namesapce doris +} // namespace doris diff --git a/be/src/agent/cgroups_mgr.h b/be/src/agent/cgroups_mgr.h index 1f2afcc2dac7990..8fec99f15113689 100644 --- a/be/src/agent/cgroups_mgr.h +++ b/be/src/agent/cgroups_mgr.h @@ -177,7 +177,7 @@ class CgroupsMgr { std::mutex _update_cgroups_mtx; // A static mapping from fe's resource type to cgroups file - static std::map _s_resource_cgroups; + static std::map _s_resource_cgroups; }; } #endif diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 411e294772c7831..7f43ded1cf7eb29 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -304,7 +304,6 @@ uint32_t TaskWorkerPool::_get_next_task_index(int32_t thread_count, void TaskWorkerPool::_create_tablet_worker_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TCreateTabletReq create_tablet_req; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -315,10 +314,10 @@ void TaskWorkerPool::_create_tablet_worker_thread_callback() { } agent_task_req = _tasks.front(); - create_tablet_req = agent_task_req.create_tablet_req; _tasks.pop_front(); } + const TCreateTabletReq& create_tablet_req = agent_task_req.create_tablet_req; TStatusCode::type status_code = TStatusCode::OK; vector error_msgs; TStatus task_status; @@ -345,7 +344,7 @@ void TaskWorkerPool::_create_tablet_worker_thread_callback() { tablet_info.row_count = 0; tablet_info.data_size = 0; tablet_info.__set_path_hash(tablet->data_dir()->path_hash()); - finish_tablet_infos.push_back(tablet_info); + finish_tablet_infos.emplace_back(tablet_info); } task_status.__set_status_code(status_code); @@ -367,7 +366,6 @@ void TaskWorkerPool::_create_tablet_worker_thread_callback() { void TaskWorkerPool::_drop_tablet_worker_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TDropTabletReq drop_tablet_req; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -378,13 +376,14 @@ void TaskWorkerPool::_drop_tablet_worker_thread_callback() { } agent_task_req = _tasks.front(); - drop_tablet_req = agent_task_req.drop_tablet_req; _tasks.pop_front(); } + const TDropTabletReq& drop_tablet_req = agent_task_req.drop_tablet_req; TStatusCode::type status_code = TStatusCode::OK; vector error_msgs; TStatus task_status; + // TODO(yingchun): Why not drop_tablet directly? TabletSharedPtr dropped_tablet = StorageEngine::instance()->tablet_manager()->get_tablet( drop_tablet_req.tablet_id, drop_tablet_req.schema_hash); if (dropped_tablet != nullptr) { @@ -392,7 +391,7 @@ void TaskWorkerPool::_drop_tablet_worker_thread_callback() { drop_tablet_req.tablet_id, drop_tablet_req.schema_hash); if (drop_status != OLAP_SUCCESS) { LOG(WARNING) << "drop table failed! signature: " << agent_task_req.signature; - error_msgs.push_back("drop table failed!"); + error_msgs.emplace_back("drop table failed!"); status_code = TStatusCode::RUNTIME_ERROR; } // if tablet is dropped by fe, then the related txn should also be removed @@ -429,6 +428,7 @@ void TaskWorkerPool::_alter_tablet_worker_thread_callback() { agent_task_req = _tasks.front(); _tasks.pop_front(); } + int64_t signatrue = agent_task_req.signature; LOG(INFO) << "get alter table task, signature: " << agent_task_req.signature; bool is_task_timeout = false; @@ -533,7 +533,7 @@ void TaskWorkerPool::_alter_tablet(const TAgentTaskRequest& agent_task_req, int6 } else if (status == DORIS_TASK_REQUEST_ERROR) { LOG(WARNING) << "alter table request task type invalid. " << "signature:" << signature; - error_msgs.push_back("alter table request new tablet id or schema count invalid."); + error_msgs.emplace_back("alter table request new tablet id or schema count invalid."); task_status.__set_status_code(TStatusCode::ANALYSIS_ERROR); } else { LOG(WARNING) << process_name << " failed. signature: " << signature; @@ -562,7 +562,6 @@ void TaskWorkerPool::_push_worker_thread_callback() { while (_is_work) { AgentStatus status = DORIS_SUCCESS; TAgentTaskRequest agent_task_req; - TPushReq push_req; int32_t index = 0; do { lock_guard worker_thread_lock(_worker_thread_lock); @@ -585,7 +584,6 @@ void TaskWorkerPool::_push_worker_thread_callback() { } agent_task_req = _tasks[index]; - push_req = agent_task_req.push_req; _tasks.erase(_tasks.begin() + index); } while (0); @@ -595,6 +593,8 @@ void TaskWorkerPool::_push_worker_thread_callback() { continue; } + const TPushReq& push_req = agent_task_req.push_req; + // TODO(yingchun): if index < 0 and breaked in while, push_req and agent_task_req are invalid. LOG(INFO) << "get push task. signature: " << agent_task_req.signature << " priority: " << priority << " push_type: " << push_req.push_type; vector tablet_infos; @@ -623,7 +623,7 @@ void TaskWorkerPool::_push_worker_thread_callback() { if (status == DORIS_SUCCESS) { VLOG(3) << "push ok. signature: " << agent_task_req.signature << ", push_type: " << push_req.push_type; - error_msgs.push_back("push success"); + error_msgs.emplace_back("push success"); ++_s_report_version; @@ -632,12 +632,12 @@ void TaskWorkerPool::_push_worker_thread_callback() { } else if (status == DORIS_TASK_REQUEST_ERROR) { LOG(WARNING) << "push request push_type invalid. type: " << push_req.push_type << ", signature: " << agent_task_req.signature; - error_msgs.push_back("push request push_type invalid."); + error_msgs.emplace_back("push request push_type invalid."); task_status.__set_status_code(TStatusCode::ANALYSIS_ERROR); } else { LOG(WARNING) << "push failed, error_code: " << status << ", signature: " << agent_task_req.signature; - error_msgs.push_back("push failed"); + error_msgs.emplace_back("push failed"); task_status.__set_status_code(TStatusCode::RUNTIME_ERROR); } task_status.__set_error_msgs(error_msgs); @@ -652,7 +652,6 @@ void TaskWorkerPool::_push_worker_thread_callback() { void TaskWorkerPool::_publish_version_worker_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TPublishVersionRequest publish_version_req; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -663,10 +662,10 @@ void TaskWorkerPool::_publish_version_worker_thread_callback() { } agent_task_req = _tasks.front(); - publish_version_req = agent_task_req.publish_version_req; _tasks.pop_front(); } + const TPublishVersionRequest& publish_version_req = agent_task_req.publish_version_req; DorisMetrics::instance()->publish_task_request_total->increment(1); LOG(INFO) << "get publish version task, signature:" << agent_task_req.signature; @@ -716,7 +715,6 @@ void TaskWorkerPool::_publish_version_worker_thread_callback() { void TaskWorkerPool::_clear_transaction_task_worker_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TClearTransactionTaskRequest clear_transaction_task_req; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -727,9 +725,10 @@ void TaskWorkerPool::_clear_transaction_task_worker_thread_callback() { } agent_task_req = _tasks.front(); - clear_transaction_task_req = agent_task_req.clear_transaction_task_req; _tasks.pop_front(); } + + const TClearTransactionTaskRequest& clear_transaction_task_req = agent_task_req.clear_transaction_task_req; LOG(INFO) << "get clear transaction task task, signature:" << agent_task_req.signature << ", transaction_id: " << clear_transaction_task_req.transaction_id << ", partition id size: " << clear_transaction_task_req.partition_id.size(); @@ -774,7 +773,6 @@ void TaskWorkerPool::_clear_transaction_task_worker_thread_callback() { void TaskWorkerPool::_update_tablet_meta_worker_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TUpdateTabletMetaInfoReq update_tablet_meta_req; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -785,11 +783,11 @@ void TaskWorkerPool::_update_tablet_meta_worker_thread_callback() { } agent_task_req = _tasks.front(); - update_tablet_meta_req = agent_task_req.update_tablet_meta_info_req; _tasks.pop_front(); } LOG(INFO) << "get update tablet meta task, signature:" << agent_task_req.signature; + const TUpdateTabletMetaInfoReq& update_tablet_meta_req = agent_task_req.update_tablet_meta_info_req; TStatusCode::type status_code = TStatusCode::OK; vector error_msgs; TStatus task_status; @@ -841,7 +839,6 @@ void TaskWorkerPool::_clone_worker_thread_callback() { while (_is_work) { AgentStatus status = DORIS_SUCCESS; TAgentTaskRequest agent_task_req; - TCloneReq clone_req; { lock_guard worker_thread_lock(_worker_thread_lock); @@ -853,10 +850,10 @@ void TaskWorkerPool::_clone_worker_thread_callback() { } agent_task_req = _tasks.front(); - clone_req = agent_task_req.clone_req; _tasks.pop_front(); } + const TCloneReq& clone_req = agent_task_req.clone_req; DorisMetrics::instance()->clone_requests_total->increment(1); LOG(INFO) << "get clone task. signature:" << agent_task_req.signature; @@ -877,7 +874,7 @@ void TaskWorkerPool::_clone_worker_thread_callback() { DorisMetrics::instance()->clone_requests_failed->increment(1); status_code = TStatusCode::RUNTIME_ERROR; LOG(WARNING) << "clone failed. signature: " << agent_task_req.signature; - error_msgs.push_back("clone failed."); + error_msgs.emplace_back("clone failed."); } else { LOG(INFO) << "clone success, set tablet infos." << "signature:" << agent_task_req.signature; @@ -895,7 +892,6 @@ void TaskWorkerPool::_clone_worker_thread_callback() { void TaskWorkerPool::_storage_medium_migrate_worker_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TStorageMediumMigrateReq storage_medium_migrate_req; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -906,10 +902,10 @@ void TaskWorkerPool::_storage_medium_migrate_worker_thread_callback() { } agent_task_req = _tasks.front(); - storage_medium_migrate_req = agent_task_req.storage_medium_migrate_req; _tasks.pop_front(); } + const TStorageMediumMigrateReq& storage_medium_migrate_req = agent_task_req.storage_medium_migrate_req; TStatusCode::type status_code = TStatusCode::OK; vector error_msgs; TStatus task_status; @@ -941,7 +937,6 @@ void TaskWorkerPool::_storage_medium_migrate_worker_thread_callback() { void TaskWorkerPool::_check_consistency_worker_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TCheckConsistencyReq check_consistency_req; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -952,10 +947,10 @@ void TaskWorkerPool::_check_consistency_worker_thread_callback() { } agent_task_req = _tasks.front(); - check_consistency_req = agent_task_req.check_consistency_req; _tasks.pop_front(); } + const TCheckConsistencyReq& check_consistency_req = agent_task_req.check_consistency_req; TStatusCode::type status_code = TStatusCode::OK; vector error_msgs; TStatus task_status; @@ -996,8 +991,10 @@ void TaskWorkerPool::_report_task_worker_thread_callback() { request.__set_backend(_backend); do { + // TODO(yingchun): 在master port = 0时,也应该sleep { lock_guard task_signatures_lock(_s_task_signatures_lock); + // TODO(yingchun): 上报task_signatures的意义是什么? request.__set_tasks(_s_task_signatures); } @@ -1025,7 +1022,7 @@ void TaskWorkerPool::_report_disk_state_worker_thread_callback() { // port == 0 means not received heartbeat yet // sleep a short time and try again LOG(INFO) << "waiting to receive first heartbeat from frontend"; - sleep(config::sleep_one_second); + sleep(config::sleep_one_second); // TODO(yingchun): should rename continue; } @@ -1130,7 +1127,6 @@ void TaskWorkerPool::_report_tablet_worker_thread_callback() { void TaskWorkerPool::_upload_worker_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TUploadReq upload_request; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -1141,10 +1137,10 @@ void TaskWorkerPool::_upload_worker_thread_callback() { } agent_task_req = _tasks.front(); - upload_request = agent_task_req.upload_req; _tasks.pop_front(); } + const TUploadReq& upload_request = agent_task_req.upload_req; LOG(INFO) << "get upload task, signature:" << agent_task_req.signature << ", job id:" << upload_request.job_id; @@ -1185,7 +1181,6 @@ void TaskWorkerPool::_upload_worker_thread_callback() { void TaskWorkerPool::_download_worker_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TDownloadReq download_request; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -1196,9 +1191,10 @@ void TaskWorkerPool::_download_worker_thread_callback() { } agent_task_req = _tasks.front(); - download_request = agent_task_req.download_req; _tasks.pop_front(); } + + const TDownloadReq& download_request = agent_task_req.download_req; LOG(INFO) << "get download task, signature: " << agent_task_req.signature << ", job id:" << download_request.job_id; @@ -1241,7 +1237,6 @@ void TaskWorkerPool::_download_worker_thread_callback() { void TaskWorkerPool::_make_snapshot_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TSnapshotRequest snapshot_request; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -1252,10 +1247,10 @@ void TaskWorkerPool::_make_snapshot_thread_callback() { } agent_task_req = _tasks.front(); - snapshot_request = agent_task_req.snapshot_req; _tasks.pop_front(); } LOG(INFO) << "get snapshot task, signature:" << agent_task_req.signature; + const TSnapshotRequest& snapshot_request = agent_task_req.snapshot_req; TStatusCode::type status_code = TStatusCode::OK; vector error_msgs; @@ -1275,7 +1270,7 @@ void TaskWorkerPool::_make_snapshot_thread_callback() { << ", version_hash:" << snapshot_request.version_hash << ", status: " << make_snapshot_status; error_msgs.push_back("make_snapshot failed. status: " + - boost::lexical_cast(make_snapshot_status)); + std::to_string(make_snapshot_status)); } else { LOG(INFO) << "make_snapshot success. tablet_id:" << snapshot_request.tablet_id << ", schema_hash:" << snapshot_request.schema_hash @@ -1322,7 +1317,6 @@ void TaskWorkerPool::_make_snapshot_thread_callback() { void TaskWorkerPool::_release_snapshot_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TReleaseSnapshotRequest release_snapshot_request; { lock_guard worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -1333,16 +1327,17 @@ void TaskWorkerPool::_release_snapshot_thread_callback() { } agent_task_req = _tasks.front(); - release_snapshot_request = agent_task_req.release_snapshot_req; _tasks.pop_front(); } + LOG(INFO) << "get release snapshot task, signature:" << agent_task_req.signature; + const TReleaseSnapshotRequest& release_snapshot_request = agent_task_req.release_snapshot_req; TStatusCode::type status_code = TStatusCode::OK; vector error_msgs; TStatus task_status; - string& snapshot_path = release_snapshot_request.snapshot_path; + const string& snapshot_path = release_snapshot_request.snapshot_path; OLAPStatus release_snapshot_status = SnapshotManager::instance()->release_snapshot(snapshot_path); if (release_snapshot_status != OLAP_SUCCESS) { @@ -1350,7 +1345,7 @@ void TaskWorkerPool::_release_snapshot_thread_callback() { LOG(WARNING) << "release_snapshot failed. snapshot_path: " << snapshot_path << ". status: " << release_snapshot_status; error_msgs.push_back("release_snapshot failed. status: " + - boost::lexical_cast(release_snapshot_status)); + std::to_string(release_snapshot_status)); } else { LOG(INFO) << "release_snapshot success. snapshot_path: " << snapshot_path << ". status: " << release_snapshot_status; @@ -1390,7 +1385,6 @@ AgentStatus TaskWorkerPool::_get_tablet_info(const TTabletId tablet_id, void TaskWorkerPool::_move_dir_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; - TMoveDirReq move_dir_req; { MutexLock worker_thread_lock(&(_worker_thread_lock)); while (_is_work && _tasks.empty()) { @@ -1401,9 +1395,9 @@ void TaskWorkerPool::_move_dir_thread_callback() { } agent_task_req = _tasks.front(); - move_dir_req = agent_task_req.move_dir_req; _tasks.pop_front(); } + const auto& move_dir_req = agent_task_req.move_dir_req; LOG(INFO) << "get move dir task, signature:" << agent_task_req.signature << ", job id:" << move_dir_req.job_id; diff --git a/be/src/agent/task_worker_pool.h b/be/src/agent/task_worker_pool.h index d7536d7a861600f..806a1339403e38f 100644 --- a/be/src/agent/task_worker_pool.h +++ b/be/src/agent/task_worker_pool.h @@ -72,7 +72,7 @@ class TaskWorkerPool { }; TaskWorkerPool( - const TaskWorkerType task_worker_type, + TaskWorkerType task_worker_type, ExecEnv* env, const TMasterInfo& master_info); virtual ~TaskWorkerPool(); @@ -118,18 +118,18 @@ class TaskWorkerPool { void _alter_tablet( const TAgentTaskRequest& alter_tablet_request, int64_t signature, - const TTaskType::type task_type, + TTaskType::type task_type, TFinishTaskRequest* finish_task_request); AgentStatus _get_tablet_info( - const TTabletId tablet_id, - const TSchemaHash schema_hash, + TTabletId tablet_id, + TSchemaHash schema_hash, int64_t signature, TTabletInfo* tablet_info); AgentStatus _move_dir( - const TTabletId tablet_id, - const TSchemaHash schema_hash, + TTabletId tablet_id, + TSchemaHash schema_hash, const std::string& src, int64_t job_id, bool overwrite, @@ -156,6 +156,7 @@ class TaskWorkerPool { static FrontendServiceClientCache _master_service_client_cache; static std::atomic_ulong _s_report_version; + // TODO(yingchun): don't use static here, each object should has its own member. static Mutex _s_task_signatures_lock; static std::map> _s_task_signatures; diff --git a/be/src/agent/utils.h b/be/src/agent/utils.h index 1bb41a5c87b3871..cdbe75eafab9bcb 100644 --- a/be/src/agent/utils.h +++ b/be/src/agent/utils.h @@ -31,10 +31,10 @@ class MasterServerClient { MasterServerClient(const TMasterInfo& master_info, FrontendServiceClientCache* client_cache); virtual ~MasterServerClient() {}; - // Reprot finished task to the master server + // Report finished task to the master server // // Input parameters: - // * request: The infomation of finished task + // * request: The information of finished task // // Output parameters: // * result: The result of report task @@ -43,7 +43,7 @@ class MasterServerClient { // Report tasks/olap tablet/disk state to the master server // // Input parameters: - // * request: The infomation to report + // * request: The information to report // // Output parameters: // * result: The result of report task diff --git a/be/src/common/config.h b/be/src/common/config.h index b074c85cbd640c9..aaacbb12736c524 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -390,7 +390,7 @@ namespace config { // you can do it as root via "sysctl -w vm.max_map_count=262144" or // "echo 262144 > /proc/sys/vm/max_map_count" // NOTE: When this is set to true, you must set chunk_reserved_bytes_limit - // to a relative large number or the performace is very very bad. + // to a relative large number or the performance is very very bad. CONF_Bool(use_mmap_allocate_chunk, "false"); // Chunk Allocator's reserved bytes limit, diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp index e126baa764b370c..f0a309eb44c4de7 100644 --- a/be/src/common/daemon.cpp +++ b/be/src/common/daemon.cpp @@ -98,7 +98,7 @@ void Daemon::memory_maintenance_thread() { // if the system is idle, we need to refresh the tracker occasionally since // untracked memory may be allocated or freed, e.g. by background threads. if (env->process_mem_tracker() != nullptr && - !env->process_mem_tracker()->is_consumption_metric_null()) { + !env->process_mem_tracker()->is_consumption_metric_null()) { env->process_mem_tracker()->RefreshConsumptionFromMetric(); } } diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp index 50a27cbb9ef15ac..bff7b7985b1b85f 100644 --- a/be/src/exec/data_sink.cpp +++ b/be/src/exec/data_sink.cpp @@ -44,8 +44,7 @@ Status DataSink::create_data_sink( const TPlanFragmentExecParams& params, const RowDescriptor& row_desc, boost::scoped_ptr* sink) { - DataSink* tmp_sink = NULL; - + DataSink* tmp_sink = nullptr; switch (thrift_sink.type) { case TDataSinkType::DATA_STREAM_SINK: { if (!thrift_sink.__isset.stream_sink) { @@ -56,7 +55,7 @@ Status DataSink::create_data_sink( // TODO: figure out good buffer size based on size of output row tmp_sink = new DataStreamSender( pool, params.sender_id, row_desc, - thrift_sink.stream_sink, params.destinations, 16 * 1024, + thrift_sink.stream_sink, params.destinations, 16 * 1024, send_query_statistics_with_every_batch); // RETURN_IF_ERROR(sender->prepare(state->obj_pool(), thrift_sink.stream_sink)); sink->reset(tmp_sink); diff --git a/be/src/olap/aggregate_func.cpp b/be/src/olap/aggregate_func.cpp index 76f515ab8546256..9255b2c9cce7da8 100644 --- a/be/src/olap/aggregate_func.cpp +++ b/be/src/olap/aggregate_func.cpp @@ -46,6 +46,7 @@ class AggregateFuncResolver { } } +private: template void add_aggregate_mapping() { _infos_mapping.emplace(std::make_pair(agg_method, field_type), diff --git a/be/src/olap/base_compaction.cpp b/be/src/olap/base_compaction.cpp index bb2404ca3be0b85..bf4059324412a80 100644 --- a/be/src/olap/base_compaction.cpp +++ b/be/src/olap/base_compaction.cpp @@ -60,9 +60,9 @@ OLAPStatus BaseCompaction::compact() { } OLAPStatus BaseCompaction::pick_rowsets_to_compact() { - _input_rowsets.clear(); + DCHECK(_input_rowsets.empty()); _tablet->pick_candicate_rowsets_to_base_compaction(&_input_rowsets); - if (_input_rowsets.size() <= 1) { + if (_input_rowsets.empty()) { return OLAP_ERR_BE_NO_SUITABLE_VERSION; } @@ -73,12 +73,13 @@ OLAPStatus BaseCompaction::pick_rowsets_to_compact() { if (_input_rowsets.size() == 2 && _input_rowsets[0]->end_version() == 1) { // the tablet is with rowset: [0-1], [2-y] // and [0-1] has no data. in this situation, no need to do base compaction. + // TODO(yingchun): why not needed? version [0-1] means have no data. return OLAP_ERR_BE_NO_SUITABLE_VERSION; } // 1. cumulative rowset must reach base_compaction_num_cumulative_deltas threshold if (_input_rowsets.size() > config::base_compaction_num_cumulative_deltas) { - LOG(INFO) << "satisfy the base compaction policy. tablet="<< _tablet->full_name() + LOG(INFO) << "satisfy the base compaction policy. tablet=" << _tablet->full_name() << ", num_cumulative_rowsets=" << _input_rowsets.size() - 1 << ", base_compaction_num_cumulative_rowsets=" << config::base_compaction_num_cumulative_deltas; return OLAP_SUCCESS; @@ -102,7 +103,6 @@ OLAPStatus BaseCompaction::pick_rowsets_to_compact() { base_size = 1; } double cumulative_base_ratio = static_cast(cumulative_total_size) / base_size; - if (cumulative_base_ratio > base_cumulative_delta_ratio) { LOG(INFO) << "satisfy the base compaction policy. tablet=" << _tablet->full_name() << ", cumualtive_total_size=" << cumulative_total_size @@ -115,7 +115,7 @@ OLAPStatus BaseCompaction::pick_rowsets_to_compact() { // 3. the interval since last base compaction reachs the threshold int64_t base_creation_time = _input_rowsets[0]->creation_time(); int64_t interval_threshold = config::base_compaction_interval_seconds_since_last_operation; - int64_t interval_since_last_base_compaction = time(NULL) - base_creation_time; + int64_t interval_since_last_base_compaction = time(nullptr) - base_creation_time; if (interval_since_last_base_compaction > interval_threshold) { LOG(INFO) << "satisfy the base compaction policy. tablet=" << _tablet->full_name() << ", interval_since_last_base_compaction=" << interval_since_last_base_compaction diff --git a/be/src/olap/column_mapping.h b/be/src/olap/column_mapping.h index 89392579c904c1b..a499b330ab3edd0 100644 --- a/be/src/olap/column_mapping.h +++ b/be/src/olap/column_mapping.h @@ -23,14 +23,11 @@ namespace doris { class WrapperField; struct ColumnMapping { - ColumnMapping() : ref_column(-1), default_value(nullptr) {} - virtual ~ColumnMapping() {} - // <0: use default value // >=0: use origin column - int32_t ref_column; + int32_t ref_column = -1; // normally for default value. stores values for filters - WrapperField* default_value; + WrapperField* default_value = nullptr; // materialize view transform function used in schema change std::string materialized_function; }; diff --git a/be/src/olap/cumulative_compaction.cpp b/be/src/olap/cumulative_compaction.cpp index 6611e5102be336a..a22f1aa5829e53d 100755 --- a/be/src/olap/cumulative_compaction.cpp +++ b/be/src/olap/cumulative_compaction.cpp @@ -133,13 +133,10 @@ OLAPStatus CumulativeCompaction::pick_rowsets_to_compact() { } } else { // init the compaction success time for first time - if (last_cumu == 0) { - _tablet->set_last_cumu_compaction_success_time(now); - } - - if (last_base == 0) { - _tablet->set_last_base_compaction_success_time(now); - } + DCHECK_EQ(last_cumu, 0); + DCHECK_EQ(last_base, 0); + _tablet->set_last_cumu_compaction_success_time(now); + _tablet->set_last_base_compaction_success_time(now); } return OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSIONS; diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index 42afdd8232b610c..1eaae4c73842710 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -345,7 +345,7 @@ void DataDir::register_tablet(Tablet* tablet) { TabletInfo tablet_info(tablet->tablet_id(), tablet->schema_hash(), tablet->tablet_uid()); std::lock_guard l(_mutex); - _tablet_set.emplace(std::move(tablet_info)); + _tablet_set.emplace(tablet_info); } void DataDir::deregister_tablet(Tablet* tablet) { @@ -756,6 +756,7 @@ OLAPStatus DataDir::load() { } } else if (rowset_meta->rowset_state() == RowsetStatePB::VISIBLE && rowset_meta->tablet_uid() == tablet->tablet_uid()) { + // TODO(yingchun): Can we add rs by batch ? OLAPStatus publish_status = tablet->add_rowset(rowset, false); if (publish_status != OLAP_SUCCESS && publish_status != OLAP_ERR_PUSH_VERSION_ALREADY_EXIST) { @@ -851,6 +852,8 @@ void DataDir::perform_path_gc_by_rowsetid() { if (config::path_gc_check_step > 0 && counter % config::path_gc_check_step == 0) { SleepFor(MonoDelta::FromMilliseconds(config::path_gc_check_step_interval_ms)); } + + // TODO(yingchun): get_tablet_id_and_schema_hash_from_path and get_rowset_id_from_path in one judgment TTabletId tablet_id = -1; TSchemaHash schema_hash = -1; bool is_valid = _tablet_manager->get_tablet_id_and_schema_hash_from_path(path, &tablet_id, diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index b24a3410a5db3ef..a4ca2f2bd04271f 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -27,15 +27,14 @@ namespace doris { -OLAPStatus DeltaWriter::open(WriteRequest* req, const std::shared_ptr& parent, +void DeltaWriter::open(const WriteRequest& req, const std::shared_ptr& parent, DeltaWriter** writer) { *writer = new DeltaWriter(req, parent, StorageEngine::instance()); - return OLAP_SUCCESS; } -DeltaWriter::DeltaWriter(WriteRequest* req, const std::shared_ptr& parent, +DeltaWriter::DeltaWriter(const WriteRequest& req, const std::shared_ptr& parent, StorageEngine* storage_engine) - : _req(*req), + : _req(req), _tablet(nullptr), _cur_rowset(nullptr), _new_rowset(nullptr), @@ -216,7 +215,7 @@ OLAPStatus DeltaWriter::close() { // which means this tablet has no data loaded, but at least one tablet // in same partition has data loaded. // so we have to also init this DeltaWriter, so that it can create a empty rowset - // for this tablet when being closd. + // for this tablet when being closed. RETURN_NOT_OK(init()); } @@ -258,7 +257,6 @@ OLAPStatus DeltaWriter::close_wait(google::protobuf::RepeatedPtrFieldtxn_manager()->commit_txn(_req.partition_id, _new_tablet, _req.txn_id, _req.load_id, _new_rowset, false); - if (res != OLAP_SUCCESS && res != OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST) { LOG(WARNING) << "Failed to save pending rowset. rowset_id:" << _new_rowset->rowset_id(); return res; diff --git a/be/src/olap/delta_writer.h b/be/src/olap/delta_writer.h index 2c4eb79a698ea2a..13aeda8b74c4432 100644 --- a/be/src/olap/delta_writer.h +++ b/be/src/olap/delta_writer.h @@ -56,8 +56,8 @@ struct WriteRequest { // This class is NOT thread-safe, external synchronization is required. class DeltaWriter { public: - static OLAPStatus open(WriteRequest* req, const std::shared_ptr& parent, - DeltaWriter** writer); + static void open(const WriteRequest& req, const std::shared_ptr& parent, + DeltaWriter** writer); ~DeltaWriter(); @@ -84,7 +84,7 @@ class DeltaWriter { int64_t mem_consumption() const; private: - DeltaWriter(WriteRequest* req, const std::shared_ptr& parent, + DeltaWriter(const WriteRequest& req, const std::shared_ptr& parent, StorageEngine* storage_engine); // push a full memtable to flush executor diff --git a/be/src/olap/field.h b/be/src/olap/field.h index f36787d00256a38..98175ae9800d821 100644 --- a/be/src/olap/field.h +++ b/be/src/olap/field.h @@ -262,7 +262,7 @@ class Field { const TypeInfo* _type_info; const KeyCoder* _key_coder; std::string _name; - uint16_t _index_size; + uint16_t _index_size; // TODO(yingchun): does it means 'cell size'? bool _is_nullable; protected: @@ -286,7 +286,7 @@ int Field::index_cmp(const LhsCellType& lhs, const RhsCellType& rhs) const { bool r_null = rhs.is_null(); if (l_null != r_null) { return l_null ? -1 : 1; - } else if (l_null){ + } else if (l_null) { return 0; } @@ -339,8 +339,7 @@ void Field::to_index(DstCellType* dst, const SrcCellType& src) const { // 先清零,再拷贝 memset(dst->mutable_cell_ptr(), 0, _index_size); const Slice* slice = reinterpret_cast(src.cell_ptr()); - size_t copy_size = slice->size < _index_size - OLAP_STRING_MAX_BYTES ? - slice->size : _index_size - OLAP_STRING_MAX_BYTES; + size_t copy_size = std::min(slice->size, _index_size - OLAP_STRING_MAX_BYTES); *reinterpret_cast(dst->mutable_cell_ptr()) = copy_size; memory_copy((char*)dst->mutable_cell_ptr() + OLAP_STRING_MAX_BYTES, slice->data, copy_size); } else if (type() == OLAP_FIELD_TYPE_CHAR) { diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp index 48d59a487920022..65e79ebd774ee0c 100644 --- a/be/src/olap/memtable.cpp +++ b/be/src/olap/memtable.cpp @@ -48,7 +48,6 @@ MemTable::MemTable(int64_t tablet_id, Schema* schema, const TabletSchema* tablet _rowset_writer(rowset_writer) {} MemTable::~MemTable() { - delete _skip_list; } MemTable::RowCursorComparator::RowCursorComparator(const Schema* schema) : _schema(schema) {} @@ -61,13 +60,13 @@ int MemTable::RowCursorComparator::operator()(const char* left, const char* righ void MemTable::insert(const Tuple* tuple) { bool overwritten = false; - uint8_t* _tuple_buf = nullptr; + uint8_t* tuple_buf = nullptr; if (_keys_type == KeysType::DUP_KEYS) { // Will insert directly, so use memory from _table_mem_pool - _tuple_buf = _table_mem_pool->allocate(_schema_size); - ContiguousRow row(_schema, _tuple_buf); + tuple_buf = _table_mem_pool->allocate(_schema_size); + ContiguousRow row(_schema, tuple_buf); _tuple_to_row(tuple, &row, _table_mem_pool.get()); - _skip_list->Insert((TableKey)_tuple_buf, &overwritten); + _skip_list->Insert((TableKey)tuple_buf, &overwritten); DCHECK(!overwritten) << "Duplicate key model meet overwrite in SkipList"; return; } @@ -76,18 +75,18 @@ void MemTable::insert(const Tuple* tuple) { // we first allocate from _buffer_mem_pool, and then check whether it already exists in // _skiplist. If it exists, we aggregate the new row into the row in skiplist. // otherwise, we need to copy it into _table_mem_pool before we can insert it. - _tuple_buf = _buffer_mem_pool->allocate(_schema_size); - ContiguousRow src_row(_schema, _tuple_buf); + tuple_buf = _buffer_mem_pool->allocate(_schema_size); + ContiguousRow src_row(_schema, tuple_buf); _tuple_to_row(tuple, &src_row, _buffer_mem_pool.get()); - bool is_exist = _skip_list->Find((TableKey)_tuple_buf, &_hint); + bool is_exist = _skip_list->Find((TableKey)tuple_buf, &_hint); if (is_exist) { _aggregate_two_row(src_row, _hint.curr->key); } else { - _tuple_buf = _table_mem_pool->allocate(_schema_size); - ContiguousRow dst_row(_schema, _tuple_buf); + tuple_buf = _table_mem_pool->allocate(_schema_size); + ContiguousRow dst_row(_schema, tuple_buf); copy_row_in_memtable(&dst_row, src_row, _table_mem_pool.get()); - _skip_list->InsertWithHint((TableKey)_tuple_buf, is_exist, &_hint); + _skip_list->InsertWithHint((TableKey)tuple_buf, is_exist, &_hint); } // Make MemPool to be reusable, but does not free its memory @@ -119,7 +118,7 @@ OLAPStatus MemTable::flush() { int64_t duration_ns = 0; { SCOPED_RAW_TIMER(&duration_ns); - Table::Iterator it(_skip_list); + Table::Iterator it(_skip_list.get()); for (it.SeekToFirst(); it.Valid(); it.Next()) { char* row = (char*)it.key(); ContiguousRow dst_row(_schema, row); diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h index 583eefdb72467cb..cd2b143f0fcae81 100644 --- a/be/src/olap/memtable.h +++ b/be/src/olap/memtable.h @@ -84,7 +84,7 @@ class MemTable { ObjectPool _agg_object_pool; size_t _schema_size; - Table* _skip_list; + std::unique_ptr _skip_list; Table::Hint _hint; RowsetWriter* _rowset_writer; diff --git a/be/src/olap/merger.cpp b/be/src/olap/merger.cpp index c8b262c515881c1..184c1228a889830 100644 --- a/be/src/olap/merger.cpp +++ b/be/src/olap/merger.cpp @@ -54,10 +54,10 @@ OLAPStatus Merger::merge_rowsets(TabletSharedPtr tablet, // The following procedure would last for long time, half of one day, etc. int64_t output_rows = 0; while (true) { - ObjectPool objectPool; + ObjectPool object_pool; bool eof = false; // Read one row into row_cursor - RETURN_NOT_OK_LOG(reader.next_row_with_aggregation(&row_cursor, mem_pool.get(), &objectPool, &eof), + RETURN_NOT_OK_LOG(reader.next_row_with_aggregation(&row_cursor, mem_pool.get(), &object_pool, &eof), "failed to read next row when merging rowsets of tablet " + tablet->full_name()); if (eof) { break; diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index 889f0e3eb23a3fd..b5039653eafcb04 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -184,6 +184,8 @@ enum ReaderType { //using Version = std::pair; struct Version { + // TODO(yingchun): rename to 'start', 'end'? + // TODO(yingchun): first <= second? int64_t first; int64_t second; @@ -203,6 +205,10 @@ struct Version { bool contains(const Version& other) const { return first <= other.first && second >= other.second; } + + bool single_version() const { + return first == second; + } }; typedef std::vector Versions; @@ -306,7 +312,7 @@ struct RowsetId { } } - // to compatiable with old version + // to compatible with old version void init(int64_t rowset_id) { init(1, rowset_id, 0, 0); } diff --git a/be/src/olap/olap_index.cpp b/be/src/olap/olap_index.cpp index bd9ac44b91cd2fb..f716a7367c39185 100644 --- a/be/src/olap/olap_index.cpp +++ b/be/src/olap/olap_index.cpp @@ -52,19 +52,12 @@ MemIndex::~MemIndex() { } } -OLAPStatus MemIndex::load_segment(const char* file, size_t *current_num_rows_per_row_block, bool use_cache) { +OLAPStatus MemIndex::load_segment(const std::string& file, size_t *current_num_rows_per_row_block, bool use_cache) { OLAPStatus res = OLAP_SUCCESS; SegmentMetaInfo meta; uint32_t adler_checksum = 0; uint32_t num_entries = 0; - - if (file == NULL) { - res = OLAP_ERR_INPUT_PARAMETER_ERROR; - LOG(WARNING) << "load index error. file=" << file << ", res=" << res; - return res; - } - FileHandler file_handler; if (use_cache) { if ((res = file_handler.open_with_cache(file, O_RDONLY)) != OLAP_SUCCESS) { @@ -286,19 +279,14 @@ OLAPStatus MemIndex::load_segment(const char* file, size_t *current_num_rows_per return OLAP_SUCCESS; } -OLAPStatus MemIndex::init(size_t short_key_len, size_t new_short_key_len, - size_t short_key_num, std::vector* short_key_columns) { - if (short_key_columns == nullptr) { - LOG(WARNING) << "fail to init MemIndex, NULL short key columns."; - return OLAP_ERR_INDEX_LOAD_ERROR; - } +void MemIndex::init(size_t short_key_len, size_t new_short_key_len, + size_t short_key_num, std::vector* short_key_columns) { + DCHECK(short_key_columns == nullptr); _key_length = short_key_len; _new_key_length = new_short_key_len; _key_num = short_key_num; _short_key_columns = short_key_columns; - - return OLAP_SUCCESS; } // Find and return the IndexOffset of the element prior to the first element which diff --git a/be/src/olap/olap_index.h b/be/src/olap/olap_index.h index c4d3714d9ddf52b..6203af2c45130f2 100644 --- a/be/src/olap/olap_index.h +++ b/be/src/olap/olap_index.h @@ -172,11 +172,11 @@ class MemIndex { ~MemIndex(); // 初始化MemIndex, 传入short_key的总长度和对应的Field数组 - OLAPStatus init(size_t short_key_len, size_t new_short_key_len, - size_t short_key_num, std::vector* short_key_columns); + void init(size_t short_key_len, size_t new_short_key_len, + size_t short_key_num, std::vector* short_key_columns); // 加载一个segment到内存 - OLAPStatus load_segment(const char* file, size_t *current_num_rows_per_row_block, bool use_cache = true); + OLAPStatus load_segment(const std::string& file, size_t *current_num_rows_per_row_block, bool use_cache = true); // Return the IndexOffset of the first element, physically, it's (0, 0) const OLAPIndexOffset begin() const { diff --git a/be/src/olap/olap_meta.cpp b/be/src/olap/olap_meta.cpp index f9ca9775b11dc22..be9960507f2d7b5 100755 --- a/be/src/olap/olap_meta.cpp +++ b/be/src/olap/olap_meta.cpp @@ -85,7 +85,7 @@ OLAPStatus OlapMeta::init() { return OLAP_SUCCESS; } -OLAPStatus OlapMeta::get(const int column_family_index, const std::string& key, std::string* value) { +OLAPStatus OlapMeta::get(int column_family_index, const std::string& key, std::string* value) { DorisMetrics::instance()->meta_read_request_total->increment(1); rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index]; int64_t duration_ns = 0; @@ -104,7 +104,7 @@ OLAPStatus OlapMeta::get(const int column_family_index, const std::string& key, return OLAP_SUCCESS; } -OLAPStatus OlapMeta::put(const int column_family_index, const std::string& key, const std::string& value) { +OLAPStatus OlapMeta::put(int column_family_index, const std::string& key, const std::string& value) { DorisMetrics::instance()->meta_write_request_total->increment(1); rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index]; int64_t duration_ns = 0; @@ -123,7 +123,7 @@ OLAPStatus OlapMeta::put(const int column_family_index, const std::string& key, return OLAP_SUCCESS; } -OLAPStatus OlapMeta::remove(const int column_family_index, const std::string& key) { +OLAPStatus OlapMeta::remove(int column_family_index, const std::string& key) { DorisMetrics::instance()->meta_write_request_total->increment(1); rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index]; rocksdb::Status s; @@ -142,11 +142,11 @@ OLAPStatus OlapMeta::remove(const int column_family_index, const std::string& ke return OLAP_SUCCESS; } -OLAPStatus OlapMeta::iterate(const int column_family_index, const std::string& prefix, +OLAPStatus OlapMeta::iterate(int column_family_index, const std::string& prefix, std::function const& func) { rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index]; std::unique_ptr it(_db->NewIterator(ReadOptions(), handle)); - if (prefix == "") { + if (prefix.empty()) { it->SeekToFirst(); } else { it->Seek(prefix); @@ -157,7 +157,7 @@ OLAPStatus OlapMeta::iterate(const int column_family_index, const std::string& p return OLAP_ERR_META_ITERATOR; } for (; it->Valid(); it->Next()) { - if (prefix != "") { + if (!prefix.empty()) { if (!it->key().starts_with(prefix)) { return OLAP_SUCCESS; } diff --git a/be/src/olap/olap_meta.h b/be/src/olap/olap_meta.h index 3574695a6babfcf..ac8bf29a3987696 100755 --- a/be/src/olap/olap_meta.h +++ b/be/src/olap/olap_meta.h @@ -35,13 +35,13 @@ class OlapMeta { OLAPStatus init(); - OLAPStatus get(const int column_family_index, const std::string& key, std::string* value); + OLAPStatus get(int column_family_index, const std::string& key, std::string* value); - OLAPStatus put(const int column_family_index, const std::string& key, const std::string& value); + OLAPStatus put(int column_family_index, const std::string& key, const std::string& valuOlapMetae); - OLAPStatus remove(const int column_family_index, const std::string& key); + OLAPStatus remove(int column_family_index, const std::string& key); - OLAPStatus iterate(const int column_family_index, const std::string& prefix, + OLAPStatus iterate(int column_family_index, const std::string& prefix, std::function const& func); std::string get_root_path(); diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp index 3f860f4f5971ecf..a47ffd0c3fe8454 100644 --- a/be/src/olap/olap_server.cpp +++ b/be/src/olap/olap_server.cpp @@ -207,31 +207,32 @@ void StorageEngine::_garbage_sweeper_thread_callback() { OLAP_LOG_WARNING("garbage sweep interval config is illegal: [max=%d min=%d].", max_interval, min_interval); min_interval = 1; - max_interval = max_interval >= min_interval ? max_interval : min_interval; + max_interval = std::max(max_interval, min_interval); LOG(INFO) << "force reset garbage sweep interval. " << "max_interval=" << max_interval << ", min_interval=" << min_interval; } const double pi = 4 * std::atan(1); - double usage = 1.0; + double max_usage = 1.0; // 程序启动后经过min_interval后触发第一轮扫描 uint32_t curr_interval = min_interval; while (!_stop_background_threads_latch.wait_for(MonoDelta::FromSeconds(curr_interval))) { - usage *= 100.0; + // TODO(yingchun): it's a big to '*= 100' in loop!!! + max_usage *= 100.0; // 该函数特性:当磁盘使用率<60%的时候,ratio接近于1; // 当使用率介于[60%, 75%]之间时,ratio急速从0.87降到0.27; // 当使用率大于75%时,ratio值开始缓慢下降 // 当usage=90%时,ratio约为0.0057 - double ratio = (1.1 * (pi / 2 - std::atan(usage / 5 - 14)) - 0.28) / pi; - ratio = ratio > 0 ? ratio : 0; + double ratio = (1.1 * (pi / 2 - std::atan(max_usage / 5 - 14)) - 0.28) / pi; + ratio = std::max(ratio, 0.0); uint32_t curr_interval = max_interval * ratio; // 此时的特性,当usage<60%时,curr_interval的时间接近max_interval, // 当usage > 80%时,curr_interval接近min_interval - curr_interval = curr_interval > min_interval ? curr_interval : min_interval; + curr_interval = std::max(curr_interval, min_interval); // 开始清理,并得到清理后的磁盘使用率 - OLAPStatus res = _start_trash_sweep(&usage); + OLAPStatus res = _start_trash_sweep(&max_usage); if (res != OLAP_SUCCESS) { OLAP_LOG_WARNING("one or more errors occur when sweep trash." "see previous message for detail. [err code=%d]", res); @@ -384,6 +385,7 @@ void StorageEngine::_tablet_checkpoint_callback(DataDir* data_dir) { _tablet_manager->do_tablet_meta_checkpoint(data_dir); int64_t used_time = (UnixMillis() - start_time) / 1000; if (used_time < config::tablet_meta_checkpoint_min_interval_secs) { + // TODO(yingchun): is it necessary to use interval difference eith others? interval = config::tablet_meta_checkpoint_min_interval_secs - used_time; } else { interval = 1; diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp index cfd221decda7105..fdbdcbd1dfddf00 100644 --- a/be/src/olap/push_handler.cpp +++ b/be/src/olap/push_handler.cpp @@ -139,6 +139,7 @@ OLAPStatus PushHandler::_do_streaming_ingestion( } else { ReadLock new_migration_rlock(related_tablet->get_migration_lock_ptr(), TRY_LOCK); if (!new_migration_rlock.own_lock()) { + // TODO(yingchun): tablet->release_push_lock(); return OLAP_ERR_RWLOCK_ERROR; } PUniqueId load_id; diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp index f925b1073a01d4f..72e69bc5578ec7f 100644 --- a/be/src/olap/reader.cpp +++ b/be/src/olap/reader.cpp @@ -482,6 +482,7 @@ OLAPStatus Reader::_capture_rs_readers(const ReaderParams& read_params) { } else if (_keys_param.end_range == "le") { is_upper_key_included = true; } else { + // TODO(yingchun): use DCHECK instead LOG(WARNING) << "reader params end_range is error. " << "range=" << _keys_param.to_string(); return OLAP_ERR_READER_GET_ITERATOR_ERROR; @@ -509,6 +510,7 @@ OLAPStatus Reader::_capture_rs_readers(const ReaderParams& read_params) { is_lower_key_included = true; is_upper_key_included = true; } else { + // TODO(yingchun): use DCHECK instead LOG(WARNING) << "reader params range is error. " << "range=" << _keys_param.to_string(); return OLAP_ERR_READER_GET_ITERATOR_ERROR; @@ -520,6 +522,10 @@ OLAPStatus Reader::_capture_rs_readers(const ReaderParams& read_params) { if (eof) { return OLAP_SUCCESS; } + // TODO(yingchun): judge whether init ok, like + // is_lower_keys_included == is_upper_keys_included == + // _keys_param.start_keys.size() == _keys_param.end_keys.size() + bool need_ordered_result = true; if (read_params.reader_type == READER_QUERY) { if (_tablet->tablet_schema().keys_type() == DUP_KEYS) { @@ -635,6 +641,7 @@ OLAPStatus Reader::_init_return_columns(const ReaderParams& read_params) { } } } else if (read_params.return_columns.empty()) { + // TODO(yingchun): how about count(*)? for (size_t i = 0; i < _tablet->tablet_schema().num_columns(); ++i) { _return_columns.push_back(i); if (_tablet->tablet_schema().column(i).is_key()) { @@ -682,6 +689,7 @@ void Reader::_init_seek_columns() { } } for (uint32_t i = 0; i < _tablet->tablet_schema().num_columns(); i++) { + // TODO(yingchun): is column id continous? if not, there is a bug if (i < max_key_column_count || column_set.find(i) != column_set.end()) { _seek_columns.push_back(i); } @@ -693,6 +701,7 @@ OLAPStatus Reader::_init_keys_param(const ReaderParams& read_params) { return OLAP_SUCCESS; } + // TODO(yingchun): validate end_range _keys_param.range = read_params.range; _keys_param.end_range = read_params.end_range; @@ -854,15 +863,18 @@ COMPARISON_PREDICATE_CONDITION_VALUE(ge, GreaterEqualPredicate) ColumnPredicate* Reader::_parse_to_predicate(const TCondition& condition) { // TODO: not equal and not in predicate is not pushed down + // TODO(yingchun): we should use column id instead of column name int index = _tablet->field_index(condition.column_name); const TabletColumn& column = _tablet->tablet_schema().column(index); if (column.aggregation() != FieldAggregationMethod::OLAP_FIELD_AGGREGATION_NONE) { return nullptr; } ColumnPredicate* predicate = nullptr; + // TODO(yingchun): we should use enum instead of string if (condition.condition_op == "*=" && condition.condition_values.size() == 1) { predicate = _new_eq_pred(column, index, condition.condition_values[0]); } else if (condition.condition_op == "<<") { + // TODO(yingchun): should add size judement 'size() == 1' for these ifs predicate = _new_lt_pred(column, index, condition.condition_values[0]); } else if (condition.condition_op == "<=") { predicate = _new_le_pred(column, index, condition.condition_values[0]); @@ -871,6 +883,7 @@ ColumnPredicate* Reader::_parse_to_predicate(const TCondition& condition) { } else if (condition.condition_op == ">=") { predicate = _new_ge_pred(column, index, condition.condition_values[0]); } else if (condition.condition_op == "*=" && condition.condition_values.size() > 1) { + // TODO(yingchun): wrap in a function switch (column.type()) { case OLAP_FIELD_TYPE_TINYINT: { std::set values; @@ -878,6 +891,7 @@ ColumnPredicate* Reader::_parse_to_predicate(const TCondition& condition) { int32_t value = 0; std::stringstream ss(cond_val); ss >> value; + // TODO(yingchun): ignore overflow? values.insert(value); } predicate = new InListPredicate(index, std::move(values)); @@ -1012,6 +1026,7 @@ void Reader::_init_load_bf_columns(const ReaderParams& read_params) { } // remove columns which have same value between start_key and end_key + // TODO(yingchun): what does these two judgement mean? int min_scan_key_len = _tablet->tablet_schema().num_columns(); for (int i = 0; i < read_params.start_key.size(); ++i) { if (read_params.start_key[i].size() < min_scan_key_len) { @@ -1025,6 +1040,7 @@ void Reader::_init_load_bf_columns(const ReaderParams& read_params) { } } + // TODO(yingchun): I don't understand what deos these code mean? int max_equal_index = -1; for (int i = 0; i < read_params.start_key.size(); ++i) { int j = 0; @@ -1034,6 +1050,7 @@ void Reader::_init_load_bf_columns(const ReaderParams& read_params) { } } + // TODO(yingchun): max_equal_index will be overwrite? if (max_equal_index < j - 1) { max_equal_index = j - 1; } diff --git a/be/src/olap/reader.h b/be/src/olap/reader.h index d5d77ea6a43be1e..243a339a349ea64 100644 --- a/be/src/olap/reader.h +++ b/be/src/olap/reader.h @@ -61,13 +61,18 @@ struct ReaderParams { // if config::disable_storage_page_cache is false, we use page cache bool use_page_cache = false; Version version = Version(-1, 0); + + // TODO(yingchun): why there is only 1 pair of range-end_range, but multi start_key-end_key pairs? // possible values are "gt", "ge", "eq" std::string range; // possible values are "lt", "le" std::string end_range; + + // TODO(yingchun): these 3 vectors should equal? std::vector start_key; std::vector end_key; std::vector conditions; + // The ColumnData will be set when using Merger, eg Cumulative, BE. std::vector rs_readers; std::vector return_columns; @@ -162,7 +167,7 @@ class Reader { return ss.str(); } - std::string range; + std::string range; // TODO(yingchun): rename to start_range std::string end_range; std::vector start_keys; std::vector end_keys; diff --git a/be/src/olap/row_block.cpp b/be/src/olap/row_block.cpp index f50acf5d347fba9..1c971d399d114af 100644 --- a/be/src/olap/row_block.cpp +++ b/be/src/olap/row_block.cpp @@ -48,13 +48,12 @@ RowBlock::~RowBlock() { delete[] _mem_buf; } -OLAPStatus RowBlock::init(const RowBlockInfo& block_info) { +void RowBlock::init(const RowBlockInfo& block_info) { _info = block_info; _null_supported = block_info.null_supported; _capacity = _info.row_num; _compute_layout(); _mem_buf = new char[_mem_buf_bytes]; - return OLAP_SUCCESS; } OLAPStatus RowBlock::finalize(uint32_t row_num) { @@ -82,7 +81,7 @@ void RowBlock::_compute_layout() { size_t memory_size = 0; for (int col_id = 0; col_id < _schema->num_columns(); ++col_id) { const TabletColumn& column = _schema->column(col_id); - if (!column_set.empty() && column_set.find(col_id) == std::end(column_set)) { + if (!column_set.empty() && column_set.count(col_id) == 0) { // which may lead BE crash _field_offset_in_memory.push_back(std::numeric_limits::max()); continue; diff --git a/be/src/olap/row_block.h b/be/src/olap/row_block.h index 1292181f8075c28..c917b9091c16651 100644 --- a/be/src/olap/row_block.h +++ b/be/src/olap/row_block.h @@ -49,7 +49,7 @@ struct RowBlockInfo { // RowBlock的内部buf中; // 2. 给定row_index,读取内部各field的值 // 3. 给定查询的key,在RowBlock内做二分查找,返回起点的行偏移; -// 4. 向量化的条件过滤下推到RowBlock级别进行,因此增加完成过滤的数据读取借口 +// 4. 向量化的条件过滤下推到RowBlock级别进行,因此增加完成过滤的数据读取接口 class RowBlock { // Please keep these classes as 'friend'. They have to use lots of private fields for // faster operation. @@ -65,7 +65,7 @@ class RowBlock { // row_num是RowBlock的最大行数,fields为了初始化各个field的起始位置。 // 在field都为定长的情况下根据这两个值可以确定RowBlock内部buffer的大小, // 目前只考虑定长,因此在函数可以分配内存资源。 - OLAPStatus init(const RowBlockInfo& block_info); + void init(const RowBlockInfo& block_info); inline void get_row(uint32_t row_index, RowCursor* cursor) const { cursor->attach(_mem_buf + row_index * _mem_row_bytes); diff --git a/be/src/olap/row_cursor.cpp b/be/src/olap/row_cursor.cpp index 9fad7ba7c5722f0..9c18a57a4bacb65 100644 --- a/be/src/olap/row_cursor.cpp +++ b/be/src/olap/row_cursor.cpp @@ -28,9 +28,7 @@ using std::string; using std::vector; namespace doris { -RowCursor::RowCursor() : - _fixed_len(0), - _variable_len(0) {} +RowCursor::RowCursor() {} RowCursor::~RowCursor() { delete [] _owned_fixed_buf; @@ -70,19 +68,7 @@ OLAPStatus RowCursor::init(const std::vector& schema) { } OLAPStatus RowCursor::init(const TabletSchema& schema, size_t column_count) { - if (column_count > schema.num_columns()) { - LOG(WARNING) << "Input param are invalid. Column count is bigger than num_columns of schema. " - << "column_count=" << column_count - << ", schema.num_columns=" << schema.num_columns(); - return OLAP_ERR_INPUT_PARAMETER_ERROR; - } - - std::vector columns; - for (size_t i = 0; i < column_count; ++i) { - columns.push_back(i); - } - RETURN_NOT_OK(_init(schema.columns(), columns)); - return OLAP_SUCCESS; + return init(schema.columns(), column_count); } OLAPStatus RowCursor::init(const std::vector& schema, size_t column_count) { @@ -97,14 +83,12 @@ OLAPStatus RowCursor::init(const std::vector& schema, size_t colum for (size_t i = 0; i < column_count; ++i) { columns.push_back(i); } - RETURN_NOT_OK(_init(schema, columns)); - return OLAP_SUCCESS; + return _init(schema, columns); } OLAPStatus RowCursor::init(const TabletSchema& schema, const vector& columns) { - RETURN_NOT_OK(_init(schema.columns(), columns)); - return OLAP_SUCCESS; + return _init(schema.columns(), columns); } OLAPStatus RowCursor::init_scan_key(const TabletSchema& schema, @@ -118,6 +102,7 @@ OLAPStatus RowCursor::init_scan_key(const TabletSchema& schema, } std::vector columns; + columns.reserve(scan_key_size); for (size_t i = 0; i < scan_key_size; ++i) { columns.push_back(i); } @@ -167,10 +152,10 @@ OLAPStatus RowCursor::init_scan_key(const TabletSchema& schema, return OLAP_SUCCESS; } -OLAPStatus RowCursor::allocate_memory_for_string_type(const TabletSchema& schema) { +void RowCursor::allocate_memory_for_string_type(const TabletSchema& schema) { // allocate memory for string type(char, varchar, hll) // The memory allocated in this function is used in aggregate and copy function - if (_variable_len == 0) { return OLAP_SUCCESS; } + if (_variable_len == 0) { return; } DCHECK(_variable_buf == nullptr) << "allocate memory twice"; _variable_buf = new (nothrow) char[_variable_len]; memset(_variable_buf, 0, _variable_len); @@ -182,28 +167,24 @@ OLAPStatus RowCursor::allocate_memory_for_string_type(const TabletSchema& schema fixed_ptr = _fixed_buf + _schema->column_offset(cid); variable_ptr = column_schema(cid)->allocate_memory(fixed_ptr + 1, variable_ptr); } - return OLAP_SUCCESS; } -OLAPStatus RowCursor::build_max_key() { +void RowCursor::build_max_key() { for (auto cid : _schema->column_ids()) { const Field* field = column_schema(cid); char* dest = cell_ptr(cid); field->set_to_max(dest); set_not_null(cid); } - return OLAP_SUCCESS; } -OLAPStatus RowCursor::build_min_key() { +void RowCursor::build_min_key() { for (auto cid : _schema->column_ids()) { const Field* field = column_schema(cid); char* dest = cell_ptr(cid); field->set_to_min(dest); set_null(cid); } - - return OLAP_SUCCESS; } OLAPStatus RowCursor::from_tuple(const OlapTuple& tuple) { @@ -235,10 +216,9 @@ OLAPStatus RowCursor::from_tuple(const OlapTuple& tuple) { OlapTuple RowCursor::to_tuple() const { OlapTuple tuple; - for (auto cid : _schema->column_ids()) { - if (_schema->column(cid) != nullptr) { - const Field* field = column_schema(cid); + const Field* field = column_schema(cid); + if (field != nullptr) { char* src = cell_ptr(cid); if (is_null(cid)) { tuple.add_null(); diff --git a/be/src/olap/row_cursor.h b/be/src/olap/row_cursor.h index 0387bb1f83b82a0..eb0dc3f5451cd84 100644 --- a/be/src/olap/row_cursor.h +++ b/be/src/olap/row_cursor.h @@ -35,8 +35,6 @@ class Field; class RowCursor { public: RowCursor(); - - // 遍历销毁field指针 ~RowCursor(); // 根据传入schema的创建RowCursor @@ -58,8 +56,8 @@ class RowCursor { OLAPStatus init_scan_key(const TabletSchema& schema, const std::vector& keys); - //allocate memory for string type, which include char, varchar, hyperloglog - OLAPStatus allocate_memory_for_string_type(const TabletSchema& schema); + // allocate memory for string type, which include char, varchar, hyperloglog + void allocate_memory_for_string_type(const TabletSchema& schema); RowCursorCell cell(uint32_t cid) const { return RowCursorCell(nullable_cell_ptr(cid)); @@ -85,6 +83,7 @@ class RowCursor { char* dst_cell = cell_ptr(index); column_schema(index)->shallow_copy_content(dst_cell, buf); } + // convert and deep copy field content OLAPStatus convert_from(size_t index, const char* src, const TypeInfo* src_type, MemPool* mem_pool) { char* dest = cell_ptr(index); @@ -92,7 +91,7 @@ class RowCursor { } // 从传入的字符串数组反序列化内部各field的值 - // 每个字符串必须是一个\0结尾的字符串 + // 每个字符串必须是一个'\0'结尾的字符串 // 要求输入字符串和row cursor有相同的列数, OLAPStatus from_tuple(const OlapTuple& tuple); @@ -118,10 +117,8 @@ class RowCursor { } // set max/min for key field in _field_array - OLAPStatus build_max_key(); - OLAPStatus build_min_key(); - - inline char* get_buf() const { return _fixed_buf; } + void build_max_key(); + void build_min_key(); // this two functions is used in unit test inline size_t get_fixed_len() const { return _fixed_len; } @@ -168,11 +165,11 @@ class RowCursor { std::unique_ptr _schema; char* _fixed_buf = nullptr; // point to fixed buf - size_t _fixed_len; + size_t _fixed_len = 0; char* _owned_fixed_buf = nullptr; // point to buf allocated in init function char* _variable_buf = nullptr; - size_t _variable_len; + size_t _variable_len = 0; DISALLOW_COPY_AND_ASSIGN(RowCursor); }; diff --git a/be/src/olap/rowset/alpha_rowset.cpp b/be/src/olap/rowset/alpha_rowset.cpp index 138b9dbd3a07a76..f38e70ee159b564 100644 --- a/be/src/olap/rowset/alpha_rowset.cpp +++ b/be/src/olap/rowset/alpha_rowset.cpp @@ -78,21 +78,24 @@ OLAPStatus AlphaRowset::remove() { return OLAP_ERR_ROWSET_DELETE_FILE_FAILED; } } + // TODO(yingchun): not clear _segment_groups? return OLAP_SUCCESS; } -void AlphaRowset::make_visible_extra(Version version, VersionHash version_hash) { +void AlphaRowset::make_visible_extra(Version version, VersionHash version_hash) { AlphaRowsetMetaSharedPtr alpha_rowset_meta = std::dynamic_pointer_cast(_rowset_meta); vector published_segment_groups; + // TODO(yingchun): Make alpha_rowset_meta->get_segment_groups return pointer or reference, to reduce data copy. alpha_rowset_meta->get_segment_groups(&published_segment_groups); - int32_t segment_group_idx = 0; + CHECK_EQ(published_segment_groups.size(), _segment_groups.size()); + int32_t i = 0; for (auto& segment_group : _segment_groups) { segment_group->set_version(version); segment_group->set_version_hash(version_hash); segment_group->set_pending_finished(); - published_segment_groups.at(segment_group_idx).clear_load_id(); - ++segment_group_idx; + published_segment_groups[i].clear_load_id(); + ++i; } alpha_rowset_meta->clear_segment_group(); for (auto& segment_group_meta : published_segment_groups) { @@ -100,7 +103,7 @@ void AlphaRowset::make_visible_extra(Version version, VersionHash version_hash) } } -OLAPStatus AlphaRowset::link_files_to(const std::string& dir, RowsetId new_rowset_id) { +OLAPStatus AlphaRowset::link_files_to(const std::string& dir, const RowsetId& new_rowset_id) { for (auto& segment_group : _segment_groups) { auto status = segment_group->link_segments_to_path(dir, new_rowset_id); if (status != OLAP_SUCCESS) { @@ -112,7 +115,7 @@ OLAPStatus AlphaRowset::link_files_to(const std::string& dir, RowsetId new_rowse return OLAP_SUCCESS; } -OLAPStatus AlphaRowset::copy_files_to(const std::string& dir) { +OLAPStatus AlphaRowset::copy_files_to(const std::string& dir) const { for (auto& segment_group : _segment_groups) { OLAPStatus status = segment_group->copy_files_to(dir); if (status != OLAP_SUCCESS) { @@ -126,7 +129,7 @@ OLAPStatus AlphaRowset::copy_files_to(const std::string& dir) { } OLAPStatus AlphaRowset::convert_from_old_files(const std::string& snapshot_path, - std::vector* success_files) { + std::vector* success_files) { for (auto& segment_group : _segment_groups) { OLAPStatus status = segment_group->convert_from_old_files(snapshot_path, success_files); if (status != OLAP_SUCCESS) { @@ -168,11 +171,6 @@ OLAPStatus AlphaRowset::split_range( const RowCursor& end_key, uint64_t request_block_row_count, vector* ranges) { - EntrySlice entry; - RowBlockPosition start_pos; - RowBlockPosition end_pos; - RowBlockPosition step_pos; - std::shared_ptr largest_segment_group = _segment_group_with_largest_size(); if (largest_segment_group == nullptr || largest_segment_group->current_num_rows_per_row_block() == 0) { LOG(WARNING) << "failed to get largest_segment_group. is null: " << (largest_segment_group == nullptr) @@ -182,8 +180,13 @@ OLAPStatus AlphaRowset::split_range( ranges->emplace_back(end_key.to_tuple()); return OLAP_SUCCESS; } - uint64_t expected_rows = request_block_row_count - / largest_segment_group->current_num_rows_per_row_block(); + + uint64_t expected_rows = 0; + size_t current_num_rows_per_row_block = largest_segment_group->current_num_rows_per_row_block(); + if (current_num_rows_per_row_block != 0) { + expected_rows = request_block_row_count / current_num_rows_per_row_block; + } + if (expected_rows == 0) { LOG(WARNING) << "expected_rows less than 1. [request_block_row_count = " << request_block_row_count << "]"; @@ -196,6 +199,8 @@ OLAPStatus AlphaRowset::split_range( LOG(WARNING) << "fail to parse strings to key with RowCursor type."; return OLAP_ERR_INVALID_SCHEMA; } + + RowBlockPosition start_pos; if (largest_segment_group->find_short_key(start_key, &helper_cursor, false, &start_pos) != OLAP_SUCCESS) { if (largest_segment_group->find_first_row_block(&start_pos) != OLAP_SUCCESS) { LOG(WARNING) << "fail to get first block pos"; @@ -203,10 +208,10 @@ OLAPStatus AlphaRowset::split_range( } } - step_pos = start_pos; VLOG(3) << "start_pos=" << start_pos.segment << ", " << start_pos.index_offset; //find last row_block is end_key is given, or using last_row_block + RowBlockPosition end_pos; if (largest_segment_group->find_short_key(end_key, &helper_cursor, false, &end_pos) != OLAP_SUCCESS) { if (largest_segment_group->find_last_row_block(&end_pos) != OLAP_SUCCESS) { LOG(WARNING) << "fail find last row block."; @@ -217,10 +222,8 @@ OLAPStatus AlphaRowset::split_range( VLOG(3) << "end_pos=" << end_pos.segment << ", " << end_pos.index_offset; //get rows between first and last - OLAPStatus res = OLAP_SUCCESS; RowCursor cur_start_key; RowCursor last_start_key; - if (cur_start_key.init(*_schema, _schema->num_short_key_columns()) != OLAP_SUCCESS || last_start_key.init(*_schema, _schema->num_short_key_columns()) != OLAP_SUCCESS) { LOG(WARNING) << "fail to init cursor"; @@ -228,10 +231,12 @@ OLAPStatus AlphaRowset::split_range( } std::vector cids; + cids.reserve(_schema->num_short_key_columns()); for (uint32_t cid = 0; cid < _schema->num_short_key_columns(); ++cid) { cids.push_back(cid); } + EntrySlice entry; if (largest_segment_group->get_row_block_entry(start_pos, &entry) != OLAP_SUCCESS) { LOG(WARNING) << "get block entry failed."; return OLAP_ERR_ROWBLOCK_FIND_ROW_EXCEPTION; @@ -243,6 +248,8 @@ OLAPStatus AlphaRowset::split_range( // start_key是last start_key, 但返回的实际上是查询层给出的key ranges->emplace_back(start_key.to_tuple()); + OLAPStatus res = OLAP_SUCCESS; + RowBlockPosition step_pos(start_pos); while (end_pos > step_pos) { res = largest_segment_group->advance_row_block(expected_rows, &step_pos); if (res == OLAP_ERR_INDEX_EOF || !(end_pos > step_pos)) { @@ -273,13 +280,14 @@ bool AlphaRowset::check_path(const std::string& path) { std::set valid_paths; for (auto& segment_group : _segment_groups) { for (int i = 0; i < segment_group->num_segments(); ++i) { - std::string data_path = segment_group->construct_data_file_path(i); - std::string index_path = segment_group->construct_index_file_path(i); - valid_paths.insert(data_path); - valid_paths.insert(index_path); + // TODO(yingchun): or some better search algorithm + if (path == segment_group->construct_data_file_path(i) || + path == segment_group->construct_index_file_path(i)) { + return true; + } } } - return valid_paths.find(path) != valid_paths.end(); + return false; } OLAPStatus AlphaRowset::init() { @@ -290,19 +298,16 @@ OLAPStatus AlphaRowset::init() { std::shared_ptr segment_group; if (_is_pending) { segment_group.reset(new SegmentGroup(_rowset_meta->tablet_id(), - _rowset_meta->rowset_id(), _schema, _rowset_path, false, segment_group_meta.segment_group_id(), - segment_group_meta.num_segments(), true, + _rowset_meta->rowset_id(), _schema, _rowset_path, + false, segment_group_meta.segment_group_id(), + segment_group_meta.num_segments(), true, _rowset_meta->partition_id(), _rowset_meta->txn_id())); } else { segment_group.reset(new SegmentGroup(_rowset_meta->tablet_id(), - _rowset_meta->rowset_id(), _schema, _rowset_path, + _rowset_meta->rowset_id(), _schema, _rowset_path, _rowset_meta->version(), _rowset_meta->version_hash(), false, segment_group_meta.segment_group_id(), segment_group_meta.num_segments())); } - if (segment_group == nullptr) { - LOG(WARNING) << "fail to create olap segment_group. rowset_id='" << _rowset_meta->rowset_id(); - return OLAP_ERR_CREATE_FILE_ERROR; - } if (segment_group_meta.has_empty()) { segment_group->set_empty(segment_group_meta.empty()); } @@ -375,21 +380,24 @@ std::shared_ptr AlphaRowset::_segment_group_with_largest_size() { OLAPStatus AlphaRowset::reset_sizeinfo() { RETURN_NOT_OK(load()); + // TODO(yingchun): optimize the way to update sizes. + // copy from _rowset_meta -> set new size value -> clear sg in _rowset_meta -> add sg into _rowset_meta one by one std::vector segment_group_metas; AlphaRowsetMetaSharedPtr alpha_rowset_meta = std::dynamic_pointer_cast(_rowset_meta); alpha_rowset_meta->get_segment_groups(&segment_group_metas); - int32_t segment_group_idx = 0; + CHECK_EQ(segment_group_metas.size(), _segment_groups.size()); + int32_t i = 0; int64_t data_disk_size = 0; int64_t index_disk_size = 0; int64_t num_rows = 0; for (auto segment_group : _segment_groups) { - segment_group_metas.at(segment_group_idx).set_data_size(segment_group->data_size()); - segment_group_metas.at(segment_group_idx).set_index_size(segment_group->index_size()); - segment_group_metas.at(segment_group_idx).set_num_rows(segment_group->num_rows()); + segment_group_metas[i].set_data_size(segment_group->data_size()); + segment_group_metas[i].set_index_size(segment_group->index_size()); + segment_group_metas[i].set_num_rows(segment_group->num_rows()); data_disk_size += segment_group->data_size(); index_disk_size += segment_group->index_size(); num_rows += segment_group->num_rows(); - ++segment_group_idx; + ++i; } alpha_rowset_meta->clear_segment_group(); alpha_rowset_meta->set_num_rows(num_rows); diff --git a/be/src/olap/rowset/alpha_rowset.h b/be/src/olap/rowset/alpha_rowset.h index 3c21e7dd21dbf6b..c48813da99bdc79 100644 --- a/be/src/olap/rowset/alpha_rowset.h +++ b/be/src/olap/rowset/alpha_rowset.h @@ -38,8 +38,6 @@ class RowsetFactory; class AlphaRowset : public Rowset { public: - virtual ~AlphaRowset() {} - OLAPStatus create_reader(std::shared_ptr* result) override; OLAPStatus create_reader(const std::shared_ptr& parent_tracker, @@ -51,9 +49,9 @@ class AlphaRowset : public Rowset { OLAPStatus remove() override; - OLAPStatus link_files_to(const std::string& dir, RowsetId new_rowset_id) override; + OLAPStatus link_files_to(const std::string& dir, const RowsetId& new_rowset_id) override; - OLAPStatus copy_files_to(const std::string& dir) override; + OLAPStatus copy_files_to(const std::string& dir) const override; OLAPStatus convert_from_old_files(const std::string& snapshot_path, std::vector* success_files); diff --git a/be/src/olap/rowset/alpha_rowset_reader.cpp b/be/src/olap/rowset/alpha_rowset_reader.cpp index 361b945e3b87dbf..1cfb93c3dc39c42 100644 --- a/be/src/olap/rowset/alpha_rowset_reader.cpp +++ b/be/src/olap/rowset/alpha_rowset_reader.cpp @@ -49,7 +49,6 @@ OLAPStatus AlphaRowsetReader::init(RowsetReaderContext* read_context) { } _is_segments_overlapping = _alpha_rowset_meta->is_segments_overlapping(); - _ordinal = 0; RETURN_NOT_OK(_init_merge_ctxs(read_context)); @@ -156,7 +155,10 @@ OLAPStatus AlphaRowsetReader::_merge_block(RowBlock** block) { VLOG(10) << "get merged row: " << row_cursor->to_string(); + // TODO(yingchun): wrap these code _read_block->get_row(_read_block->pos(), _dst_cursor); + // TODO(yingchun): optimize to avoid copy row, as long as this row is in memory + // (i.e. merge_ctx didn't switch to next block), we can use pointer and not copy. copy_row(_dst_cursor, *row_cursor, _read_block->mem_pool()); _read_block->pos_inc(); num_rows_in_block++; @@ -318,6 +320,7 @@ OLAPStatus AlphaRowsetReader::_pull_first_block(AlphaMergeContext* merge_ctx) { } OLAPStatus AlphaRowsetReader::_init_merge_ctxs(RowsetReaderContext* read_context) { + // TODO(yingchun): do these check in upper layer if (read_context->reader_type == READER_QUERY) { if (read_context->lower_bound_keys->size() != read_context->is_lower_keys_included->size() || read_context->lower_bound_keys->size() != read_context->upper_bound_keys->size() @@ -359,8 +362,7 @@ OLAPStatus AlphaRowsetReader::_init_merge_ctxs(RowsetReaderContext* read_context if (new_column_data->rowset_pruning_filter()) { _stats->rows_stats_filtered += new_column_data->num_rows(); VLOG(3) << "filter segment group in query in condition. version=" - << new_column_data->version().first - << "-" << new_column_data->version().second; + << new_column_data->version(); continue; } } @@ -369,15 +371,15 @@ OLAPStatus AlphaRowsetReader::_init_merge_ctxs(RowsetReaderContext* read_context if (ret == DEL_SATISFIED) { _stats->rows_del_filtered += new_column_data->num_rows(); VLOG(3) << "filter segment group in delete predicate:" - << new_column_data->version().first << ", " << new_column_data->version().second; + << new_column_data->version(); continue; } else if (ret == DEL_PARTIAL_SATISFIED) { VLOG(3) << "filter segment group partially in delete predicate:" - << new_column_data->version().first << ", " << new_column_data->version().second; + << new_column_data->version(); new_column_data->set_delete_status(DEL_PARTIAL_SATISFIED); } else { VLOG(3) << "not filter segment group in delete predicate:" - << new_column_data->version().first << ", " << new_column_data->version().second; + << new_column_data->version(); new_column_data->set_delete_status(DEL_NOT_SATISFIED); } AlphaMergeContext merge_ctx; diff --git a/be/src/olap/rowset/alpha_rowset_reader.h b/be/src/olap/rowset/alpha_rowset_reader.h index 711f4c2a60c01ba..ce6e9f99f57b4b3 100644 --- a/be/src/olap/rowset/alpha_rowset_reader.h +++ b/be/src/olap/rowset/alpha_rowset_reader.h @@ -92,6 +92,7 @@ class AlphaRowsetReader : public RowsetReader { // merge by priority queue(_merge_heap) // this method has same function with _pull_next_row_for_merge_rowset, but using heap merge. // and this should replace the _pull_next_row_for_merge_rowset later. + // TODO(yingchun): remove _pull_next_row_for_merge_rowset now OLAPStatus _pull_next_row_for_merge_rowset_v2(RowCursor** row); // init the merge heap, this should be call before calling _pull_next_row_for_merge_rowset_v2(); OLAPStatus _init_merge_heap(); @@ -113,15 +114,15 @@ class AlphaRowsetReader : public RowsetReader { std::unique_ptr _read_block; OLAPStatus (AlphaRowsetReader::*_next_block)(RowBlock** block) = nullptr; RowCursor* _dst_cursor = nullptr; - int _key_range_size; + int _key_range_size = 0; // In streaming ingestion, row among different segment // groups may overlap, and is necessary to be taken // into consideration deliberately. bool _is_segments_overlapping; - // ordinal of ColumnData upon reading - size_t _ordinal; + // ordinal of ColumnData upon reading, only used for _union_block (non-ordered read?). + size_t _ordinal = 0; RowsetReaderContext* _current_read_context; OlapReaderStatistics _owned_stats; diff --git a/be/src/olap/rowset/alpha_rowset_writer.cpp b/be/src/olap/rowset/alpha_rowset_writer.cpp index 979a1ae3f1255f6..8a772741b114118 100644 --- a/be/src/olap/rowset/alpha_rowset_writer.cpp +++ b/be/src/olap/rowset/alpha_rowset_writer.cpp @@ -57,8 +57,8 @@ OLAPStatus AlphaRowsetWriter::init(const RowsetWriterContext& rowset_writer_cont _current_rowset_meta->set_tablet_id(_rowset_writer_context.tablet_id); _current_rowset_meta->set_tablet_schema_hash(_rowset_writer_context.tablet_schema_hash); _current_rowset_meta->set_rowset_type(_rowset_writer_context.rowset_type); - _current_rowset_meta->set_rowset_state(rowset_writer_context.rowset_state); - _current_rowset_meta->set_segments_overlap(rowset_writer_context.segments_overlap); + _current_rowset_meta->set_rowset_state(_rowset_writer_context.rowset_state); + _current_rowset_meta->set_segments_overlap(_rowset_writer_context.segments_overlap); RowsetStatePB rowset_state = _rowset_writer_context.rowset_state; if (rowset_state == PREPARED || rowset_state == COMMITTED) { @@ -98,9 +98,10 @@ OLAPStatus AlphaRowsetWriter::add_rowset(RowsetSharedPtr rowset) { // this api is for clone AlphaRowsetSharedPtr alpha_rowset = std::dynamic_pointer_cast(rowset); for (auto& segment_group : alpha_rowset->_segment_groups) { + // TODO(yingchun): now that rowset is hard linked to dest, why need init and flush? RETURN_NOT_OK(_init()); RETURN_NOT_OK(segment_group->link_segments_to_path(_rowset_writer_context.rowset_path_prefix, - _rowset_writer_context.rowset_id)); + _rowset_writer_context.rowset_id)); _cur_segment_group->set_empty(segment_group->empty()); _cur_segment_group->set_num_segments(segment_group->num_segments()); _cur_segment_group->add_zone_maps(segment_group->get_zone_maps()); @@ -124,7 +125,7 @@ OLAPStatus AlphaRowsetWriter::add_rowset_for_linked_schema_change( for (auto& segment_group : alpha_rowset->_segment_groups) { RETURN_NOT_OK(_init()); RETURN_NOT_OK(segment_group->link_segments_to_path(_rowset_writer_context.rowset_path_prefix, - _rowset_writer_context.rowset_id)); + _rowset_writer_context.rowset_id)); _cur_segment_group->set_empty(segment_group->empty()); _cur_segment_group->set_num_segments(segment_group->num_segments()); _cur_segment_group->add_zone_maps_for_linked_schema_change(segment_group->get_zone_maps(), @@ -139,19 +140,19 @@ OLAPStatus AlphaRowsetWriter::flush() { if (_writer_state == WRITER_FLUSHED) { return OLAP_SUCCESS; } - DCHECK(_writer_state == WRITER_INITED); + DCHECK_EQ(_writer_state, WRITER_INITED); if (_need_column_data_writer) { // column_data_writer finalize will call segment_group->set_empty() RETURN_NOT_OK(_column_data_writer->finalize()); } - SAFE_DELETE(_column_data_writer); + SAFE_DELETE(_column_data_writer); // TODO(yingchun): why do it like this? _writer_state = WRITER_FLUSHED; return OLAP_SUCCESS; } RowsetSharedPtr AlphaRowsetWriter::build() { if (_current_rowset_meta->rowset_id().version == 0) { - LOG(WARNING) << "invalid rowset id, version == 0, rowset id=" + LOG(WARNING) << "invalid rowset id, version == 0, rowset id=" << _current_rowset_meta->rowset_id().to_string(); return nullptr; } @@ -160,6 +161,8 @@ RowsetSharedPtr AlphaRowsetWriter::build() { return nullptr; } int total_num_segments = 0; + AlphaRowsetMetaSharedPtr alpha_rowset_meta + = std::dynamic_pointer_cast(_current_rowset_meta); for (auto& segment_group : _segment_groups) { if (segment_group->load() != OLAP_SUCCESS) { return nullptr; @@ -171,6 +174,7 @@ RowsetSharedPtr AlphaRowsetWriter::build() { _current_rowset_meta->set_index_disk_size(_current_rowset_meta->index_disk_size() + segment_group->index_size()); _current_rowset_meta->set_total_disk_size(_current_rowset_meta->total_disk_size() + segment_group->index_size() + segment_group->data_size()); + // TODO(yingchun): need a function to convert SegmentGroup to SegmentGroupPB SegmentGroupPB segment_group_pb; segment_group_pb.set_segment_group_id(segment_group->segment_group_id()); segment_group_pb.set_num_segments(segment_group->num_segments()); @@ -179,13 +183,11 @@ RowsetSharedPtr AlphaRowsetWriter::build() { segment_group_pb.set_data_size(segment_group->data_size()); segment_group_pb.set_num_rows(segment_group->num_rows()); const std::vector& zone_maps = segment_group->get_zone_maps(); - if (!zone_maps.empty()) { - for (size_t i = 0; i < zone_maps.size(); ++i) { - ZoneMap* new_zone_map = segment_group_pb.add_zone_maps(); - new_zone_map->set_min(zone_maps.at(i).first->to_string()); - new_zone_map->set_max(zone_maps.at(i).second->to_string()); - new_zone_map->set_null_flag(zone_maps.at(i).first->is_null()); - } + for (const auto& zone_map : zone_maps) { + ZoneMap* new_zone_map = segment_group_pb.add_zone_maps(); + new_zone_map->set_min(zone_map.first->to_string()); + new_zone_map->set_max(zone_map.second->to_string()); + new_zone_map->set_null_flag(zone_map.first->is_null()); } if (_is_pending_rowset) { PUniqueId* unique_id = segment_group_pb.mutable_load_id(); @@ -193,8 +195,6 @@ RowsetSharedPtr AlphaRowsetWriter::build() { unique_id->set_lo(_rowset_writer_context.load_id.lo()); } segment_group_pb.set_empty(segment_group->empty()); - AlphaRowsetMetaSharedPtr alpha_rowset_meta - = std::dynamic_pointer_cast(_current_rowset_meta); alpha_rowset_meta->add_segment_group(segment_group_pb); } _current_rowset_meta->set_num_segments(total_num_segments); @@ -266,7 +266,7 @@ OLAPStatus AlphaRowsetWriter::_init() { _rowset_writer_context.version_hash, false, _segment_group_id, 0); } - DCHECK(_cur_segment_group != nullptr) << "failed to malloc SegmentGroup"; + DCHECK(_cur_segment_group) << "failed to malloc SegmentGroup"; _cur_segment_group->acquire(); //_cur_segment_group->set_load_id(_rowset_writer_context.load_id); _segment_groups.push_back(_cur_segment_group); @@ -274,7 +274,7 @@ OLAPStatus AlphaRowsetWriter::_init() { _column_data_writer = ColumnDataWriter::create(_cur_segment_group, true, _rowset_writer_context.tablet_schema->compress_kind(), _rowset_writer_context.tablet_schema->bloom_filter_fpp()); - DCHECK(_column_data_writer != nullptr) << "memory error occurs when creating writer"; + DCHECK(_column_data_writer) << "memory error occurs when creating writer"; OLAPStatus res = _column_data_writer->init(); if (res != OLAP_SUCCESS) { LOG(WARNING) << "column data writer init failed"; diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp index cc531493dd677cf..a1bdfa652fb3084 100644 --- a/be/src/olap/rowset/beta_rowset.cpp +++ b/be/src/olap/rowset/beta_rowset.cpp @@ -110,7 +110,7 @@ void BetaRowset::do_close() { _segments.clear(); } -OLAPStatus BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset_id) { +OLAPStatus BetaRowset::link_files_to(const std::string& dir, const RowsetId& new_rowset_id) { for (int i = 0; i < num_segments(); ++i) { std::string dst_link_path = segment_file_path(dir, new_rowset_id, i); // TODO(lingbin): use Env API? or EnvUtil? @@ -130,7 +130,7 @@ OLAPStatus BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset return OLAP_SUCCESS; } -OLAPStatus BetaRowset::copy_files_to(const std::string& dir) { +OLAPStatus BetaRowset::copy_files_to(const std::string& dir) const { for (int i = 0; i < num_segments(); ++i) { std::string dst_path = segment_file_path(dir, rowset_id(), i); if (FileUtils::check_exist(dst_path)) { diff --git a/be/src/olap/rowset/beta_rowset.h b/be/src/olap/rowset/beta_rowset.h index b85e8750b7609fc..1aa9223c294dc09 100644 --- a/be/src/olap/rowset/beta_rowset.h +++ b/be/src/olap/rowset/beta_rowset.h @@ -51,9 +51,9 @@ class BetaRowset : public Rowset { OLAPStatus remove() override; - OLAPStatus link_files_to(const std::string& dir, RowsetId new_rowset_id) override; + OLAPStatus link_files_to(const std::string& dir, const RowsetId& new_rowset_id) override; - OLAPStatus copy_files_to(const std::string& dir) override; + OLAPStatus copy_files_to(const std::string& dir) const override; // only applicable to alpha rowset, no op here OLAPStatus remove_old_files(std::vector* files_to_remove) override { diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp index 2f6ce767643804d..ebe1b3113a958df 100644 --- a/be/src/olap/rowset/beta_rowset_reader.cpp +++ b/be/src/olap/rowset/beta_rowset_reader.cpp @@ -107,7 +107,7 @@ OLAPStatus BetaRowsetReader::init(RowsetReaderContext* read_context) { // the output block's schema should be seek_columns to comform to v1 // TODO(hkp): this should be optimized to use return_columns output_block_info.column_ids = *(_context->seek_columns); - RETURN_NOT_OK(_output_block->init(output_block_info)); + _output_block->init(output_block_info); _row.reset(new RowCursor()); RETURN_NOT_OK(_row->init(*(read_context->tablet_schema), *(_context->seek_columns))); diff --git a/be/src/olap/rowset/column_data.cpp b/be/src/olap/rowset/column_data.cpp index b1b15cba5ae6f1c..8fc8330b0b91a0c 100644 --- a/be/src/olap/rowset/column_data.cpp +++ b/be/src/olap/rowset/column_data.cpp @@ -26,30 +26,21 @@ namespace doris { ColumnData* ColumnData::create(SegmentGroup* segment_group, const std::shared_ptr& parent_tracker) { - ColumnData* data = new (std::nothrow) ColumnData(segment_group, parent_tracker); - return data; + return new (std::nothrow) ColumnData(segment_group, parent_tracker); } ColumnData::ColumnData(SegmentGroup* segment_group, const std::shared_ptr& parent_tracker) : _segment_group(segment_group), _parent_tracker(parent_tracker), - _eof(false), - _conditions(nullptr), - _col_predicates(nullptr), - _delete_status(DEL_NOT_SATISFIED), - _runtime_state(nullptr), _schema(segment_group->get_tablet_schema()), - _is_using_cache(false), - _segment_reader(nullptr), - _lru_cache(nullptr) { + _num_rows_per_block(segment_group->get_num_rows_per_row_block()) { if (StorageEngine::instance() != nullptr) { _lru_cache = StorageEngine::instance()->index_stream_lru_cache(); } else { // for independent usage, eg: unit test/segment tool _lru_cache = FileHandler::get_fd_cache(); } - _num_rows_per_block = _segment_group->get_num_rows_per_row_block(); } ColumnData::~ColumnData() { @@ -95,6 +86,7 @@ OLAPStatus ColumnData::_next_row(const RowCursor** row, bool without_filter) { return OLAP_SUCCESS; } + // TODO(yingchun): what does DEL_NOT_SATISFIED mean? // when without_filter is true, _include_blocks is nullptr if (_read_block->block_status() == DEL_NOT_SATISFIED) { *row = &_cursor; @@ -133,8 +125,7 @@ OLAPStatus ColumnData::_seek_to_block(const RowBlockPosition& block_pos, bool wi return OLAP_ERR_DATA_EOF; } SAFE_DELETE(_segment_reader); - std::string file_name; - file_name = segment_group()->construct_data_file_path(block_pos.segment); + std::string file_name = segment_group()->construct_data_file_path(block_pos.segment); _segment_reader = new(std::nothrow) SegmentReader( file_name, segment_group(), block_pos.segment, _seek_columns, _load_bf_columns, _conditions, @@ -176,7 +167,8 @@ OLAPStatus ColumnData::_find_position_by_short_key( } return res; } - res = segment_group()->find_prev_point(tmp_pos, position); + // TODO(yingchun): why get prev? + res = _segment_group->find_prev_point(tmp_pos, position); if (res != OLAP_SUCCESS) { OLAP_LOG_WARNING("find prev row block failed. [res=%d]", res); return res; @@ -197,7 +189,7 @@ OLAPStatus ColumnData::_find_position_by_full_key( return res; } RowBlockPosition start_position; - res = segment_group()->find_prev_point(tmp_pos, &start_position); + res = _segment_group->find_prev_point(tmp_pos, &start_position); if (res != OLAP_SUCCESS) { OLAP_LOG_WARNING("find prev row block failed. [res=%d]", res); return res; @@ -223,7 +215,7 @@ OLAPStatus ColumnData::_find_position_by_full_key( OLAPIndexOffset index_offset; index_offset.segment = _end_segment; index_offset.offset = _end_block; - res = segment_group()->get_row_block_position(index_offset, &end_position); + res = _segment_group->get_row_block_position(index_offset, &end_position); if (res != OLAP_SUCCESS) { OLAP_LOG_WARNING("fail to get row block position. [res=%d]", res); return res; @@ -279,6 +271,7 @@ OLAPStatus ColumnData::_find_position_by_full_key( return OLAP_SUCCESS; } +// TODO(yingchun): find_last_key -> include_last_key? OLAPStatus ColumnData::_seek_to_row(const RowCursor& key, bool find_last_key, bool is_end_key) { RowBlockPosition position; OLAPStatus res = OLAP_SUCCESS; @@ -327,7 +320,7 @@ OLAPStatus ColumnData::_seek_to_row(const RowCursor& key, bool find_last_key, bo } else { // 找last key。返回大于这个key的第一个。也就是 // row_cursor > key - while (res == OLAP_SUCCESS && compare_row_key(*row_cursor,key) <= 0) { + while (res == OLAP_SUCCESS && compare_row_key(*row_cursor, key) <= 0) { res = _next_row(&row_cursor, without_filter); } } @@ -356,7 +349,7 @@ OLAPStatus ColumnData::prepare_block_read( const RowCursor* end_key, bool find_end_key, RowBlock** first_block) { SCOPED_RAW_TIMER(&_stats->block_fetch_ns); - set_eof(false); + _eof = false; _end_key_is_set = false; _is_normal_read = false; // set end position @@ -369,13 +362,13 @@ OLAPStatus ColumnData::prepare_block_read( _end_row_index = _read_block->pos(); _end_key_is_set = true; } else if (res != OLAP_ERR_DATA_EOF) { - LOG(WARNING) << "Find end key failed.key=" << end_key->to_string(); + LOG(WARNING) << "Find end key failed. key=" << end_key->to_string(); return res; } // res == OLAP_ERR_DATA_EOF means there is no end key, then we read to // the end of this ColumnData } - set_eof(false); + _eof = false; if (start_key != nullptr) { auto res = _seek_to_row(*start_key, !find_start_key, false); if (res == OLAP_SUCCESS) { @@ -385,7 +378,7 @@ OLAPStatus ColumnData::prepare_block_read( *first_block = nullptr; return res; } else { - LOG(WARNING) << "start_key can't be found.key=" << start_key->to_string(); + LOG(WARNING) << "start_key can't be found. key=" << start_key->to_string(); return res; } } else { diff --git a/be/src/olap/rowset/column_data.h b/be/src/olap/rowset/column_data.h index 7124dda1408a8b5..16204bc2cc87446 100644 --- a/be/src/olap/rowset/column_data.h +++ b/be/src/olap/rowset/column_data.h @@ -37,6 +37,7 @@ namespace doris { class Tablet; class SegmentReader; +// TODO(yingchun): why not rename it to ColumnDataReader? // This class is column data reader. this class will be used in two case. class ColumnData { public: @@ -102,7 +103,6 @@ class ColumnData { // 开放接口查询_eof,让外界知道数据读取是否正常终止 // 因为这个函数被频繁访问, 从性能考虑, 放在基类而不是虚函数 bool eof() { return _eof; } - void set_eof(bool eof) { _eof = eof; } bool* eof_ptr() { return &_eof; } bool empty() const { return _segment_group->empty(); } @@ -112,6 +112,7 @@ class ColumnData { int delete_pruning_filter(); uint64_t get_filted_rows(); + // TODO(yingchun): remove it SegmentGroup* segment_group() const { return _segment_group; } void set_segment_group(SegmentGroup* segment_group) { _segment_group = segment_group; } int64_t num_rows() const { return _segment_group->num_rows(); } @@ -156,22 +157,22 @@ class ColumnData { return &_cursor; } private: - SegmentGroup* _segment_group; + SegmentGroup* _segment_group = nullptr; std::shared_ptr _parent_tracker; // 当到达文件末尾或者到达end key时设置此标志 - bool _eof; - const Conditions* _conditions; - const std::vector* _col_predicates; + bool _eof = false; + const Conditions* _conditions = nullptr; + const std::vector* _col_predicates = nullptr; const DeleteHandler*_delete_handler = nullptr; - DelCondSatisfied _delete_status; - RuntimeState* _runtime_state; - OlapReaderStatistics* _stats; + DelCondSatisfied _delete_status = DEL_NOT_SATISFIED; + RuntimeState* _runtime_state = nullptr; + OlapReaderStatistics* _stats = nullptr; const TabletSchema& _schema; // whether in normal read, use return columns to load block bool _is_normal_read = false; bool _end_key_is_set = false; - bool _is_using_cache; + bool _is_using_cache = false; bool _segment_eof = false; bool _need_eval_predicates = false; @@ -179,7 +180,7 @@ class ColumnData { std::vector _seek_columns; std::set _load_bf_columns; - SegmentReader* _segment_reader; + SegmentReader* _segment_reader = nullptr; std::unique_ptr _seek_vector_batch; std::unique_ptr _read_vector_batch; @@ -198,7 +199,7 @@ class ColumnData { int64_t _end_row_index = 0; size_t _num_rows_per_block; - Cache* _lru_cache; + Cache* _lru_cache = nullptr; }; class ColumnDataComparator { diff --git a/be/src/olap/rowset/column_data_writer.cpp b/be/src/olap/rowset/column_data_writer.cpp index 2a501e452fe7781..08509259afae27d 100644 --- a/be/src/olap/rowset/column_data_writer.cpp +++ b/be/src/olap/rowset/column_data_writer.cpp @@ -42,8 +42,8 @@ ColumnDataWriter::ColumnDataWriter(SegmentGroup* segment_group, _bloom_filter_fpp(bloom_filter_fpp), _zone_maps(segment_group->get_num_zone_map_columns(), KeyRange(NULL, NULL)), _row_index(0), - _row_block(NULL), - _segment_writer(NULL), + _row_block(nullptr), + _segment_writer(nullptr), _num_rows(0), _block_id(0), _max_segment_size(OLAP_MAX_COLUMN_SEGMENT_FILE_SIZE), @@ -63,7 +63,6 @@ ColumnDataWriter::~ColumnDataWriter() { OLAPStatus ColumnDataWriter::init() { OLAPStatus res = OLAP_SUCCESS; - for (size_t i = 0; i < _zone_maps.size(); ++i) { _zone_maps[i].first = WrapperField::create(_segment_group->get_tablet_schema().column(i)); DCHECK(_zone_maps[i].first != nullptr) << "fail to create column statistics field."; @@ -80,8 +79,7 @@ OLAPStatus ColumnDataWriter::init() { _max_segment_size = static_cast(lround(size)); _row_block = new(std::nothrow) RowBlock(&(_segment_group->get_tablet_schema())); - - if (NULL == _row_block) { + if (nullptr == _row_block) { LOG(WARNING) << "fail to new RowBlock."; return OLAP_ERR_MALLOC_ERROR; } @@ -97,11 +95,7 @@ OLAPStatus ColumnDataWriter::init() { RowBlockInfo block_info(0U, _segment_group->get_num_rows_per_row_block()); block_info.null_supported = true; - res = _row_block->init(block_info); - if (OLAP_SUCCESS != res) { - LOG(WARNING) << "fail to initiate row block. [res=" << res << "]"; - return res; - } + _row_block->init(block_info); return OLAP_SUCCESS; } @@ -190,11 +184,7 @@ OLAPStatus ColumnDataWriter::finalize() { return res; } - res = _segment_group->add_zone_maps(_zone_maps); - if (res != OLAP_SUCCESS) { - LOG(WARNING) << "Fail to set zone_map! res=" << res; - return res; - } + _segment_group->add_zone_maps(_zone_maps); return OLAP_SUCCESS; } @@ -236,18 +226,15 @@ OLAPStatus ColumnDataWriter::_flush_row_block(bool finalize) { } OLAPStatus ColumnDataWriter::_add_segment() { - std::string file_name; - - if (NULL != _segment_writer) { + if (nullptr != _segment_writer) { OLAP_LOG_WARNING("previous segment is not finalized before add new segment."); return OLAP_ERR_WRITER_SEGMENT_NOT_FINALIZED; } - file_name = _segment_group->construct_data_file_path(_segment); + std::string file_name = _segment_group->construct_data_file_path(_segment); _segment_writer = new(std::nothrow) SegmentWriter(file_name, _segment_group, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, _compress_kind, _bloom_filter_fpp); - - if (NULL == _segment_writer) { + if (nullptr == _segment_writer) { OLAP_LOG_WARNING("fail to allocate SegmentWriter"); return OLAP_ERR_MALLOC_ERROR; } @@ -256,8 +243,7 @@ OLAPStatus ColumnDataWriter::_add_segment() { if (_is_push_write) { res = _segment_writer->init(config::push_write_mbytes_per_sec); } else { - res = _segment_writer->init( - config::base_compaction_write_mbytes_per_sec); + res = _segment_writer->init(config::base_compaction_write_mbytes_per_sec); } if (OLAP_SUCCESS != res) { @@ -306,8 +292,7 @@ OLAPStatus ColumnDataWriter::_finalize_segment() { } uint64_t ColumnDataWriter::written_bytes() { - uint64_t size = _segment * _max_segment_size + _segment_writer->estimate_segment_size(); - return size; + return _segment * _max_segment_size + _segment_writer->estimate_segment_size(); } MemPool* ColumnDataWriter::mem_pool() { diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h index aac920c2f447c17..cb4d3bbb1174420 100644 --- a/be/src/olap/rowset/rowset.h +++ b/be/src/olap/rowset/rowset.h @@ -178,11 +178,11 @@ class Rowset : public std::enable_shared_from_this { OLAPStatus st = OLAP_SUCCESS; { std::lock_guard close_lock(_lock); - uint64_t current_refs = _refs_by_reader; old_state = _rowset_state_machine.rowset_state(); if (old_state != ROWSET_LOADED) { return; } + uint64_t current_refs = _refs_by_reader; if (current_refs == 0) { do_close(); } @@ -198,11 +198,11 @@ class Rowset : public std::enable_shared_from_this { << ", tabletid:" << _rowset_meta->tablet_id(); } - // hard link all files in this rowset to `dir` to form a new rowset with id `new_rowset_id`. - virtual OLAPStatus link_files_to(const std::string& dir, RowsetId new_rowset_id) = 0; + // hard link all files in this rowset with id `new_rowset_id` to `dir` to form a new rowset. + virtual OLAPStatus link_files_to(const std::string& dir, const RowsetId& new_rowset_id) = 0; // copy all files to `dir` - virtual OLAPStatus copy_files_to(const std::string& dir) = 0; + virtual OLAPStatus copy_files_to(const std::string& dir) const = 0; virtual OLAPStatus remove_old_files(std::vector* files_to_remove) = 0; @@ -236,7 +236,7 @@ class Rowset : public std::enable_shared_from_this { } void release() { - // if the refs by reader is 0 and the rowset is closed, should release the resouce + // if the refs by reader is 0 and the rowset is closed, should release the resource uint64_t current_refs = --_refs_by_reader; if (current_refs == 0 && _rowset_state_machine.rowset_state() == ROWSET_UNLOADING) { { diff --git a/be/src/olap/rowset/rowset_factory.h b/be/src/olap/rowset/rowset_factory.h index b3258f1cee98c5d..4fa22769d6de7fd 100644 --- a/be/src/olap/rowset/rowset_factory.h +++ b/be/src/olap/rowset/rowset_factory.h @@ -33,9 +33,9 @@ class RowsetFactory { // return OLAP_SUCCESS and set inited rowset in `*rowset`. // return others if failed to create or init rowset. static OLAPStatus create_rowset(const TabletSchema* schema, - const std::string& rowset_path, - RowsetMetaSharedPtr rowset_meta, - RowsetSharedPtr* rowset); + const std::string& rowset_path, + RowsetMetaSharedPtr rowset_meta, + RowsetSharedPtr* rowset); // create and init rowset writer. // return OLAP_SUCCESS and set `*output` to inited rowset writer. diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h index 882f46609f28cc0..56435a0dc4daad3 100644 --- a/be/src/olap/rowset/rowset_meta.h +++ b/be/src/olap/rowset/rowset_meta.h @@ -70,8 +70,7 @@ class RowsetMeta { virtual bool json_rowset_meta(std::string* json_rowset_meta) { json2pb::Pb2JsonOptions json_options; json_options.pretty_json = true; - bool ret = json2pb::ProtoMessageToJson(_rowset_meta_pb, json_rowset_meta, json_options); - return ret; + return json2pb::ProtoMessageToJson(_rowset_meta_pb, json_rowset_meta, json_options); } RowsetId rowset_id() const { @@ -385,7 +384,7 @@ class RowsetMeta { // ATTN(cmy): the num segments should be read from rowset meta pb. // But the previous code error caused this value not to be set in some cases. // So when init the rowset meta and find that the num_segments is 0(not set), - // we will try to calculate the num segmengts from AlphaRowsetExtraMetaPB, + // we will try to calculate the num segments from AlphaRowsetExtraMetaPB, // and then set the num_segments field. // This should only happen in some rowsets converted from old version. // and for all newly created rowsets, the num_segments field must be set. diff --git a/be/src/olap/rowset/rowset_meta_manager.cpp b/be/src/olap/rowset/rowset_meta_manager.cpp index 53840a2df12c702..ce4aa04767dd014 100644 --- a/be/src/olap/rowset/rowset_meta_manager.cpp +++ b/be/src/olap/rowset/rowset_meta_manager.cpp @@ -34,6 +34,7 @@ namespace doris { const std::string ROWSET_PREFIX = "rst_"; +// TODO(yingchun): key的生成和解析封装成函数 bool RowsetMetaManager::check_rowset_meta(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id) { std::string key = ROWSET_PREFIX + tablet_uid.to_string() + "_" + rowset_id.to_string(); std::string value; @@ -127,7 +128,7 @@ OLAPStatus RowsetMetaManager::load_json_rowset_meta(OlapMeta* meta, const std::s std::string json_rowset_meta; while (!infile.eof()) { infile.getline(buffer, 1024); - json_rowset_meta = json_rowset_meta + buffer; + json_rowset_meta.append(buffer); } boost::algorithm::trim(json_rowset_meta); RowsetMeta rowset_meta; diff --git a/be/src/olap/rowset/rowset_writer.h b/be/src/olap/rowset/rowset_writer.h index e89d217d04dea76..2323463edbd2127 100644 --- a/be/src/olap/rowset/rowset_writer.h +++ b/be/src/olap/rowset/rowset_writer.h @@ -26,7 +26,7 @@ namespace doris { -class ContiguousRow; +struct ContiguousRow; class RowCursor; class RowsetWriter { diff --git a/be/src/olap/rowset/segment_group.cpp b/be/src/olap/rowset/segment_group.cpp index 11f9abffb961389..9234a0e0caf438d 100644 --- a/be/src/olap/rowset/segment_group.cpp +++ b/be/src/olap/rowset/segment_group.cpp @@ -89,6 +89,7 @@ SegmentGroup::SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const T _new_segment_created = false; _empty = false; + _short_key_columns.reserve(_schema->num_short_key_columns()); for (size_t i = 0; i < _schema->num_short_key_columns(); ++i) { const TabletColumn& column = _schema->column(i); _short_key_columns.push_back(column); @@ -121,10 +122,11 @@ SegmentGroup::SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const T _ref_count = 0; _short_key_length = 0; _new_short_key_length = 0; - _short_key_buf = NULL; + _short_key_buf = nullptr; _new_segment_created = false; _empty = false; + _short_key_columns.reserve(_schema->num_short_key_columns()); for (size_t i = 0; i < _schema->num_short_key_columns(); ++i) { const TabletColumn& column = _schema->column(i); _short_key_columns.push_back(column); @@ -217,6 +219,7 @@ bool SegmentGroup::delete_all_files() { string data_path = construct_data_file_path(seg_id); VLOG(3) << "delete index file. path=" << index_path; + // TODO(yingchun): unify to use Env? if (remove(index_path.c_str()) != 0) { // if the errno is not ENOENT, log the error msg. // ENOENT stands for 'No such file or directory' @@ -241,12 +244,12 @@ bool SegmentGroup::delete_all_files() { return success; } -OLAPStatus SegmentGroup::add_zone_maps_for_linked_schema_change( +void SegmentGroup::add_zone_maps_for_linked_schema_change( const std::vector>& zone_map_fields, const SchemaMapping& schema_mapping) { //When add rollup tablet, the base tablet index maybe empty - if (zone_map_fields.size() == 0) { - return OLAP_SUCCESS; + if (zone_map_fields.empty()) { + return; } // 1. rollup tablet get_num_zone_map_columns() will less than base tablet zone_map_fields.size(). @@ -306,11 +309,9 @@ OLAPStatus SegmentGroup::add_zone_maps_for_linked_schema_change( // can be nullptr, and it is checked in olap_cond.cpp eval function. _zone_maps.push_back(std::make_pair(first, second)); } - - return OLAP_SUCCESS; } -OLAPStatus SegmentGroup::add_zone_maps( +void SegmentGroup::add_zone_maps( const std::vector>& zone_map_fields) { DCHECK(_empty || zone_map_fields.size() == get_num_zone_map_columns()); for (size_t i = 0; i < zone_map_fields.size(); ++i) { @@ -325,7 +326,6 @@ OLAPStatus SegmentGroup::add_zone_maps( _zone_maps.push_back(std::make_pair(first, second)); } - return OLAP_SUCCESS; } OLAPStatus SegmentGroup::add_zone_maps( @@ -341,9 +341,11 @@ OLAPStatus SegmentGroup::add_zone_maps( //[min, max] -> [NULL, max] first->set_null(); } + WrapperField* second = WrapperField::create(column); DCHECK(first != NULL) << "failed to allocate memory for field: " << i ; RETURN_NOT_OK(second->from_string(zone_map_strings[i].second)); + _zone_maps.push_back(std::make_pair(first, second)); } return OLAP_SUCCESS; @@ -366,27 +368,22 @@ OLAPStatus SegmentGroup::load(bool use_cache) { return res; } - if (_index.init(_short_key_length, _new_short_key_length, - _schema->num_short_key_columns(), &_short_key_columns) != OLAP_SUCCESS) { - LOG(WARNING) << "fail to create MemIndex. num_segment=" << _num_segments; - return res; - } + _index.init(_short_key_length, _new_short_key_length, + _schema->num_short_key_columns(), &_short_key_columns); // for each segment for (uint32_t seg_id = 0; seg_id < _num_segments; ++seg_id) { + // get full path for one segment string seg_path = construct_data_file_path(seg_id); if (OLAP_SUCCESS != (res = load_pb(seg_path.c_str(), seg_id))) { LOG(WARNING) << "failed to load pb structures. [seg_path='" << seg_path << "']"; - return res; } - - // get full path for one segment + std::string path = construct_index_file_path(seg_id); - if ((res = _index.load_segment(path.c_str(), &_current_num_rows_per_row_block, use_cache)) + if ((res = _index.load_segment(path, &_current_num_rows_per_row_block, use_cache)) != OLAP_SUCCESS) { LOG(WARNING) << "fail to load segment. [path='" << path << "']"; - return res; } } @@ -397,7 +394,7 @@ OLAPStatus SegmentGroup::load(bool use_cache) { return OLAP_SUCCESS; } -OLAPStatus SegmentGroup::load_pb(const char* file, uint32_t seg_id) { +OLAPStatus SegmentGroup::load_pb(const string& file, uint32_t seg_id) { OLAPStatus res = OLAP_SUCCESS; FileHeader seg_file_header; @@ -465,9 +462,9 @@ bool SegmentGroup::check() { } OLAPStatus SegmentGroup::find_short_key(const RowCursor& key, - RowCursor* helper_cursor, - bool find_last, - RowBlockPosition* pos) const { + RowCursor* helper_cursor, + bool find_last, + RowBlockPosition* pos) const { SEGMENT_GROUP_PARAM_VALIDATE(); POS_PARAM_VALIDATE(pos); @@ -526,9 +523,9 @@ OLAPStatus SegmentGroup::find_next_row_block(RowBlockPosition* pos, bool* eof) c } OLAPStatus SegmentGroup::find_mid_point(const RowBlockPosition& low, - const RowBlockPosition& high, - RowBlockPosition* output, - uint32_t* dis) const { + const RowBlockPosition& high, + RowBlockPosition* output, + uint32_t* dis) const { *dis = compute_distance(low, high); if (*dis >= _index.count()) { return OLAP_ERR_INDEX_EOF; @@ -565,7 +562,7 @@ OLAPStatus SegmentGroup::advance_row_block(int64_t num_row_blocks, RowBlockPosit // PRECONDITION position1 < position2 uint32_t SegmentGroup::compute_distance(const RowBlockPosition& position1, - const RowBlockPosition& position2) const { + const RowBlockPosition& position2) const { iterator_offset_t offset1 = _index.get_absolute_offset(_index.get_offset(position1)); iterator_offset_t offset2 = _index.get_absolute_offset(_index.get_offset(position2)); @@ -576,9 +573,8 @@ OLAPStatus SegmentGroup::add_segment() { // 打开文件 ++_num_segments; - OLAPIndexHeaderMessage* index_header = NULL; // 构造Proto格式的Header - index_header = _file_header.mutable_message(); + OLAPIndexHeaderMessage* index_header = _file_header.mutable_message(); index_header->set_start_version(_version.first); index_header->set_end_version(_version.second); index_header->set_cumulative_version_hash(_version_hash); @@ -588,9 +584,9 @@ OLAPStatus SegmentGroup::add_segment() { index_header->set_null_supported(true); // 分配一段存储short key的内存, 初始化index_row - if (_short_key_buf == NULL) { + if (_short_key_buf == nullptr) { _short_key_buf = new(std::nothrow) char[_short_key_length]; - if (_short_key_buf == NULL) { + if (_short_key_buf == nullptr) { OLAP_LOG_WARNING("malloc short_key_buf error."); return OLAP_ERR_MALLOC_ERROR; } @@ -618,6 +614,7 @@ OLAPStatus SegmentGroup::add_short_key(const RowCursor& short_key, const uint32_ if (!_new_segment_created) { string file_path = construct_index_file_path(_num_segments - 1); StorageEngine* engine = StorageEngine::instance(); + // TODO(yingchun): should use DCHECK? if (engine != nullptr) { boost::filesystem::path tablet_path(_rowset_path_prefix); boost::filesystem::path data_dir_path = tablet_path.parent_path().parent_path().parent_path().parent_path(); @@ -628,6 +625,7 @@ OLAPStatus SegmentGroup::add_short_key(const RowCursor& short_key, const uint32_ res = _current_file_handler.open_with_mode( file_path.c_str(), O_CREAT | O_EXCL | O_WRONLY, S_IRUSR | S_IWUSR); if (res != OLAP_SUCCESS) { + // TODO(yingchun): remove_pending_ids ? char errmsg[64]; LOG(WARNING) << "can not create file. file_path=" << file_path << ", err='" << strerror_r(errno, errmsg, 64); @@ -637,12 +635,14 @@ OLAPStatus SegmentGroup::add_short_key(const RowCursor& short_key, const uint32_ // 准备FileHeader if ((res = _file_header.prepare(&_current_file_handler)) != OLAP_SUCCESS) { + // TODO(yingchun): remove_pending_ids ? OLAP_LOG_WARNING("write file header error. [err=%m]"); return res; } // 跳过FileHeader if (_current_file_handler.seek(_file_header.size(), SEEK_SET) == -1) { + // TODO(yingchun): remove_pending_ids ? OLAP_LOG_WARNING("lseek header file error. [err=%m]"); res = OLAP_ERR_IO_ERROR; return res; @@ -662,7 +662,6 @@ OLAPStatus SegmentGroup::add_short_key(const RowCursor& short_key, const uint32_ // 写入Short Key对应的数据 if ((res = _current_file_handler.write(_short_key_buf, _short_key_length)) != OLAP_SUCCESS) { OLAP_LOG_WARNING("write short key failed. [err=%m]"); - return res; } @@ -698,7 +697,6 @@ OLAPStatus SegmentGroup::finalize_segment(uint32_t data_segment_size, int64_t nu // 写入更新之后的FileHeader if ((res = _file_header.serialize(&_current_file_handler)) != OLAP_SUCCESS) { OLAP_LOG_WARNING("write file header error. [err=%m]"); - return res; } @@ -707,7 +705,6 @@ OLAPStatus SegmentGroup::finalize_segment(uint32_t data_segment_size, int64_t nu if ((res = _current_file_handler.close()) != OLAP_SUCCESS) { OLAP_LOG_WARNING("close file error. [err=%m]"); - return res; } @@ -754,7 +751,7 @@ int64_t SegmentGroup::get_tablet_id() { return _tablet_id; } -OLAPStatus SegmentGroup::copy_files_to(const std::string& dir) { +OLAPStatus SegmentGroup::copy_files_to(const std::string& dir) const { if (_empty) { return OLAP_SUCCESS; } @@ -792,7 +789,7 @@ OLAPStatus SegmentGroup::copy_files_to(const std::string& dir) { // case 1: clone from old version be // case 2: upgrade to new version be OLAPStatus SegmentGroup::convert_from_old_files(const std::string& snapshot_path, - std::vector* success_links) { + std::vector* success_links) { if (_empty) { // the segment group is empty, it does not have files, just return return OLAP_SUCCESS; @@ -839,7 +836,7 @@ OLAPStatus SegmentGroup::convert_from_old_files(const std::string& snapshot_path } OLAPStatus SegmentGroup::convert_to_old_files(const std::string& snapshot_path, - std::vector* success_links) { + std::vector* success_links) { if (_empty) { return OLAP_SUCCESS; } diff --git a/be/src/olap/rowset/segment_group.h b/be/src/olap/rowset/segment_group.h index 7f79a48981c0796..1e6f680841746c6 100644 --- a/be/src/olap/rowset/segment_group.h +++ b/be/src/olap/rowset/segment_group.h @@ -61,17 +61,17 @@ class SegmentGroup { // Load the index into memory. OLAPStatus load(bool use_cache = true); bool index_loaded(); - OLAPStatus load_pb(const char* file, uint32_t seg_id); + OLAPStatus load_pb(const string& file, uint32_t seg_id); bool has_zone_maps() { - return _zone_maps.size() != 0; + return !_zone_maps.empty(); } - OLAPStatus add_zone_maps_for_linked_schema_change( + void add_zone_maps_for_linked_schema_change( const std::vector>& zone_map_fields, const SchemaMapping& schema_mapping); - OLAPStatus add_zone_maps( + void add_zone_maps( const std::vector>& zone_map_fields); OLAPStatus add_zone_maps( @@ -91,9 +91,9 @@ class SegmentGroup { // Finds position of first row block contain the smallest key equal // to or greater than 'key'. Returns true on success. OLAPStatus find_short_key(const RowCursor& key, - RowCursor* helper_cursor, - bool find_last, - RowBlockPosition* position) const; + RowCursor* helper_cursor, + bool find_last, + RowBlockPosition* position) const; // Returns position of the first row block in the index. OLAPStatus find_first_row_block(RowBlockPosition* position) const; @@ -110,9 +110,9 @@ class SegmentGroup { // the midpoint between those two positions. Returns the distance // between low and high as computed by ComputeDistance. OLAPStatus find_mid_point(const RowBlockPosition& low, - const RowBlockPosition& high, - RowBlockPosition* output, - uint32_t* dis) const; + const RowBlockPosition& high, + RowBlockPosition* output, + uint32_t* dis) const; OLAPStatus find_prev_point(const RowBlockPosition& current, RowBlockPosition* prev) const; @@ -222,10 +222,10 @@ class SegmentGroup { } std::string construct_index_file_path(const std::string& snapshot_path, - int32_t segment_id) const; + int32_t segment_id) const; std::string construct_index_file_path(int32_t segment_id) const; std::string construct_data_file_path(const std::string& snapshot_path, - int32_t segment_id) const; + int32_t segment_id) const; std::string construct_data_file_path(int32_t segment_id) const; // these two functions are for compatible, and will be deleted later @@ -256,14 +256,14 @@ class SegmentGroup { } OLAPStatus convert_from_old_files(const std::string& snapshot_path, - std::vector* success_links); + std::vector* success_links); OLAPStatus convert_to_old_files(const std::string& snapshot_path, - std::vector* success_links); + std::vector* success_links); OLAPStatus remove_old_files(std::vector* linkes_to_remove); - OLAPStatus copy_files_to(const std::string& dir); + OLAPStatus copy_files_to(const std::string& dir) const; OLAPStatus link_segments_to_path(const std::string& dest_path, const RowsetId& rowset_id); @@ -301,7 +301,7 @@ class SegmentGroup { TTransactionId _txn_id; // short key对应的column information - std::vector _short_key_columns; + std::vector _short_key_columns; // TODO(yingchun): duplicate, can use _schema instead // short key对应的总长度 size_t _short_key_length; size_t _new_short_key_length; diff --git a/be/src/olap/schema.h b/be/src/olap/schema.h index 63b858c35cbab6e..a766e8cd782a342 100644 --- a/be/src/olap/schema.h +++ b/be/src/olap/schema.h @@ -115,7 +115,7 @@ class Schema { return _col_offsets[cid]; } - // TODO(lingbin): What is the difference between colun_size() and index_size() + // TODO(lingbin): What is the difference between column_size() and index_size() size_t column_size(ColumnId cid) const { return _cols[cid]->size(); } diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index 81ace25c7808cb0..91d0db6f3f2a384 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -769,18 +769,14 @@ OLAPStatus RowBlockAllocator::allocate(RowBlock** row_block, size_t num_rows, bo RowBlockInfo row_block_info(0U, num_rows); row_block_info.null_supported = null_supported; - OLAPStatus res = OLAP_SUCCESS; - - if ((res = (*row_block)->init(row_block_info)) != OLAP_SUCCESS) { - LOG(WARNING) << "failed to init row block."; - SAFE_DELETE(*row_block); - return res; - } + (*row_block)->init(row_block_info); _memory_allocated += row_block_size; - VLOG(3) << "RowBlockAllocator::allocate() this=" << this << ", num_rows=" << num_rows - << ", m_memory_allocated=" << _memory_allocated << ", row_block_addr=" << *row_block; - return res; + VLOG(3) << "RowBlockAllocator::allocate() this=" << this + << ", num_rows=" << num_rows + << ", m_memory_allocated=" << _memory_allocated + << ", row_block_addr=" << *row_block; + return OLAP_SUCCESS; } void RowBlockAllocator::release(RowBlock* row_block) { @@ -1764,11 +1760,8 @@ OLAPStatus SchemaChangeHandler::_get_versions_to_be_changed( } vector span_versions; - RETURN_NOT_OK(base_tablet->capture_consistent_versions(Version(0, rowset->version().second), - &span_versions)); - for (uint32_t i = 0; i < span_versions.size(); i++) { - versions_to_be_changed->push_back(span_versions[i]); - } + 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(), span_versions.begin(), span_versions.end()); return OLAP_SUCCESS; } @@ -1794,6 +1787,7 @@ OLAPStatus SchemaChangeHandler::_add_alter_task(AlterTabletType alter_tablet_typ vector(), // empty versions alter_tablet_type); new_tablet->save_meta(); + LOG(INFO) << "successfully add alter task to both base and new"; return OLAP_SUCCESS; } @@ -1815,6 +1809,7 @@ OLAPStatus SchemaChangeHandler::_save_alter_state(AlterTabletState state, return res; } base_tablet->save_meta(); + AlterTabletTaskSharedPtr new_alter_task = new_tablet->alter_task(); if (new_alter_task == nullptr) { LOG(INFO) << "could not find alter task info from new tablet " << new_tablet->full_name(); @@ -1941,6 +1936,7 @@ OLAPStatus SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangePa sc_params.new_tablet->release_push_lock(); goto PROCESS_ALTER_EXIT; } + // TODO(yingchun): Can we add rs by batch ? res = sc_params.new_tablet->add_rowset(new_rowset, false); if (res == OLAP_ERR_PUSH_VERSION_ALREADY_EXIST) { LOG(WARNING) << "version already exist, version revert occured. " diff --git a/be/src/olap/schema_change.h b/be/src/olap/schema_change.h index 6ce83dd14e860a3..ff6686e061adef5 100644 --- a/be/src/olap/schema_change.h +++ b/be/src/olap/schema_change.h @@ -179,12 +179,11 @@ class SchemaChangeHandler { SchemaChangeHandler() {} virtual ~SchemaChangeHandler() {} - OLAPStatus process_alter_tablet(AlterTabletType alter_tablet_type, - const TAlterTabletReq& request); - - OLAPStatus schema_version_convert(TabletSharedPtr base_tablet, TabletSharedPtr new_tablet, - RowsetSharedPtr* base_rowset, RowsetSharedPtr* new_rowset); - + OLAPStatus schema_version_convert(TabletSharedPtr base_tablet, + TabletSharedPtr new_tablet, + RowsetSharedPtr* base_rowset, + RowsetSharedPtr* new_rowset); + // schema change v2, it will not set alter task in base tablet OLAPStatus process_alter_tablet_v2(const TAlterTabletReqV2& request); diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp index 7d5f74ee3ed350c..dae6f1fb3daf985 100644 --- a/be/src/olap/snapshot_manager.cpp +++ b/be/src/olap/snapshot_manager.cpp @@ -55,27 +55,17 @@ using std::list; namespace doris { -SnapshotManager* SnapshotManager::_s_instance = nullptr; -std::mutex SnapshotManager::_mlock; - SnapshotManager* SnapshotManager::instance() { - if (_s_instance == nullptr) { - std::lock_guard lock(_mlock); - if (_s_instance == nullptr) { - _s_instance = new SnapshotManager(); - } - } - return _s_instance; + static SnapshotManager _s_instance; + // TODO(yingchun): or we can return reference object + return &_s_instance; } OLAPStatus SnapshotManager::make_snapshot( const TSnapshotRequest& request, string* snapshot_path) { OLAPStatus res = OLAP_SUCCESS; - if (snapshot_path == nullptr) { - LOG(WARNING) << "output parameter cannot be NULL"; - return OLAP_ERR_INPUT_PARAMETER_ERROR; - } + DCHECK(snapshot_path == nullptr) << "snapshot_path cannot be nullptr"; TabletSharedPtr ref_tablet = StorageEngine::instance()->tablet_manager()->get_tablet(request.tablet_id, request.schema_hash); if (ref_tablet == nullptr) { @@ -98,14 +88,14 @@ OLAPStatus SnapshotManager::make_snapshot( } LOG(INFO) << "success to make snapshot. path=['" << *snapshot_path << "']"; - return res; + return OLAP_SUCCESS; } OLAPStatus SnapshotManager::release_snapshot(const string& snapshot_path) { // 如果请求的snapshot_path位于root/snapshot文件夹下,则认为是合法的,可以删除 // 否则认为是非法请求,返回错误结果 auto stores = StorageEngine::instance()->get_stores(); - for (auto store : stores) { + for (const auto& store : stores) { std::string abs_path; RETURN_WITH_WARN_IF_ERROR(FileUtils::canonicalize(store->path(), &abs_path), OLAP_ERR_DIR_NOT_EXIST, @@ -148,8 +138,7 @@ OLAPStatus SnapshotManager::convert_rowset_ids(const string& clone_dir, int64_t TabletMetaPB cloned_tablet_meta_pb; cloned_tablet_meta.to_meta_pb(&cloned_tablet_meta_pb); - TabletMetaPB new_tablet_meta_pb; - new_tablet_meta_pb = cloned_tablet_meta_pb; + TabletMetaPB new_tablet_meta_pb(cloned_tablet_meta_pb); new_tablet_meta_pb.clear_rs_metas(); new_tablet_meta_pb.clear_inc_rs_metas(); // should modify tablet id and schema hash because in restore process the tablet id is not @@ -159,21 +148,21 @@ OLAPStatus SnapshotManager::convert_rowset_ids(const string& clone_dir, int64_t TabletSchema tablet_schema; tablet_schema.init_from_pb(new_tablet_meta_pb.schema()); - std::unordered_map _rs_version_map; + std::unordered_map rs_version_map; for (auto& visible_rowset : cloned_tablet_meta_pb.rs_metas()) { RowsetMetaPB* rowset_meta = new_tablet_meta_pb.add_rs_metas(); - RowsetId rowset_id = StorageEngine::instance()->next_rowset_id(); + RowsetId rowset_id = StorageEngine::instance()->next_rowset_id(); // generate a new rowset id RETURN_NOT_OK(_rename_rowset_id(visible_rowset, clone_dir, tablet_schema, rowset_id, rowset_meta)); rowset_meta->set_tablet_id(tablet_id); rowset_meta->set_tablet_schema_hash(schema_hash); Version rowset_version = {visible_rowset.start_version(), visible_rowset.end_version()}; - _rs_version_map[rowset_version] = rowset_meta; + rs_version_map[rowset_version] = rowset_meta; } for (auto& inc_rowset : cloned_tablet_meta_pb.inc_rs_metas()) { Version rowset_version = {inc_rowset.start_version(), inc_rowset.end_version()}; - auto exist_rs = _rs_version_map.find(rowset_version); - if (exist_rs != _rs_version_map.end()) { + auto exist_rs = rs_version_map.find(rowset_version); + if (exist_rs != rs_version_map.end()) { RowsetMetaPB* rowset_meta = new_tablet_meta_pb.add_inc_rs_metas(); *rowset_meta = *(exist_rs->second); continue; @@ -244,28 +233,22 @@ OLAPStatus SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb, co // get snapshot path: curtime.seq.timeout // eg: 20190819221234.3.86400 -OLAPStatus SnapshotManager::_calc_snapshot_id_path( +OLAPStatus SnapshotManager::_gen_snapshot_id_path( const TabletSharedPtr& tablet, int64_t timeout_s, string* out_path) { - OLAPStatus res = OLAP_SUCCESS; - if (out_path == nullptr) { - LOG(WARNING) << "output parameter cannot be NULL"; - return OLAP_ERR_INPUT_PARAMETER_ERROR; - } + DCHECK(out_path != nullptr); - // get current timestamp string + OLAPStatus res = OLAP_SUCCESS; string time_str; if ((res = gen_timestamp_string(&time_str)) != OLAP_SUCCESS) { - LOG(WARNING) << "failed to generate time_string when move file to trash." - << "err code=" << res; + LOG(WARNING) << "failed to gen_timestamp_string. err code=" << res; return res; } 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++ + << "/" << time_str << "." << ++_snapshot_base_id << "." << timeout_s; *out_path = snapshot_id_path_stream.str(); return res; @@ -278,9 +261,7 @@ string SnapshotManager::get_schema_hash_full_path( schema_full_path_stream << location << "/" << ref_tablet->tablet_id() << "/" << ref_tablet->schema_hash(); - string schema_full_path = schema_full_path_stream.str(); - - return schema_full_path; + return schema_full_path_stream.str(); } string SnapshotManager::_get_header_full_path( @@ -307,31 +288,24 @@ OLAPStatus SnapshotManager::_create_snapshot_files( const TSnapshotRequest& request, string* snapshot_path, int32_t snapshot_version) { + DCHECK(snapshot_path == nullptr); LOG(INFO) << "receive a make snapshot request," << " request detail is " << apache::thrift::ThriftDebugString(request) << " snapshot_path is " << *snapshot_path << " snapshot_version is " << snapshot_version; - OLAPStatus res = OLAP_SUCCESS; - if (snapshot_path == nullptr) { - LOG(WARNING) << "output parameter cannot be NULL"; - return OLAP_ERR_INPUT_PARAMETER_ERROR; - } string snapshot_id_path; - int64_t timeout_s = config::snapshot_expire_time_sec; - if (request.__isset.timeout) { - timeout_s = request.timeout; - } - res = _calc_snapshot_id_path(ref_tablet, timeout_s, &snapshot_id_path); + int64_t timeout_s = request.__isset.timeout ? request.timeout : config::snapshot_expire_time_sec; + + OLAPStatus res = _gen_snapshot_id_path(ref_tablet, timeout_s, &snapshot_id_path); if (res != OLAP_SUCCESS) { - LOG(WARNING) << "failed to calc snapshot_id_path, ref tablet=" + LOG(WARNING) << "failed to gen snapshot_id_path, tablet=" << ref_tablet->data_dir()->path(); return res; } - string schema_full_path = get_schema_hash_full_path( - ref_tablet, snapshot_id_path); + string schema_full_path = get_schema_hash_full_path(ref_tablet, snapshot_id_path); string header_path = _get_header_full_path(ref_tablet, schema_full_path); if (FileUtils::check_exist(schema_full_path)) { VLOG(10) << "remove the old schema_full_path."; @@ -347,6 +321,7 @@ OLAPStatus SnapshotManager::_create_snapshot_files( do { TabletMetaSharedPtr new_tablet_meta(new (nothrow) TabletMeta()); + // TODO(yingchun): do we need such a protection? if (new_tablet_meta == nullptr) { LOG(WARNING) << "fail to malloc TabletMeta."; res = OLAP_ERR_MALLOC_ERROR; @@ -364,7 +339,7 @@ OLAPStatus SnapshotManager::_create_snapshot_files( LOG(WARNING) << "failed to find missed version when snapshot. " << " tablet=" << request.tablet_id << " schema_hash=" << request.schema_hash - << " version=" << version.first << "-" << version.second; + << " version=" << version; res = OLAP_ERR_VERSION_NOT_EXIST; break; } @@ -508,7 +483,6 @@ 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) { - OLAPStatus res = OLAP_SUCCESS; RowsetConverter rowset_converter(new_tablet_meta); std::vector new_rowset_metas; bool modified = false; @@ -516,35 +490,34 @@ OLAPStatus SnapshotManager::_convert_beta_rowsets_to_alpha(const TabletMetaShare if (rowset_meta->rowset_type() == BETA_ROWSET) { modified = true; RowsetMetaPB rowset_meta_pb; - auto st = rowset_converter.convert_beta_to_alpha(rowset_meta, dst_path, &rowset_meta_pb); - if (st != OLAP_SUCCESS) { - res = st; + OLAPStatus res = rowset_converter.convert_beta_to_alpha(rowset_meta, dst_path, &rowset_meta_pb); + if (res != OLAP_SUCCESS) { LOG(WARNING) << "convert beta to alpha failed" << ", tablet_id:" << new_tablet_meta->tablet_id() << ", schema hash:" << new_tablet_meta->schema_hash() << ", src rowset:" << rowset_meta->rowset_id() - << ", error:" << st; - break; + << ", error:" << res; + return res; } RowsetMetaSharedPtr new_rowset_meta(new AlphaRowsetMeta()); - bool ret = new_rowset_meta->init_from_pb(rowset_meta_pb); - if (!ret) { - res = OLAP_ERR_INIT_FAILED; - break; + if (!new_rowset_meta->init_from_pb(rowset_meta_pb)) { + return OLAP_ERR_INIT_FAILED; } new_rowset_metas.push_back(new_rowset_meta); } else { + DCHECK_EQ(rowset_meta->rowset_type(), ALPHA_ROWSET); new_rowset_metas.push_back(rowset_meta); } } - if (res == OLAP_SUCCESS && modified) { + + if (modified) { if (is_incremental) { new_tablet_meta->revise_inc_rs_metas(std::move(new_rowset_metas)); } else { new_tablet_meta->revise_rs_metas(std::move(new_rowset_metas)); } } - return res; + return OLAP_SUCCESS; } } // namespace doris diff --git a/be/src/olap/snapshot_manager.h b/be/src/olap/snapshot_manager.h index 509dc7b7b365485..17b9ae93efc8a58 100644 --- a/be/src/olap/snapshot_manager.h +++ b/be/src/olap/snapshot_manager.h @@ -45,15 +45,14 @@ namespace doris { class SnapshotManager { public: - ~SnapshotManager() {} // @brief 创建snapshot - // @param tablet_id [in] 原表的id - // @param schema_hash [in] 原表的schema,与tablet_id参数合起来唯一确定一张表 + // @param TSnapshotRequest [in] 包含原表的id, schema等信息,用来唯一确定一张表 // @param snapshot_path [out] 新生成的snapshot的路径 OLAPStatus make_snapshot( const TSnapshotRequest& request, std::string* snapshot_path); + // TODO(yingchun): these functions are not owned by SnapshotManager, should move to Tablet std::string get_schema_hash_full_path( const TabletSharedPtr& ref_tablet, const std::string& location) const; @@ -68,10 +67,10 @@ class SnapshotManager { const int32_t& schema_hash); private: - SnapshotManager() - : _snapshot_base_id(0) {} + SnapshotManager() : _snapshot_base_id(1) {} + ~SnapshotManager() {} - OLAPStatus _calc_snapshot_id_path( + OLAPStatus _gen_snapshot_id_path( const TabletSharedPtr& tablet, int64_t timeout_s, std::string* out_path); @@ -102,13 +101,9 @@ class SnapshotManager { bool is_incremental); private: - static SnapshotManager* _s_instance; - static std::mutex _mlock; - + std::atomic _snapshot_base_id; - // snapshot - Mutex _snapshot_mutex; - uint64_t _snapshot_base_id; + DISALLOW_COPY_AND_ASSIGN(SnapshotManager); }; // SnapshotManager } // doris diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index a07d6b9eb935588..e335a7b42cfbeaa 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -240,6 +240,7 @@ void StorageEngine::_update_storage_medium_type_count() { } Status StorageEngine::_judge_and_update_effective_cluster_id(int32_t cluster_id) { + // TODO(yingchun): do some refactor? if (cluster_id == -1 && _effective_cluster_id == -1) { // maybe this is a new cluster, cluster id will get from heartbeat message return Status::OK(); @@ -266,6 +267,7 @@ void StorageEngine::set_store_used_flag(const string& path, bool is_used) { auto it = _store_map.find(path); if (it == _store_map.end()) { LOG(WARNING) << "store not exist, path=" << path; + return; } it->second->set_is_used(is_used); @@ -380,7 +382,7 @@ Status StorageEngine::_check_all_root_path_cluster_id() { if (tmp_cluster_id == -1) { _is_all_cluster_id_exist = false; } else if (tmp_cluster_id == cluster_id) { - // both hava right cluster id, do nothing + // both have right cluster id, do nothing } else if (cluster_id == -1) { cluster_id = tmp_cluster_id; } else { @@ -448,7 +450,7 @@ std::vector StorageEngine::get_stores_for_create_tablet( DataDir* StorageEngine::get_store(const std::string& path) { // _store_map is unchanged, no need to lock auto it = _store_map.find(path); - if (it == std::end(_store_map)) { + if (it == _store_map.end()) { return nullptr; } return it->second; @@ -465,7 +467,7 @@ bool StorageEngine::_delete_tablets_on_unused_root_path() { uint32_t total_root_path_num = 0; std::lock_guard l(_store_lock); - if (_store_map.size() == 0) { + if (_store_map.empty()) { return false; } @@ -653,7 +655,7 @@ void StorageEngine::get_cache_status(rapidjson::Document* document) const { return _index_stream_lru_cache->get_cache_status(document); } -OLAPStatus StorageEngine::_start_trash_sweep(double* usage) { +OLAPStatus StorageEngine::_start_trash_sweep(double* max_usage) { OLAPStatus res = OLAP_SUCCESS; LOG(INFO) << "start trash and snapshot sweep."; @@ -664,6 +666,7 @@ OLAPStatus StorageEngine::_start_trash_sweep(double* usage) { RETURN_NOT_OK_LOG(get_all_data_dir_info(&data_dir_infos, false), "failed to get root path stat info when sweep trash.") + // TODO(yingchun): make these functions as common utils time_t now = time(nullptr); //获取UTC时间 tm local_tm_now; if (localtime_r(&now, &local_tm_now) == nullptr) { @@ -678,7 +681,7 @@ OLAPStatus StorageEngine::_start_trash_sweep(double* usage) { } double curr_usage = (double) (info.disk_capacity - info.available) / info.disk_capacity; - *usage = *usage > curr_usage ? *usage : curr_usage; + *max_usage = std::max(*max_usage, curr_usage); OLAPStatus curr_res = OLAP_SUCCESS; string snapshot_path = info.path + SNAPSHOT_PREFIX; @@ -836,8 +839,7 @@ void StorageEngine::start_delete_unused_rowset() { ++it; } else if (it->second->need_delete_file()) { VLOG(3) << "start to remove rowset:" << it->second->rowset_id() - << ", version:" << it->second->version().first << "-" - << it->second->version().second; + << ", version: " << it->second->version(); OLAPStatus status = it->second->remove(); VLOG(3) << "remove rowset:" << it->second->rowset_id() << " finished. status:" << status; @@ -852,9 +854,8 @@ void StorageEngine::add_unused_rowset(RowsetSharedPtr rowset) { } VLOG(3) << "add unused rowset, rowset id:" << rowset->rowset_id() - << ", version:" << rowset->version().first << "-" << rowset->version().second + << ", version:" << rowset->version() << ", unique id:" << rowset->unique_id(); - auto rowset_id = rowset->rowset_id().to_string(); MutexLock lock(&_gc_mutex); @@ -881,13 +882,10 @@ OLAPStatus StorageEngine::create_tablet(const TCreateTabletReq& request) { OLAPStatus StorageEngine::obtain_shard_path( TStorageMedium::type storage_medium, std::string* shard_path, DataDir** store) { + DCHECK(shard_path); + DCHECK(store); LOG(INFO) << "begin to process obtain root path. storage_medium=" << storage_medium; - if (shard_path == NULL) { - LOG(WARNING) << "invalid output parameter which is null pointer."; - return OLAP_ERR_CE_CMD_PARAMS_ERROR; - } - auto stores = get_stores_for_create_tablet(storage_medium); if (stores.empty()) { LOG(WARNING) << "no available disk can be used to create tablet."; diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index 0b3aacfee3e81e3..dc5494f48981c05 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -241,7 +241,7 @@ class StorageEngine { void _perform_cumulative_compaction(DataDir* data_dir); void _perform_base_compaction(DataDir* data_dir); // 清理trash和snapshot文件,返回清理后的磁盘使用量 - OLAPStatus _start_trash_sweep(double *usage); + OLAPStatus _start_trash_sweep(double *max_usage); // 磁盘状态监测。监测unused_flag路劲新的对应root_path unused标识位, // 当检测到有unused标识时,从内存中删除对应表信息,磁盘数据不动。 // 当磁盘状态为不可用,但未检测到unused标识时,需要从root_path上 @@ -283,7 +283,7 @@ class StorageEngine { int32_t _effective_cluster_id; bool _is_all_cluster_id_exist; - Cache* _file_descriptor_lru_cache; + Cache* _file_descriptor_lru_cache; // TODO(yingchun): add cache Cache* _index_stream_lru_cache; // _file_cache is a lru_cache for file descriptors of files opened by doris, @@ -336,7 +336,7 @@ class StorageEngine { std::unique_ptr _block_manager; - // Used to control the migration from segment_v1 to segment_v2, can be deleted in futrue. + // Used to control the migration from segment_v1 to segment_v2, can be deleted in the future. // Type of new loaded data RowsetTypePB _default_rowset_type; diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index ea841642c0504b0..e201ceb1c8cb58d 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -147,6 +147,7 @@ void Tablet::save_meta() { _schema = _tablet_meta->tablet_schema(); } +// TODO(yingchun): _tablet_meta is modified here, why not lock? if has been locked outside, add '_unlocked' OLAPStatus Tablet::revise_tablet_meta( const vector& rowsets_to_clone, const vector& versions_to_delete) { @@ -162,6 +163,7 @@ OLAPStatus Tablet::revise_tablet_meta( // delete versions from new local tablet_meta for (const Version& version : versions_to_delete) { new_tablet_meta->delete_rs_meta_by_version(version, nullptr); + // TODO(yingchun): remove it directly if (new_tablet_meta->version_for_delete_predicate(version)) { new_tablet_meta->remove_delete_predicate_by_version(version); } @@ -170,8 +172,12 @@ OLAPStatus Tablet::revise_tablet_meta( } for (auto& rs_meta : rowsets_to_clone) { - new_tablet_meta->add_rs_meta(rs_meta); + res = new_tablet_meta->add_rs_meta(rs_meta); } + if (res != OLAP_SUCCESS) { + break; + } + VLOG(3) << "load rowsets successfully when clone. tablet=" << full_name() << ", added rowset size=" << rowsets_to_clone.size(); // save and reload tablet_meta @@ -214,6 +220,7 @@ OLAPStatus Tablet::revise_tablet_meta( return res; } +// TODO(yingchun): why not add it to _inc_rs_version_map ? OLAPStatus Tablet::add_rowset(RowsetSharedPtr rowset, bool need_persist) { DCHECK(rowset != nullptr); WriteLock wrlock(&_meta_lock); @@ -225,6 +232,7 @@ OLAPStatus Tablet::add_rowset(RowsetSharedPtr rowset, bool need_persist) { // Otherwise, the version shoud be not contained in any existing rowset. RETURN_NOT_OK(_contains_version(rowset->version())); + // TODO(yingchun): why not add this rs by modify_rowsets in L241 together? RETURN_NOT_OK(_tablet_meta->add_rs_meta(rowset->rowset_meta())); _rs_version_map[rowset->version()] = rowset; _timestamped_version_tracker.add_version(rowset->version()); @@ -361,8 +369,7 @@ OLAPStatus Tablet::add_inc_rowset(const RowsetSharedPtr& rowset) { _inc_rs_version_map[rowset->version()] = rowset; _timestamped_version_tracker.add_version(rowset->version()); - - RETURN_NOT_OK(_tablet_meta->add_inc_rs_meta(rowset->rowset_meta())); + _tablet_meta->add_inc_rs_meta(rowset->rowset_meta()); ++_newly_created_rowset_num; return OLAP_SUCCESS; } @@ -371,11 +378,6 @@ void Tablet::_delete_inc_rowset_by_version(const Version& version, const VersionHash& version_hash) { // delete incremental rowset from map _inc_rs_version_map.erase(version); - - RowsetMetaSharedPtr rowset_meta = _tablet_meta->acquire_inc_rs_meta_by_version(version); - if (rowset_meta == nullptr) { - return; - } _tablet_meta->delete_inc_rs_meta_by_version(version); VLOG(3) << "delete incremental rowset. tablet=" << full_name() << ", version=" << version; } @@ -394,6 +396,7 @@ void Tablet::delete_expired_inc_rowsets() { int64_t now = UnixSeconds(); vector> expired_versions; WriteLock wrlock(&_meta_lock); + // TODO(yingchun): make sure _tablet_meta->all_inc_rs_metas() are the same with myself for (auto& rs_meta : _tablet_meta->all_inc_rs_metas()) { double diff = ::difftime(now, rs_meta->creation_time()); if (diff >= config::inc_rowset_expired_sec) { @@ -556,6 +559,7 @@ void Tablet::delete_expired_stale_rowset() { #endif } +// TODO(yingchun): add _unlocked postfix too? OLAPStatus Tablet::capture_consistent_versions(const Version& spec_version, vector* version_path) const { // OLAPStatus status = _rs_graph.capture_consistent_versions(spec_version, version_path); @@ -788,6 +792,7 @@ void Tablet::calc_missed_versions(int64_t spec_version, vector* missed_ // for example: // [0-4][5-5][8-8][9-9] // if spec_version = 6, we still return {6, 7} other than {7} +// TODO(yingchun): +1 void Tablet::calc_missed_versions_unlocked(int64_t spec_version, vector* missed_versions) const { DCHECK(spec_version > 0) << "invalid spec_version: " << spec_version; @@ -802,7 +807,7 @@ void Tablet::calc_missed_versions_unlocked(int64_t spec_version, return a.first < b.first; }); - // From the first version(=0), find the missing version until spec_version + // From the first version(=0), find the missing version until spec_version int64_t last_version = -1; for (const Version& version : existing_versions) { if (version.first > last_version + 1) { @@ -876,6 +881,7 @@ OLAPStatus Tablet::split_range(const OlapTuple& start_key_strings, RowCursor start_key; // 如果有startkey,用startkey初始化;反之则用minkey初始化 if (start_key_strings.size() > 0) { + // TODO(yingchun): can we combine the two function? if (start_key.init_scan_key(_schema, start_key_strings.values()) != OLAP_SUCCESS) { LOG(WARNING) << "fail to initial key strings with RowCursor type."; return OLAP_ERR_INIT_FAILED; @@ -948,6 +954,7 @@ void Tablet::delete_all_files() { _stale_rs_version_map.clear(); } +// TODO(yingchun): can we remove it? bool Tablet::check_path(const std::string& path_to_check) const { ReadLock rdlock(&_meta_lock); if (path_to_check == _tablet_path) { @@ -1028,6 +1035,7 @@ OLAPStatus Tablet::_contains_version(const Version& version) { // because the value type is std::shared_ptr, when will it be nullptr? // In addition, in this class, there are many places that do not make this judgment // when access _rs_version_map's value. + // TODO(yingchun): +1 CHECK(it.second != nullptr) << "there exist a version=" << it.first << " contains the input rs with version=" << version << ", but the related rs is null"; @@ -1221,6 +1229,7 @@ bool Tablet::rowset_meta_is_useful(RowsetMetaSharedPtr rowset_meta) { return find_rowset_id || !find_version; } +// TODO(yingchun): if it will frenquently invoked, it's better to add a related mapping. bool Tablet::_contains_rowset(const RowsetId rowset_id) { for (auto& version_rowset : _rs_version_map) { if (version_rowset.second->rowset_id() == rowset_id) { diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index c32daad3b510086..d5c4d079180db0f 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -57,6 +57,7 @@ class Tablet : public BaseTablet { const std::string& cumulative_compaction_type = config::cumulative_compaction_policy); OLAPStatus init(); + // TODO(yingchun): inline is useless. inline bool init_succeeded(); bool is_used(); @@ -98,6 +99,7 @@ class Tablet : public BaseTablet { // _rs_version_map and _inc_rs_version_map should be protected by _meta_lock // The caller must call hold _meta_lock when call this two function. + // TODO(yingchun): add '_unlock' posifix const RowsetSharedPtr get_rowset_by_version(const Version& version) const; const RowsetSharedPtr get_inc_rowset_by_version(const Version& version) const; const RowsetSharedPtr get_stale_rowset_by_version(const Version& version) const; @@ -138,6 +140,7 @@ class Tablet : public BaseTablet { void delete_alter_task(); OLAPStatus set_alter_state(AlterTabletState state); + // TODO(yingchun): we'd better use a RAII method for these locks. // meta lock inline void obtain_header_rdlock() { _meta_lock.rdlock(); } inline void obtain_header_wrlock() { _meta_lock.wrlock(); } @@ -255,6 +258,8 @@ class Tablet : public BaseTablet { TimestampedVersionTracker _timestamped_version_tracker; DorisCallOnce _init_once; + + // TODO(yingchun): Can we reduce lock count? // meta store lock is used for prevent 2 threads do checkpoint concurrently // it will be used in econ-mode in the future RWMutex _meta_store_lock; @@ -339,6 +344,7 @@ inline void Tablet::set_cumulative_layer_point(int64_t new_point) { // TODO(lingbin): Why other methods that need to get information from _tablet_meta // are not locked, here needs a comment to explain. +// TODO(yingchun): +1 inline size_t Tablet::tablet_footprint() { ReadLock rdlock(&_meta_lock); return _tablet_meta->tablet_footprint(); @@ -346,6 +352,7 @@ inline size_t Tablet::tablet_footprint() { // TODO(lingbin): Why other methods which need to get information from _tablet_meta // are not locked, here needs a comment to explain. +// TODO(yingchun): +1 inline size_t Tablet::num_rows() { ReadLock rdlock(&_meta_lock); return _tablet_meta->num_rows(); diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 6260f2163184e33..18e3103b0ba3913 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -166,8 +166,7 @@ OLAPStatus TabletManager::_add_tablet_to_map_unlocked(TTabletId tablet_id, Schem const TabletSharedPtr& tablet, bool update_meta, bool keep_files, bool drop_old) { - // check if new tablet's meta is in store and add new tablet's meta to meta store - OLAPStatus res = OLAP_SUCCESS; + // check if new tablet's meta is in store and add new tablet's meta to meta store if (update_meta) { // call tablet save meta in order to valid the meta tablet->save_meta(); @@ -190,7 +189,7 @@ OLAPStatus TabletManager::_add_tablet_to_map_unlocked(TTabletId tablet_id, Schem VLOG(3) << "add tablet to map successfully." << " tablet_id=" << tablet_id << ", schema_hash=" << schema_hash; - return res; + return OLAP_SUCCESS; } bool TabletManager::check_tablet_id_exist(TTabletId tablet_id) { @@ -200,8 +199,9 @@ bool TabletManager::check_tablet_id_exist(TTabletId tablet_id) { } bool TabletManager::_check_tablet_id_exist_unlocked(TTabletId tablet_id) { + // TODO(yingchun): uint64_t or int64_t tablet_map_t& tablet_map = _get_tablet_map(tablet_id); - tablet_map_t::iterator it = tablet_map.find(tablet_id); + const auto& it = tablet_map.find(tablet_id); return it != tablet_map.end() && !it->second.table_arr.empty(); } @@ -309,13 +309,14 @@ TabletSharedPtr TabletManager::_internal_create_tablet_unlocked( break; } // TODO(lingbin): is it needed? because all type of create_tablet will be true. + // TODO(yingchun): use DCHECK? // 1. !is_schema_change: not in schema-change state; // 2. request.base_tablet_id > 0: in schema-change state; if (!is_schema_change || (request.__isset.base_tablet_id && request.base_tablet_id > 0)) { // Create init version if this is not a restore mode replica and request.version is set // bool in_restore_mode = request.__isset.in_restore_mode && request.in_restore_mode; // if (!in_restore_mode && request.__isset.version) { - // create inital rowset before add it to storage engine could omit many locks + // create initial rowset before add it to storage engine could omit many locks res = _create_inital_rowset_unlocked(request, tablet.get()); if (res != OLAP_SUCCESS) { LOG(WARNING) << "fail to create initial version for tablet. res=" << res; @@ -326,7 +327,7 @@ TabletSharedPtr TabletManager::_internal_create_tablet_unlocked( if (request.__isset.base_tablet_id && request.base_tablet_id > 0) { LOG(INFO) << "request for alter-tablet v2, do not add alter task to tablet"; // if this is a new alter tablet, has to set its state to not ready - // because schema change hanlder depends on it to check whether history data + // because schema change handler depends on it to check whether history data // convert finished tablet->set_tablet_state(TabletState::TABLET_NOTREADY); } else { @@ -349,7 +350,7 @@ TabletSharedPtr TabletManager::_internal_create_tablet_unlocked( tablet->set_creation_time(new_creation_time); } } - // Add tablet to StorageEngine will make it visiable to user + // Add tablet to StorageEngine will make it visible to user res = _add_tablet_unlocked(new_tablet_id, new_schema_hash, tablet, true, false); if (res != OLAP_SUCCESS) { LOG(WARNING) << "fail to add tablet to StorageEngine. res=" << res; @@ -405,7 +406,7 @@ TabletSharedPtr TabletManager::_create_tablet_meta_and_dir_unlocked( last_dir = data_dir; TabletMetaSharedPtr tablet_meta; - // if create meta faild, do not need to clean dir, because it is only in memory + // if create meta failed, do not need to clean dir, because it is only in memory OLAPStatus res = _create_tablet_meta_unlocked( request, data_dir, is_schema_change, base_tablet, &tablet_meta); if (res != OLAP_SUCCESS) { @@ -414,6 +415,7 @@ TabletSharedPtr TabletManager::_create_tablet_meta_and_dir_unlocked( continue; } + // TODO(yingchun): combine the 2 lines string tablet_dir = _gen_tablet_dir( data_dir->path(), tablet_meta->shard_id(), request.tablet_id); string schema_hash_dir = path_util::join_path_segments( @@ -461,7 +463,7 @@ OLAPStatus TabletManager::_drop_tablet_unlocked( LOG(INFO) << "begin drop tablet. tablet_id=" << tablet_id << ", schema_hash=" << schema_hash; DorisMetrics::instance()->drop_tablet_requests_total->increment(1); - // Fetch tablet which need to be droped + // Fetch tablet which need to be dropped TabletSharedPtr to_drop_tablet = _get_tablet_unlocked(tablet_id, schema_hash); if (to_drop_tablet == nullptr) { LOG(WARNING) << "fail to drop tablet because it does not exist. " @@ -680,7 +682,7 @@ void TabletManager::get_tablet_stat(TTabletStatResult* result) { } TabletSharedPtr TabletManager::find_best_tablet_to_compaction(CompactionType compaction_type, - DataDir* data_dir) { + DataDir* data_dir) { int64_t now_ms = UnixMillis(); const string& compaction_type_str = compaction_type == CompactionType::BASE_COMPACTION ? "base" : "cumulative"; uint32_t highest_score = 0; @@ -878,7 +880,7 @@ OLAPStatus TabletManager::load_tablet_from_dir(DataDir* store, TTabletId tablet_ LOG(WARNING) << "fail to load tablet_meta. file_path=" << header_path; return OLAP_ERR_ENGINE_LOAD_INDEX_TABLE_ERROR; } - // has to change shard id here, because meta file maybe copyed from other source + // has to change shard id here, because meta file maybe copyid from other source // its shard is different from local shard tablet_meta->set_shard_id(shard); string meta_binary; @@ -896,7 +898,7 @@ void TabletManager::release_schema_change_lock(TTabletId tablet_id) { ReadLock rlock(&tablet_map_lock); tablet_map_t& tablet_map = _get_tablet_map(tablet_id); - tablet_map_t::iterator it = tablet_map.find(tablet_id); + auto it = tablet_map.find(tablet_id); if (it == tablet_map.end()) { LOG(WARNING) << "tablet does not exists. tablet=" << tablet_id; } else { @@ -940,7 +942,7 @@ OLAPStatus TabletManager::report_all_tablets_info(std::map* ReadLock rlock(&_tablet_map_lock_array[i]); tablet_map_t& tablet_map = _tablet_map_array[i]; for (const auto& item : tablet_map) { - if (item.second.table_arr.size() == 0) { + if (item.second.table_arr.empty()) { continue; } @@ -970,6 +972,7 @@ OLAPStatus TabletManager::report_all_tablets_info(std::map* } OLAPStatus TabletManager::start_trash_sweep() { + std::vector tablets_to_clean; { std::vector tablets_to_clean; std::vector all_tablets; // we use this vector to save all tablet ptr for saving lock time. @@ -1049,9 +1052,10 @@ OLAPStatus TabletManager::start_trash_sweep() { if (Env::Default()->path_exists(tablet_path).ok()) { // take snapshot of tablet meta string meta_file_path = path_util::join_path_segments( - (*it)->tablet_path(), std::to_string((*it)->tablet_id()) + ".hdr"); + tablet_path, std::to_string((*it)->tablet_id()) + ".hdr"); (*it)->tablet_meta()->save(meta_file_path); LOG(INFO) << "start to move tablet to trash. tablet_path = " << tablet_path; + // TODO(yingchun): this operation is time costy, should move outside of lock! OLAPStatus rm_st = move_to_trash(tablet_path, tablet_path); if (rm_st != OLAP_SUCCESS) { LOG(WARNING) << "fail to move dir to trash. dir=" << tablet_path; @@ -1060,8 +1064,7 @@ OLAPStatus TabletManager::start_trash_sweep() { } } // remove tablet meta - TabletMetaManager::remove( - (*it)->data_dir(), (*it)->tablet_id(), (*it)->schema_hash()); + TabletMetaManager::remove((*it)->data_dir(), (*it)->tablet_id(), (*it)->schema_hash()); LOG(INFO) << "successfully move tablet to trash. " << "tablet_id=" << (*it)->tablet_id() << ", schema_hash=" << (*it)->schema_hash() @@ -1158,7 +1161,7 @@ bool TabletManager::try_schema_change_lock(TTabletId tablet_id) { void TabletManager::update_root_path_info(std::map* path_map, size_t* tablet_count) { - DCHECK(tablet_count != 0); + DCHECK(tablet_count); *tablet_count = 0; for (int32 i = 0; i < _tablet_map_lock_shard_size; i++) { ReadLock rlock(&_tablet_map_lock_array[i]); @@ -1168,10 +1171,7 @@ void TabletManager::update_root_path_info(std::map* path_ma ++(*tablet_count); int64_t data_size = tablet->tablet_footprint(); auto iter = path_map->find(tablet->data_dir()->path()); - if (iter == path_map->end()) { - continue; - } - if (iter->second.is_used) { + if (iter != path_map->end() && iter->second.is_used) { iter->second.data_used_capacity += data_size; } } @@ -1182,8 +1182,9 @@ void TabletManager::update_root_path_info(std::map* path_ma void TabletManager::get_partition_related_tablets(int64_t partition_id, std::set* tablet_infos) { ReadLock rlock(&_partition_tablet_map_lock); - if (_partition_tablet_map.find(partition_id) != _partition_tablet_map.end()) { - *tablet_infos = _partition_tablet_map[partition_id]; + auto it = _partition_tablet_map.find(partition_id); + if (it != _partition_tablet_map.end()) { + *tablet_infos = it->second; } } @@ -1211,7 +1212,6 @@ void TabletManager::do_tablet_meta_checkpoint(DataDir* data_dir) { for (TabletSharedPtr tablet : related_tablets) { tablet->do_tablet_meta_checkpoint(); } - return; } void TabletManager::_build_tablet_stat() { @@ -1269,6 +1269,7 @@ OLAPStatus TabletManager::_create_inital_rowset_unlocked(const TCreateTabletReq& } else { LOG(ERROR) << "invalid TStorageFormat: " << request.storage_format; DCHECK(false); + // TODO(yingchun): why not assert? context.rowset_type = StorageEngine::instance()->default_rowset_type(); } context.rowset_path_prefix = tablet->tablet_path(); @@ -1379,7 +1380,7 @@ OLAPStatus TabletManager::_drop_tablet_directly_unlocked( } tablet_map_t& tablet_map = _get_tablet_map(tablet_id); list& candidate_tablets = tablet_map[tablet_id].table_arr; - list::iterator it = candidate_tablets.begin(); + auto it = candidate_tablets.begin(); while (it != candidate_tablets.end()) { if (!(*it)->equal(tablet_id, schema_hash)) { ++it; @@ -1417,7 +1418,7 @@ OLAPStatus TabletManager::_drop_tablet_directly_unlocked( TabletSharedPtr TabletManager::_get_tablet_unlocked(TTabletId tablet_id, SchemaHash schema_hash) { VLOG(3) << "begin to get tablet. tablet_id=" << tablet_id << ", schema_hash=" << schema_hash; tablet_map_t& tablet_map = _get_tablet_map(tablet_id); - tablet_map_t::iterator it = tablet_map.find(tablet_id); + const auto& it = tablet_map.find(tablet_id); if (it != tablet_map.end()) { for (TabletSharedPtr tablet : it->second.table_arr) { CHECK(tablet != nullptr) << "tablet is nullptr. tablet_id=" << tablet_id; diff --git a/be/src/olap/tablet_manager.h b/be/src/olap/tablet_manager.h index ea4de8c69e96c7c..cf3cb4e1a606dee 100644 --- a/be/src/olap/tablet_manager.h +++ b/be/src/olap/tablet_manager.h @@ -113,8 +113,6 @@ class TabletManager { void release_schema_change_lock(TTabletId tablet_id); - // 获取所有tables的名字 - // // Return OLAP_SUCCESS, if run ok // OLAP_ERR_INPUT_PARAMETER_ERROR, if tables is null OLAPStatus report_tablet_info(TTabletInfo* tablet_info); diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index ded65217f5f58ab..464aeafbcdf4e72 100755 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -307,6 +307,7 @@ OLAPStatus TabletMeta::serialize(string* meta_binary) { if (!serialize_success) { LOG(FATAL) << "failed to serialize meta " << full_name(); } + // TODO(yingchun): not needed, or there must be a bug in protobuf // deserialize the meta to check the result is correct TabletMetaPB de_tablet_meta_pb; bool parsed = de_tablet_meta_pb.ParseFromString(*meta_binary); @@ -404,7 +405,7 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { } } -void TabletMeta::to_meta_pb(TabletMetaPB* tablet_meta_pb) { +void TabletMeta::to_meta_pb(TabletMetaPB* tablet_meta_pb) const { tablet_meta_pb->set_table_id(table_id()); tablet_meta_pb->set_partition_id(partition_id()); tablet_meta_pb->set_tablet_id(tablet_id()); @@ -454,14 +455,15 @@ void TabletMeta::to_meta_pb(TabletMetaPB* tablet_meta_pb) { } } -void TabletMeta::to_json(string* json_string, json2pb::Pb2JsonOptions& options) { +void TabletMeta::to_json(string* json_string, json2pb::Pb2JsonOptions& options) const { TabletMetaPB tablet_meta_pb; to_meta_pb(&tablet_meta_pb); json2pb::ProtoMessageToJson(tablet_meta_pb, json_string, options); } Version TabletMeta::max_version() const { - Version max_version = { -1, 0 }; + // TODO(yingchun): define a const kInvalidVersion + Version max_version(-1, 0); for (auto& rs_meta : _rs_metas) { if (rs_meta->end_version() > max_version.second) { max_version = rs_meta->version(); @@ -475,6 +477,7 @@ OLAPStatus TabletMeta::add_rs_meta(const RowsetMetaSharedPtr& rs_meta) { for (auto& rs : _rs_metas) { if (rs->version() == rs_meta->version()) { if (rs->rowset_id() != rs_meta->rowset_id()) { + // TODO(yingchun): should CHECK? LOG(WARNING) << "version already exist. rowset_id=" << rs->rowset_id() << " version=" << rs->version() << ", tablet=" << full_name(); @@ -498,10 +501,12 @@ void TabletMeta::delete_rs_meta_by_version(const Version& version, vector* deleted_rs_metas) { auto it = _rs_metas.begin(); while (it != _rs_metas.end()) { + // TODO(yingchun): are versions ordered? or can we sort them? if ((*it)->version() == version) { if (deleted_rs_metas != nullptr) { deleted_rs_metas->push_back(*it); } + // TODO(yingchun): no need to remove_delete_predicate_by_version like in modify_rs_metas _rs_metas.erase(it); return; } else { @@ -596,6 +601,7 @@ void TabletMeta::delete_inc_rs_meta_by_version(const Version& version) { } RowsetMetaSharedPtr TabletMeta::acquire_inc_rs_meta_by_version(const Version& version) const { + // TODO(yingchun): too many loop and compare, can we use map? for (auto it : _inc_rs_metas) { if (it->version() == version) { return it; @@ -622,6 +628,7 @@ void TabletMeta::remove_delete_predicate_by_version(const Version& version) { for (int ordinal = 0; ordinal < _del_pred_array.size(); ++ordinal) { const DeletePredicatePB& temp = _del_pred_array.Get(ordinal); if (temp.version() == version.first) { + // TODO(yingchun): just log in some log level // log delete condition string del_cond_str; for (const auto& it : temp.sub_predicates()) { @@ -630,6 +637,7 @@ void TabletMeta::remove_delete_predicate_by_version(const Version& version) { LOG(INFO) << "remove one del_pred. version=" << temp.version() << ", condition=" << del_cond_str; + // TODO(yingchun): Swap here to avoid move many elements? // remove delete condition from PB _del_pred_array.SwapElements(ordinal, _del_pred_array.size() - 1); _del_pred_array.RemoveLast(); diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index 4cc75de64d3d21f..25719403f2874fd 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -131,8 +131,8 @@ class TabletMeta { OLAPStatus deserialize(const std::string& meta_binary); void init_from_pb(const TabletMetaPB& tablet_meta_pb); - void to_meta_pb(TabletMetaPB* tablet_meta_pb); - void to_json(std::string* json_string, json2pb::Pb2JsonOptions& options); + void to_meta_pb(TabletMetaPB* tablet_meta_pb) const; + void to_json(std::string* json_string, json2pb::Pb2JsonOptions& options) const; inline TabletTypePB tablet_type() const { return _tablet_type; } inline TabletUid tablet_uid() const; @@ -163,6 +163,7 @@ class TabletMeta { inline TabletSchema* mutable_tablet_schema(); + // rowset meta inline const std::vector& all_rs_metas() const; OLAPStatus add_rs_meta(const RowsetMetaSharedPtr& rs_meta); void delete_rs_meta_by_version(const Version& version, @@ -171,6 +172,7 @@ class TabletMeta { const std::vector& to_delete); void revise_rs_metas(std::vector&& rs_metas); + // inc rowset meta void revise_inc_rs_metas(std::vector&& rs_metas); inline const std::vector& all_inc_rs_metas() const; @@ -181,11 +183,15 @@ class TabletMeta { void delete_stale_rs_meta_by_version(const Version& version); RowsetMetaSharedPtr acquire_stale_rs_meta_by_version(const Version& version) const; + // delete predicate void add_delete_predicate(const DeletePredicatePB& delete_predicate, int64_t version); void remove_delete_predicate_by_version(const Version& version); DelPredicateArray delete_predicates() const; bool version_for_delete_predicate(const Version& version); + + // alter_task AlterTabletTaskSharedPtr alter_task(); + // TODO(yingchun): rename to set_alter_task? void add_alter_task(const AlterTabletTask& alter_task); void delete_alter_task(); OLAPStatus set_alter_state(AlterTabletState alter_state); diff --git a/be/src/olap/tablet_meta_manager.cpp b/be/src/olap/tablet_meta_manager.cpp index 74fdc1653c026ab..80f62a840c122dc 100755 --- a/be/src/olap/tablet_meta_manager.cpp +++ b/be/src/olap/tablet_meta_manager.cpp @@ -106,6 +106,7 @@ OLAPStatus TabletMetaManager::save(DataDir* store, VLOG(3) << "save tablet meta to meta store: key = " << key; OlapMeta* meta = store->get_meta(); + // TODO(yingchun): make sure outside TabletMetaPB de_tablet_meta_pb; bool parsed = de_tablet_meta_pb.ParseFromString(meta_binary); if (!parsed) { diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp index 02e9c5e89cfe3bd..535fa1f7b64854e 100644 --- a/be/src/olap/tablet_schema.cpp +++ b/be/src/olap/tablet_schema.cpp @@ -27,6 +27,7 @@ FieldType TabletColumn::get_field_type_by_string(const std::string& type_str) { std::transform(type_str.begin(), type_str.end(), upper_type_str.begin(), toupper); FieldType type; + // TODO(yingchun): maybe it's less effective than strcmp? if (0 == upper_type_str.compare("TINYINT")) { type = OLAP_FIELD_TYPE_TINYINT; } else if (0 == upper_type_str.compare("SMALLINT")) { @@ -314,7 +315,7 @@ void TabletColumn::init_from_pb(const ColumnPB& column) { } } -void TabletColumn::to_schema_pb(ColumnPB* column) { +void TabletColumn::to_schema_pb(ColumnPB* column) const { column->set_unique_id(_unique_id); column->set_name(_col_name); column->set_type(get_string_by_field_type(_type)); @@ -376,7 +377,7 @@ void TabletSchema::init_from_pb(const TabletSchemaPB& schema) { _sequence_col_idx = schema.sequence_col_idx(); } -void TabletSchema::to_schema_pb(TabletSchemaPB* tablet_meta_pb) { +void TabletSchema::to_schema_pb(TabletSchemaPB* tablet_meta_pb) const { tablet_meta_pb->set_keys_type(_keys_type); for (auto& col : _cols) { ColumnPB* column = tablet_meta_pb->add_column(); diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h index da7aedfaf1ad9de..c7adeef22c6f01f 100644 --- a/be/src/olap/tablet_schema.h +++ b/be/src/olap/tablet_schema.h @@ -32,7 +32,7 @@ class TabletColumn { TabletColumn(FieldAggregationMethod agg, FieldType type); TabletColumn(FieldAggregationMethod agg, FieldType filed_type, bool is_nullable); void init_from_pb(const ColumnPB& column); - void to_schema_pb(ColumnPB* column); + void to_schema_pb(ColumnPB* column) const; inline int32_t unique_id() const { return _unique_id; } inline std::string name() const { return _col_name; } @@ -97,7 +97,8 @@ class TabletSchema { public: TabletSchema() = default; void init_from_pb(const TabletSchemaPB& schema); - void to_schema_pb(TabletSchemaPB* tablet_meta_pb); + void to_schema_pb(TabletSchemaPB* tablet_meta_pb) const; + // TODO(yingchun): precalculate these result size_t row_size() const; size_t field_index(const std::string& field_name) const; const TabletColumn& column(size_t ordinal) const; @@ -124,6 +125,7 @@ class TabletSchema { private: KeysType _keys_type = DUP_KEYS; + // TODO(yingchun): we should index it by column name std::vector _cols; size_t _num_columns = 0; size_t _num_key_columns = 0; diff --git a/be/src/olap/task/engine_alter_tablet_task.cpp b/be/src/olap/task/engine_alter_tablet_task.cpp index 0e7e1172e4e499e..41a1a4919045fc0 100644 --- a/be/src/olap/task/engine_alter_tablet_task.cpp +++ b/be/src/olap/task/engine_alter_tablet_task.cpp @@ -33,6 +33,7 @@ EngineAlterTabletTask::EngineAlterTabletTask(const TAlterTabletReqV2& request, _process_name(process_name) { } OLAPStatus EngineAlterTabletTask::execute() { + // TODO(yingchun): why this metric? DorisMetrics::instance()->create_rollup_requests_total->increment(1); SchemaChangeHandler handler; diff --git a/be/src/olap/task/engine_batch_load_task.cpp b/be/src/olap/task/engine_batch_load_task.cpp index cf9c5a83cc810d1..f51695a75369e1c 100644 --- a/be/src/olap/task/engine_batch_load_task.cpp +++ b/be/src/olap/task/engine_batch_load_task.cpp @@ -43,7 +43,7 @@ using std::vector; namespace doris { -EngineBatchLoadTask::EngineBatchLoadTask(TPushReq& push_req, +EngineBatchLoadTask::EngineBatchLoadTask(const TPushReq& push_req, std::vector* tablet_infos, int64_t signature, AgentStatus* res_status) : @@ -66,7 +66,6 @@ OLAPStatus EngineBatchLoadTask::execute() { uint32_t retry_time = 0; while (retry_time < PUSH_MAX_RETRY) { status = _process(); - if (status == DORIS_PUSH_HAD_LOADED) { OLAP_LOG_WARNING("transaction exists when realtime push, " "but unfinished, do not report to fe, signature: %ld", @@ -232,7 +231,8 @@ AgentStatus EngineBatchLoadTask::_process() { } } // NOTE: change http_file_path is not good design - _push_req.http_file_path = _local_file_path; + // TODO(yingchun): seems not used later, so disable this line + // _push_req.http_file_path = _local_file_path; return Status::OK(); }; @@ -356,6 +356,7 @@ OLAPStatus EngineBatchLoadTask::_delete_data( OLAPStatus res = OLAP_SUCCESS; + // TODO(yingchun): use CHECK? if (tablet_info_vec == nullptr) { LOG(WARNING) << "invalid tablet info parameter which is nullptr pointer."; return OLAP_ERR_CE_CMD_PARAMS_ERROR; diff --git a/be/src/olap/task/engine_batch_load_task.h b/be/src/olap/task/engine_batch_load_task.h index 51a003d3b4fc6dd..6926609e8d7c620 100644 --- a/be/src/olap/task/engine_batch_load_task.h +++ b/be/src/olap/task/engine_batch_load_task.h @@ -36,9 +36,9 @@ const uint32_t MAX_RETRY = 3; const uint32_t DEFAULT_DOWNLOAD_TIMEOUT = 3600; class StorageEngine; -class EngineBatchLoadTask : public EngineTask{ +class EngineBatchLoadTask : public EngineTask { public: - EngineBatchLoadTask(TPushReq& push_req, std::vector* tablet_infos, + EngineBatchLoadTask(const TPushReq& push_req, std::vector* tablet_infos, int64_t signature, AgentStatus* res_status); virtual ~EngineBatchLoadTask(); @@ -59,7 +59,7 @@ class EngineBatchLoadTask : public EngineTask{ // but not actually deleted util delay_delete_time run out. // // @param [in] request specify tablet and delete conditions - // @param [out] tablet_info_vec return tablet lastest status, which + // @param [out] tablet_info_vec return tablet latest status, which // include version info, row count, data size, etc // @return OLAP_SUCCESS if submit delete_data success virtual OLAPStatus _delete_data(const TPushReq& request, @@ -71,7 +71,7 @@ class EngineBatchLoadTask : public EngineTask{ void _get_file_name_from_path(const std::string& file_path, std::string* file_name); bool _is_init = false; - TPushReq& _push_req; + const TPushReq& _push_req; std::vector* _tablet_infos; int64_t _signature; AgentStatus _download_status; diff --git a/be/src/olap/task/engine_checksum_task.cpp b/be/src/olap/task/engine_checksum_task.cpp index c201e5aae289ee4..66314ea6b4ea97d 100644 --- a/be/src/olap/task/engine_checksum_task.cpp +++ b/be/src/olap/task/engine_checksum_task.cpp @@ -57,8 +57,6 @@ OLAPStatus EngineChecksumTask::_compute_checksum() { return OLAP_ERR_TABLE_NOT_FOUND; } - - Reader reader; ReaderParams reader_params; reader_params.tablet = tablet; reader_params.reader_type = READER_CHECKSUM; @@ -84,6 +82,7 @@ OLAPStatus EngineChecksumTask::_compute_checksum() { reader_params.return_columns.push_back(i); } + Reader reader; res = reader.init(reader_params); if (res != OLAP_SUCCESS) { OLAP_LOG_WARNING("initiate reader fail. [res=%d]", res); diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp index ba4a6c753f5c8d7..ef43b6735953ae1 100644 --- a/be/src/olap/task/engine_clone_task.cpp +++ b/be/src/olap/task/engine_clone_task.cpp @@ -433,6 +433,7 @@ Status EngineCloneTask::_make_snapshot( return Status(result.status); } + // TODO(yingchun): why not init outside? snapshot_path, allow_incremental_clone, snapshot_version if (result.__isset.snapshot_path) { *snapshot_path = result.snapshot_path; if (snapshot_path->at(snapshot_path->length() - 1) != '/') { @@ -468,7 +469,7 @@ Status EngineCloneTask::_download_files( const std::string& remote_url_prefix, const std::string& local_path) { // Check local path exist, if exist, remove it, then create the dir - // local_file_full_path = tabletid/clone, for a specific tablet, there should be only one folder + // local_file_full_path = tabletid/clone, for a specific tablet, there should be only one folder // if this folder exists, then should remove it // for example, BE clone from BE 1 to download file 1 with version (2,2), but clone from BE 1 failed // then it will try to clone from BE 2, but it will find the file 1 already exist, but file 1 with same @@ -487,7 +488,7 @@ Status EngineCloneTask::_download_files( 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()); - // If the header file is not exist, the table could't loaded by olap engine. + // 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. // The header file's name is end of .hdr. for (int i = 0; i < file_name_list.size() - 1; ++i) { @@ -582,7 +583,6 @@ OLAPStatus EngineCloneTask::_convert_to_new_snapshot(const string& clone_dir, in string cloned_meta_file = clone_dir + "/" + std::to_string(tablet_id) + ".hdr"; FileHeader file_header; FileHandler file_handler; - OLAPHeaderMessage olap_header_msg; if (file_handler.open(cloned_meta_file.c_str(), O_RDONLY) != OLAP_SUCCESS) { LOG(WARNING) << "fail to open ordinal file. file=" << cloned_meta_file; return OLAP_ERR_IO_ERROR; @@ -595,12 +595,12 @@ OLAPStatus EngineCloneTask::_convert_to_new_snapshot(const string& clone_dir, in } set clone_files; - RETURN_WITH_WARN_IF_ERROR( FileUtils::list_dirs_files(clone_dir, NULL, &clone_files, Env::Default()), OLAP_ERR_DISK_FAILURE, "failed to dir walk when clone. clone_dir=" + clone_dir); + OLAPHeaderMessage olap_header_msg; try { olap_header_msg.CopyFrom(file_header.message()); } catch (...) { @@ -752,20 +752,18 @@ OLAPStatus EngineCloneTask::_clone_incremental_data(Tablet* tablet, const Tablet vector missed_versions; tablet->calc_missed_versions_unlocked(committed_version, &missed_versions); - vector versions_to_delete; - vector rowsets_to_clone; - VLOG(3) << "get missed versions again when finish incremental clone. " << "tablet=" << tablet->full_name() << ", committed_version=" << committed_version << ", missed_versions_size=" << missed_versions.size(); // check missing versions exist in clone src + vector rowsets_to_clone; for (Version version : missed_versions) { RowsetMetaSharedPtr inc_rs_meta = cloned_tablet_meta.acquire_inc_rs_meta_by_version(version); if (inc_rs_meta == nullptr) { LOG(WARNING) << "missed version is not found in cloned tablet meta." - << ", missed_version=" << version.first << "-" << version.second; + << " missed_version=" << version.first << "-" << version.second; return OLAP_ERR_VERSION_NOT_EXIST; } @@ -773,6 +771,7 @@ OLAPStatus EngineCloneTask::_clone_incremental_data(Tablet* tablet, const Tablet } // clone_data to tablet + vector versions_to_delete; OLAPStatus clone_res = tablet->revise_tablet_meta(rowsets_to_clone, versions_to_delete); LOG(INFO) << "finish to incremental clone. [tablet=" << tablet->full_name() << " res=" << clone_res << "]"; return clone_res; @@ -781,8 +780,7 @@ OLAPStatus EngineCloneTask::_clone_incremental_data(Tablet* tablet, const Tablet OLAPStatus EngineCloneTask::_clone_full_data(Tablet* tablet, TabletMeta* cloned_tablet_meta) { Version cloned_max_version = cloned_tablet_meta->max_version(); LOG(INFO) << "begin to full clone. tablet=" << tablet->full_name() - << ", cloned_max_version=" << cloned_max_version.first - << "-" << cloned_max_version.second; + << ", cloned_max_version=" << cloned_max_version; vector versions_to_delete; vector rs_metas_found_in_src; // check local versions @@ -790,7 +788,7 @@ OLAPStatus EngineCloneTask::_clone_full_data(Tablet* tablet, TabletMeta* cloned_ Version local_version(rs_meta->start_version(), rs_meta->end_version()); LOG(INFO) << "check local delta when full clone." << "tablet=" << tablet->full_name() - << ", local_version=" << local_version.first << "-" << local_version.second; + << ", local_version=" << local_version; // if local version cross src latest, clone failed // if local version is : 0-0, 1-1, 2-10, 12-14, 15-15,16-16 @@ -802,7 +800,7 @@ OLAPStatus EngineCloneTask::_clone_full_data(Tablet* tablet, TabletMeta* cloned_ && local_version.second > cloned_max_version.second) { LOG(WARNING) << "stop to full clone, version cross src latest." << "tablet=" << tablet->full_name() - << ", local_version=" << local_version.first << "-" << local_version.second; + << ", local_version=" << local_version; return OLAP_ERR_TABLE_VERSION_DUPLICATE_ERROR; } else if (local_version.second <= cloned_max_version.second) { @@ -811,9 +809,9 @@ OLAPStatus EngineCloneTask::_clone_full_data(Tablet* tablet, TabletMeta* cloned_ // if delta labeled with local_version is same with the specified version in clone header, // there is no necessity to clone it. + // TODO(yingchun): if '_rs_metas' is sorted by rs meta version, we can use binary search. for (auto& rs_meta : cloned_tablet_meta->all_rs_metas()) { - if (rs_meta->version().first == local_version.first - && rs_meta->version().second == local_version.second) { + if (rs_meta->version() == local_version) { existed_in_src = true; break; } @@ -823,7 +821,7 @@ OLAPStatus EngineCloneTask::_clone_full_data(Tablet* tablet, TabletMeta* cloned_ cloned_tablet_meta->delete_rs_meta_by_version(local_version, &rs_metas_found_in_src); LOG(INFO) << "Delta has already existed in local header, no need to clone." << "tablet=" << tablet->full_name() - << ", version='" << local_version.first<< "-" << local_version.second; + << ", version='" << local_version; } else { // Delta labeled in local_version is not existed in clone header, // some overlapping delta will be cloned to replace it. @@ -831,17 +829,15 @@ OLAPStatus EngineCloneTask::_clone_full_data(Tablet* tablet, TabletMeta* cloned_ versions_to_delete.push_back(local_version); LOG(INFO) << "Delete delta not included by the clone header, should delete it from local header." << "tablet=" << tablet->full_name() << "," - << ", version=" << local_version.first<< "-" << local_version.second; + << ", version=" << local_version; } } } - vector rowsets_to_clone; - for (auto& rs_meta : cloned_tablet_meta->all_rs_metas()) { - rowsets_to_clone.push_back(rs_meta); + vector rowsets_to_clone(cloned_tablet_meta->all_rs_metas()); + for (const auto& rs_meta : rowsets_to_clone) { LOG(INFO) << "Delta to clone." << "tablet=" << tablet->full_name() - << ", version=" << rs_meta->version().first << "-" - << rs_meta->version().second; + << ", version=" << rs_meta->version(); } // clone_data to tablet diff --git a/be/src/olap/task/engine_clone_task.h b/be/src/olap/task/engine_clone_task.h index 57a8e2eb9b05e50..dd24e285c870437 100644 --- a/be/src/olap/task/engine_clone_task.h +++ b/be/src/olap/task/engine_clone_task.h @@ -73,6 +73,7 @@ class EngineCloneTask : public EngineTask { const std::string& remote_url_prefix, const std::string& local_path); + // TODO(yingchun): use 'const &' would be better than 'const *', like missed_versions Status _make_snapshot( const std::string& ip, int port, TTableId tablet_id, diff --git a/be/src/olap/task/engine_publish_version_task.cpp b/be/src/olap/task/engine_publish_version_task.cpp index 28202ab9cb61b2a..124ed2d6bdeca5d 100644 --- a/be/src/olap/task/engine_publish_version_task.cpp +++ b/be/src/olap/task/engine_publish_version_task.cpp @@ -25,7 +25,7 @@ namespace doris { using std::map; -EnginePublishVersionTask::EnginePublishVersionTask(TPublishVersionRequest& publish_version_req, +EnginePublishVersionTask::EnginePublishVersionTask(const TPublishVersionRequest& publish_version_req, vector* error_tablet_ids) : _publish_version_req(publish_version_req), _error_tablet_ids(error_tablet_ids) {} @@ -42,6 +42,7 @@ OLAPStatus EnginePublishVersionTask::finish() { std::set partition_related_tablet_infos; StorageEngine::instance()->tablet_manager()->get_partition_related_tablets( partition_id, &partition_related_tablet_infos); + // TODO(yingchun): what dose 'strict_mode' mean? Should be '!_publish_version_req.strict_mode' here? if (_publish_version_req.strict_mode && partition_related_tablet_infos.empty()) { LOG(INFO) << "could not find related tablet for partition " << partition_id << ", skip publish version"; @@ -56,10 +57,10 @@ OLAPStatus EnginePublishVersionTask::finish() { VersionHash version_hash = par_ver_info.version_hash; // each tablet - for (auto& tablet_rs : tablet_related_rs) { + for (const auto& tablet_rs : tablet_related_rs) { OLAPStatus publish_status = OLAP_SUCCESS; - TabletInfo tablet_info = tablet_rs.first; - RowsetSharedPtr rowset = tablet_rs.second; + const TabletInfo& tablet_info = tablet_rs.first; + const RowsetSharedPtr& rowset = tablet_rs.second; LOG(INFO) << "begin to publish version on tablet. " << "tablet_id=" << tablet_info.tablet_id << ", schema_hash=" << tablet_info.schema_hash @@ -116,34 +117,38 @@ OLAPStatus EnginePublishVersionTask::finish() { << ", res=" << publish_status; } + // has to use strict mode to check if check all tablets + if (!_publish_version_req.strict_mode) { + continue; + } // check if the related tablet remained all have the version for (auto& tablet_info : partition_related_tablet_infos) { - // has to use strict mode to check if check all tablets - if (!_publish_version_req.strict_mode) { - break; - } TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet( tablet_info.tablet_id, tablet_info.schema_hash); if (tablet == nullptr) { _error_tablet_ids->push_back(tablet_info.tablet_id); + continue; + } + // check if the version exist, if not exist, then set publish failed + if (tablet->check_version_exist(version)) { + continue; + } + _error_tablet_ids->push_back(tablet_info.tablet_id); + // generate a pull rowset meta task to pull rowset from remote meta store and storage + // pull rowset meta using tablet_id + txn_id + // it depends on the tablet type to download file or only meta + // TODO(yingchun): Should use '!tablet->in_eco_mode()' ? + if (!tablet->in_eco_mode()) { + continue; + } + + if (tablet->is_primary_replica()) { + // primary replica should fetch the meta using txn id + // it will fetch the rowset to meta store, and will be published in next publish version task + StorageEngine::instance()->tablet_sync_service()->fetch_rowset(tablet, transaction_id, FETCH_DATA); } else { - // check if the version exist, if not exist, then set publish failed - if (!tablet->check_version_exist(version)) { - _error_tablet_ids->push_back(tablet_info.tablet_id); - // generate a pull rowset meta task to pull rowset from remote meta store and storage - // pull rowset meta using tablet_id + txn_id - // it depends on the tablet type to download file or only meta - if (tablet->in_eco_mode()) { - if (tablet->is_primary_replica()) { - // primary replica should fetch the meta using txn id - // it will fetch the rowset to meta store, and will be published in next publish version task - StorageEngine::instance()->tablet_sync_service()->fetch_rowset(tablet, transaction_id, FETCH_DATA); - } else { - // shadow replica should fetch the meta using version - StorageEngine::instance()->tablet_sync_service()->fetch_rowset(tablet, version, NOT_FETCH_DATA); - } - } - } + // shadow replica should fetch the meta using version + StorageEngine::instance()->tablet_sync_service()->fetch_rowset(tablet, version, NOT_FETCH_DATA); } } } diff --git a/be/src/olap/task/engine_publish_version_task.h b/be/src/olap/task/engine_publish_version_task.h index 188e1ea9cba0d49..ba4bbf7ef002bd6 100644 --- a/be/src/olap/task/engine_publish_version_task.h +++ b/be/src/olap/task/engine_publish_version_task.h @@ -26,7 +26,7 @@ namespace doris { class EnginePublishVersionTask : public EngineTask { public: - EnginePublishVersionTask(TPublishVersionRequest& publish_version_req, + EnginePublishVersionTask(const TPublishVersionRequest& publish_version_req, vector* error_tablet_ids); ~EnginePublishVersionTask() {} diff --git a/be/src/olap/task/engine_storage_migration_task.cpp b/be/src/olap/task/engine_storage_migration_task.cpp index c7ffa374529c9f0..c9b980d83f0e720 100644 --- a/be/src/olap/task/engine_storage_migration_task.cpp +++ b/be/src/olap/task/engine_storage_migration_task.cpp @@ -24,9 +24,8 @@ namespace doris { using std::stringstream; -EngineStorageMigrationTask::EngineStorageMigrationTask(TStorageMediumMigrateReq& storage_medium_migrate_req) : +EngineStorageMigrationTask::EngineStorageMigrationTask(const TStorageMediumMigrateReq& storage_medium_migrate_req) : _storage_medium_migrate_req(storage_medium_migrate_req) { - } OLAPStatus EngineStorageMigrationTask::execute() { @@ -117,6 +116,7 @@ OLAPStatus EngineStorageMigrationTask::_storage_medium_migrate( } // check disk capacity + // TODO(yingchun): should try another store? int64_t tablet_size = tablet->tablet_footprint(); if (stores[0]->reach_capacity_limit(tablet_size)) { res = OLAP_ERR_DISK_REACH_CAPACITY_LIMIT; @@ -133,7 +133,8 @@ OLAPStatus EngineStorageMigrationTask::_storage_medium_migrate( stringstream root_path_stream; root_path_stream << stores[0]->path() << DATA_PREFIX << "/" << shard; - string schema_hash_path = SnapshotManager::instance()->get_schema_hash_full_path(tablet, root_path_stream.str()); + string schema_hash_path = SnapshotManager::instance()->get_schema_hash_full_path( + tablet, root_path_stream.str()); // if dir already exist then return err, it should not happen // should not remove the dir directly if (FileUtils::check_exist(schema_hash_path)) { @@ -166,6 +167,7 @@ OLAPStatus EngineStorageMigrationTask::_storage_medium_migrate( LOG(WARNING) << "fail to copy index and data files when migrate. res=" << res; break; } + // TODO(yingchun): Should be 'obtain_header_wdlock' ? tablet->obtain_header_rdlock(); _generate_new_header(stores[0], shard, tablet, consistent_rowsets, new_tablet_meta); tablet->release_header_lock(); diff --git a/be/src/olap/task/engine_storage_migration_task.h b/be/src/olap/task/engine_storage_migration_task.h index 7a449f2d6e2d849..650f055c3551134 100644 --- a/be/src/olap/task/engine_storage_migration_task.h +++ b/be/src/olap/task/engine_storage_migration_task.h @@ -32,7 +32,7 @@ class EngineStorageMigrationTask : public EngineTask { virtual OLAPStatus execute(); public: - EngineStorageMigrationTask(TStorageMediumMigrateReq& storage_medium_migrate_req); + EngineStorageMigrationTask(const TStorageMediumMigrateReq& storage_medium_migrate_req); ~EngineStorageMigrationTask() {} private: diff --git a/be/src/olap/txn_manager.cpp b/be/src/olap/txn_manager.cpp index 2d0baa445697f48..ff5f6c134c06e8f 100755 --- a/be/src/olap/txn_manager.cpp +++ b/be/src/olap/txn_manager.cpp @@ -129,7 +129,7 @@ OLAPStatus TxnManager::prepare_txn( if (load_itr != it->second.end()) { // found load for txn,tablet // case 1: user commit rowset, then the load id must be equal - TabletTxnInfo& load_info = load_itr->second; + const TabletTxnInfo& load_info = load_itr->second; // check if load id is equal if (load_info.load_id.hi() == load_id.hi() && load_info.load_id.lo() == load_id.lo() @@ -169,6 +169,7 @@ OLAPStatus TxnManager::commit_txn( OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id, TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid, const PUniqueId& load_id, const RowsetSharedPtr& rowset_ptr, bool is_recovery) { + // TODO(yingchun): Why not check in prepare phase? if (partition_id < 1 || transaction_id < 1 || tablet_id < 1) { LOG(FATAL) << "invalid commit req " << " partition_id=" << partition_id @@ -196,31 +197,30 @@ OLAPStatus TxnManager::commit_txn( if (load_itr != it->second.end()) { // found load for txn,tablet // case 1: user commit rowset, then the load id must be equal - TabletTxnInfo& load_info = load_itr->second; + const TabletTxnInfo& load_info = load_itr->second; // check if load id is equal if (load_info.load_id.hi() == load_id.hi() && load_info.load_id.lo() == load_id.lo() - && load_info.rowset != nullptr - && load_info.rowset->rowset_id() == rowset_ptr->rowset_id()) { - // find a rowset with same rowset id, then it means a duplicate call - LOG(INFO) << "find transaction exists when add to engine." - << "partition_id: " << key.first - << ", transaction_id: " << key.second - << ", tablet: " << tablet_info.to_string() - << ", rowset_id: " << load_info.rowset->rowset_id(); - return OLAP_SUCCESS; - } else if (load_info.load_id.hi() == load_id.hi() - && load_info.load_id.lo() == load_id.lo() - && load_info.rowset != nullptr - && load_info.rowset->rowset_id() != rowset_ptr->rowset_id()) { - // find a rowset with different rowset id, then it should not happen, just return errors - LOG(WARNING) << "find transaction exists when add to engine. but rowset ids are not same." - << "partition_id: " << key.first - << ", transaction_id: " << key.second - << ", tablet: " << tablet_info.to_string() - << ", exist rowset_id: " << load_info.rowset->rowset_id() - << ", new rowset_id: " << rowset_ptr->rowset_id(); - return OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST; + && load_info.rowset != nullptr) { + if (load_info.rowset->rowset_id() == rowset_ptr->rowset_id()) { + // find a rowset with same rowset id, then it means a duplicate call + LOG(INFO) << "find transaction exists when add to engine." + << "partition_id: " << key.first + << ", transaction_id: " << key.second + << ", tablet: " << tablet_info.to_string() + << ", rowset_id: " << load_info.rowset->rowset_id(); + return OLAP_SUCCESS; + } else { + DCHECK(load_info.rowset->rowset_id() != rowset_ptr->rowset_id()); + // find a rowset with different rowset id, then it should not happen, just return errors + LOG(WARNING) << "find transaction exists when add to engine. but rowset ids are not same." + << "partition_id: " << key.first + << ", transaction_id: " << key.second + << ", tablet: " << tablet_info.to_string() + << ", exist rowset_id: " << load_info.rowset->rowset_id() + << ", new rowset_id: " << rowset_ptr->rowset_id(); + return OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST; + } } } } @@ -275,8 +275,7 @@ OLAPStatus TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, TT if (load_itr != it->second.end()) { // found load for txn,tablet // case 1: user commit rowset, then the load id must be equal - TabletTxnInfo& load_info = load_itr->second; - rowset_ptr = load_info.rowset; + rowset_ptr = load_itr->second.rowset; } } } @@ -336,14 +335,14 @@ OLAPStatus TxnManager::rollback_txn(TPartitionId partition_id, TTransactionId tr if (load_itr != it->second.end()) { // found load for txn,tablet // case 1: user commit rowset, then the load id must be equal - TabletTxnInfo& load_info = load_itr->second; + const TabletTxnInfo& load_info = load_itr->second; if (load_info.rowset != nullptr) { // if rowset is not null, it means other thread may commit the rowset // should not delete txn any more return OLAP_ERR_TRANSACTION_ALREADY_COMMITTED; } + it->second.erase(tablet_info); } - it->second.erase(tablet_info); LOG(INFO) << "rollback transaction from engine successfully." << " partition_id: " << key.first << ", transaction_id: " << key.second @@ -372,8 +371,9 @@ OLAPStatus TxnManager::delete_txn(OlapMeta* meta, TPartitionId partition_id, TTr if (load_itr != it->second.end()) { // found load for txn,tablet // case 1: user commit rowset, then the load id must be equal - TabletTxnInfo& load_info = load_itr->second; - if (load_info.rowset != nullptr && meta != nullptr) { + const TabletTxnInfo& load_info = load_itr->second; + if (load_info.rowset != nullptr) { + DCHECK(meta); if (load_info.rowset->version().first > 0) { LOG(WARNING) << "could not delete transaction from engine, " << "just remove it from memory not delete from disk" @@ -396,8 +396,8 @@ OLAPStatus TxnManager::delete_txn(OlapMeta* meta, TPartitionId partition_id, TTr << ", rowset: " << (load_info.rowset != nullptr ? load_info.rowset->rowset_id().to_string(): "0"); } } + it->second.erase(tablet_info); } - it->second.erase(tablet_info); if (it->second.empty()) { txn_tablet_map.erase(it); _clear_txn_partition_map_unlocked(transaction_id, partition_id); @@ -424,6 +424,7 @@ void TxnManager::get_tablet_related_txns(TTabletId tablet_id, SchemaHash schema_ << "partition_id: " << it.first.first << ", transaction_id: " << it.first.second << ", tablet: " << tablet_info.to_string(); + break; } } } @@ -436,11 +437,13 @@ void TxnManager::force_rollback_tablet_related_txns(OlapMeta* meta, TTabletId ta for (int32_t i = 0; i < _txn_map_shard_size; i++) { WriteLock txn_wrlock(&_txn_map_locks[i]); txn_tablet_map_t& txn_tablet_map = _txn_tablet_maps[i]; + // TODO(yingchun): unify iterator names for (auto it = txn_tablet_map.begin(); it != txn_tablet_map.end();) { auto load_itr = it->second.find(tablet_info); if (load_itr != it->second.end()) { TabletTxnInfo& load_info = load_itr->second; - if (load_info.rowset != nullptr && meta != nullptr) { + if (load_info.rowset != nullptr) { + DCHECK(meta != nullptr); // TODO(yingchun): check all usage LOG(INFO) << " delete transaction from engine " << ", tablet: " << tablet_info.to_string() << ", rowset id: " << load_info.rowset->rowset_id(); @@ -467,7 +470,7 @@ void TxnManager::force_rollback_tablet_related_txns(OlapMeta* meta, TTabletId ta } } -void TxnManager::get_txn_related_tablets(const TTransactionId transaction_id, +void TxnManager::get_txn_related_tablets(TTransactionId transaction_id, TPartitionId partition_id, std::map* tablet_infos) { // get tablets in this transaction @@ -481,14 +484,11 @@ void TxnManager::get_txn_related_tablets(const TTransactionId transaction_id, << ", transaction_id=" << transaction_id; return; } - std::map& load_info_map = it->second; - // each tablet - for (auto& load_info : load_info_map) { - const TabletInfo& tablet_info = load_info.first; + for (const auto& load_info : it->second) { // must not check rowset == null here, because if rowset == null // publish version should failed - tablet_infos->emplace(tablet_info, load_info.second.rowset); + tablet_infos->emplace(load_info.first, load_info.second.rowset); } } @@ -510,10 +510,8 @@ bool TxnManager::has_txn(TPartitionId partition_id, TTransactionId transaction_i ReadLock txn_rdlock(&_get_txn_map_lock(transaction_id)); txn_tablet_map_t& txn_tablet_map = _get_txn_tablet_map(transaction_id); auto it = txn_tablet_map.find(key); - bool found = it != txn_tablet_map.end() - && it->second.find(tablet_info) != it->second.end(); - - return found; + return it != txn_tablet_map.end() + && it->second.find(tablet_info) != it->second.end(); } void TxnManager::build_expire_txn_map(std::map>* expire_txn_map) { @@ -526,6 +524,7 @@ void TxnManager::build_expire_txn_map(std::map> for (auto& t_map : it.second) { double diff = difftime(now, t_map.second.creation_time); if (diff >= config::pending_data_expire_time_sec) { + // TODO(yingchun): optimize find and insert (*expire_txn_map)[t_map.first].push_back(txn_id); if (VLOG_IS_ON(3)) { VLOG(3) << "find expired txn." @@ -544,13 +543,12 @@ void TxnManager::get_partition_ids(const TTransactionId transaction_id, std::vec txn_partition_map_t& txn_partition_map = _get_txn_partition_map(transaction_id); auto it = txn_partition_map.find(transaction_id); if (it != txn_partition_map.end()) { - for (int64_t partition_id : it->second) { - partition_ids->push_back(partition_id); - } + partition_ids->insert(partition_ids->end(), it->second.begin(), it->second.end()); } } void TxnManager::_insert_txn_partition_map_unlocked(int64_t transaction_id, int64_t partition_id) { + // TODO(yingchun): optimize find and insert txn_partition_map_t& txn_partition_map = _get_txn_partition_map(transaction_id); auto find = txn_partition_map.find(transaction_id); if (find == txn_partition_map.end()) { diff --git a/be/src/olap/txn_manager.h b/be/src/olap/txn_manager.h index d16e4532c9b889b..9dcf5b7558c55b6 100755 --- a/be/src/olap/txn_manager.h +++ b/be/src/olap/txn_manager.h @@ -52,16 +52,16 @@ namespace doris { struct TabletTxnInfo { PUniqueId load_id; RowsetSharedPtr rowset; - int64_t creation_time; + int64_t creation_time = 0; TabletTxnInfo( PUniqueId load_id, RowsetSharedPtr rowset) : - load_id(load_id), + load_id(std::move(load_id)), rowset(rowset), creation_time(UnixSeconds()) {} - TabletTxnInfo() {} + TabletTxnInfo() = default; }; @@ -123,7 +123,7 @@ class TxnManager { void get_tablet_related_txns(TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid, int64_t* partition_id, std::set* transaction_ids); - void get_txn_related_tablets(const TTransactionId transaction_id, + void get_txn_related_tablets(TTransactionId transaction_id, TPartitionId partition_ids, std::map* tablet_infos); @@ -133,16 +133,15 @@ class TxnManager { bool has_txn(TPartitionId partition_id, TTransactionId transaction_id, TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid); - // get all expired txns and save tham in expire_txn_map. + // get all expired txns and save them in expire_txn_map. // This is currently called before reporting all tablet info, to avoid iterating txn map for every tablets. void build_expire_txn_map(std::map>* expire_txn_map); void force_rollback_tablet_related_txns(OlapMeta* meta, TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid); - void get_partition_ids(const TTransactionId transaction_id, std::vector* partition_ids); + void get_partition_ids(TTransactionId transaction_id, std::vector* partition_ids); private: - using TxnKey = std::pair; // partition_id, transaction_id; // implement TxnKey hash function to support TxnKey as a key for unordered_map diff --git a/be/src/olap/types.h b/be/src/olap/types.h index 0067092203d4088..5f72528607bd85a 100644 --- a/be/src/olap/types.h +++ b/be/src/olap/types.h @@ -68,7 +68,7 @@ class TypeInfo { } //convert and deep copy value from other type's source - OLAPStatus convert_from(void* dest, const void* src, const TypeInfo* src_type, MemPool* mem_pool) const{ + OLAPStatus convert_from(void* dest, const void* src, const TypeInfo* src_type, MemPool* mem_pool) const { return _convert_from(dest, src, src_type, mem_pool); } @@ -244,6 +244,7 @@ struct BaseFieldtypeTraits : public CppTypeTraits { } static std::string to_string(const void* src) { + // TODO(yingchun): use std::to_string(xxx); std::stringstream stream; stream << *reinterpret_cast(src); return stream.str(); @@ -251,7 +252,8 @@ struct BaseFieldtypeTraits : public CppTypeTraits { static OLAPStatus from_string(void* buf, const std::string& scan_key) { CppType value = 0; - if (scan_key.length() > 0) { + if (!scan_key.empty()) { + // TODO(yingchun): should be strtoll? and overflow judge? value = static_cast(strtol(scan_key.c_str(), NULL, 10)); } *reinterpret_cast(buf) = value; @@ -365,6 +367,7 @@ struct FieldTypeTraits : public BaseFieldtypeTraits struct FieldTypeTraits : public BaseFieldtypeTraits { + // TODO(yingchun): improve performance static OLAPStatus from_string(void* buf, const std::string& scan_key) { int128_t value = 0; @@ -475,7 +478,7 @@ template<> struct FieldTypeTraits : public BaseFieldtypeTraits { static OLAPStatus from_string(void* buf, const std::string& scan_key) { CppType value = 0.0f; - if (scan_key.length() > 0) { + if (!scan_key.empty()) { value = static_cast(atof(scan_key.c_str())); } *reinterpret_cast(buf) = value; @@ -499,7 +502,7 @@ template<> struct FieldTypeTraits : public BaseFieldtypeTraits { static OLAPStatus from_string(void* buf, const std::string& scan_key) { CppType value = 0.0; - if (scan_key.length() > 0) { + if (!scan_key.empty()) { value = atof(scan_key.c_str()); } *reinterpret_cast(buf) = value; diff --git a/be/src/olap/utils.cpp b/be/src/olap/utils.cpp index f34108f28839ba2..e750dbb5d0ab987 100644 --- a/be/src/olap/utils.cpp +++ b/be/src/olap/utils.cpp @@ -918,6 +918,7 @@ int operator-(const BinarySearchIterator& left, const BinarySearchIterator& righ return *left - *right; } +// TODO(yingchun): move to FileUtil OLAPStatus copy_file(const string& src, const string& dest) { int src_fd = -1; int dest_fd = -1; diff --git a/be/src/olap/utils.h b/be/src/olap/utils.h index 00121c4fccb3c1e..988478b86246790 100644 --- a/be/src/olap/utils.h +++ b/be/src/olap/utils.h @@ -159,6 +159,7 @@ uint32_t olap_adler32(uint32_t adler, const char* buf, size_t len); uint32_t olap_crc32(uint32_t crc32, const char* buf, size_t len); // 获取系统当前时间,并将时间转换为字符串 +// TODO(yingchun): add comments OLAPStatus gen_timestamp_string(std::string* out_string); // 将file移到回收站,回收站位于storage_root/trash, file可以是文件或目录 diff --git a/be/src/olap/wrapper_field.h b/be/src/olap/wrapper_field.h index 62c0233bde24198..97fb81a2fd58f03 100644 --- a/be/src/olap/wrapper_field.h +++ b/be/src/olap/wrapper_field.h @@ -112,13 +112,13 @@ class WrapperField { private: Field* _rep = nullptr; - bool _is_string_type; + bool _is_string_type = false; char* _field_buf = nullptr; char* _owned_buf = nullptr; //include fixed and variable length and null bytes - size_t _length; - size_t _var_length; + size_t _length = 0; + size_t _var_length = 0; // memory for string type field std::unique_ptr _string_content; }; diff --git a/be/src/runtime/broker_mgr.cpp b/be/src/runtime/broker_mgr.cpp index 214995245375707..874528f5fddde7d 100644 --- a/be/src/runtime/broker_mgr.cpp +++ b/be/src/runtime/broker_mgr.cpp @@ -68,7 +68,6 @@ const std::string& BrokerMgr::get_client_id(const TNetworkAddress& address) { void BrokerMgr::ping(const TNetworkAddress& addr) { TBrokerPingBrokerRequest request; - request.__set_version(TBrokerVersion::VERSION_ONE); request.__set_clientId(_client_id); @@ -102,12 +101,10 @@ void BrokerMgr::ping(const TNetworkAddress& addr) { void BrokerMgr::ping_worker() { do { - std::vector addresses; + std::unordered_set addresses; { std::lock_guard l(_mutex); - for (auto& addr : _broker_set) { - addresses.emplace_back(addr); - } + addresses = _broker_set; } for (auto& addr : addresses) { ping(addr); diff --git a/be/src/runtime/broker_mgr.h b/be/src/runtime/broker_mgr.h index 7842ebb5585edf4..4df02416e733ef5 100644 --- a/be/src/runtime/broker_mgr.h +++ b/be/src/runtime/broker_mgr.h @@ -32,6 +32,7 @@ namespace doris { class ExecEnv; +// TODO(yingchun): seems useless, can we remove it? class BrokerMgr { public: BrokerMgr(ExecEnv* exec_env); diff --git a/be/src/runtime/bufferpool/buffer_allocator.cc b/be/src/runtime/bufferpool/buffer_allocator.cc index 0f0f31fa594d092..7f992fb394b8396 100644 --- a/be/src/runtime/bufferpool/buffer_allocator.cc +++ b/be/src/runtime/bufferpool/buffer_allocator.cc @@ -241,7 +241,7 @@ Status BufferPool::BufferAllocator::AllocateInternal(int64_t len, BufferHandle* } if (UNLIKELY(len > system_bytes_limit_)) { err_stream << "Tried to allocate buffer of " << len << " bytes" - << " > buffer pool limit of " << MAX_BUFFER_BYTES << " bytes"; + << " > buffer pool limit of " << system_bytes_limit_ << " bytes"; return Status::InternalError(err_stream.str()); } diff --git a/be/src/runtime/data_stream_mgr.cpp b/be/src/runtime/data_stream_mgr.cpp index 3152ded3b80d496..1b301252d290e98 100644 --- a/be/src/runtime/data_stream_mgr.cpp +++ b/be/src/runtime/data_stream_mgr.cpp @@ -71,7 +71,7 @@ shared_ptr DataStreamMgr::create_recvr(RuntimeState* state, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, int buffer_size, RuntimeProfile* profile, bool is_merging, std::shared_ptr sub_plan_query_statistics_recvr) { - DCHECK(profile != NULL); + DCHECK(profile != nullptr); VLOG_FILE << "creating receiver for fragment=" << fragment_instance_id << ", node=" << dest_node_id; shared_ptr recvr( @@ -93,6 +93,9 @@ shared_ptr DataStreamMgr::find_recvr( if (acquire_lock) { _lock.lock(); } + + // There may be duplicate hash value of different ids. + // TODO(yingchun): use ralated container or hash of ids, which one is more effective? we should do benchmard std::pair range = _receiver_map.equal_range(hash_value); while (range.first != range.second) { @@ -112,7 +115,7 @@ shared_ptr DataStreamMgr::find_recvr( return shared_ptr(); } -Status DataStreamMgr::transmit_data(const PTransmitDataParams* request, ::google::protobuf::Closure** done) { +void DataStreamMgr::transmit_data(const PTransmitDataParams* request, ::google::protobuf::Closure** done) { const PUniqueId& finst_id = request->finst_id(); TUniqueId t_finst_id; t_finst_id.hi = finst_id.hi(); @@ -126,7 +129,7 @@ Status DataStreamMgr::transmit_data(const PTransmitDataParams* request, ::google // in acquiring _lock. // TODO: Rethink the lifecycle of DataStreamRecvr to distinguish // errors from receiver-initiated teardowns. - return Status::OK(); + return; } // request can only be used before calling recvr's add_batch or when request @@ -145,12 +148,11 @@ Status DataStreamMgr::transmit_data(const PTransmitDataParams* request, ::google if (eos) { recvr->remove_sender(request->sender_id(), request->be_number()); } - return Status::OK(); } -Status DataStreamMgr::deregister_recvr( - const TUniqueId& fragment_instance_id, PlanNodeId node_id) { +Status DataStreamMgr::deregister_recvr(const TUniqueId& fragment_instance_id, PlanNodeId node_id) { boost::shared_ptr targert_recvr; + // TODO(yingchun): add a to_string() for DataStreamRecvr, then we can reuse it in these log. VLOG_QUERY << "deregister_recvr(): fragment_instance_id=" << fragment_instance_id << ", node=" << node_id; size_t hash_value = get_hash_value(fragment_instance_id, node_id); @@ -163,6 +165,7 @@ Status DataStreamMgr::deregister_recvr( if (recvr->fragment_instance_id() == fragment_instance_id && recvr->dest_node_id() == node_id) { targert_recvr = recvr; + // TODO(yingchun): has been remove in cancel() _fragment_stream_set.erase(std::make_pair( recvr->fragment_instance_id(), recvr->dest_node_id())); _receiver_map.erase(range.first); @@ -174,6 +177,7 @@ Status DataStreamMgr::deregister_recvr( // Notify concurrent add_data() requests that the stream has been terminated. // cancel_stream maybe take a long time, so we handle it out of lock. + // TODO(yingchun): dep with it->cancel_stream() in cancel() if (targert_recvr) { targert_recvr->cancel_stream(); return Status::OK(); @@ -186,6 +190,7 @@ Status DataStreamMgr::deregister_recvr( } } +// TODO(yingchun): didn't remove it from _receiver_map? void DataStreamMgr::cancel(const TUniqueId& fragment_instance_id) { VLOG_QUERY << "cancelling all streams for fragment=" << fragment_instance_id; std::vector> recvrs; diff --git a/be/src/runtime/data_stream_mgr.h b/be/src/runtime/data_stream_mgr.h index 6ace3829103664a..4f446945308e148 100644 --- a/be/src/runtime/data_stream_mgr.h +++ b/be/src/runtime/data_stream_mgr.h @@ -81,7 +81,7 @@ class DataStreamMgr { int num_senders, int buffer_size, RuntimeProfile* profile, bool is_merging, std::shared_ptr sub_plan_query_statistics_recvr); - Status transmit_data(const PTransmitDataParams* request, ::google::protobuf::Closure** done); + void transmit_data(const PTransmitDataParams* request, ::google::protobuf::Closure** done); // Closes all receivers registered for fragment_instance_id immediately. void cancel(const TUniqueId& fragment_instance_id); @@ -105,14 +105,11 @@ class DataStreamMgr { struct ComparisonOp { bool operator()(const std::pair& a, const std::pair& b) { - if (a.first.hi < b.first.hi) { - return true; - } else if (a.first.hi > b.first.hi) { - return false; - } else if (a.first.lo < b.first.lo) { - return true; - } else if (a.first.lo > b.first.lo) { - return false; + if (a.first.hi != b.first.hi) { + return a.first.hi < b.first.hi; + } + if (a.first.lo != b.first.lo) { + return a.first.lo < b.first.lo; } return a.second < b.second; } @@ -132,7 +129,7 @@ class DataStreamMgr { // Remove receiver block for fragment_instance_id/node_id from the map. Status deregister_recvr(const TUniqueId& fragment_instance_id, PlanNodeId node_id); - inline uint32_t get_hash_value(const TUniqueId& fragment_instance_id, PlanNodeId node_id); + static inline uint32_t get_hash_value(const TUniqueId& fragment_instance_id, PlanNodeId node_id); }; } diff --git a/be/src/runtime/data_stream_recvr.cc b/be/src/runtime/data_stream_recvr.cc index 9fc643d8cdbf287..3d31ab140ac1b4f 100644 --- a/be/src/runtime/data_stream_recvr.cc +++ b/be/src/runtime/data_stream_recvr.cc @@ -88,8 +88,8 @@ class DataStreamRecvr::SenderQueue { void close(); // Returns the current batch from this queue being processed by a consumer. - RowBatch* current_batch() const { { - return _current_batch.get(); } + RowBatch* current_batch() const { + return _current_batch.get(); } private: @@ -110,7 +110,7 @@ class DataStreamRecvr::SenderQueue { condition_variable _data_arrival_cv; // signal removal of data by stream consumer - condition_variable _data_removal_cv; + // condition_variable _data_removal_cv; // queue of (batch length, batch) pairs. The SenderQueue block owns memory to // these batches. They are handed off to the caller via get_batch. @@ -128,6 +128,7 @@ class DataStreamRecvr::SenderQueue { std::unordered_set _sender_eos_set; // sender_id std::unordered_map _packet_seq_map; // be_number => packet_seq + // TODO(yingchun): what does it means? delay request? std::deque> _pending_closures; }; @@ -284,10 +285,11 @@ void DataStreamRecvr::SenderQueue::cancel() { return; } _is_cancelled = true; - VLOG_QUERY << "cancelled stream: _fragment_instance_id=" - << _recvr->fragment_instance_id() - << " node_id=" << _recvr->dest_node_id(); } + VLOG_QUERY << "cancelled stream: _fragment_instance_id=" + << _recvr->fragment_instance_id() + << " node_id=" << _recvr->dest_node_id(); + // Wake up all threads waiting to produce/consume batches. They will all // notice that the stream is cancelled and handle it. _data_arrival_cv.notify_all(); @@ -309,7 +311,7 @@ void DataStreamRecvr::SenderQueue::close() { // If _is_cancelled is not set to true, there may be concurrent send // which add batch to _batch_queue. The batch added after _batch_queue // is clear will be memory leak - boost::lock_guard l(_lock); + lock_guard l(_lock); _is_cancelled = true; for (auto closure_pair : _pending_closures) { @@ -319,10 +321,10 @@ void DataStreamRecvr::SenderQueue::close() { } // Delete any batches queued in _batch_queue - for (RowBatchQueue::iterator it = _batch_queue.begin(); - it != _batch_queue.end(); ++it) { - delete it->second; + for (auto it : _batch_queue) { + delete it.second; } + _batch_queue.clear(); _current_batch.reset(); } diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index a880419921d658c..b65edfba991995c 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -67,10 +67,11 @@ class HeartbeatFlags; // Contains all required global structures, and handles to // singleton services. Clients must call StartServices exactly // once to properly initialise service state. +// TODO(yingchun): this class is too complex, it better to inlcude and use the embeded class when needed. class ExecEnv { public: // Initial exec enviorment. must call this to init all - static Status init(ExecEnv* env, const std::vector& store_paths); + static void init(ExecEnv* env, const std::vector& store_paths); static void destroy(ExecEnv* exec_env); /// Returns the first created exec env instance. In a normal doris, this is @@ -139,7 +140,7 @@ class ExecEnv { PluginMgr* plugin_mgr() { return _plugin_mgr; } private: - Status _init(const std::vector& store_paths); + void _init(const std::vector& store_paths); void _destory(); Status _init_mem_tracker(); diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index fb84bb114a37351..dba389dae1d2da9 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -65,11 +65,11 @@ namespace doris { -Status ExecEnv::init(ExecEnv* env, const std::vector& store_paths) { - return env->_init(store_paths); +void ExecEnv::init(ExecEnv* env, const std::vector& store_paths) { + env->_init(store_paths); } -Status ExecEnv::_init(const std::vector& store_paths) { +void ExecEnv::_init(const std::vector& store_paths) { _store_paths = store_paths; _external_scan_context_mgr = new ExternalScanContextMgr(this); _stream_mgr = new DataStreamMgr(); @@ -112,17 +112,13 @@ Status ExecEnv::_init(const std::vector& store_paths) { _cgroups_mgr->init_cgroups(); _etl_job_mgr->init(); Status status = _load_path_mgr->init(); - if (!status.ok()) { - LOG(ERROR) << "load path mgr init failed." << status.get_error_msg(); - exit(-1); - } + CHECK(status.ok()) << "load path mgr init failed. " << status.to_string(); _broker_mgr->init(); _small_file_mgr->init(); - _init_mem_tracker(); - - RETURN_IF_ERROR(_load_channel_mgr->init(_mem_tracker->limit())); + status = _init_mem_tracker(); + CHECK(status.ok()) << "init mem tracker failed. " << status.to_string(); + _load_channel_mgr->init(_mem_tracker->limit()); _heartbeat_flags = new HeartbeatFlags(); - return Status::OK(); } Status ExecEnv::_init_mem_tracker() { diff --git a/be/src/runtime/external_scan_context_mgr.cpp b/be/src/runtime/external_scan_context_mgr.cpp index 9a9e4886e662077..c617a87f771c522 100644 --- a/be/src/runtime/external_scan_context_mgr.cpp +++ b/be/src/runtime/external_scan_context_mgr.cpp @@ -52,7 +52,8 @@ ExternalScanContextMgr::~ExternalScanContextMgr() { } } -Status ExternalScanContextMgr::create_scan_context(std::shared_ptr* p_context) { +void ExternalScanContextMgr::create_scan_context(std::shared_ptr* p_context) { + // TODO(yingchun): use int may be faster than string std::string context_id = generate_uuid_string(); std::shared_ptr context(new ScanContext(context_id)); // context->last_access_time = time(NULL); @@ -61,26 +62,26 @@ Status ExternalScanContextMgr::create_scan_context(std::shared_ptr* _active_contexts.insert(std::make_pair(context_id, context)); } *p_context = context; - return Status::OK(); } Status ExternalScanContextMgr::get_scan_context(const std::string& context_id, std::shared_ptr* p_context) { { - std::lock_guard l(_lock); + std::lock_guard l(_lock); auto iter = _active_contexts.find(context_id); if (iter != _active_contexts.end()) { *p_context = iter->second; - } else { - LOG(WARNING) << "get_scan_context error: context id [ " << context_id << " ] not found"; - std::stringstream msg; - msg << "context_id: " << context_id << " not found"; - return Status::NotFound(msg.str()); + return Status::OK(); } } - return Status::OK(); + + // TODO(yingchun): use substitude + LOG(WARNING) << "get_scan_context error: context id [ " << context_id << " ] not found"; + std::stringstream msg; + msg << "context_id: " << context_id << " not found"; + return Status::NotFound(msg.str()); } -Status ExternalScanContextMgr::clear_scan_context(const std::string& context_id) { +void ExternalScanContextMgr::clear_scan_context(const std::string& context_id) { std::shared_ptr context; { std::lock_guard l(_lock); @@ -89,9 +90,9 @@ Status ExternalScanContextMgr::clear_scan_context(const std::string& context_id) context = iter->second; if (context == nullptr) { _active_contexts.erase(context_id); - Status::OK(); + return; } - iter = _active_contexts.erase(iter); + _active_contexts.erase(iter); } } if (context != nullptr) { @@ -101,7 +102,6 @@ Status ExternalScanContextMgr::clear_scan_context(const std::string& context_id) _exec_env->result_queue_mgr()->cancel(context->fragment_instance_id); LOG(INFO) << "close scan context: context id [ " << context_id << " ]"; } - return Status::OK(); } void ExternalScanContextMgr::gc_expired_context() { @@ -111,7 +111,7 @@ void ExternalScanContextMgr::gc_expired_context() { std::vector> expired_contexts; { std::lock_guard l(_lock); - for(auto iter = _active_contexts.begin(); iter != _active_contexts.end(); ) { + for (auto iter = _active_contexts.begin(); iter != _active_contexts.end();) { auto context = iter->second; if (context == nullptr) { iter = _active_contexts.erase(iter); @@ -133,6 +133,7 @@ void ExternalScanContextMgr::gc_expired_context() { } } for (auto expired_context : expired_contexts) { + // TODO(yingchun): can e cancel it gracefully, like use ref count, cancle in ScanContext's destructor? // must cancel the fragment instance, otherwise return thrift transport TTransportException _exec_env->fragment_mgr()->cancel(expired_context->fragment_instance_id); _exec_env->result_queue_mgr()->cancel(expired_context->fragment_instance_id); @@ -141,3 +142,4 @@ void ExternalScanContextMgr::gc_expired_context() { #endif } } + diff --git a/be/src/runtime/external_scan_context_mgr.h b/be/src/runtime/external_scan_context_mgr.h index bc8665c9a79d7c8..1b8e29d99d93928 100644 --- a/be/src/runtime/external_scan_context_mgr.h +++ b/be/src/runtime/external_scan_context_mgr.h @@ -47,6 +47,7 @@ struct ScanContext { ScanContext(const TUniqueId& fragment_id, int64_t offset) : fragment_instance_id(fragment_id), offset(offset) {} }; +// TODO(yingchun): we can move this class object to BackendService, that's the only one place use it. class ExternalScanContextMgr { public: @@ -54,11 +55,11 @@ class ExternalScanContextMgr { ExternalScanContextMgr(ExecEnv* exec_env); ~ExternalScanContextMgr(); - Status create_scan_context(std::shared_ptr* p_context); + void create_scan_context(std::shared_ptr* p_context); Status get_scan_context(const std::string& context_id, std::shared_ptr* p_context); - Status clear_scan_context(const std::string& context_id); + void clear_scan_context(const std::string& context_id); private: ExecEnv* _exec_env; diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 6aecb843176c074..836f4e31a001d2e 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -87,9 +87,9 @@ class FragmentExecState { // just no use now void callback(const Status& status, RuntimeProfile* profile, bool done); - std::string to_http_path(const std::string& file_name); + std::string to_http_path(const std::string& file_name) const; - Status execute(); + void execute(); Status cancel_before_execute(); @@ -126,10 +126,7 @@ class FragmentExecState { if (_timeout_second <= 0) { return false; } - if (now.second_diff(_start_time) > _timeout_second) { - return true; - } - return false; + return now.second_diff(_start_time) > _timeout_second; } int get_timeout_second() const { return _timeout_second; } @@ -206,7 +203,7 @@ static void register_cgroups(const std::string& user, const std::string& group) CgroupsMgr::apply_cgroup(new_info->user, new_info->group); } -Status FragmentExecState::execute() { +void FragmentExecState::execute() { int64_t duration_ns = 0; { SCOPED_RAW_TIMER(&duration_ns); @@ -220,9 +217,9 @@ Status FragmentExecState::execute() { print_id(_fragment_instance_id))); _executor.close(); } + DorisMetrics::instance()->fragment_requests_total->increment(1); DorisMetrics::instance()->fragment_request_duration_us->increment(duration_ns / 1000); - return Status::OK(); } Status FragmentExecState::cancel_before_execute() { @@ -245,13 +242,11 @@ Status FragmentExecState::cancel(const PPlanFragmentCancelReason& reason) { void FragmentExecState::callback(const Status& status, RuntimeProfile* profile, bool done) { } -std::string FragmentExecState::to_http_path(const std::string& file_name) { - std::stringstream url; - url << "http://" << BackendOptions::get_localhost() << ":" << config::webserver_port - << "/api/_download_load?" - << "token=" << _exec_env->token() - << "&file=" << file_name; - return url.str(); +std::string FragmentExecState::to_http_path(const std::string& file_name) const { + static std::string base_path(std::string("http://") + BackendOptions::get_localhost() + + ":" + std::to_string(config::webserver_port) + + "/api/_download_load?token=" + _exec_env->token() + "&file="); + return base_path + file_name; } // There can only be one of these callbacks in-flight at any moment, because @@ -284,7 +279,7 @@ void FragmentExecState::coordinator_callback( params.__set_done(done); RuntimeState* runtime_state = _executor.runtime_state(); - DCHECK(runtime_state != NULL); + DCHECK(runtime_state != nullptr); if (runtime_state->query_options().query_type == TQueryType::LOAD && !done && status.ok()) { // this is a load plan, and load is not finished, just make a brief report params.__set_loaded_rows(runtime_state->num_rows_load_total()); @@ -352,7 +347,6 @@ void FragmentExecState::coordinator_callback( } catch (TTransportException& e) { LOG(WARNING) << "Retrying ReportExecStatus: " << e.what(); rpc_status = coord.reopen(); - if (!rpc_status.ok()) { // we need to cancel the execution of this fragment update_status(rpc_status); @@ -450,7 +444,6 @@ Status FragmentMgr::exec_plan_fragment( const TExecPlanFragmentParams& params, FinishCallback cb) { const TUniqueId& fragment_instance_id = params.params.fragment_instance_id; - std::shared_ptr exec_state; { std::lock_guard lock(_lock); auto iter = _fragment_map.find(fragment_instance_id); @@ -459,7 +452,8 @@ Status FragmentMgr::exec_plan_fragment( return Status::OK(); } } - exec_state.reset(new FragmentExecState( + std::shared_ptr exec_state( + new FragmentExecState( params.params.query_id, fragment_instance_id, params.backend_num, @@ -531,13 +525,13 @@ void FragmentMgr::cancel_worker() { LOG(INFO) << "FragmentMgr cancel worker is going to exit."; } -Status FragmentMgr::trigger_profile_report(const PTriggerProfileReportRequest* request) { - if (request->instance_ids_size() > 0) { - for (int i = 0; i < request->instance_ids_size(); i++) { - const PUniqueId& p_fragment_id = request->instance_ids(i); +void FragmentMgr::trigger_profile_report(const PTriggerProfileReportRequest* request) { + const auto& instance_ids = request->instance_ids(); + if (!instance_ids.empty()) { + for (const auto& instance_id : instance_ids) { TUniqueId id; - id.__set_hi(p_fragment_id.hi()); - id.__set_lo(p_fragment_id.lo()); + id.__set_hi(instance_id.hi()); + id.__set_lo(instance_id.lo()); { std::lock_guard lock(_lock); auto iter = _fragment_map.find(id); @@ -548,15 +542,12 @@ Status FragmentMgr::trigger_profile_report(const PTriggerProfileReportRequest* r } } else { std::lock_guard lock(_lock); - auto iter = _fragment_map.begin(); - for (; iter != _fragment_map.end(); iter++) { - iter->second->executor()->report_profile_once(); + for (const auto& iter : _fragment_map) { + iter.second->executor()->report_profile_once(); } } - return Status::OK(); } - void FragmentMgr::debug(std::stringstream& ss) { // Keep things simple std::lock_guard lock(_lock); @@ -576,7 +567,10 @@ void FragmentMgr::debug(std::stringstream& ss) { * 1. resolve opaqued_query_plan to thrift structure * 2. build TExecPlanFragmentParams */ -Status FragmentMgr::exec_external_plan_fragment(const TScanOpenParams& params, const TUniqueId& fragment_instance_id, std::vector* selected_columns) { +Status FragmentMgr::exec_external_plan_fragment( + const TScanOpenParams& params, + const TUniqueId& fragment_instance_id, + std::vector* selected_columns) { const std::string& opaqued_query_plan = params.opaqued_query_plan; std::string query_plan_info; // base64 decode query plan @@ -620,13 +614,13 @@ Status FragmentMgr::exec_external_plan_fragment(const TScanOpenParams& params, c TScanColumnDesc col; col.__set_name(slot->col_name()); col.__set_type(to_thrift(slot->type().type)); - selected_columns->emplace_back(std::move(col)); + selected_columns->emplace_back(col); } LOG(INFO) << "BackendService execute open() TQueryPlanInfo: " << apache::thrift::ThriftDebugString(t_query_plan_info); // assign the param used to execute PlanFragment TExecPlanFragmentParams exec_fragment_params; - exec_fragment_params.protocol_version = (PaloInternalServiceVersion::type)0; + exec_fragment_params.protocol_version = PaloInternalServiceVersion::type::V1; exec_fragment_params.__set_fragment(t_query_plan_info.plan_fragment); exec_fragment_params.__set_desc_tbl(t_query_plan_info.desc_tbl); @@ -640,7 +634,7 @@ Status FragmentMgr::exec_external_plan_fragment(const TScanOpenParams& params, c TNetworkAddress address; address.hostname = BackendOptions::get_localhost(); address.port = doris::config::be_port; - std::map tablet_info = t_query_plan_info.tablet_info; + const std::map& tablet_info = t_query_plan_info.tablet_info; for (auto tablet_id : params.tablet_ids) { TPaloScanRange scan_range; scan_range.db_name = params.database; @@ -663,7 +657,7 @@ Status FragmentMgr::exec_external_plan_fragment(const TScanOpenParams& params, c doris_scan_range.__set_palo_scan_range(scan_range); TScanRangeParams scan_range_params; scan_range_params.scan_range = doris_scan_range; - scan_ranges.push_back(scan_range_params); + scan_ranges.emplace_back(scan_range_params); } per_node_scan_ranges.insert(std::make_pair((::doris::TPlanNodeId)0, scan_ranges)); fragment_exec_params.per_node_scan_ranges = per_node_scan_ranges; diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h index 287664e63f0b95a..8dd532ea8d58c53 100644 --- a/be/src/runtime/fragment_mgr.h +++ b/be/src/runtime/fragment_mgr.h @@ -70,12 +70,14 @@ class FragmentMgr : public RestMonitorIface { virtual void debug(std::stringstream& ss); - Status trigger_profile_report(const PTriggerProfileReportRequest* request); + void trigger_profile_report(const PTriggerProfileReportRequest* request); // input: TScanOpenParams fragment_instance_id // output: selected_columns // execute external query, all query info are packed in TScanOpenParams - Status exec_external_plan_fragment(const TScanOpenParams& params, const TUniqueId& fragment_instance_id, std::vector* selected_columns); + Status exec_external_plan_fragment(const TScanOpenParams& params, + const TUniqueId& fragment_instance_id, + std::vector* selected_columns); private: void exec_actual(std::shared_ptr exec_state, diff --git a/be/src/runtime/load_channel.cpp b/be/src/runtime/load_channel.cpp index 72ace2357783f5f..8620ce470b8c469 100644 --- a/be/src/runtime/load_channel.cpp +++ b/be/src/runtime/load_channel.cpp @@ -65,7 +65,6 @@ Status LoadChannel::open(const PTabletWriterOpenRequest& params) { Status LoadChannel::add_batch( const PTabletWriterAddBatchRequest& request, google::protobuf::RepeatedPtrField* tablet_vec) { - int64_t index_id = request.index_id(); // 1. get tablets channel std::shared_ptr channel; @@ -93,7 +92,6 @@ Status LoadChannel::add_batch( } // 4. handle eos - Status st; if (request.has_eos() && request.eos()) { bool finished = false; RETURN_IF_ERROR(channel->close(request.sender_id(), &finished, @@ -105,7 +103,7 @@ Status LoadChannel::add_batch( } } _last_updated_time.store(time(nullptr)); - return st; + return Status::OK(); } void LoadChannel::handle_mem_exceed_limit(bool force) { diff --git a/be/src/runtime/load_channel_mgr.cpp b/be/src/runtime/load_channel_mgr.cpp index 5a20ec7d6ffd3e7..a49c73f2ba42842 100644 --- a/be/src/runtime/load_channel_mgr.cpp +++ b/be/src/runtime/load_channel_mgr.cpp @@ -44,7 +44,7 @@ static int64_t calc_process_max_load_memory(int64_t process_mem_limit) { static int64_t calc_job_max_load_memory(int64_t mem_limit_in_req, int64_t total_mem_limit) { // default mem limit is used to be compatible with old request. // new request should be set load_mem_limit. - const int64_t default_load_mem_limit = 2 * 1024 * 1024 * 1024L; // 2GB + static const int64_t default_load_mem_limit = 2 * 1024 * 1024 * 1024L; // 2GB int64_t load_mem_limit = default_load_mem_limit; if (mem_limit_in_req != -1) { // mem-limit of a certain load should between config::write_buffer_size @@ -80,11 +80,10 @@ LoadChannelMgr::~LoadChannelMgr() { delete _lastest_success_channel; } -Status LoadChannelMgr::init(int64_t process_mem_limit) { +void LoadChannelMgr::init(int64_t process_mem_limit) { int64_t load_mem_limit = calc_process_max_load_memory(process_mem_limit); _mem_tracker = MemTracker::CreateTracker(load_mem_limit, "load channel mgr"); - RETURN_IF_ERROR(_start_bg_worker()); - return Status::OK(); + _start_bg_worker(); } Status LoadChannelMgr::open(const PTabletWriterOpenRequest& params) { @@ -159,6 +158,7 @@ Status LoadChannelMgr::add_batch( _load_channels.erase(load_id); auto handle = _lastest_success_channel->insert( load_id.to_string(), nullptr, 1, dummy_deleter); + // TODO(yingchun): why release after insert? _lastest_success_channel->release(handle); } VLOG(1) << "removed load channel " << load_id; @@ -195,14 +195,15 @@ void LoadChannelMgr::_handle_mem_exceed_limit() { channel->handle_mem_exceed_limit(true); } -Status LoadChannelMgr::cancel(const PTabletWriterCancelRequest& params) { +void LoadChannelMgr::cancel(const PTabletWriterCancelRequest& params) { UniqueId load_id(params.id()); std::shared_ptr cancelled_channel; { std::lock_guard l(_lock); - if (_load_channels.find(load_id) != _load_channels.end()) { - cancelled_channel = _load_channels[load_id]; - _load_channels.erase(load_id); + const auto& it = _load_channels.find(load_id); + if (it != _load_channels.end()) { + cancelled_channel = it->second; + _load_channels.erase(it); } } @@ -210,12 +211,10 @@ Status LoadChannelMgr::cancel(const PTabletWriterCancelRequest& params) { cancelled_channel->cancel(); LOG(INFO) << "load channel has been cancelled: " << load_id; } - - return Status::OK(); } -Status LoadChannelMgr::_start_bg_worker() { - RETURN_IF_ERROR( +void LoadChannelMgr::_start_bg_worker() { + CHECK( Thread::create("LoadChannelMgr", "cancel_timeout_load_channels", [this]() { #ifdef GOOGLE_PROFILER @@ -229,33 +228,28 @@ Status LoadChannelMgr::_start_bg_worker() { while (!_stop_background_threads_latch.wait_for(MonoDelta::FromSeconds(interval))) { _start_load_channels_clean(); }}, - &_load_channels_clean_thread)); - - return Status::OK(); + &_load_channels_clean_thread).ok()); } -Status LoadChannelMgr::_start_load_channels_clean() { +void LoadChannelMgr::_start_load_channels_clean() { std::vector> need_delete_channels; LOG(INFO) << "cleaning timed out load channels"; time_t now = time(nullptr); { - std::vector need_delete_channel_ids; std::lock_guard l(_lock); VLOG(1) << "there are " << _load_channels.size() << " running load channels"; int i = 0; - for (auto& kv : _load_channels) { - VLOG(1) << "load channel[" << i++ << "]: " << *(kv.second); - time_t last_updated_time = kv.second->last_updated_time(); - if (difftime(now, last_updated_time) >= kv.second->timeout()) { - need_delete_channel_ids.emplace_back(kv.first); - need_delete_channels.emplace_back(kv.second); + for (auto it = _load_channels.begin(); it != _load_channels.end();) { + VLOG(1) << "load channel[" << i++ << "]: " << *(it->second); + time_t last_updated_time = it->second->last_updated_time(); + if (difftime(now, last_updated_time) >= it->second->timeout()) { + need_delete_channels.push_back(it->second); + LOG(INFO) << "erase timeout load channel: " << it->first; + it = _load_channels.erase(it); + } else { + ++it; } } - - for(auto& key: need_delete_channel_ids) { - _load_channels.erase(key); - LOG(INFO) << "erase timeout load channel: " << key; - } } // we must cancel these load channels before destroying them. @@ -272,8 +266,6 @@ Status LoadChannelMgr::_start_load_channels_clean() { LOG(INFO) << "load mem consumption(bytes). limit: " << _mem_tracker->limit() << ", current: " << _mem_tracker->consumption() << ", peak: " << _mem_tracker->peak_consumption(); - - return Status::OK(); } } diff --git a/be/src/runtime/load_channel_mgr.h b/be/src/runtime/load_channel_mgr.h index e506e192bb5f856..cb90e2fdc3d41c0 100644 --- a/be/src/runtime/load_channel_mgr.h +++ b/be/src/runtime/load_channel_mgr.h @@ -45,7 +45,7 @@ class LoadChannelMgr { LoadChannelMgr(); ~LoadChannelMgr(); - Status init(int64_t process_mem_limit); + void init(int64_t process_mem_limit); // open a new load channel if not exist Status open(const PTabletWriterOpenRequest& request); @@ -55,15 +55,16 @@ class LoadChannelMgr { int64_t* wait_lock_time_ns); // cancel all tablet stream for 'load_id' load - Status cancel(const PTabletWriterCancelRequest& request); - + void cancel(const PTabletWriterCancelRequest& request); private: // check if the total load mem consumption exceeds limit. // If yes, it will pick a load channel to try to reduce memory consumption. void _handle_mem_exceed_limit(); - Status _start_bg_worker(); + void _start_bg_worker(); + + void _start_load_channels_clean(); private: // lock protect the load channel map @@ -78,7 +79,6 @@ class LoadChannelMgr { CountDownLatch _stop_background_threads_latch; // thread to clean timeout load channels scoped_refptr _load_channels_clean_thread; - Status _start_load_channels_clean(); }; } diff --git a/be/src/runtime/load_path_mgr.cpp b/be/src/runtime/load_path_mgr.cpp index d7e637cdd5d2377..243ce9379a162ae 100644 --- a/be/src/runtime/load_path_mgr.cpp +++ b/be/src/runtime/load_path_mgr.cpp @@ -47,6 +47,7 @@ LoadPathMgr::~LoadPathMgr() { } Status LoadPathMgr::init() { + // TODO(yingchun): can only init once? we can remove _lock? _path_vec.clear(); for (auto& path : _exec_env->store_paths()) { _path_vec.push_back(path.path + MINI_PREFIX); @@ -121,18 +122,17 @@ bool LoadPathMgr::is_too_old(time_t cur_time, const std::string& label_dir, int6 void LoadPathMgr::get_load_data_path(std::vector* data_paths) { data_paths->insert(data_paths->end(), _path_vec.begin(), _path_vec.end()); - return; } const std::string ERROR_FILE_NAME = "error_log"; Status LoadPathMgr::get_load_error_file_name( const std::string& db, - const std::string&label, + const std::string& label, const TUniqueId& fragment_instance_id, std::string* error_path) { std::stringstream ss; - std::string shard = ""; + std::string shard; { std::lock_guard l(_lock); shard = SHARD_PREFIX + std::to_string(_error_path_next_shard++ % MAX_SHARD_NUM); @@ -152,14 +152,11 @@ Status LoadPathMgr::get_load_error_file_name( } std::string LoadPathMgr::get_load_error_absolute_path(const std::string& file_path) { - std::string path; - path.append(_error_log_dir); - path.append("/"); - path.append(file_path); - return path; + return _error_log_dir + "/" + file_path; } void LoadPathMgr::process_path(time_t now, const std::string& path, int64_t reserve_hours) { + // TODO(yingchun): 只用时间判断,但如果有一个文件一直在使用呢? if (!is_too_old(now, path, reserve_hours)) { return; } diff --git a/be/src/runtime/load_path_mgr.h b/be/src/runtime/load_path_mgr.h index fd7fe6f4bc45e11..39e2d4dfa2e360b 100644 --- a/be/src/runtime/load_path_mgr.h +++ b/be/src/runtime/load_path_mgr.h @@ -48,7 +48,7 @@ class LoadPathMgr { Status get_load_error_file_name( const std::string& db, - const std::string&label, + const std::string& label, const TUniqueId& fragment_instance_id, std::string* error_path); std::string get_load_error_absolute_path(const std::string& file_path); diff --git a/be/src/runtime/memory/chunk.h b/be/src/runtime/memory/chunk.h index 703489a4530f1c3..8bc958009788b58 100644 --- a/be/src/runtime/memory/chunk.h +++ b/be/src/runtime/memory/chunk.h @@ -27,9 +27,9 @@ namespace doris { // will result in recompilation of all files. So, we put it in a // file to keep this file simple and infrequently changed. struct Chunk { - uint8_t* data; - size_t size; - int core_id; + uint8_t* data = nullptr; + size_t size = 0; + int core_id = 0; }; } diff --git a/be/src/runtime/memory/chunk_allocator.h b/be/src/runtime/memory/chunk_allocator.h index 38f84bbf1f47c26..2347cc2ad9f9600 100644 --- a/be/src/runtime/memory/chunk_allocator.h +++ b/be/src/runtime/memory/chunk_allocator.h @@ -36,7 +36,7 @@ class MetricEntity; // ChunkAllocator has one ChunkArena for each CPU core, it will try to allocate // memory from current core arena firstly. In this way, there will be no lock contention // between concurrently-running threads. If this fails, ChunkAllocator will try to allocate -// memroy from other core's arena. +// memory from other core's arena. // // Memory Reservation // ChunkAllocator has a limit about how much free chunk bytes it can reserve, above which diff --git a/be/src/runtime/result_buffer_mgr.cpp b/be/src/runtime/result_buffer_mgr.cpp index 8983298be288a1f..9ec039c8b8dcb69 100644 --- a/be/src/runtime/result_buffer_mgr.cpp +++ b/be/src/runtime/result_buffer_mgr.cpp @@ -52,51 +52,47 @@ ResultBufferMgr::~ResultBufferMgr() { } } -Status ResultBufferMgr::init() { - RETURN_IF_ERROR(Thread::create("ResultBufferMgr", "cancel_timeout_result", +void ResultBufferMgr::init() { + CHECK(Thread::create("ResultBufferMgr", "cancel_timeout_result", [this]() { this->cancel_thread(); }, - &_clean_thread)); - return Status::OK(); + &_clean_thread).ok()); } -Status ResultBufferMgr::create_sender( +void ResultBufferMgr::create_sender( const TUniqueId& query_id, int buffer_size, boost::shared_ptr* sender) { *sender = find_control_block(query_id); if (*sender != nullptr) { LOG(WARNING) << "already have buffer control block for this instance " << query_id; - return Status::OK(); + return; } - boost::shared_ptr control_block( - new BufferControlBlock(query_id, buffer_size)); + boost::shared_ptr control_block(new BufferControlBlock(query_id, buffer_size)); { boost::lock_guard l(_lock); _buffer_map.insert(std::make_pair(query_id, control_block)); } *sender = control_block; - return Status::OK(); } boost::shared_ptr ResultBufferMgr::find_control_block( const TUniqueId& query_id) { // TODO(zhaochun): this lock can be bottleneck? + // TODO(yingchun): maybe we can shard it boost::lock_guard l(_lock); - BufferMap::iterator iter = _buffer_map.find(query_id); - + auto iter = _buffer_map.find(query_id); if (_buffer_map.end() != iter) { return iter->second; } - return boost::shared_ptr(); + return nullptr; } Status ResultBufferMgr::fetch_data( const TUniqueId& query_id, TFetchDataResult* result) { boost::shared_ptr cb = find_control_block(query_id); - - if (NULL == cb) { + if (nullptr == cb) { // the sender tear down its buffer block return Status::InternalError("no result for this query."); } @@ -117,22 +113,18 @@ void ResultBufferMgr::fetch_data(const PUniqueId& finst_id, GetResultBatchCtx* c cb->get_batch(ctx); } -Status ResultBufferMgr::cancel(const TUniqueId& query_id) { +void ResultBufferMgr::cancel(const TUniqueId& query_id) { boost::lock_guard l(_lock); BufferMap::iterator iter = _buffer_map.find(query_id); - if (_buffer_map.end() != iter) { iter->second->cancel(); _buffer_map.erase(iter); } - - return Status::OK(); } -Status ResultBufferMgr::cancel_at_time(time_t cancel_time, const TUniqueId& query_id) { +void ResultBufferMgr::cancel_at_time(time_t cancel_time, const TUniqueId& query_id) { boost::lock_guard l(_timeout_lock); TimeoutMap::iterator iter = _timeout_map.find(cancel_time); - if (_timeout_map.end() == iter) { _timeout_map.insert(std::pair>( cancel_time, std::vector())); @@ -140,7 +132,6 @@ Status ResultBufferMgr::cancel_at_time(time_t cancel_time, const TUniqueId& quer } iter->second.push_back(query_id); - return Status::OK(); } void ResultBufferMgr::cancel_thread() { @@ -153,19 +144,15 @@ void ResultBufferMgr::cancel_thread() { { boost::lock_guard l(_timeout_lock); TimeoutMap::iterator end = _timeout_map.upper_bound(now_time + 1); - for (TimeoutMap::iterator iter = _timeout_map.begin(); iter != end; ++iter) { - for (int i = 0; i < iter->second.size(); ++i) { - query_to_cancel.push_back(iter->second[i]); - } + query_to_cancel.insert(query_to_cancel.end(), iter->second.begin(), iter->second.end()); } - _timeout_map.erase(_timeout_map.begin(), end); } // cancel query - for (int i = 0; i < query_to_cancel.size(); ++i) { - cancel(query_to_cancel[i]); + for (const auto& it : query_to_cancel) { + cancel(it); } } while (!_stop_background_threads_latch.wait_for(MonoDelta::FromSeconds(1))); diff --git a/be/src/runtime/result_buffer_mgr.h b/be/src/runtime/result_buffer_mgr.h index 07b69bdc586a185..9b35f2b01052e7a 100644 --- a/be/src/runtime/result_buffer_mgr.h +++ b/be/src/runtime/result_buffer_mgr.h @@ -45,22 +45,21 @@ class ResultBufferMgr { ResultBufferMgr(); ~ResultBufferMgr(); // init Result Buffer Mgr, start cancel thread - Status init(); + void init(); // create one result sender for this query_id // the returned sender do not need release // sender is not used when call cancel or unregister - Status create_sender(const TUniqueId& query_id, int buffer_size, - boost::shared_ptr* sender); + void create_sender(const TUniqueId& query_id, int buffer_size, + boost::shared_ptr* sender); // fetch data, used by RPC Status fetch_data(const TUniqueId& fragment_id, TFetchDataResult* result); void fetch_data(const PUniqueId& finst_id, GetResultBatchCtx* ctx); - // cancel - Status cancel(const TUniqueId& fragment_id); + void cancel(const TUniqueId& fragment_id); // cancel one query at a future time. - Status cancel_at_time(time_t cancel_time, const TUniqueId& query_id); + void cancel_at_time(time_t cancel_time, const TUniqueId& query_id); private: typedef boost::unordered_map> BufferMap; diff --git a/be/src/runtime/result_queue_mgr.cpp b/be/src/runtime/result_queue_mgr.cpp index cfdd60ab014f5f6..f2db921469d0de5 100644 --- a/be/src/runtime/result_queue_mgr.cpp +++ b/be/src/runtime/result_queue_mgr.cpp @@ -56,21 +56,20 @@ Status ResultQueueMgr::fetch_result(const TUniqueId& fragment_instance_id, std:: } // check queue status before get result RETURN_IF_ERROR(queue->status()); - bool sucess = queue->blocking_get(result); - if (sucess) { - // sentinel nullptr indicates scan end - if (*result == nullptr) { - *eos = true; - // put sentinel for consistency, avoid repeated invoking fetch result when hava no rowbatch - if (queue != nullptr) { - queue->blocking_put(nullptr); - } - } else { - *eos = false; - } - } else { + if (!queue->blocking_get(result)) { *eos = true; + return Status::OK(); } + + // sentinel nullptr indicates scan end + if (*result == nullptr) { + *eos = true; + // put sentinel for consistency, avoid repeated invoking fetch result when hava no rowbatch + queue->blocking_put(nullptr); + return Status::OK(); + } + + *eos = false; return Status::OK(); } @@ -87,17 +86,19 @@ void ResultQueueMgr::create_queue(const TUniqueId& fragment_instance_id, BlockQu } } -Status ResultQueueMgr::cancel(const TUniqueId& fragment_instance_id) { - std::lock_guard l(_lock); - auto iter = _fragment_queue_map.find(fragment_instance_id); - if (iter != _fragment_queue_map.end()) { - // first remove RecordBatch from queue - // avoid MemoryScratchSink block on send or close operation - iter->second->shutdown(); - // remove this queue from map - _fragment_queue_map.erase(fragment_instance_id); +void ResultQueueMgr::cancel(const TUniqueId& fragment_instance_id) { + BlockQueueSharedPtr queue; + { + std::lock_guard l(_lock); + auto iter = _fragment_queue_map.find(fragment_instance_id); + if (iter != _fragment_queue_map.end()) { + queue = iter->second; + _fragment_queue_map.erase(fragment_instance_id); + } + } + if (queue != nullptr) { + queue->shutdown(); } - return Status::OK(); } void ResultQueueMgr::update_queue_status(const TUniqueId& fragment_instance_id, const Status& status) { diff --git a/be/src/runtime/result_queue_mgr.h b/be/src/runtime/result_queue_mgr.h index 9ceba7922c887ef..a83016ff7aa78b3 100644 --- a/be/src/runtime/result_queue_mgr.h +++ b/be/src/runtime/result_queue_mgr.h @@ -51,7 +51,7 @@ class ResultQueueMgr { void create_queue(const TUniqueId& fragment_instance_id, BlockQueueSharedPtr* queue); - Status cancel(const TUniqueId& fragment_id); + void cancel(const TUniqueId& fragment_id); void update_queue_status(const TUniqueId& fragment_id, const Status& status); diff --git a/be/src/runtime/result_sink.cpp b/be/src/runtime/result_sink.cpp index 0a39f0672dbda1b..e26c0b58dd08028 100644 --- a/be/src/runtime/result_sink.cpp +++ b/be/src/runtime/result_sink.cpp @@ -49,8 +49,7 @@ ResultSink::ResultSink(const RowDescriptor& row_desc, const std::vector& } } -ResultSink::~ResultSink() { -} +ResultSink::~ResultSink() {} Status ResultSink::prepare_exprs(RuntimeState* state) { // From the thrift expressions create the real exprs. @@ -71,8 +70,9 @@ Status ResultSink::prepare(RuntimeState* state) { RETURN_IF_ERROR(prepare_exprs(state)); // create sender - RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender( - state->fragment_instance_id(), _buf_size, &_sender)); + state->exec_env()->result_mgr()->create_sender( + state->fragment_instance_id(), _buf_size, &_sender); + // create writer based on sink type switch (_sink_type) { diff --git a/be/src/runtime/result_sink.h b/be/src/runtime/result_sink.h index bf732b1c5d44573..7f2431d66ca1b50 100644 --- a/be/src/runtime/result_sink.h +++ b/be/src/runtime/result_sink.h @@ -35,7 +35,7 @@ class BufferControlBlock; class ExprContext; class ResultWriter; class MemTracker; -class ResultFileOptions; +struct ResultFileOptions; class ResultSink : public DataSink { public: @@ -44,7 +44,7 @@ class ResultSink : public DataSink { // buffer_size is the buffer size allocated to each query ResultSink(const RowDescriptor& row_desc, const std::vector& select_exprs, const TResultSink& sink, int buffer_size); - virtual ~ResultSink(); + ~ResultSink() override; virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); // send data in 'batch' to this backend stream mgr diff --git a/be/src/runtime/routine_load/data_consumer_group.cpp b/be/src/runtime/routine_load/data_consumer_group.cpp index d900eb5bef066e9..5845cee5ddb4329 100644 --- a/be/src/runtime/routine_load/data_consumer_group.cpp +++ b/be/src/runtime/routine_load/data_consumer_group.cpp @@ -68,11 +68,9 @@ Status KafkaDataConsumerGroup::start_all(StreamLoadContext* ctx) { for(auto& consumer : _consumers) { if (!_thread_pool.offer( boost::bind(&KafkaDataConsumerGroup::actual_consume, this, consumer, &_queue, ctx->max_interval_s * 1000, - [this, &result_st] (const Status& st) { - std::unique_lock lock(_mutex); - _counter--; - VLOG(1) << "group counter is: " << _counter << ", grp: " << _grp_id; - if (_counter == 0) { + [this, &result_st] (const Status& st) { + VLOG(1) << "group counter is: " << _counter.load() << ", grp: " << _grp_id; + if (_counter.fetch_sub(1) == 0) { _queue.shutdown(); LOG(INFO) << "all consumers are finished. shutdown queue. group id: " << _grp_id; } diff --git a/be/src/runtime/routine_load/data_consumer_group.h b/be/src/runtime/routine_load/data_consumer_group.h index 9083835f9098de3..ff6778051e72fdf 100644 --- a/be/src/runtime/routine_load/data_consumer_group.h +++ b/be/src/runtime/routine_load/data_consumer_group.h @@ -48,8 +48,8 @@ class DataConsumerGroup { void add_consumer(std::shared_ptr consumer) { consumer->set_grp(_grp_id); - _consumers.push_back(consumer); ++_counter; + _consumers.push_back(consumer); } // start all consumers @@ -60,12 +60,10 @@ class DataConsumerGroup { std::vector> _consumers; // thread pool to run each consumer in multi thread PriorityThreadPool _thread_pool; - // mutex to protect counter. // the counter is init as the number of consumers. // once a consumer is done, decrease the counter. // when the counter becomes zero, shutdown the queue to finish - std::mutex _mutex; - int _counter; + std::atomic _counter; }; // for kafka diff --git a/be/src/runtime/routine_load/data_consumer_pool.cpp b/be/src/runtime/routine_load/data_consumer_pool.cpp index 3fb80459453f45f..6bc89db405c6ea2 100644 --- a/be/src/runtime/routine_load/data_consumer_pool.cpp +++ b/be/src/runtime/routine_load/data_consumer_pool.cpp @@ -29,14 +29,14 @@ Status DataConsumerPool::get_consumer( // check if there is an available consumer. // if has, return it, also remove it from the pool - auto iter = std::begin(_pool); - while (iter != std::end(_pool)) { + auto iter = _pool.begin(); + while (iter != _pool.end()) { if ((*iter)->match(ctx)) { VLOG(3) << "get an available data consumer from pool: " << (*iter)->id(); (*iter)->reset(); *ret = *iter; - iter = _pool.erase(iter); - return Status::OK(); + _pool.erase(iter); + return Status::OK(); } else { ++iter; } @@ -105,13 +105,13 @@ void DataConsumerPool::return_consumer(std::shared_ptr consumer) { } void DataConsumerPool::return_consumers(DataConsumerGroup* grp) { - for (std::shared_ptr consumer : grp->consumers()) { + for (auto& consumer : grp->consumers()) { return_consumer(consumer); } } -Status DataConsumerPool::start_bg_worker() { - RETURN_IF_ERROR(Thread::create("ResultBufferMgr", "clean_idle_consumer", +void DataConsumerPool::start_bg_worker() { + CHECK(Thread::create("ResultBufferMgr", "clean_idle_consumer", [this]() { #ifdef GOOGLE_PROFILER ProfilerRegisterThread(); @@ -120,8 +120,7 @@ Status DataConsumerPool::start_bg_worker() { _clean_idle_consumer_bg(); } while (!_stop_background_threads_latch.wait_for(MonoDelta::FromSeconds(60))); }, - &_clean_idle_consumer_thread)); - return Status::OK(); + &_clean_idle_consumer_thread).ok()); } void DataConsumerPool::_clean_idle_consumer_bg() { @@ -130,8 +129,8 @@ void DataConsumerPool::_clean_idle_consumer_bg() { std::unique_lock l(_lock); time_t now = time(nullptr); - auto iter = std::begin(_pool); - while (iter != std::end(_pool)) { + auto iter = _pool.begin(); + while (iter != _pool.end()) { if (difftime(now, (*iter)->last_visit_time()) >= max_idle_time_second) { LOG(INFO) << "remove data consumer " << (*iter)->id() << ", since it last visit: " << (*iter)->last_visit_time() diff --git a/be/src/runtime/routine_load/data_consumer_pool.h b/be/src/runtime/routine_load/data_consumer_pool.h index 05dc01538042723..d182321950f5cd7 100644 --- a/be/src/runtime/routine_load/data_consumer_pool.h +++ b/be/src/runtime/routine_load/data_consumer_pool.h @@ -66,12 +66,13 @@ class DataConsumerPool { // return the consumers in consumer group to the pool void return_consumers(DataConsumerGroup* grp); - Status start_bg_worker(); + void start_bg_worker(); private: void _clean_idle_consumer_bg(); private: + // Protect _pool. std::mutex _lock; std::list> _pool; int64_t _max_pool_size; diff --git a/be/src/runtime/routine_load/routine_load_task_executor.cpp b/be/src/runtime/routine_load/routine_load_task_executor.cpp index 343825153aaaa15..87a3aaec21a8f24 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.cpp +++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp @@ -105,8 +105,8 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { return Status::OK(); } - // thread pool's queue size > 0 means there are tasks waiting to be executed, so no more tasks should be submitted. - if (_thread_pool.get_queue_size() > 0) { + // thread pool's queue not empty means there are tasks waiting to be executed, so no more tasks should be submitted. + if (!_thread_pool.empty()) { LOG(INFO) << "too many tasks in thread pool. reject task: " << UniqueId(task.id); return Status::TooManyTasks(UniqueId(task.id).to_string()); } @@ -138,7 +138,8 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { if(task.__isset.format) { ctx->format = task.format; } - // the routine load task'txn has alreay began in FE. + // the routine load task's txn has alreay began in FE. + // so it need to rollback if encounter error. ctx->need_rollback = true; ctx->max_filter_ratio = 1.0; @@ -155,6 +156,7 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { } VLOG(1) << "receive a new routine load task: " << ctx->brief(); + // TODO(yingchun): use shared_ptr would be better? // register the task ctx->ref(); _task_map[ctx->id] = ctx; @@ -259,6 +261,7 @@ void RoutineLoadTaskExecutor::exec_task( // commit txn HANDLE_ERROR(_exec_env->stream_load_executor()->commit_txn(ctx), "commit failed"); + // TODO(yingchun): _data_consumer_pool and consumer_pool are the same // commit kafka offset switch (ctx->load_src_type) { case TLoadSourceType::KAFKA: { @@ -292,8 +295,8 @@ void RoutineLoadTaskExecutor::exec_task( [](RdKafka::TopicPartition* tp1) { delete tp1; }); }; DeferOp delete_tp(std::bind(tp_deleter)); - } break; + } default: return; } @@ -304,7 +307,6 @@ void RoutineLoadTaskExecutor::err_handler( StreamLoadContext* ctx, const Status& st, const std::string& err_msg) { - LOG(WARNING) << err_msg; ctx->status = st; if (ctx->need_rollback) { diff --git a/be/src/runtime/sorted_run_merger.cc b/be/src/runtime/sorted_run_merger.cc index 60ed6d86b57ee07..121af6dbb5cd114 100644 --- a/be/src/runtime/sorted_run_merger.cc +++ b/be/src/runtime/sorted_run_merger.cc @@ -164,6 +164,7 @@ Status SortedRunMerger::get_next(RowBatch* output_batch, bool* eos) { return Status::OK(); } + // TODO(yingchun): optimize to copy batch by batch, not row by row while (!output_batch->at_capacity()) { BatchedRowSupplier* min = _min_heap[0]; int output_row_index = output_batch->add_row(); @@ -191,6 +192,7 @@ Status SortedRunMerger::get_next(RowBatch* output_batch, bool* eos) { if (_min_heap.empty()) break; } + // TODO(yingchun): need reorder when min is not swaped? heapify(0); } diff --git a/be/src/runtime/sorted_run_merger.h b/be/src/runtime/sorted_run_merger.h index c9d24b913f4ba28..22ada122234a8d8 100644 --- a/be/src/runtime/sorted_run_merger.h +++ b/be/src/runtime/sorted_run_merger.h @@ -61,10 +61,6 @@ class SortedRunMerger { // Return the next batch of sorted rows from this merger. Status get_next(RowBatch* output_batch, bool* eos); - // Called to finalize a merge when deep_copy is false. Transfers resources from - // all input batches to the specified output batch. - void transfer_all_resources(RowBatch* transfer_resource_batch); - private: class BatchedRowSupplier; diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp index 7cab4febce23a47..d3be5693e56cc3e 100644 --- a/be/src/runtime/tablets_channel.cpp +++ b/be/src/runtime/tablets_channel.cpp @@ -63,10 +63,9 @@ Status TabletsChannel::open(const PTabletWriterOpenRequest& params) { _txn_id = params.txn_id(); _index_id = params.index_id(); _schema = new OlapTableSchemaParam(); - RETURN_IF_ERROR(_schema->init(params.schema())); + _schema->init(params.schema()); _tuple_desc = _schema->tuple_desc(); _row_desc = new RowDescriptor(_tuple_desc, false); - _num_remaining_senders = params.num_senders(); _next_seqs.resize(_num_remaining_senders, 0); _closed_senders.Reset(_num_remaining_senders); @@ -86,6 +85,7 @@ Status TabletsChannel::add_batch(const PTabletWriterAddBatchRequest& params) { : Status::InternalError(strings::Substitute("TabletsChannel $0 state: $1", _key.to_string(), _state)); } + DCHECK_LT(params.sender_id(), _next_seqs.size()); auto next_seq = _next_seqs[params.sender_id()]; // check packet if (params.packet_seq() < next_seq) { @@ -104,10 +104,11 @@ Status TabletsChannel::add_batch(const PTabletWriterAddBatchRequest& params) { for (int i = 0; i < params.tablet_ids_size(); ++i) { auto tablet_id = params.tablet_ids(i); auto it = _tablet_writers.find(tablet_id); - if (it == std::end(_tablet_writers)) { + if (it == _tablet_writers.end()) { return Status::InternalError(strings::Substitute( "unknown tablet to append data, tablet=$0", tablet_id)); } + // TODO(yingchun): now write in tablets are in sequential, we can optimize to write concurrently auto st = it->second->write(row_batch.get_row(i)->get_tuple(0)); if (st != OLAP_SUCCESS) { const std::string& err_msg = strings::Substitute( @@ -147,11 +148,13 @@ Status TabletsChannel::close(int sender_id, bool* finished, std::vector need_wait_writers; for (auto& it : _tablet_writers) { if (_partition_ids.count(it.second->partition_id()) > 0) { + // TODO(yingchun): rename it to async_flush? auto st = it.second->close(); if (st != OLAP_SUCCESS) { LOG(WARNING) << "close tablet writer failed, tablet_id=" << it.first << ", transaction_id=" << _txn_id << ", err=" << st; // just skip this tablet(writer) and continue to close others + // TODO(yingchun): we need a deterministic result, should not continue simplely continue; } need_wait_writers.push_back(it.second); @@ -227,29 +230,21 @@ Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& params) ss << "unknown index id, key=" << _key; return Status::InternalError(ss.str()); } + + WriteRequest request; + request.schema_hash = schema_hash; + request.write_type = WriteType::LOAD; + request.txn_id = _txn_id; + request.load_id = params.id(); + request.need_gen_rollup = params.need_gen_rollup(); + request.tuple_desc = _tuple_desc; + request.slots = index_slots; for (auto& tablet : params.tablets()) { - WriteRequest request; request.tablet_id = tablet.tablet_id(); - request.schema_hash = schema_hash; - request.write_type = WriteType::LOAD; - request.txn_id = _txn_id; request.partition_id = tablet.partition_id(); - request.load_id = params.id(); - request.need_gen_rollup = params.need_gen_rollup(); - request.tuple_desc = _tuple_desc; - request.slots = index_slots; DeltaWriter* writer = nullptr; - auto st = DeltaWriter::open(&request, _mem_tracker, &writer); - if (st != OLAP_SUCCESS) { - std::stringstream ss; - ss << "open delta writer failed, tablet_id=" << tablet.tablet_id() - << ", txn_id=" << _txn_id - << ", partition_id=" << tablet.partition_id() - << ", err=" << st; - LOG(WARNING) << ss.str(); - return Status::InternalError(ss.str()); - } + DeltaWriter::open(request, _mem_tracker, &writer); _tablet_writers.emplace(tablet.tablet_id(), writer); } _s_tablet_writer_count += _tablet_writers.size(); diff --git a/be/src/runtime/tablets_channel.h b/be/src/runtime/tablets_channel.h index 6e7851152b508b4..cb34bcefd0f8917 100644 --- a/be/src/runtime/tablets_channel.h +++ b/be/src/runtime/tablets_channel.h @@ -116,6 +116,7 @@ class TabletsChannel { Bitmap _closed_senders; // status to return when operate on an already closed/cancelled channel // currently it's OK. + // TODO(yingchun): always OK, should remove it Status _close_status; // tablet_id -> TabletChannel diff --git a/be/src/service/backend_options.cpp b/be/src/service/backend_options.cpp index 6935920c461fae4..d8fda70f57986c9 100644 --- a/be/src/service/backend_options.cpp +++ b/be/src/service/backend_options.cpp @@ -40,7 +40,6 @@ bool BackendOptions::init() { } std::vector hosts; Status status = get_hosts_v4(&hosts); - if (!status.ok()) { LOG(FATAL) << status.get_error_msg(); return false; @@ -52,21 +51,22 @@ bool BackendOptions::init() { } std::string loopback; - std::vector::iterator addr_it = hosts.begin(); - for (; addr_it != hosts.end(); ++addr_it) { - if ((*addr_it).is_address_v4()) { - VLOG(2) << "check ip=" << addr_it->get_host_address_v4(); - if ((*addr_it).is_loopback_v4()) { - loopback = addr_it->get_host_address_v4(); - } else if (!_s_priority_cidrs.empty()) { - if (is_in_prior_network(addr_it->get_host_address_v4())) { - _s_localhost = addr_it->get_host_address_v4(); - break; - } - } else { - _s_localhost = addr_it->get_host_address_v4(); + for (auto& addr_it : hosts) { + if (!addr_it.is_address_v4()) { + continue; + } + + VLOG(2) << "check ip=" << addr_it.get_host_address_v4(); + if (addr_it.is_loopback_v4()) { + loopback = addr_it.get_host_address_v4(); + } else if (!_s_priority_cidrs.empty()) { + if (is_in_prior_network(addr_it.get_host_address_v4())) { + _s_localhost = addr_it.get_host_address_v4(); break; } + } else { + _s_localhost = addr_it.get_host_address_v4(); + break; } } diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index 09f22dd1146c1ad..71e7e3eb0dd127d 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -70,6 +70,7 @@ using apache::thrift::concurrency::PosixThreadFactory; BackendService::BackendService(ExecEnv* exec_env) : _exec_env(exec_env), _agent_server(new AgentServer(exec_env, *exec_env->master_info())) { + // TODO(yingchun): waht does this used for? char buf[64]; DateTimeValue value = DateTimeValue::local_time(); value.to_string(buf); @@ -311,11 +312,8 @@ void BackendService::get_next(TScanBatchResult& result_, const TScanNextBatchPar } void BackendService::close_scanner(TScanCloseResult& result_, const TScanCloseParams& params) { - std::string context_id = params.context_id; - TStatus t_status; - Status st = _exec_env->external_scan_context_mgr()->clear_scan_context(context_id); - st.to_thrift(&t_status); - result_.status = t_status; + _exec_env->external_scan_context_mgr()->clear_scan_context(params.context_id); + result_.status.__set_status_code(TStatusCode::type::OK); } } // namespace doris diff --git a/be/src/service/backend_service.h b/be/src/service/backend_service.h index a2b9f3a60b3a8aa..1a0c8498a7919d2 100644 --- a/be/src/service/backend_service.h +++ b/be/src/service/backend_service.h @@ -61,7 +61,7 @@ class TExportTaskRequest; class TExportStatusResult; -// This class just forword rpc for actual handler +// This class just forward rpc for actual handler // make this class because we can bind multiple service on single point class BackendService : public BackendServiceIf { public: diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 68dfdfd63813ada..273843b7ee59304 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -130,12 +130,7 @@ void PInternalServiceImpl::tablet_writer_cancel(google::protobuf::RpcControll << ", index_id=" << request->index_id() << ", sender_id=" << request->sender_id(); brpc::ClosureGuard closure_guard(done); - auto st = _exec_env->load_channel_mgr()->cancel(*request); - if (!st.ok()) { - LOG(WARNING) << "tablet writer cancel failed, id=" << request->id() - << ", index_id=" << request->index_id() - << ", sender_id=" << request->sender_id(); - } + _exec_env->load_channel_mgr()->cancel(*request); } template @@ -195,8 +190,8 @@ void PInternalServiceImpl::trigger_profile_report( PTriggerProfileReportResult* result, google::protobuf::Closure* done) { brpc::ClosureGuard closure_guard(done); - auto st = _exec_env->fragment_mgr()->trigger_profile_report(request); - st.to_protobuf(result->mutable_status()); + _exec_env->fragment_mgr()->trigger_profile_report(request); + result->mutable_status()->set_status_code(0); // TODO(yingchun): should remove it } template diff --git a/be/src/util/batch_process_thread_pool.hpp b/be/src/util/batch_process_thread_pool.hpp index e4148b556debae3..38a1073f99f8a72 100644 --- a/be/src/util/batch_process_thread_pool.hpp +++ b/be/src/util/batch_process_thread_pool.hpp @@ -107,7 +107,7 @@ class BatchProcessThreadPool { void drain_and_shutdown() { { boost::unique_lock l(_lock); - while (_work_queue.get_size() != 0) { + while (!_work_queue.empty()) { _empty_cv.wait(l); } } @@ -143,7 +143,7 @@ class BatchProcessThreadPool { if (!tasks.empty()) { _work_func(tasks); } - if (_work_queue.get_size() == 0) { + if (_work_queue.empty()) { _empty_cv.notify_all(); } } diff --git a/be/src/util/blocking_priority_queue.hpp b/be/src/util/blocking_priority_queue.hpp index c68dfbeed31775a..25aa8db54ff6d6d 100644 --- a/be/src/util/blocking_priority_queue.hpp +++ b/be/src/util/blocking_priority_queue.hpp @@ -87,6 +87,7 @@ class BlockingPriorityQueue { while (true) { if (!_queue.empty()) { + // TODO(yingchun): the 2 blocks are the same // 定期提高队列中残留的任务优先级 // 保证优先级较低的大查询不至于完全饿死 if (_upgrade_counter > config::priority_queue_remaining_tasks_increased_frequency) { @@ -108,9 +109,6 @@ class BlockingPriorityQueue { _put_cv.notify_one(); return true; } - if (_shutdown) { - return false; - } return false; } } @@ -154,12 +152,19 @@ class BlockingPriorityQueue { return _queue.size(); } + bool empty() const { + boost::unique_lock l(_lock); + return _queue.empty(); + } + + // TODO(yingchun): should lock free // Returns the total amount of time threads have blocked in blocking_get. uint64_t total_get_wait_time() const { boost::lock_guard guard(_lock); return _total_get_wait_time; } + // TODO(yingchun): should lock free // Returns the total amount of time threads have blocked in blocking_put. uint64_t total_put_wait_time() const { boost::lock_guard guard(_lock); diff --git a/be/src/util/cpu_info.cpp b/be/src/util/cpu_info.cpp index 2b7f9cce9e5b224..62faaefccca70e1 100755 --- a/be/src/util/cpu_info.cpp +++ b/be/src/util/cpu_info.cpp @@ -123,7 +123,7 @@ int64_t ParseCPUFlags(const string& values) { } void CpuInfo::init() { - if (initialized_) return; + if (initialized_) return; string line; string name; string value; diff --git a/be/src/util/file_utils.cpp b/be/src/util/file_utils.cpp index 469983f53ffe3f0..9724cdcbff309a4 100644 --- a/be/src/util/file_utils.cpp +++ b/be/src/util/file_utils.cpp @@ -121,10 +121,10 @@ Status FileUtils::remove_paths(const std::vector& paths) { } Status FileUtils::list_files(Env* env, const std::string& dir, - std::vector* files) { + std::vector* files) { auto cb = [files](const char* name) -> bool { if (!is_dot_or_dotdot(name)) { - files->push_back(name); + files->emplace_back(name); } return true; }; diff --git a/be/src/util/filesystem_util.h b/be/src/util/filesystem_util.h index 5aaf468a8518bbf..53c2ab9c5eb2066 100644 --- a/be/src/util/filesystem_util.h +++ b/be/src/util/filesystem_util.h @@ -22,6 +22,7 @@ namespace doris { +// TODO(yingchun): we would better move these files from impala to a special dir named 'impala' // Utility class for common local file system operations such as file creation and // deletion. This class should NOT be used to read or write data (DiskIoMgr is used // for that). Errors are indicated by the status code RUNTIME_ERROR, and are not diff --git a/be/src/util/mem_util.hpp b/be/src/util/mem_util.hpp index 473c8a37bed00ac..846c5258967cff8 100644 --- a/be/src/util/mem_util.hpp +++ b/be/src/util/mem_util.hpp @@ -49,6 +49,7 @@ template<> inline void fixed_size_memory_copy<8>(void* dst, const void* src) { *(reinterpret_cast(dst)) = * (reinterpret_cast(src)); } +// TODO(yingchun): do benckmark on it inline void memory_copy(void* dst, const void* src, size_t size) { // Function fixed_size_memory_copy will report a stack-use-after-scope error in ASAN mode. #if !defined(ADDRESS_SANITIZER) diff --git a/be/src/util/priority_thread_pool.hpp b/be/src/util/priority_thread_pool.hpp index 68f50d7e4343f63..9ac2db01bad2539 100644 --- a/be/src/util/priority_thread_pool.hpp +++ b/be/src/util/priority_thread_pool.hpp @@ -110,8 +110,8 @@ class PriorityThreadPool { _threads.join_all(); } - uint32_t get_queue_size() const { - return _work_queue.get_size(); + bool empty() const { + return _work_queue.empty(); } // Blocks until the work queue is empty, and then calls shutdown to stop the worker @@ -120,7 +120,7 @@ class PriorityThreadPool { void drain_and_shutdown() { { boost::unique_lock l(_lock); - while (_work_queue.get_size() != 0) { + while (!_work_queue.empty()) { _empty_cv.wait(l); } } @@ -137,7 +137,7 @@ class PriorityThreadPool { if (_work_queue.blocking_get(&task)) { task.work_function(); } - if (_work_queue.get_size() == 0) { + if (_work_queue.empty()) { _empty_cv.notify_all(); } } diff --git a/be/src/util/thrift_server.cpp b/be/src/util/thrift_server.cpp index 8af993a461c6079..d590786398cfa63 100644 --- a/be/src/util/thrift_server.cpp +++ b/be/src/util/thrift_server.cpp @@ -126,7 +126,7 @@ Status ThriftServer::ThriftServerEventProcessor::start_and_wait_for_server() { // _started == true only if preServe was called. May be false if there was an exception // after preServe that was caught by Supervise, causing it to reset the error condition. - if (_thrift_server->_started == false) { + if (!_thrift_server->_started) { std::stringstream ss; ss << "ThriftServer '" << _thrift_server->_name << "' (on port: " << _thrift_server->_port << ") did not start correctly "; @@ -138,7 +138,7 @@ Status ThriftServer::ThriftServerEventProcessor::start_and_wait_for_server() { } void ThriftServer::ThriftServerEventProcessor::supervise() { - DCHECK(_thrift_server->_server.get() != NULL); + DCHECK(_thrift_server->_server.get()); try { _thrift_server->_server->serve(); diff --git a/be/tags b/be/tags new file mode 100644 index 000000000000000..3386431839f1396 --- /dev/null +++ b/be/tags @@ -0,0 +1,27799 @@ +!_TAG_FILE_FORMAT 2 /extended format; --format=1 will not append ;" to lines/ +!_TAG_FILE_SORTED 1 /0=unsorted, 1=sorted, 2=foldcase/ +!_TAG_PROGRAM_AUTHOR Darren Hiebert /dhiebert@users.sourceforge.net/ +!_TAG_PROGRAM_NAME Exuberant Ctags // +!_TAG_PROGRAM_URL http://ctags.sourceforge.net /official site/ +!_TAG_PROGRAM_VERSION 5.8 // +ABSTRACT src/gutil/macros.h 20;" d +ACCEPT src/http/http_headers.cpp /^const char* HttpHeaders::ACCEPT = "Accept";$/;" m class:doris::HttpHeaders file: +ACCEPT src/http/http_headers.h /^ static const char* ACCEPT;$/;" m class:doris::HttpHeaders +ACCEPTED src/http/http_status.h /^ ACCEPTED = 202,$/;" e enum:doris::HttpStatus +ACCEPT_CHARSET src/http/http_headers.cpp /^const char* HttpHeaders::ACCEPT_CHARSET = "Accept-Charset";$/;" m class:doris::HttpHeaders file: +ACCEPT_CHARSET src/http/http_headers.h /^ static const char* ACCEPT_CHARSET;$/;" m class:doris::HttpHeaders +ACCEPT_ENCODING src/http/http_headers.cpp /^const char* HttpHeaders::ACCEPT_ENCODING = "Accept-Encoding";$/;" m class:doris::HttpHeaders file: +ACCEPT_ENCODING src/http/http_headers.h /^ static const char* ACCEPT_ENCODING;$/;" m class:doris::HttpHeaders +ACCEPT_LANGUAGE src/http/http_headers.cpp /^const char* HttpHeaders::ACCEPT_LANGUAGE = "Accept-Language";$/;" m class:doris::HttpHeaders file: +ACCEPT_LANGUAGE src/http/http_headers.h /^ static const char* ACCEPT_LANGUAGE;$/;" m class:doris::HttpHeaders +ACCEPT_PATCH src/http/http_headers.cpp /^const char* HttpHeaders::ACCEPT_PATCH = "Accept-Patch";$/;" m class:doris::HttpHeaders file: +ACCEPT_PATCH src/http/http_headers.h /^ static const char* ACCEPT_PATCH;$/;" m class:doris::HttpHeaders +ACCEPT_RANGES src/http/http_headers.cpp /^const char* HttpHeaders::ACCEPT_RANGES = "Accept-Ranges";$/;" m class:doris::HttpHeaders file: +ACCEPT_RANGES src/http/http_headers.h /^ static const char* ACCEPT_RANGES;$/;" m class:doris::HttpHeaders +ACCESS_CONTROL_ALLOW_CREDENTIALS src/http/http_headers.cpp /^const char* HttpHeaders::ACCESS_CONTROL_ALLOW_CREDENTIALS = "Access-Control-Allow-Credentials";$/;" m class:doris::HttpHeaders file: +ACCESS_CONTROL_ALLOW_CREDENTIALS src/http/http_headers.h /^ static const char* ACCESS_CONTROL_ALLOW_CREDENTIALS;$/;" m class:doris::HttpHeaders +ACCESS_CONTROL_ALLOW_HEADERS src/http/http_headers.cpp /^const char* HttpHeaders::ACCESS_CONTROL_ALLOW_HEADERS = "Access-Control-Allow-Headers";$/;" m class:doris::HttpHeaders file: +ACCESS_CONTROL_ALLOW_HEADERS src/http/http_headers.h /^ static const char* ACCESS_CONTROL_ALLOW_HEADERS;$/;" m class:doris::HttpHeaders +ACCESS_CONTROL_ALLOW_METHODS src/http/http_headers.cpp /^const char* HttpHeaders::ACCESS_CONTROL_ALLOW_METHODS = "Access-Control-Allow-Methods";$/;" m class:doris::HttpHeaders file: +ACCESS_CONTROL_ALLOW_METHODS src/http/http_headers.h /^ static const char* ACCESS_CONTROL_ALLOW_METHODS;$/;" m class:doris::HttpHeaders +ACCESS_CONTROL_ALLOW_ORIGIN src/http/http_headers.cpp /^const char* HttpHeaders::ACCESS_CONTROL_ALLOW_ORIGIN = "Access-Control-Allow-Origin";$/;" m class:doris::HttpHeaders file: +ACCESS_CONTROL_ALLOW_ORIGIN src/http/http_headers.h /^ static const char* ACCESS_CONTROL_ALLOW_ORIGIN;$/;" m class:doris::HttpHeaders +ACCESS_CONTROL_EXPOSE_HEADERS src/http/http_headers.cpp /^const char* HttpHeaders::ACCESS_CONTROL_EXPOSE_HEADERS = "Access-Control-Expose-Headers";$/;" m class:doris::HttpHeaders file: +ACCESS_CONTROL_EXPOSE_HEADERS src/http/http_headers.h /^ static const char* ACCESS_CONTROL_EXPOSE_HEADERS;$/;" m class:doris::HttpHeaders +ACCESS_CONTROL_MAX_AGE src/http/http_headers.cpp /^const char* HttpHeaders::ACCESS_CONTROL_MAX_AGE = "Access-Control-Max-Age";$/;" m class:doris::HttpHeaders file: +ACCESS_CONTROL_MAX_AGE src/http/http_headers.h /^ static const char* ACCESS_CONTROL_MAX_AGE;$/;" m class:doris::HttpHeaders +ACCESS_CONTROL_REQUEST_HEADERS src/http/http_headers.cpp /^const char* HttpHeaders::ACCESS_CONTROL_REQUEST_HEADERS = "Access-Control-Request-Headers";$/;" m class:doris::HttpHeaders file: +ACCESS_CONTROL_REQUEST_HEADERS src/http/http_headers.h /^ static const char* ACCESS_CONTROL_REQUEST_HEADERS;$/;" m class:doris::HttpHeaders +ACCESS_CONTROL_REQUEST_METHOD src/http/http_headers.cpp /^const char* HttpHeaders::ACCESS_CONTROL_REQUEST_METHOD = "Access-Control-Request-Method";$/;" m class:doris::HttpHeaders file: +ACCESS_CONTROL_REQUEST_METHOD src/http/http_headers.h /^ static const char* ACCESS_CONTROL_REQUEST_METHOD;$/;" m class:doris::HttpHeaders +ADD src/exprs/arithmetic_expr.h /^ ADD,$/;" e enum:doris::ArithmeticExpr::BinaryOpType +ADD_CHILD_TIMER src/util/runtime_profile.h 48;" d +ADD_COUNTER src/util/runtime_profile.h 46;" d +ADD_COUNTER_IMPL src/util/runtime_profile.cpp 403;" d file: +ADD_ROWBATCH src/exec/olap_scan_node.h /^ ADD_ROWBATCH = 6,$/;" e enum:doris::TransferStatus +ADD_THREAD_COUNTERS src/util/runtime_profile.h 56;" d +ADD_TIMER src/util/runtime_profile.h 47;" d +ADLER32_INIT src/olap/utils.h 153;" d +ADLER32_INIT_VALUE src/exec/decompressor.h /^ const static uint32_t ADLER32_INIT_VALUE;$/;" m class:doris::LzopDecompressor +ADLER32_INIT_VALUE src/exec/lzo_decompressor.cpp /^const uint32_t LzopDecompressor::ADLER32_INIT_VALUE = 1;$/;" m class:doris::LzopDecompressor file: +AES_128_CBC src/util/aes_util.h /^ AES_128_CBC,$/;" e enum:doris::AesMode +AES_128_ECB src/util/aes_util.h /^ AES_128_ECB,$/;" e enum:doris::AesMode +AES_192_CBC src/util/aes_util.h /^ AES_192_CBC,$/;" e enum:doris::AesMode +AES_192_ECB src/util/aes_util.h /^ AES_192_ECB,$/;" e enum:doris::AesMode +AES_256_CBC src/util/aes_util.h /^ AES_256_CBC$/;" e enum:doris::AesMode +AES_256_ECB src/util/aes_util.h /^ AES_256_ECB,$/;" e enum:doris::AesMode +AES_BAD_DATA src/util/aes_util.h /^ AES_BAD_DATA = -1$/;" e enum:doris::AesState +AES_MAX_KEY_LENGTH src/util/aes_util.cpp /^static const int AES_MAX_KEY_LENGTH = 256;$/;" m namespace:doris file: +AES_SUCCESS src/util/aes_util.h /^ AES_SUCCESS = 0,$/;" e enum:doris::AesState +AGE src/http/http_headers.cpp /^const char* HttpHeaders::AGE = "Age";$/;" m class:doris::HttpHeaders file: +AGE src/http/http_headers.h /^ static const char* AGE;$/;" m class:doris::HttpHeaders +ALIGN src/util/crc32c.cpp 344;" d file: +ALIGN src/util/crc32c.cpp 376;" d file: +ALIGNED_CHAR_ARRAY src/gutil/port.h 686;" d +ALIGNED_CHAR_ARRAY src/gutil/port.h 724;" d +ALIGNED_CHAR_ARRAY src/gutil/port.h 731;" d +ALIGNED_CHAR_ARRAY src/gutil/port.h 740;" d +ALIGNED_CHAR_ARRAY src/gutil/port.h 744;" d +ALIGN_CACHE_LINE src/common/compiler_util.h 47;" d +ALIGN_DOWN src/util/alignment.h 22;" d +ALIGN_TAG_PREFIX src/olap/olap_define.h /^static const std::string ALIGN_TAG_PREFIX = "\/align_tag";$/;" m namespace:doris +ALIGN_UP src/util/alignment.h 25;" d +ALL output/udf/include/uda_test_harness.h /^ ALL = 0,$/;" e enum:doris_udf::UdaExecutionMode +ALL src/udf/uda_test_harness.h /^ ALL = 0,$/;" e enum:doris_udf::UdaExecutionMode +ALLOC src/util/minizip/unzip.c 112;" d file: +ALLOW src/http/http_headers.cpp /^const char* HttpHeaders::ALLOW = "Allow";$/;" m class:doris::HttpHeaders file: +ALLOW src/http/http_headers.h /^ static const char* ALLOW;$/;" m class:doris::HttpHeaders +ALTER_TABLE src/agent/task_worker_pool.h /^ ALTER_TABLE,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +ALWAYS_INLINE src/common/compiler_util.h 45;" d +ANNOTALYSIS_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 402;" d +ANNOTALYSIS_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 420;" d +ANNOTALYSIS_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 421;" d +ANNOTALYSIS_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 442;" d +ANNOTALYSIS_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 443;" d +ANNOTALYSIS_IGNORE_READS_END src/gutil/dynamic_annotations.h 403;" d +ANNOTALYSIS_IGNORE_READS_END src/gutil/dynamic_annotations.h 422;" d +ANNOTALYSIS_IGNORE_READS_END src/gutil/dynamic_annotations.h 423;" d +ANNOTALYSIS_IGNORE_READS_END src/gutil/dynamic_annotations.h 445;" d +ANNOTALYSIS_IGNORE_READS_END src/gutil/dynamic_annotations.h 446;" d +ANNOTALYSIS_IGNORE_WRITES_BEGIN src/gutil/dynamic_annotations.h 404;" d +ANNOTALYSIS_IGNORE_WRITES_BEGIN src/gutil/dynamic_annotations.h 424;" d +ANNOTALYSIS_IGNORE_WRITES_BEGIN src/gutil/dynamic_annotations.h 425;" d +ANNOTALYSIS_IGNORE_WRITES_END src/gutil/dynamic_annotations.h 405;" d +ANNOTALYSIS_IGNORE_WRITES_END src/gutil/dynamic_annotations.h 426;" d +ANNOTALYSIS_IGNORE_WRITES_END src/gutil/dynamic_annotations.h 427;" d +ANNOTALYSIS_ONLY src/gutil/dynamic_annotations.h 411;" d +ANNOTALYSIS_ONLY src/gutil/dynamic_annotations.h 676;" d +ANNOTALYSIS_SEMICOLON_OR_EMPTY_BODY src/gutil/dynamic_annotations.h 401;" d +ANNOTALYSIS_SEMICOLON_OR_EMPTY_BODY src/gutil/dynamic_annotations.h 414;" d +ANNOTALYSIS_SEMICOLON_OR_EMPTY_BODY src/gutil/dynamic_annotations.h 415;" d +ANNOTALYSIS_SEMICOLON_OR_EMPTY_BODY src/gutil/dynamic_annotations.h 456;" d +ANNOTALYSIS_SEMICOLON_OR_EMPTY_BODY src/gutil/dynamic_annotations.h 457;" d +ANNOTALYSIS_SEMICOLON_OR_EMPTY_BODY src/gutil/dynamic_annotations.h 769;" d +ANNOTALYSIS_STATIC_INLINE src/gutil/dynamic_annotations.h 400;" d +ANNOTALYSIS_STATIC_INLINE src/gutil/dynamic_annotations.h 412;" d +ANNOTALYSIS_STATIC_INLINE src/gutil/dynamic_annotations.h 413;" d +ANNOTALYSIS_STATIC_INLINE src/gutil/dynamic_annotations.h 454;" d +ANNOTALYSIS_STATIC_INLINE src/gutil/dynamic_annotations.h 455;" d +ANNOTALYSIS_STATIC_INLINE src/gutil/dynamic_annotations.h 768;" d +ANNOTALYSIS_UNPROTECTED_READ src/gutil/dynamic_annotations.h 406;" d +ANNOTALYSIS_UNPROTECTED_READ src/gutil/dynamic_annotations.h 428;" d +ANNOTALYSIS_UNPROTECTED_READ src/gutil/dynamic_annotations.h 429;" d +ANNOTATE_BARRIER_DESTROY src/gutil/dynamic_annotations.h 327;" d +ANNOTATE_BARRIER_DESTROY src/gutil/dynamic_annotations.h 358;" d +ANNOTATE_BARRIER_INIT src/gutil/dynamic_annotations.h 314;" d +ANNOTATE_BARRIER_INIT src/gutil/dynamic_annotations.h 355;" d +ANNOTATE_BARRIER_WAIT_AFTER src/gutil/dynamic_annotations.h 323;" d +ANNOTATE_BARRIER_WAIT_AFTER src/gutil/dynamic_annotations.h 357;" d +ANNOTATE_BARRIER_WAIT_BEFORE src/gutil/dynamic_annotations.h 319;" d +ANNOTATE_BARRIER_WAIT_BEFORE src/gutil/dynamic_annotations.h 356;" d +ANNOTATE_BENIGN_RACE src/gutil/dynamic_annotations.h 206;" d +ANNOTATE_BENIGN_RACE src/gutil/dynamic_annotations.h 374;" d +ANNOTATE_BENIGN_RACE_SIZED src/gutil/dynamic_annotations.h 212;" d +ANNOTATE_BENIGN_RACE_SIZED src/gutil/dynamic_annotations.h 375;" d +ANNOTATE_BENIGN_RACE_STATIC src/gutil/dynamic_annotations.h 645;" d +ANNOTATE_BENIGN_RACE_STATIC src/gutil/dynamic_annotations.h 660;" d +ANNOTATE_CONDVAR_LOCK_WAIT src/gutil/dynamic_annotations.h 106;" d +ANNOTATE_CONDVAR_LOCK_WAIT src/gutil/dynamic_annotations.h 359;" d +ANNOTATE_CONDVAR_SIGNAL src/gutil/dynamic_annotations.h 116;" d +ANNOTATE_CONDVAR_SIGNAL src/gutil/dynamic_annotations.h 361;" d +ANNOTATE_CONDVAR_SIGNAL_ALL src/gutil/dynamic_annotations.h 120;" d +ANNOTATE_CONDVAR_SIGNAL_ALL src/gutil/dynamic_annotations.h 362;" d +ANNOTATE_CONDVAR_WAIT src/gutil/dynamic_annotations.h 111;" d +ANNOTATE_CONDVAR_WAIT src/gutil/dynamic_annotations.h 360;" d +ANNOTATE_ENABLE_RACE_DETECTION src/gutil/dynamic_annotations.h 261;" d +ANNOTATE_ENABLE_RACE_DETECTION src/gutil/dynamic_annotations.h 388;" d +ANNOTATE_EXPECT_RACE src/gutil/dynamic_annotations.h 335;" d +ANNOTATE_EXPECT_RACE src/gutil/dynamic_annotations.h 373;" d +ANNOTATE_FLUSH_STATE src/gutil/dynamic_annotations.h 344;" d +ANNOTATE_FLUSH_STATE src/gutil/dynamic_annotations.h 390;" d +ANNOTATE_HAPPENS_AFTER src/gutil/dynamic_annotations.h 125;" d +ANNOTATE_HAPPENS_AFTER src/gutil/dynamic_annotations.h 364;" d +ANNOTATE_HAPPENS_BEFORE src/gutil/dynamic_annotations.h 124;" d +ANNOTATE_HAPPENS_BEFORE src/gutil/dynamic_annotations.h 363;" d +ANNOTATE_IGNORE_READS_AND_WRITES_BEGIN src/gutil/dynamic_annotations.h 236;" d +ANNOTATE_IGNORE_READS_AND_WRITES_BEGIN src/gutil/dynamic_annotations.h 384;" d +ANNOTATE_IGNORE_READS_AND_WRITES_BEGIN src/gutil/dynamic_annotations.h 695;" d +ANNOTATE_IGNORE_READS_AND_WRITES_BEGIN src/gutil/dynamic_annotations.h 696;" d +ANNOTATE_IGNORE_READS_AND_WRITES_BEGIN src/gutil/dynamic_annotations.h 739;" d +ANNOTATE_IGNORE_READS_AND_WRITES_BEGIN src/gutil/dynamic_annotations.h 740;" d +ANNOTATE_IGNORE_READS_AND_WRITES_END src/gutil/dynamic_annotations.h 243;" d +ANNOTATE_IGNORE_READS_AND_WRITES_END src/gutil/dynamic_annotations.h 385;" d +ANNOTATE_IGNORE_READS_AND_WRITES_END src/gutil/dynamic_annotations.h 702;" d +ANNOTATE_IGNORE_READS_AND_WRITES_END src/gutil/dynamic_annotations.h 703;" d +ANNOTATE_IGNORE_READS_AND_WRITES_END src/gutil/dynamic_annotations.h 746;" d +ANNOTATE_IGNORE_READS_AND_WRITES_END src/gutil/dynamic_annotations.h 747;" d +ANNOTATE_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 220;" d +ANNOTATE_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 380;" d +ANNOTATE_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 679;" d +ANNOTATE_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 680;" d +ANNOTATE_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 731;" d +ANNOTATE_IGNORE_READS_BEGIN src/gutil/dynamic_annotations.h 732;" d +ANNOTATE_IGNORE_READS_END src/gutil/dynamic_annotations.h 224;" d +ANNOTATE_IGNORE_READS_END src/gutil/dynamic_annotations.h 381;" d +ANNOTATE_IGNORE_READS_END src/gutil/dynamic_annotations.h 683;" d +ANNOTATE_IGNORE_READS_END src/gutil/dynamic_annotations.h 684;" d +ANNOTATE_IGNORE_READS_END src/gutil/dynamic_annotations.h 735;" d +ANNOTATE_IGNORE_READS_END src/gutil/dynamic_annotations.h 736;" d +ANNOTATE_IGNORE_SYNC_BEGIN src/gutil/dynamic_annotations.h 251;" d +ANNOTATE_IGNORE_SYNC_BEGIN src/gutil/dynamic_annotations.h 386;" d +ANNOTATE_IGNORE_SYNC_END src/gutil/dynamic_annotations.h 255;" d +ANNOTATE_IGNORE_SYNC_END src/gutil/dynamic_annotations.h 387;" d +ANNOTATE_IGNORE_WRITES_BEGIN src/gutil/dynamic_annotations.h 228;" d +ANNOTATE_IGNORE_WRITES_BEGIN src/gutil/dynamic_annotations.h 382;" d +ANNOTATE_IGNORE_WRITES_BEGIN src/gutil/dynamic_annotations.h 687;" d +ANNOTATE_IGNORE_WRITES_BEGIN src/gutil/dynamic_annotations.h 688;" d +ANNOTATE_IGNORE_WRITES_END src/gutil/dynamic_annotations.h 232;" d +ANNOTATE_IGNORE_WRITES_END src/gutil/dynamic_annotations.h 383;" d +ANNOTATE_IGNORE_WRITES_END src/gutil/dynamic_annotations.h 691;" d +ANNOTATE_IGNORE_WRITES_END src/gutil/dynamic_annotations.h 692;" d +ANNOTATE_MUTEX_IS_USED_AS_CONDVAR src/gutil/dynamic_annotations.h 159;" d +ANNOTATE_MUTEX_IS_USED_AS_CONDVAR src/gutil/dynamic_annotations.h 377;" d +ANNOTATE_NEW_MEMORY src/gutil/dynamic_annotations.h 170;" d +ANNOTATE_NEW_MEMORY src/gutil/dynamic_annotations.h 372;" d +ANNOTATE_NO_OP src/gutil/dynamic_annotations.h 339;" d +ANNOTATE_NO_OP src/gutil/dynamic_annotations.h 389;" d +ANNOTATE_PCQ_CREATE src/gutil/dynamic_annotations.h 181;" d +ANNOTATE_PCQ_CREATE src/gutil/dynamic_annotations.h 368;" d +ANNOTATE_PCQ_DESTROY src/gutil/dynamic_annotations.h 185;" d +ANNOTATE_PCQ_DESTROY src/gutil/dynamic_annotations.h 369;" d +ANNOTATE_PCQ_GET src/gutil/dynamic_annotations.h 194;" d +ANNOTATE_PCQ_GET src/gutil/dynamic_annotations.h 371;" d +ANNOTATE_PCQ_PUT src/gutil/dynamic_annotations.h 190;" d +ANNOTATE_PCQ_PUT src/gutil/dynamic_annotations.h 370;" d +ANNOTATE_PUBLISH_MEMORY_RANGE src/gutil/dynamic_annotations.h 134;" d +ANNOTATE_PUBLISH_MEMORY_RANGE src/gutil/dynamic_annotations.h 365;" d +ANNOTATE_PURE_HAPPENS_BEFORE_MUTEX src/gutil/dynamic_annotations.h 155;" d +ANNOTATE_PURE_HAPPENS_BEFORE_MUTEX src/gutil/dynamic_annotations.h 376;" d +ANNOTATE_RWLOCK_ACQUIRED src/gutil/dynamic_annotations.h 299;" d +ANNOTATE_RWLOCK_ACQUIRED src/gutil/dynamic_annotations.h 353;" d +ANNOTATE_RWLOCK_CREATE src/gutil/dynamic_annotations.h 281;" d +ANNOTATE_RWLOCK_CREATE src/gutil/dynamic_annotations.h 350;" d +ANNOTATE_RWLOCK_CREATE_STATIC src/gutil/dynamic_annotations.h 287;" d +ANNOTATE_RWLOCK_CREATE_STATIC src/gutil/dynamic_annotations.h 290;" d +ANNOTATE_RWLOCK_CREATE_STATIC src/gutil/dynamic_annotations.h 351;" d +ANNOTATE_RWLOCK_DESTROY src/gutil/dynamic_annotations.h 294;" d +ANNOTATE_RWLOCK_DESTROY src/gutil/dynamic_annotations.h 352;" d +ANNOTATE_RWLOCK_RELEASED src/gutil/dynamic_annotations.h 303;" d +ANNOTATE_RWLOCK_RELEASED src/gutil/dynamic_annotations.h 354;" d +ANNOTATE_SWAP_MEMORY_RANGE src/gutil/dynamic_annotations.h 142;" d +ANNOTATE_SWAP_MEMORY_RANGE src/gutil/dynamic_annotations.h 367;" d +ANNOTATE_THREAD_NAME src/gutil/dynamic_annotations.h 272;" d +ANNOTATE_THREAD_NAME src/gutil/dynamic_annotations.h 379;" d +ANNOTATE_TRACE_MEMORY src/gutil/dynamic_annotations.h 268;" d +ANNOTATE_TRACE_MEMORY src/gutil/dynamic_annotations.h 378;" d +ANNOTATE_UNPROTECTED_READ src/gutil/dynamic_annotations.h /^ inline T ANNOTATE_UNPROTECTED_READ(const volatile T &x) {$/;" f +ANNOTATE_UNPROTECTED_READ src/gutil/dynamic_annotations.h 659;" d +ANNOTATE_UNPROTECTED_READ src/gutil/dynamic_annotations.h 710;" d +ANNOTATE_UNPROTECTED_READ src/gutil/dynamic_annotations.h 754;" d +ANNOTATE_UNPUBLISH_MEMORY_RANGE src/gutil/dynamic_annotations.h 138;" d +ANNOTATE_UNPUBLISH_MEMORY_RANGE src/gutil/dynamic_annotations.h 366;" d +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 440;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 443;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 446;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 449;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 453;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 455;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 457;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 461;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 464;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 467;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 472;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 475;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 478;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 483;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 486;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 489;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 492;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 425;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 428;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 431;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 434;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 438;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 440;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 442;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 446;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 449;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 452;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 457;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 460;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 463;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 468;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 471;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 474;" d file: +ARCHITECTURE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 477;" d file: +ARCH_CPU_32_BITS src/gutil/cpu.h 26;" d +ARCH_CPU_32_BITS src/gutil/cpu.h 30;" d +ARCH_CPU_64_BITS src/gutil/cpu.h 22;" d +ARCH_CPU_64_BITS src/gutil/cpu.h 34;" d +ARCH_CPU_ARM64 src/gutil/cpu.h 33;" d +ARCH_CPU_ARMEL src/gutil/cpu.h 29;" d +ARCH_CPU_ARM_FAMILY src/gutil/cpu.h 28;" d +ARCH_CPU_ARM_FAMILY src/gutil/cpu.h 32;" d +ARCH_CPU_X86 src/gutil/cpu.h 25;" d +ARCH_CPU_X86_64 src/gutil/cpu.h 21;" d +ARCH_CPU_X86_FAMILY src/gutil/cpu.h 20;" d +ARCH_CPU_X86_FAMILY src/gutil/cpu.h 24;" d +ARG_TYPE_SIZE src/exprs/expr.h /^ ARG_TYPE_SIZE \/\/ int[]$/;" e enum:doris::Expr::ExprConstant +ARITHMETIC_RIGHT_SHIFT src/util/thrift_util.cpp 41;" d file: +ARMV2 src/gutil/arm_instruction_set_select.h 49;" d +ARMV3 src/gutil/arm_instruction_set_select.h 44;" d +ARMV4 src/gutil/arm_instruction_set_select.h 38;" d +ARMV5 src/gutil/arm_instruction_set_select.h 32;" d +ARMV6 src/gutil/arm_instruction_set_select.h 24;" d +ARMV7 src/gutil/arm_instruction_set_select.h 14;" d +ARM_INSTRUCTION_SET_SELECT_H_ src/gutil/arm_instruction_set_select.h 9;" d +ARRAYSIZE src/gutil/macros.h 189;" d +ARRAY_ELEMENTS src/exprs/math_functions.cpp 70;" d file: +ARROW_UTIL_LOGGING_H test/util/arrow/arrow_row_batch_test.cpp 27;" d file: +ARROW_UTIL_LOGGING_H test/util/arrow/arrow_row_block_test.cpp 25;" d file: +ASAN_POISON_MEMORY_REGION src/gutil/dynamic_annotations.h 599;" d +ASAN_POISON_MEMORY_REGION src/gutil/dynamic_annotations.h 604;" d +ASAN_SET_DEATH_CALLBACK src/gutil/dynamic_annotations.h 615;" d +ASAN_SET_DEATH_CALLBACK src/gutil/dynamic_annotations.h 618;" d +ASAN_UNPOISON_MEMORY_REGION src/gutil/dynamic_annotations.h 601;" d +ASAN_UNPOISON_MEMORY_REGION src/gutil/dynamic_annotations.h 606;" d +ASCII_PRECISION src/runtime/raw_value.cpp /^const int RawValue::ASCII_PRECISION = 16; \/\/ print 16 digits for double\/float$/;" m class:doris::RawValue file: +ASCII_PRECISION src/runtime/raw_value.h /^ static const int ASCII_PRECISION;$/;" m class:doris::RawValue +ASSIGN_DEFAULT_VALUE src/olap/schema_change.cpp 188;" d file: +ASSIGN_DEFAULT_VALUE src/olap/schema_change.cpp 440;" d file: +AS_STRING src/gutil/macros.h 195;" d +AS_STRING_INTERNAL src/gutil/macros.h 196;" d +ATOMICOPS_COMPILER_BARRIER src/gutil/atomicops-internals-tsan.h 215;" d +ATOMICOPS_COMPILER_BARRIER src/gutil/atomicops-internals-tsan.h 29;" d +ATOMICOPS_COMPILER_BARRIER src/gutil/atomicops-internals-x86.h 498;" d +ATOMICOPS_COMPILER_BARRIER src/gutil/atomicops-internals-x86.h 53;" d +ATTRIBUTE_ALWAYS_INLINE src/gutil/port.h 367;" d +ATTRIBUTE_COLD src/gutil/port.h 610;" d +ATTRIBUTE_COLD src/gutil/port.h 613;" d +ATTRIBUTE_DEPRECATED src/gutil/port.h 383;" d +ATTRIBUTE_HOT src/gutil/port.h 609;" d +ATTRIBUTE_HOT src/gutil/port.h 612;" d +ATTRIBUTE_INITIAL_EXEC src/gutil/port.h 390;" d +ATTRIBUTE_MEMBER_UNUSED src/gutil/port.h 359;" d +ATTRIBUTE_NOINLINE src/gutil/port.h 369;" d +ATTRIBUTE_NONNULL src/gutil/port.h 398;" d +ATTRIBUTE_NORETURN src/gutil/port.h 403;" d +ATTRIBUTE_NO_ADDRESS_SAFETY_ANALYSIS src/gutil/port.h 410;" d +ATTRIBUTE_NO_SANITIZE_THREAD src/gutil/port.h 420;" d +ATTRIBUTE_STACK_ALIGN_FOR_OLD_LIBC src/gutil/port.h 445;" d +ATTRIBUTE_UNUSED src/gutil/port.h 351;" d +ATTRIBUTE_UNUSED src/gutil/port.h 352;" d +ATTRIBUTE_WEAK src/gutil/port.h 373;" d +ATTRIBUTE_WEAK src/gutil/port.h 374;" d +AT_CAPACITY_MEM_USAGE src/runtime/row_batch.cpp /^const int RowBatch::AT_CAPACITY_MEM_USAGE = 8 * 1024 * 1024;$/;" m class:doris::RowBatch file: +AT_CAPACITY_MEM_USAGE src/runtime/row_batch.h /^ static const int AT_CAPACITY_MEM_USAGE;$/;" m class:doris::RowBatch +AT_FDCWD src/gutil/linux_syscall_support.h 562;" d +AT_REMOVEDIR src/gutil/linux_syscall_support.h 568;" d +AT_SYMLINK_NOFOLLOW src/gutil/linux_syscall_support.h 565;" d +AUTHORITY src/util/url_parser.h /^ AUTHORITY,$/;" e enum:doris::UrlParser::UrlPart +AUTHORIZATION src/http/http_headers.cpp /^const char* HttpHeaders::AUTHORIZATION = "Authorization";$/;" m class:doris::HttpHeaders file: +AUTHORIZATION src/http/http_headers.h /^ static const char* AUTHORIZATION;$/;" m class:doris::HttpHeaders +AVG src/exprs/agg_fn.h /^ AVG,$/;" e enum:doris::AggFn::AggregationOp +AVG src/exprs/agg_fn_evaluator.h /^ AVG,$/;" e enum:doris::AggFnEvaluator::AggregationOp +AVX src/gutil/cpu.h /^ AVX,$/;" e enum:base::final::IntelMicroArchitecture +AVX src/util/cpu_info.h /^ static const int64_t AVX = (1 << 5);$/;" m class:doris::CpuInfo +AVX2 src/gutil/cpu.h /^ AVX2,$/;" e enum:base::final::IntelMicroArchitecture +AVX2 src/util/cpu_info.h /^ static const int64_t AVX2 = (1 << 6);$/;" m class:doris::CpuInfo +Aborted src/common/status.h /^ static Status Aborted(const Slice& msg,$/;" f class:doris::Status +Acquire_AtomicExchange src/gutil/atomicops-internals-gcc.h /^inline Atomic32 Acquire_AtomicExchange(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Acquire_AtomicExchange src/gutil/atomicops-internals-gcc.h /^inline Atomic64 Acquire_AtomicExchange(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Acquire_AtomicExchange src/gutil/atomicops-internals-tsan.h /^inline Atomic32 Acquire_AtomicExchange(volatile Atomic32 *ptr,$/;" f namespace:base::subtle +Acquire_AtomicExchange src/gutil/atomicops-internals-tsan.h /^inline Atomic64 Acquire_AtomicExchange(volatile Atomic64 *ptr,$/;" f namespace:base::subtle +Acquire_AtomicExchange src/gutil/atomicops-internals-x86.h /^inline Atomic32 Acquire_AtomicExchange(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Acquire_AtomicExchange src/gutil/atomicops-internals-x86.h /^inline Atomic64 Acquire_AtomicExchange(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Acquire_AtomicExchange src/gutil/atomicops.h /^inline AtomicWord Acquire_AtomicExchange(volatile AtomicWord* ptr,$/;" f namespace:base::subtle +Acquire_CompareAndSwap src/gutil/atomicops-internals-gcc.h /^inline Atomic32 Acquire_CompareAndSwap(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Acquire_CompareAndSwap src/gutil/atomicops-internals-gcc.h /^inline Atomic64 Acquire_CompareAndSwap(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Acquire_CompareAndSwap src/gutil/atomicops-internals-tsan.h /^inline Atomic32 Acquire_CompareAndSwap(volatile Atomic32 *ptr,$/;" f namespace:base::subtle +Acquire_CompareAndSwap src/gutil/atomicops-internals-tsan.h /^inline Atomic64 Acquire_CompareAndSwap(volatile Atomic64 *ptr,$/;" f namespace:base::subtle +Acquire_CompareAndSwap src/gutil/atomicops-internals-x86.h /^inline Atomic32 Acquire_CompareAndSwap(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Acquire_CompareAndSwap src/gutil/atomicops-internals-x86.h /^inline Atomic64 Acquire_CompareAndSwap(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Acquire_CompareAndSwap src/gutil/atomicops.h /^inline Atomic32 Acquire_CompareAndSwap(volatile Atomic32* ptr,$/;" f +Acquire_CompareAndSwap src/gutil/atomicops.h /^inline AtomicWord Acquire_CompareAndSwap(volatile AtomicWord* ptr,$/;" f +Acquire_CompareAndSwap src/gutil/atomicops.h /^inline AtomicWord Acquire_CompareAndSwap(volatile AtomicWord* ptr,$/;" f namespace:base::subtle +Acquire_CompareAndSwap src/gutil/atomicops.h /^inline base::subtle::Atomic64 Acquire_CompareAndSwap($/;" f +Acquire_Load src/gutil/atomicops-internals-gcc.h /^inline Atomic32 Acquire_Load(volatile const Atomic32* ptr) {$/;" f namespace:base::subtle +Acquire_Load src/gutil/atomicops-internals-gcc.h /^inline Atomic64 Acquire_Load(volatile const Atomic64* ptr) {$/;" f namespace:base::subtle +Acquire_Load src/gutil/atomicops-internals-tsan.h /^inline Atomic32 Acquire_Load(volatile const Atomic32 *ptr) {$/;" f namespace:base::subtle +Acquire_Load src/gutil/atomicops-internals-tsan.h /^inline Atomic64 Acquire_Load(volatile const Atomic64 *ptr) {$/;" f namespace:base::subtle +Acquire_Load src/gutil/atomicops-internals-x86.h /^inline Atomic32 Acquire_Load(volatile const Atomic32* ptr) {$/;" f namespace:base::subtle +Acquire_Load src/gutil/atomicops-internals-x86.h /^inline Atomic64 Acquire_Load(volatile const Atomic64* ptr) {$/;" f namespace:base::subtle +Acquire_Load src/gutil/atomicops.h /^inline Atomic32 Acquire_Load(volatile const Atomic32* ptr) {$/;" f +Acquire_Load src/gutil/atomicops.h /^inline AtomicWord Acquire_Load(volatile const AtomicWord* ptr) {$/;" f +Acquire_Load src/gutil/atomicops.h /^inline AtomicWord Acquire_Load(volatile const AtomicWord* ptr) {$/;" f namespace:base::subtle +Acquire_Load src/gutil/atomicops.h /^inline base::subtle::Atomic64 Acquire_Load($/;" f +Acquire_Store src/gutil/atomicops-internals-gcc.h /^inline void Acquire_Store(volatile Atomic32* ptr, Atomic32 value) {$/;" f namespace:base::subtle +Acquire_Store src/gutil/atomicops-internals-gcc.h /^inline void Acquire_Store(volatile Atomic64* ptr, Atomic64 value) {$/;" f namespace:base::subtle +Acquire_Store src/gutil/atomicops-internals-tsan.h /^inline void Acquire_Store(volatile Atomic32 *ptr, Atomic32 value) {$/;" f namespace:base::subtle +Acquire_Store src/gutil/atomicops-internals-tsan.h /^inline void Acquire_Store(volatile Atomic64 *ptr, Atomic64 value) {$/;" f namespace:base::subtle +Acquire_Store src/gutil/atomicops-internals-x86.h /^inline void Acquire_Store(volatile Atomic32* ptr, Atomic32 value) {$/;" f namespace:base::subtle +Acquire_Store src/gutil/atomicops-internals-x86.h /^inline void Acquire_Store(volatile Atomic64* ptr, Atomic64 value) {$/;" f namespace:base::subtle +Acquire_Store src/gutil/atomicops.h /^inline void Acquire_Store($/;" f +Acquire_Store src/gutil/atomicops.h /^inline void Acquire_Store(volatile Atomic32* ptr, Atomic32 value) {$/;" f +Acquire_Store src/gutil/atomicops.h /^inline void Acquire_Store(volatile AtomicWord* ptr, AtomicWord value) {$/;" f +Acquire_Store src/gutil/atomicops.h /^inline void Acquire_Store(volatile AtomicWord* ptr, AtomicWord value) {$/;" f namespace:base::subtle +Active src/runtime/disk_io_mgr_internal.h /^ Active,$/;" e enum:doris::DiskIoMgr::RequestContext::State +Add src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::Add(const TupleRow* row, Tuple* dst) {$/;" f class:doris::NewAggFnEvaluator +Add src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::Add(const std::vector& evals,$/;" f class:doris::NewAggFnEvaluator +Add src/gutil/strings/charset.h /^ void Add(unsigned char c) { bits_[Word(c)] |= BitMask(c); }$/;" f class:strings::CharSet +AddBatch src/exec/exec_node.cpp /^void ExecNode::RowBatchQueue::AddBatch(RowBatch* batch) {$/;" f class:doris::ExecNode::RowBatchQueue +AddBatchCounter src/exec/tablet_sink.h /^struct AddBatchCounter {$/;" s namespace:doris::stream_load +AddBatchWithTimeout src/exec/exec_node.cpp /^bool ExecNode::RowBatchQueue::AddBatchWithTimeout(RowBatch* batch,$/;" f class:doris::ExecNode::RowBatchQueue +AddCleanPage src/runtime/bufferpool/buffer_allocator.cc /^void BufferPool::BufferAllocator::AddCleanPage($/;" f class:doris::BufferPool::BufferAllocator +AddCleanPage src/runtime/bufferpool/buffer_allocator.cc /^void BufferPool::FreeBufferArena::AddCleanPage(Page* page) {$/;" f class:doris::BufferPool::FreeBufferArena +AddDelta src/util/monotime.cpp /^void MonoTime::AddDelta(const MonoDelta &delta) {$/;" f class:doris::MonoTime +AddExpr src/exprs/arithmetic_expr.h /^ AddExpr(const TExprNode& node) : ArithmeticExpr(node) { }$/;" f class:doris::AddExpr +AddExpr src/exprs/arithmetic_expr.h /^class AddExpr : public ArithmeticExpr {$/;" c namespace:doris +AddFreeBuffer src/runtime/bufferpool/buffer_allocator.cc /^ void AddFreeBuffer(BufferHandle&& buffer) {$/;" f struct:doris::BufferPool::FreeBufferArena::PerSizeLists +AddFreeBuffer src/runtime/bufferpool/buffer_allocator.cc /^void BufferPool::FreeBufferArena::AddFreeBuffer(BufferHandle&& buffer) {$/;" f class:doris::BufferPool::FreeBufferArena +AddFreeBuffer src/runtime/bufferpool/free_list.h /^ void AddFreeBuffer(BufferHandle&& buffer) {$/;" f class:doris::FreeList +AddGcFunction src/runtime/mem_tracker.cpp /^void MemTracker::AddGcFunction(GcFunction f) {$/;" f class:doris::MemTracker +AddIntermediateTuple src/exec/new_partitioned_aggregation_node_ir.cc /^Status NewPartitionedAggregationNode::AddIntermediateTuple(Partition* partition,$/;" f class:NewPartitionedAggregationNode +AddRef src/gutil/ref_counted.cc /^void RefCountedBase::AddRef() const {$/;" f class:doris::subtle::RefCountedBase +AddRef src/gutil/ref_counted.cc /^void RefCountedThreadSafeBase::AddRef() const {$/;" f class:doris::subtle::RefCountedThreadSafeBase +AddRef src/gutil/ref_counted.h /^ void AddRef() const {$/;" f class:doris::RefCounted +AddRef src/gutil/ref_counted.h /^ void AddRef() const {$/;" f class:doris::RefCountedThreadSafe +AddRowCustomBegin src/runtime/buffered_tuple_stream3.inline.h /^inline uint8_t* BufferedTupleStream3::AddRowCustomBegin(int64_t size, Status* status) {$/;" f class:doris::BufferedTupleStream3 +AddRowCustomEnd src/runtime/buffered_tuple_stream3.inline.h /^inline void BufferedTupleStream3::AddRowCustomEnd(int64_t size) {$/;" f class:doris::BufferedTupleStream3 +AddSample src/util/streaming_sampler.h /^ void AddSample(T sample, int ms) {$/;" f class:doris::StreamingSampler +AddToFreeList src/runtime/bufferpool/suballocator.cc /^void Suballocator::AddToFreeList(unique_ptr node) {$/;" f class:doris::Suballocator +AddTokenCounts src/gutil/map-util.h /^void AddTokenCounts($/;" f +AddUdf src/udf_samples/udf_sample.cpp /^IntVal AddUdf(FunctionContext* context, const IntVal& arg1, const IntVal& arg2) {$/;" f namespace:doris_udf +AddUdfClose src/udf_samples/udf_sample.cpp /^void AddUdfClose(FunctionContext* context, FunctionContext::FunctionStateScope scope) {}$/;" f namespace:doris_udf +AddUdfPrepare src/udf_samples/udf_sample.cpp /^void AddUdfPrepare(FunctionContext* context, FunctionContext::FunctionStateScope scope) {}$/;" f namespace:doris_udf +AdjustedLastPos src/gutil/strings/util.cc /^char* AdjustedLastPos(const char* str, char separator, int n) {$/;" f +AdvanceIdentifier src/gutil/strings/util.cc /^const char *AdvanceIdentifier(const char *str) {$/;" f +AdvanceIdentifier src/gutil/strings/util.h /^inline char* AdvanceIdentifier(char* str) {$/;" f +AesMode src/util/aes_util.h /^enum AesMode {$/;" g namespace:doris +AesState src/util/aes_util.h /^enum AesState {$/;" g namespace:doris +AesUtil src/util/aes_util.h /^class AesUtil {$/;" c namespace:doris +AesUtilTest test/util/aes_util_test.cpp /^ AesUtilTest() {$/;" f class:doris::AesUtilTest +AesUtilTest test/util/aes_util_test.cpp /^class AesUtilTest : public testing::Test {$/;" c namespace:doris file: +AgentServer src/agent/agent_server.cpp /^AgentServer::AgentServer(ExecEnv* exec_env, const TMasterInfo& master_info) :$/;" f class:doris::AgentServer +AgentServer src/agent/agent_server.h /^class AgentServer {$/;" c namespace:doris +AgentStatus src/agent/status.h /^enum AgentStatus {$/;" g namespace:doris +AgentUtils src/agent/utils.h /^ AgentUtils() {};$/;" f class:doris::AgentUtils +AgentUtils src/agent/utils.h /^class AgentUtils {$/;" c namespace:doris +AggFn src/exprs/agg_fn.cc /^AggFn::AggFn(const TExprNode& tnode, const SlotDescriptor& intermediate_slot_desc,$/;" f class:doris::AggFn +AggFn src/exprs/agg_fn.h /^class AggFn : public Expr {$/;" c namespace:doris +AggFnEvaluator src/exprs/agg_fn_evaluator.cpp /^AggFnEvaluator::AggFnEvaluator(const TExprNode& desc, bool is_analytic_fn) :$/;" f class:doris::AggFnEvaluator +AggFnEvaluator src/exprs/agg_fn_evaluator.h /^class AggFnEvaluator {$/;" c namespace:doris +AggregateFuncMapHash src/olap/aggregate_func.cpp /^struct AggregateFuncMapHash {$/;" s namespace:doris file: +AggregateFuncResolver src/olap/aggregate_func.cpp /^AggregateFuncResolver::AggregateFuncResolver() {$/;" f class:doris::AggregateFuncResolver +AggregateFuncResolver src/olap/aggregate_func.cpp /^class AggregateFuncResolver {$/;" c namespace:doris file: +AggregateFuncTest test/olap/aggregate_func_test.cpp /^ AggregateFuncTest() { }$/;" f class:doris::AggregateFuncTest +AggregateFuncTest test/olap/aggregate_func_test.cpp /^class AggregateFuncTest : public testing::Test {$/;" c namespace:doris file: +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits : public BaseAggregateFuncs {$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits {$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits {$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFuncTraits src/olap/aggregate_func.h /^struct AggregateFuncTraits :$/;" s namespace:doris +AggregateFunctions src/exprs/aggregate_functions.h /^class AggregateFunctions {$/;" c namespace:doris +AggregateInfo src/olap/aggregate_func.cpp /^AggregateInfo::AggregateInfo(const Traits& traits) $/;" f class:doris::AggregateInfo +AggregateInfo src/olap/aggregate_func.h /^class AggregateInfo {$/;" c namespace:doris +AggregateTraits src/olap/aggregate_func.h /^struct AggregateTraits : public AggregateFuncTraits {$/;" s namespace:doris +AggregationNode src/exec/aggregation_node.cpp /^AggregationNode::AggregationNode($/;" f class:doris::AggregationNode +AggregationNode src/exec/aggregation_node.h /^class AggregationNode : public ExecNode {$/;" c namespace:doris +AggregationOp src/exprs/agg_fn.h /^ enum AggregationOp {$/;" g class:doris::AggFn +AggregationOp src/exprs/agg_fn_evaluator.h /^ enum AggregationOp {$/;" g class:doris::AggFnEvaluator +AlignType src/gutil/port.h /^struct AlignType { typedef char result[Size]; };$/;" s +AlignType src/gutil/port.h /^template struct AlignType { };$/;" s +AlignType src/gutil/port.h /^template struct AlignType<0, size> { typedef char result[size]; };$/;" s +AllWritesComplete test/runtime/buffered_block_mgr2_test.cpp /^ static bool AllWritesComplete(const vector& block_mgrs) {$/;" f class:doris::BufferedBlockMgrTest +Allocate src/runtime/bufferpool/buffer_allocator.cc /^Status BufferPool::BufferAllocator::Allocate($/;" f class:doris::BufferPool::BufferAllocator +Allocate src/runtime/bufferpool/suballocator.cc /^Status Suballocator::Allocate(int64_t bytes, unique_ptr* result) {$/;" f class:doris::Suballocator +Allocate src/runtime/bufferpool/system_allocator.cc /^Status SystemAllocator::Allocate(int64_t len, BufferPool::BufferHandle* buffer) {$/;" f class:doris::SystemAllocator +AllocateBlocks test/runtime/buffered_block_mgr2_test.cpp /^ void AllocateBlocks(BufferedBlockMgr2* block_mgr, BufferedBlockMgr2::Client* client,$/;" f class:doris::BufferedBlockMgrTest +AllocateBuffer src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::AllocateBuffer($/;" f class:doris::BufferPool +AllocateBuffer src/runtime/bufferpool/suballocator.cc /^Status Suballocator::AllocateBuffer(int64_t bytes, unique_ptr* result) {$/;" f class:doris::Suballocator +AllocateFrom src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::AllocateFrom(int64_t bytes) {$/;" f class:doris::ReservationTracker +AllocateInternal src/runtime/bufferpool/buffer_allocator.cc /^Status BufferPool::BufferAllocator::AllocateInternal(int64_t len, BufferHandle* buffer) {$/;" f class:doris::BufferPool::BufferAllocator +AllocateViaMMap src/runtime/bufferpool/system_allocator.cc /^Status SystemAllocator::AllocateViaMMap(int64_t len, uint8_t** buffer_mem) {$/;" f class:doris::SystemAllocator +AllocateViaMalloc src/runtime/bufferpool/system_allocator.cc /^Status SystemAllocator::AllocateViaMalloc(int64_t len, uint8_t** buffer_mem) {$/;" f class:doris::SystemAllocator +AllowEmpty src/gutil/strings/split.h /^struct AllowEmpty {$/;" s namespace:strings +AlphaMergeContext src/olap/rowset/alpha_rowset_reader.h /^struct AlphaMergeContext {$/;" s namespace:doris +AlphaMergeContextComparator src/olap/rowset/alpha_rowset_reader.h /^struct AlphaMergeContextComparator {$/;" s namespace:doris +AlphaNum src/gutil/strings/strcat.h /^ AlphaNum(StringPiece pc)$/;" f struct:AlphaNum +AlphaNum src/gutil/strings/strcat.h /^ AlphaNum(const char *c_str) : piece(c_str) {} \/\/ NOLINT(runtime\/explicit)$/;" f struct:AlphaNum +AlphaNum src/gutil/strings/strcat.h /^ AlphaNum(const string &s) : piece(s) {} \/\/ NOLINT(runtime\/explicit)$/;" f struct:AlphaNum +AlphaNum src/gutil/strings/strcat.h /^ AlphaNum(double f) \/\/ NOLINT(runtime\/explicit)$/;" f struct:AlphaNum +AlphaNum src/gutil/strings/strcat.h /^ AlphaNum(float f) \/\/ NOLINT(runtime\/explicit)$/;" f struct:AlphaNum +AlphaNum src/gutil/strings/strcat.h /^ AlphaNum(int32 i32) \/\/ NOLINT(runtime\/explicit)$/;" f struct:AlphaNum +AlphaNum src/gutil/strings/strcat.h /^ AlphaNum(int64 i64) \/\/ NOLINT(runtime\/explicit)$/;" f struct:AlphaNum +AlphaNum src/gutil/strings/strcat.h /^ AlphaNum(size_t size) \/\/ NOLINT(runtime\/explicit)$/;" f struct:AlphaNum +AlphaNum src/gutil/strings/strcat.h /^ AlphaNum(uint32 u32) \/\/ NOLINT(runtime\/explicit)$/;" f struct:AlphaNum +AlphaNum src/gutil/strings/strcat.h /^ AlphaNum(uint64 u64) \/\/ NOLINT(runtime\/explicit)$/;" f struct:AlphaNum +AlphaNum src/gutil/strings/strcat.h /^struct AlphaNum {$/;" s +AlphaRowset src/olap/rowset/alpha_rowset.cpp /^AlphaRowset::AlphaRowset(const TabletSchema* schema,$/;" f class:doris::AlphaRowset +AlphaRowset src/olap/rowset/alpha_rowset.h /^class AlphaRowset : public Rowset {$/;" c namespace:doris +AlphaRowsetMeta src/olap/rowset/alpha_rowset_meta.h /^class AlphaRowsetMeta : public RowsetMeta {$/;" c namespace:doris +AlphaRowsetReader src/olap/rowset/alpha_rowset_reader.cpp /^AlphaRowsetReader::AlphaRowsetReader($/;" f class:doris::AlphaRowsetReader +AlphaRowsetReader src/olap/rowset/alpha_rowset_reader.h /^class AlphaRowsetReader : public RowsetReader {$/;" c namespace:doris +AlphaRowsetTest test/olap/rowset/alpha_rowset_test.cpp /^class AlphaRowsetTest : public testing::Test {$/;" c namespace:doris file: +AlphaRowsetWriter src/olap/rowset/alpha_rowset_writer.cpp /^AlphaRowsetWriter::AlphaRowsetWriter() :$/;" f class:doris::AlphaRowsetWriter +AlphaRowsetWriter src/olap/rowset/alpha_rowset_writer.h /^class AlphaRowsetWriter : public RowsetWriter {$/;" c namespace:doris +AlreadyExist src/common/status.h /^ static Status AlreadyExist(const Slice& msg,$/;" f class:doris::Status +AlterTabletTask src/olap/tablet_meta.h /^ AlterTabletTask() {}$/;" f class:doris::AlterTabletTask +AlterTabletTask src/olap/tablet_meta.h /^class AlterTabletTask {$/;" c namespace:doris +AlterTabletTaskSharedPtr src/olap/tablet_meta.h /^typedef std::shared_ptr AlterTabletTaskSharedPtr;$/;" t namespace:doris +AnalyticEvalNode src/exec/analytic_eval_node.cpp /^AnalyticEvalNode::AnalyticEvalNode(ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::AnalyticEvalNode +AnalyticEvalNode src/exec/analytic_eval_node.h /^class AnalyticEvalNode : public ExecNode {$/;" c namespace:doris +AnalyticFnScope src/exec/analytic_eval_node.h /^ enum AnalyticFnScope {$/;" g class:doris::AnalyticEvalNode +AndPredicate src/exprs/compound_predicate.h /^ AndPredicate(const TExprNode& node) : CompoundPredicate(node) { }$/;" f class:doris::AndPredicate +AndPredicate src/exprs/compound_predicate.h /^class AndPredicate: public CompoundPredicate {$/;" c namespace:doris +AnnotateBarrierDestroy src/gutil/dynamic_annotations.c /^void AnnotateBarrierDestroy(const char *file, int line,$/;" f +AnnotateBarrierInit src/gutil/dynamic_annotations.c /^void AnnotateBarrierInit(const char *file, int line,$/;" f +AnnotateBarrierWaitAfter src/gutil/dynamic_annotations.c /^void AnnotateBarrierWaitAfter(const char *file, int line,$/;" f +AnnotateBarrierWaitBefore src/gutil/dynamic_annotations.c /^void AnnotateBarrierWaitBefore(const char *file, int line,$/;" f +AnnotateBenignRace src/gutil/dynamic_annotations.c /^void AnnotateBenignRace(const char *file, int line,$/;" f +AnnotateBenignRaceSized src/gutil/dynamic_annotations.c /^void AnnotateBenignRaceSized(const char *file, int line,$/;" f +AnnotateCondVarSignal src/gutil/dynamic_annotations.c /^void AnnotateCondVarSignal(const char *file, int line,$/;" f +AnnotateCondVarSignalAll src/gutil/dynamic_annotations.c /^void AnnotateCondVarSignalAll(const char *file, int line,$/;" f +AnnotateCondVarWait src/gutil/dynamic_annotations.c /^void AnnotateCondVarWait(const char *file, int line,$/;" f +AnnotateEnableRaceDetection src/gutil/dynamic_annotations.c /^void AnnotateEnableRaceDetection(const char *file, int line, int enable){}$/;" f +AnnotateExpectRace src/gutil/dynamic_annotations.c /^void AnnotateExpectRace(const char *file, int line,$/;" f +AnnotateFlushState src/gutil/dynamic_annotations.c /^void AnnotateFlushState(const char *file, int line){}$/;" f +AnnotateIgnoreReadsBegin src/gutil/dynamic_annotations.c /^void AnnotateIgnoreReadsBegin(const char *file, int line){}$/;" f +AnnotateIgnoreReadsEnd src/gutil/dynamic_annotations.c /^void AnnotateIgnoreReadsEnd(const char *file, int line){}$/;" f +AnnotateIgnoreWritesBegin src/gutil/dynamic_annotations.c /^void AnnotateIgnoreWritesBegin(const char *file, int line){}$/;" f +AnnotateIgnoreWritesEnd src/gutil/dynamic_annotations.c /^void AnnotateIgnoreWritesEnd(const char *file, int line){}$/;" f +AnnotateMutexIsUsedAsCondVar src/gutil/dynamic_annotations.c /^void AnnotateMutexIsUsedAsCondVar(const char *file, int line,$/;" f +AnnotateNewMemory src/gutil/dynamic_annotations.c /^void AnnotateNewMemory(const char *file, int line,$/;" f +AnnotateNoOp src/gutil/dynamic_annotations.c /^void AnnotateNoOp(const char *file, int line,$/;" f +AnnotatePCQCreate src/gutil/dynamic_annotations.c /^void AnnotatePCQCreate(const char *file, int line,$/;" f +AnnotatePCQDestroy src/gutil/dynamic_annotations.c /^void AnnotatePCQDestroy(const char *file, int line,$/;" f +AnnotatePCQGet src/gutil/dynamic_annotations.c /^void AnnotatePCQGet(const char *file, int line,$/;" f +AnnotatePCQPut src/gutil/dynamic_annotations.c /^void AnnotatePCQPut(const char *file, int line,$/;" f +AnnotatePublishMemoryRange src/gutil/dynamic_annotations.c /^void AnnotatePublishMemoryRange(const char *file, int line,$/;" f +AnnotateRWLockAcquired src/gutil/dynamic_annotations.c /^void AnnotateRWLockAcquired(const char *file, int line,$/;" f +AnnotateRWLockCreate src/gutil/dynamic_annotations.c /^void AnnotateRWLockCreate(const char *file, int line,$/;" f +AnnotateRWLockDestroy src/gutil/dynamic_annotations.c /^void AnnotateRWLockDestroy(const char *file, int line,$/;" f +AnnotateRWLockReleased src/gutil/dynamic_annotations.c /^void AnnotateRWLockReleased(const char *file, int line,$/;" f +AnnotateThreadName src/gutil/dynamic_annotations.c /^void AnnotateThreadName(const char *file, int line,$/;" f +AnnotateTraceMemory src/gutil/dynamic_annotations.c /^void AnnotateTraceMemory(const char *file, int line,$/;" f +AnnotateUnpublishMemoryRange src/gutil/dynamic_annotations.c /^void AnnotateUnpublishMemoryRange(const char *file, int line,$/;" f +AnyOf src/gutil/strings/split.cc /^AnyOf::AnyOf(StringPiece sp) : delimiters_(sp.ToString()) {$/;" f class:strings::delimiter::AnyOf +AnyOf src/gutil/strings/split.h /^class AnyOf {$/;" c namespace:strings::delimiter +AnyOfPolicy src/gutil/strings/split.cc /^struct AnyOfPolicy {$/;" s namespace:strings::delimiter::__anon29 file: +AnyVal output/udf/include/udf.h /^ AnyVal() : is_null(false) {}$/;" f struct:doris_udf::AnyVal +AnyVal output/udf/include/udf.h /^ AnyVal(bool is_null) : is_null(is_null) {}$/;" f struct:doris_udf::AnyVal +AnyVal output/udf/include/udf.h /^struct AnyVal {$/;" s namespace:doris_udf +AnyVal src/udf/udf.h /^ AnyVal() : is_null(false) {}$/;" f struct:doris_udf::AnyVal +AnyVal src/udf/udf.h /^ AnyVal(bool is_null) : is_null(is_null) {}$/;" f struct:doris_udf::AnyVal +AnyVal src/udf/udf.h /^struct AnyVal {$/;" s namespace:doris_udf +AnyValUtil src/exprs/anyval_util.h /^class AnyValUtil {$/;" c namespace:doris +Append1 src/gutil/strings/strcat.cc /^static char *Append1(char *out, const AlphaNum &x) {$/;" f file: +Append2 src/gutil/strings/strcat.cc /^static char *Append2(char *out, const AlphaNum &x1, const AlphaNum &x2) {$/;" f file: +Append4 src/gutil/strings/strcat.cc /^static char *Append4(char *out,$/;" f file: +AppendKeysFromMap src/gutil/map-util.h /^void AppendKeysFromMap(const MapContainer& map_container,$/;" f +AppendNextNode src/exec/new_partitioned_hash_table.inline.h /^inline NewPartitionedHashTable::DuplicateNode* NewPartitionedHashTable::AppendNextNode(Bucket* bucket) {$/;" f class:doris::NewPartitionedHashTable +AppendRowTuples test/runtime/buffered_tuple_stream2_test.cpp /^ void AppendRowTuples(TupleRow* row, vector* results) {$/;" f class:doris::SimpleTupleStreamTest +AppendRowTuples test/runtime/buffered_tuple_stream2_test.cpp /^ void AppendRowTuples(TupleRow* row, vector* results) {$/;" f class:doris::SimpleTupleStreamTest +AppendSpilledRow src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::AppendSpilledRow($/;" f class:doris::NewPartitionedAggregationNode +AppendTo src/gutil/strings/split.cc /^void AppendTo(Container* container, Splitter splitter) {$/;" f namespace:__anon30 +AppendToImpl src/gutil/strings/split.cc /^void AppendToImpl(Container* container, Splitter splitter) {$/;" f namespace:__anon30 +AppendToImpl src/gutil/strings/split.cc /^void AppendToImpl(map* map_container, Splitter splitter) {$/;" f namespace:__anon30 +AppendToImpl src/gutil/strings/split.cc /^void AppendToImpl(unordered_map* map_container, Splitter splitter) {$/;" f namespace:__anon30 +AppendToImpl src/gutil/strings/split.cc /^void AppendToImpl(vector* container, Splitter splitter) {$/;" f namespace:__anon30 +AppendToMap src/gutil/strings/split.cc /^void AppendToMap(Map* m, Splitter splitter) {$/;" f namespace:__anon30 +AppendToString src/gutil/strings/stringpiece.cc /^void StringPiece::AppendToString(string* target) const {$/;" f class:StringPiece +AppendValue test/runtime/buffered_tuple_stream2_test.cpp /^ void AppendValue(Tuple* t, vector* results) {$/;" f class:doris::SimpleTupleStreamTest +AppendValue test/runtime/buffered_tuple_stream2_test.cpp /^ void AppendValue(Tuple* t, vector* results) {$/;" f class:doris::SimpleTupleStreamTest +AppendValuesFromMap src/gutil/map-util.h /^void AppendValuesFromMap(const MapContainer& map_container,$/;" f +ArgumentMap src/http/web_page_handler.h /^ typedef std::map ArgumentMap;$/;" t class:doris::WebPageHandler +ArithmeticExpr src/exprs/arithmetic_expr.h /^ ArithmeticExpr(const TExprNode& node) : Expr(node) { }$/;" f class:doris::ArithmeticExpr +ArithmeticExpr src/exprs/arithmetic_expr.h /^class ArithmeticExpr : public Expr {$/;" c namespace:doris +ArrowRowBatchTest test/util/arrow/arrow_row_batch_test.cpp /^ ArrowRowBatchTest() { }$/;" f class:doris::ArrowRowBatchTest +ArrowRowBatchTest test/util/arrow/arrow_row_batch_test.cpp /^class ArrowRowBatchTest : public testing::Test {$/;" c namespace:doris file: +ArrowRowBlockTest test/util/arrow/arrow_row_block_test.cpp /^ ArrowRowBlockTest() { }$/;" f class:doris::ArrowRowBlockTest +ArrowRowBlockTest test/util/arrow/arrow_row_block_test.cpp /^class ArrowRowBlockTest : public testing::Test {$/;" c namespace:doris file: +ArrowWorkFlowTest test/util/arrow/arrow_work_flow_test.cpp /^ ArrowWorkFlowTest(){}$/;" f class:doris::ArrowWorkFlowTest +ArrowWorkFlowTest test/util/arrow/arrow_work_flow_test.cpp /^class ArrowWorkFlowTest : public testing::Test {$/;" c namespace:doris file: +AssertNumRowsNode src/exec/assert_num_rows_node.cpp /^AssertNumRowsNode::AssertNumRowsNode($/;" f class:doris::AssertNumRowsNode +AssertNumRowsNode src/exec/assert_num_rows_node.h /^class AssertNumRowsNode : public ExecNode {$/;" c namespace:doris +AsserterBase src/gutil/threading/thread_collision_warner.h /^struct BASE_EXPORT AsserterBase {$/;" s namespace:base +AtEnd src/exec/new_partitioned_hash_table.h /^ bool ALWAYS_INLINE AtEnd() const { return bucket_idx_ == BUCKET_NOT_FOUND; }$/;" f class:doris::NewPartitionedHashTable::Iterator +AtEnd src/exec/new_partitioned_hash_table.h /^ bool ALWAYS_INLINE AtEnd() const {$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +Atomic32 src/gutil/atomicops-internals-gcc.h /^typedef int32_t Atomic32;$/;" t +Atomic32 src/gutil/atomicops-internals-tsan.h /^typedef int32_t Atomic32;$/;" t +Atomic32 src/gutil/atomicops-internals-tsan.h /^typedef int32_t Atomic32;$/;" t namespace:base::subtle +Atomic32 src/gutil/atomicops-internals-x86.h /^typedef int32_t Atomic32;$/;" t +Atomic32 src/gutil/atomicops-internals-x86.h /^typedef int32_t Atomic32;$/;" t namespace:base::subtle +Atomic64 src/gutil/atomicops-internals-gcc.h /^typedef int64_t Atomic64;$/;" t namespace:base::subtle +Atomic64 src/gutil/atomicops-internals-tsan.h /^typedef int64_t Atomic64;$/;" t +Atomic64 src/gutil/atomicops-internals-tsan.h /^typedef int64_t Atomic64;$/;" t namespace:base::subtle +Atomic64 src/gutil/atomicops-internals-x86.h /^typedef int64_t Atomic64;$/;" t +Atomic64 src/gutil/atomicops-internals-x86.h /^typedef int64_t Atomic64;$/;" t namespace:base::subtle +AtomicInt src/common/atomic.h /^ AtomicInt() : _value(0) {}$/;" f class:doris::AtomicInt +AtomicInt src/common/atomic.h /^ AtomicInt(T initial) : _value(initial) {}$/;" f class:doris::AtomicInt +AtomicInt src/common/atomic.h /^class AtomicInt {$/;" c namespace:doris +AtomicInt32 src/common/atomic.h /^typedef AtomicInt AtomicInt32;$/;" t namespace:doris +AtomicInt64 src/common/atomic.h /^typedef AtomicInt AtomicInt64;$/;" t namespace:doris +AtomicOps_Internalx86CPUFeaturesInit src/gutil/atomicops-internals-x86.cc /^static void AtomicOps_Internalx86CPUFeaturesInit() {$/;" f file: +AtomicOps_x86CPUFeatureStruct src/gutil/atomicops-internals-tsan.h /^struct AtomicOps_x86CPUFeatureStruct {$/;" s +AtomicOps_x86CPUFeaturesInit src/gutil/atomicops-internals-x86.cc /^void AtomicOps_x86CPUFeaturesInit() {$/;" f +AtomicPtr src/common/atomic.h /^ AtomicPtr(T* initial = nullptr) : _ptr(reinterpret_cast(initial)) {}$/;" f class:doris::AtomicPtr +AtomicPtr src/common/atomic.h /^class AtomicPtr {$/;" c namespace:doris +AtomicRefCount src/gutil/ref_counted.h /^typedef Atomic32 AtomicRefCount;$/;" t namespace:doris::subtle +AtomicUtil src/common/atomic.h /^class AtomicUtil {$/;" c namespace:doris +AtomicWord src/gutil/atomicops.h /^typedef intptr_t AtomicWord;$/;" t +AuthInfo src/common/utils.h /^struct AuthInfo {$/;" s namespace:doris +AutoDigitLessThan src/gutil/strings/numbers.cc /^bool AutoDigitLessThan(const char* a, int alen, const char* b, int blen) {$/;" f +AutoDigitStrCmp src/gutil/strings/numbers.cc /^int AutoDigitStrCmp(const char* a, int alen,$/;" f +AutoIncrementIterator src/olap/generic_iterators.cpp /^ AutoIncrementIterator(const Schema& schema, size_t num_rows)$/;" f class:doris::AutoIncrementIterator +AutoIncrementIterator src/olap/generic_iterators.cpp /^class AutoIncrementIterator : public RowwiseIterator {$/;" c namespace:doris file: +AvgState src/exprs/aggregate_functions.cpp /^struct AvgState {$/;" s namespace:doris file: +BAD_GATEWAY src/http/http_status.h /^ BAD_GATEWAY = 502,$/;" e enum:doris::HttpStatus +BAD_REQUEST src/http/http_status.h /^ BAD_REQUEST = 400,$/;" e enum:doris::HttpStatus +BASE_ATOMICOPS_INTERNALS_GCC_GENERIC_H_ src/gutil/atomicops-internals-gcc.h 39;" d +BASE_ATOMICOPS_INTERNALS_TSAN_H_ src/gutil/atomicops-internals-tsan.h 9;" d +BASE_ATOMIC_REFCOUNT_H_ src/gutil/atomic_refcount.h 2;" d +BASE_BASICTYPES_H_ src/gutil/basictypes.h 6;" d +BASE_CASTS_H_ src/gutil/casts.h 11;" d +BASE_COMPACTION src/olap/olap_common.h /^ BASE_COMPACTION = 1,$/;" e enum:doris::CompactionType +BASE_CPU_H_ src/gutil/cpu.h 5;" d +BASE_EXPORT src/gutil/atomicops-internals-tsan.h 13;" d +BASE_EXPORT src/gutil/threading/thread_collision_warner.h 14;" d +BASE_HAS_ATOMIC64 src/gutil/atomicops-internals-x86.h 34;" d +BASE_INT128_H_ src/gutil/int128.h 6;" d +BASE_INTEGRAL_TYPES_H_ src/gutil/integral_types.h 11;" d +BASE_MACROS_H_ src/gutil/macros.h 11;" d +BASE_MEMORY_REF_COUNTED_H_ src/gutil/ref_counted.h 6;" d +BASE_MOVE_H_ src/gutil/move.h 6;" d +BASE_ONCE_H_ src/gutil/once.h 25;" d +BASE_PORT_H_ src/gutil/port.h 9;" d +BASE_PORT_H_ALIGNTYPE_TEMPLATE src/gutil/port.h 703;" d +BASE_PORT_H_ALIGNTYPE_TEMPLATE src/gutil/port.h 727;" d +BASE_PORT_H_ALIGN_ATTRIBUTE src/gutil/port.h 694;" d +BASE_PORT_H_ALIGN_ATTRIBUTE src/gutil/port.h 697;" d +BASE_PORT_H_ALIGN_ATTRIBUTE src/gutil/port.h 728;" d +BASE_PORT_H_ALIGN_OF src/gutil/port.h 695;" d +BASE_PORT_H_ALIGN_OF src/gutil/port.h 698;" d +BASE_SPINLOCK_INTERNAL_H_ src/gutil/spinlock_internal.h 37;" d +BASE_STRTOINT_H_ src/gutil/strtoint.h 31;" d +BASE_TEMPLATE_UTIL_H_ src/gutil/template_util.h 50;" d +BASE_THREADING_THREAD_COLLISION_WARNER_H_ src/gutil/threading/thread_collision_warner.h 6;" d +BASE_TYPE_TRAITS_H_ src/gutil/type_traits.h 60;" d +BATCH_CAPACITY test/runtime/data_stream_test.cpp /^ static const int BATCH_CAPACITY = 100; \/\/ rows$/;" m class:doris::DataStreamTest file: +BATCH_SIZE test/runtime/buffered_tuple_stream2_test.cpp /^static const int BATCH_SIZE = 250;$/;" v file: +BE_EXEC_ES_HTTP_SCANNER_H src/exec/es_http_scanner.h 20;" d +BE_EXEC_ES_HTTP_SCAN_NODE_H src/exec/es_http_scan_node.h 19;" d +BE_EXEC_ES_PREDICATE_H src/exec/es/es_predicate.h 19;" d +BE_SRC_EXEC_BASE_SCANNER_H_ src/exec/base_scanner.h 19;" d +BE_TEST test/agent/cgroups_mgr_test.cpp 27;" d file: +BE_TEST test/olap/file_helper_test.cpp 29;" d file: +BE_TEST test/olap/file_utils_test.cpp 33;" d file: +BE_TEST test/olap/olap_meta_test.cpp 29;" d file: +BE_TEST test/olap/olap_snapshot_converter_test.cpp 39;" d file: +BE_TEST test/olap/rowset/alpha_rowset_test.cpp 39;" d file: +BE_TEST test/olap/rowset/rowset_converter_test.cpp 41;" d file: +BE_TEST test/olap/rowset/rowset_meta_manager_test.cpp 31;" d file: +BE_TEST test/olap/rowset/rowset_meta_test.cpp 31;" d file: +BE_TEST test/olap/tablet_meta_manager_test.cpp 31;" d file: +BE_TEST test/olap/tablet_mgr_test.cpp 36;" d file: +BE_TEST test/olap/txn_manager_test.cpp 35;" d file: +BIGINT_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t BIGINT_SIZE = sizeof(int64_t);$/;" m class:doris::AggFnEvaluator +BIGINT_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t BIGINT_SIZE = sizeof(int64_t);$/;" m class:doris::NewAggFnEvaluator +BIG_CONSTANT src/util/murmur_hash3.cpp 26;" d file: +BIG_CONSTANT src/util/murmur_hash3.cpp 47;" d file: +BILLION src/util/debug_util.cpp 42;" d file: +BILLION src/util/pretty_printer.h /^ static const int64_t BILLION = MILLION * 1000;$/;" m class:doris::PrettyPrinter +BINARY_ARITH_FNS src/exprs/arithmetic_expr.cpp 77;" d file: +BINARY_BIT_FNS src/exprs/arithmetic_expr.cpp 134;" d file: +BINARY_DICT_PAGE_HEADER_SIZE src/olap/rowset/segment_v2/binary_dict_page.h /^ BINARY_DICT_PAGE_HEADER_SIZE = 4$/;" e enum:doris::segment_v2::__anon53 +BINARY_DIV_FNS src/exprs/arithmetic_expr.cpp 90;" d file: +BINARY_MOD_FNS src/exprs/arithmetic_expr.cpp 101;" d file: +BINARY_OP_CHECK_ZERO_FN src/exprs/arithmetic_expr.cpp 51;" d file: +BINARY_OP_CHECK_ZERO_FN src/exprs/operators.cpp 38;" d file: +BINARY_OP_CHECK_ZERO_INT_TYPES src/exprs/operators.cpp 94;" d file: +BINARY_OP_FN src/exprs/arithmetic_expr.cpp 64;" d file: +BINARY_OP_FN src/exprs/operators.cpp 31;" d file: +BINARY_OP_INT_TYPES src/exprs/operators.cpp 87;" d file: +BINARY_OP_NUMERIC_TYPES src/exprs/operators.cpp 78;" d file: +BINARY_PREDICATE_ALL_TYPES src/exprs/operators.cpp 101;" d file: +BINARY_PREDICATE_NONNUMERIC_FN src/exprs/operators.cpp 69;" d file: +BINARY_PREDICATE_NONNUMERIC_FNS src/exprs/decimal_operators.cpp 181;" d file: +BINARY_PREDICATE_NONNUMERIC_FNS src/exprs/decimalv2_operators.cpp 190;" d file: +BINARY_PREDICATE_NUMERIC_FN src/exprs/operators.cpp 62;" d file: +BINARY_PRED_FLOAT_FNS src/exprs/binary_predicate.cpp 299;" d file: +BINARY_PRED_FN src/exprs/binary_predicate.cpp 270;" d file: +BINARY_PRED_FOR_NULL_FLOAT_FNS src/exprs/binary_predicate.cpp 422;" d file: +BINARY_PRED_FOR_NULL_FN src/exprs/binary_predicate.cpp 399;" d file: +BINARY_PRED_FOR_NULL_INT_FNS src/exprs/binary_predicate.cpp 412;" d file: +BINARY_PRED_INT_FNS src/exprs/binary_predicate.cpp 284;" d file: +BIN_PRED_CLASSES_DEFINE src/exprs/binary_predicate.h 55;" d +BIN_PRED_CLASS_DEFINE src/exprs/binary_predicate.h 44;" d +BIN_PRED_FOR_NULL_CLASSES_DEFINE src/exprs/binary_predicate.h 88;" d +BIN_PRED_FOR_NULL_CLASS_DEFINE src/exprs/binary_predicate.h 77;" d +BITFIELD_POSITIONS src/olap/rowset/segment_reader.h /^ static const int32_t BITFIELD_POSITIONS = RUN_LENGTH_BYTE_POSITIONS + 1;$/;" m class:doris::SegmentReader +BITMAP src/util/bitmap_value.h /^ BITMAP = 2 \/\/ more than one elements$/;" e enum:doris::BitmapValue::BitmapDataType +BITMAP32 src/util/bitmap_value.h /^ BITMAP32 = 2,$/;" e enum:doris::BitmapTypeCode::type +BITMAP64 src/util/bitmap_value.h /^ BITMAP64 = 4$/;" e enum:doris::BitmapTypeCode::type +BITMAP_COMPARE src/olap/comparison_predicate.cpp 193;" d file: +BITMAP_COMPARE_EqualPredicate src/olap/comparison_predicate.cpp 143;" d file: +BITMAP_COMPARE_GreaterEqualPredicate src/olap/comparison_predicate.cpp 186;" d file: +BITMAP_COMPARE_GreaterPredicate src/olap/comparison_predicate.cpp 176;" d file: +BITMAP_COMPARE_LessEqualPredicate src/olap/comparison_predicate.cpp 167;" d file: +BITMAP_COMPARE_LessPredicate src/olap/comparison_predicate.cpp 161;" d file: +BITMAP_COMPARE_NotEqualPredicate src/olap/comparison_predicate.cpp 152;" d file: +BITNOT_FN src/exprs/operators.cpp 45;" d file: +BITNOT_FNS src/exprs/arithmetic_expr.cpp 154;" d file: +BITNOT_OP_FN src/exprs/arithmetic_expr.cpp 145;" d file: +BITSHUFFLE_H src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp 28;" d file: +BITSHUFFLE_PAGE_HEADER_SIZE src/olap/rowset/segment_v2/bitshuffle_page.h /^ BITSHUFFLE_PAGE_HEADER_SIZE = 16$/;" e enum:doris::segment_v2::__anon43 +BITS_SET_PER_BLOCK src/olap/rowset/segment_v2/block_split_bloom_filter.h /^ static const int BITS_SET_PER_BLOCK = 8;$/;" m class:doris::segment_v2::BlockSplitBloomFilter +BIT_AND src/exprs/arithmetic_expr.h /^ BIT_AND,$/;" e enum:doris::ArithmeticExpr::BinaryOpType +BIT_INDEX_MASK src/util/bitmap.h /^ static const int64_t BIT_INDEX_MASK = 63;$/;" m class:doris::Bitmap +BIT_NOT src/exprs/arithmetic_expr.h /^ BIT_NOT,$/;" e enum:doris::ArithmeticExpr::BinaryOpType +BIT_OR src/exprs/arithmetic_expr.h /^ BIT_OR,$/;" e enum:doris::ArithmeticExpr::BinaryOpType +BIT_XOR src/exprs/arithmetic_expr.h /^ BIT_XOR,$/;" e enum:doris::ArithmeticExpr::BinaryOpType +BLOCKS_REQUIRED_FOR_MERGE src/runtime/spill_sorter.cc /^const int BLOCKS_REQUIRED_FOR_MERGE = 3;$/;" m namespace:doris file: +BLOCK_MASK src/runtime/buffered_tuple_stream.h /^ static const uint64_t BLOCK_MASK = 0xFFFF;$/;" m struct:doris::BufferedTupleStream::RowIdx +BLOCK_MASK src/runtime/buffered_tuple_stream2.h /^ static const uint64_t BLOCK_MASK = 0xFFFF;$/;" m struct:doris::BufferedTupleStream2::RowIdx +BLOCK_SHIFT src/runtime/buffered_tuple_stream.h /^ static const uint64_t BLOCK_SHIFT = 0;$/;" m struct:doris::BufferedTupleStream::RowIdx +BLOCK_SHIFT src/runtime/buffered_tuple_stream2.h /^ static const uint64_t BLOCK_SHIFT = 0;$/;" m struct:doris::BufferedTupleStream2::RowIdx +BLOCK_SIZE src/util/core_local.cpp /^constexpr int BLOCK_SIZE = 4096;$/;" m namespace:doris file: +BLOOM_FILTER_DEFAULT_FPP src/olap/olap_define.h /^static const double BLOOM_FILTER_DEFAULT_FPP = 0.05;$/;" m namespace:doris +BLOOM_FILTER_NULL_HASHCODE src/olap/bloom_filter.hpp /^static const uint64_t BLOOM_FILTER_NULL_HASHCODE = 2862933555777941757ULL;$/;" m namespace:doris +BLOOM_WIDTH src/runtime/string_search.hpp /^ static const int BLOOM_WIDTH = 64;$/;" m class:doris::StringSearch +BRpcService src/service/brpc_service.cpp /^BRpcService::BRpcService(ExecEnv* exec_env)$/;" f class:doris::BRpcService +BRpcService src/service/brpc_service.h /^class BRpcService {$/;" c namespace:doris +BUCKET_NOT_FOUND src/exec/new_partitioned_hash_table.h /^ static const int64_t BUCKET_NOT_FOUND = -1;$/;" m class:doris::NewPartitionedHashTable::Iterator +BUCKET_NOT_FOUND src/exec/partitioned_hash_table.h /^ static const int64_t BUCKET_NOT_FOUND = -1;$/;" m class:doris::PartitionedHashTable::Iterator +BUFFER_SIZE src/util/perf_counters.cpp 37;" d file: +BUFREADCOMMENT src/util/minizip/unzip.c 405;" d file: +BUF_SIZE src/olap/utils.h /^ static const int BUF_SIZE = 256;$/;" m class:doris::Errno +BUILD_ROWBATCH src/exec/olap_scan_node.h /^ BUILD_ROWBATCH = 3,$/;" e enum:doris::TransferStatus +BUILD_VERSION src/olap/olap_define.h 462;" d +BUTIL_BASICTYPES_H_ src/service/brpc.h 41;" d +BYTES_PER_BLOCK src/olap/rowset/segment_v2/block_split_bloom_filter.h /^ static const uint32_t BYTES_PER_BLOCK = 32;$/;" m class:doris::segment_v2::BlockSplitBloomFilter +BYTE_STREAM_POSITIONS src/olap/rowset/segment_reader.h /^ static const int32_t BYTE_STREAM_POSITIONS = 1;$/;" m class:doris::SegmentReader +BZIP2 src/exec/decompressor.h /^ BZIP2,$/;" e enum:doris::CompressType +BackInsertPolicy src/gutil/strings/split.h /^struct BackInsertPolicy {$/;" s namespace:strings::internal +BackendOptions src/service/backend_options.h /^class BackendOptions {$/;" c namespace:doris +BackendService src/service/backend_service.cpp /^BackendService::BackendService(ExecEnv* exec_env) :$/;" f class:doris::BackendService +BackendService src/service/backend_service.h /^class BackendService : public BackendServiceIf {$/;" c namespace:doris +BackendServiceClientCache src/runtime/client_cache.h /^typedef ClientCache BackendServiceClientCache;$/;" t namespace:doris +BackendServiceConnection src/runtime/client_cache.h /^typedef ClientConnection BackendServiceConnection;$/;" t namespace:doris +BackslashEscape src/gutil/strings/escaping.cc /^void BackslashEscape(const StringPiece& src,$/;" f namespace:strings +BackslashEscape src/gutil/strings/escaping.h /^inline string BackslashEscape(const StringPiece& src,$/;" f namespace:strings +BackslashUnescape src/gutil/strings/escaping.cc /^void BackslashUnescape(const StringPiece& src,$/;" f namespace:strings +BackslashUnescape src/gutil/strings/escaping.h /^inline string BackslashUnescape(const StringPiece& src,$/;" f namespace:strings +Bad src/gutil/utf/rune.c /^ Bad = Runeerror,$/;" e enum:__anon8 file: +Barrier src/util/barrier.h /^ explicit Barrier(int count) :$/;" f class:doris::Barrier +Barrier src/util/barrier.h /^class Barrier {$/;" c namespace:doris +Barrier_AtomicIncrement src/gutil/atomicops-internals-gcc.h /^inline Atomic32 Barrier_AtomicIncrement(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Barrier_AtomicIncrement src/gutil/atomicops-internals-gcc.h /^inline Atomic64 Barrier_AtomicIncrement(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Barrier_AtomicIncrement src/gutil/atomicops-internals-tsan.h /^inline Atomic32 Barrier_AtomicIncrement(volatile Atomic32 *ptr,$/;" f namespace:base::subtle +Barrier_AtomicIncrement src/gutil/atomicops-internals-tsan.h /^inline Atomic64 Barrier_AtomicIncrement(volatile Atomic64 *ptr,$/;" f namespace:base::subtle +Barrier_AtomicIncrement src/gutil/atomicops-internals-x86.h /^inline Atomic32 Barrier_AtomicIncrement(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Barrier_AtomicIncrement src/gutil/atomicops-internals-x86.h /^inline Atomic64 Barrier_AtomicIncrement(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Barrier_AtomicIncrement src/gutil/atomicops.h /^inline AtomicWord Barrier_AtomicIncrement(volatile AtomicWord* ptr,$/;" f namespace:base::subtle +Barrier_CompareAndSwap src/gutil/atomicops-internals-x86.h /^inline Atomic32 Barrier_CompareAndSwap(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Barrier_CompareAndSwap src/gutil/atomicops-internals-x86.h /^inline Atomic64 Barrier_CompareAndSwap(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Base32Alphabet src/gutil/strings/escaping.cc /^static const char Base32Alphabet[] = {$/;" m namespace:strings file: +Base32Escape src/gutil/strings/escaping.cc /^bool Base32Escape(const string& src, string* dest) {$/;" f namespace:strings +Base32Escape src/gutil/strings/escaping.cc /^int Base32Escape(const unsigned char* src, size_t szsrc,$/;" f namespace:strings +Base32HexAlphabet src/gutil/strings/escaping.cc /^static const char Base32HexAlphabet[] = {$/;" m namespace:strings file: +Base32HexEscape src/gutil/strings/escaping.cc /^bool Base32HexEscape(const string& src, string* dest) {$/;" f namespace:strings +Base32HexEscape src/gutil/strings/escaping.cc /^int Base32HexEscape(const unsigned char* src, size_t szsrc,$/;" f namespace:strings +Base32Unescape src/gutil/strings/escaping.cc /^bool Base32Unescape(const char* src, int slen, string* dest) {$/;" f namespace:strings +Base32Unescape src/gutil/strings/escaping.cc /^int Base32Unescape(const char* src, int slen, char* dest, int szdest) {$/;" f namespace:strings +Base32Unescape src/gutil/strings/escaping.h /^inline bool Base32Unescape(const string& src, string* dest) {$/;" f namespace:strings +Base64Escape src/gutil/strings/escaping.cc /^int Base64Escape(const unsigned char *src, int szsrc, char *dest, int szdest) {$/;" f namespace:strings +Base64Escape src/gutil/strings/escaping.cc /^void Base64Escape(const string& src, string* dest) {$/;" f namespace:strings +Base64Escape src/gutil/strings/escaping.cc /^void Base64Escape(const unsigned char *src, int szsrc,$/;" f namespace:strings +Base64EscapeInternal src/gutil/strings/escaping.cc /^int Base64EscapeInternal(const unsigned char *src, int szsrc,$/;" f namespace:strings +Base64EscapeInternal src/gutil/strings/escaping.cc /^void Base64EscapeInternal(const unsigned char* src, int szsrc,$/;" f namespace:strings +Base64Unescape src/gutil/strings/escaping.cc /^bool Base64Unescape(const char *src, int slen, string* dest) {$/;" f namespace:strings +Base64Unescape src/gutil/strings/escaping.cc /^int Base64Unescape(const char *src, int szsrc, char *dest, int szdest) {$/;" f namespace:strings +Base64Unescape src/gutil/strings/escaping.h /^inline bool Base64Unescape(const string& src, string* dest) {$/;" f namespace:strings +Base64UnescapeInternal src/gutil/strings/escaping.cc /^int Base64UnescapeInternal(const char *src, int szsrc,$/;" f namespace:strings +Base64UnescapeInternal src/gutil/strings/escaping.cc /^static bool Base64UnescapeInternal(const char* src, int slen, string* dest,$/;" f namespace:strings +BaseAggregateFuncs src/olap/aggregate_func.h /^struct BaseAggregateFuncs {$/;" s namespace:doris +BaseClass output/udf/include/uda_test_harness.h /^ typedef UdaTestHarnessBase BaseClass;$/;" t class:doris_udf::UdaTestHarness +BaseClass output/udf/include/uda_test_harness.h /^ typedef UdaTestHarnessBase BaseClass;$/;" t class:doris_udf::UdaTestHarness2 +BaseClass output/udf/include/uda_test_harness.h /^ typedef UdaTestHarnessBase BaseClass;$/;" t class:doris_udf::UdaTestHarness3 +BaseClass output/udf/include/uda_test_harness.h /^ typedef UdaTestHarnessBase BaseClass;$/;" t class:doris_udf::UdaTestHarness4 +BaseClass src/udf/uda_test_harness.h /^ typedef UdaTestHarnessBase BaseClass;$/;" t class:doris_udf::UdaTestHarness +BaseClass src/udf/uda_test_harness.h /^ typedef UdaTestHarnessBase BaseClass;$/;" t class:doris_udf::UdaTestHarness2 +BaseClass src/udf/uda_test_harness.h /^ typedef UdaTestHarnessBase BaseClass;$/;" t class:doris_udf::UdaTestHarness3 +BaseClass src/udf/uda_test_harness.h /^ typedef UdaTestHarnessBase BaseClass;$/;" t class:doris_udf::UdaTestHarness4 +BaseCompaction src/olap/base_compaction.cpp /^BaseCompaction::BaseCompaction(TabletSharedPtr tablet)$/;" f class:doris::BaseCompaction +BaseCompaction src/olap/base_compaction.h /^class BaseCompaction : public Compaction {$/;" c namespace:doris +BaseDeleter src/gutil/stl_util.h /^ BaseDeleter() {}$/;" f class:BaseDeleter +BaseDeleter src/gutil/stl_util.h /^class BaseDeleter {$/;" c +BaseFieldtypeTraits src/olap/types.h /^struct BaseFieldtypeTraits : public CppTypeTraits {$/;" s namespace:doris +BaseScanner src/exec/base_scanner.cpp /^BaseScanner::BaseScanner(RuntimeState* state, RuntimeProfile* profile, const TBrokerScanRangeParams& params, ScannerCounter* counter) :$/;" f class:doris::BaseScanner +BaseScanner src/exec/base_scanner.h /^class BaseScanner {$/;" c namespace:doris +BasicInsertPolicy src/gutil/strings/split.h /^struct BasicInsertPolicy {$/;" s namespace:strings::internal +BasicTest test/exec/partitioned_hash_table_test.cpp /^ void BasicTest(bool quadratic, int initial_num_buckets) {$/;" f class:doris::PartitionedHashTableTest +BatchProcessFunction src/util/batch_process_thread_pool.hpp /^ typedef std::function)> BatchProcessFunction;$/;" t class:doris::BatchProcessThreadPool +BatchProcessThreadPool src/util/batch_process_thread_pool.hpp /^ BatchProcessThreadPool(uint32_t num_threads, uint32_t queue_size, uint32_t batch_size,$/;" f class:doris::BatchProcessThreadPool +BatchProcessThreadPool src/util/batch_process_thread_pool.hpp /^class BatchProcessThreadPool {$/;" c namespace:doris +BatchedRowSupplier src/runtime/sorted_run_merger.cc /^ BatchedRowSupplier(SortedRunMerger* parent, const RunBatchSupplier& sorted_run) :$/;" f class:doris::SortedRunMerger::BatchedRowSupplier +BatchedRowSupplier src/runtime/sorted_run_merger.cc /^class SortedRunMerger::BatchedRowSupplier {$/;" c class:doris::SortedRunMerger file: +Begin src/exec/new_partitioned_hash_table.inline.h /^inline NewPartitionedHashTable::Iterator NewPartitionedHashTable::Begin($/;" f class:doris::NewPartitionedHashTable +BetaRowset src/olap/rowset/beta_rowset.cpp /^BetaRowset::BetaRowset(const TabletSchema* schema,$/;" f class:doris::BetaRowset +BetaRowset src/olap/rowset/beta_rowset.h /^class BetaRowset : public Rowset {$/;" c namespace:doris +BetaRowsetReader src/olap/rowset/beta_rowset_reader.cpp /^BetaRowsetReader::BetaRowsetReader(BetaRowsetSharedPtr rowset)$/;" f class:doris::BetaRowsetReader +BetaRowsetReader src/olap/rowset/beta_rowset_reader.h /^class BetaRowsetReader : public RowsetReader {$/;" c namespace:doris +BetaRowsetTest test/olap/rowset/beta_rowset_test.cpp /^class BetaRowsetTest : public testing::Test {$/;" c namespace:doris file: +BetaRowsetWriter src/olap/rowset/beta_rowset_writer.cpp /^BetaRowsetWriter::BetaRowsetWriter() :$/;" f class:doris::BetaRowsetWriter +BetaRowsetWriter src/olap/rowset/beta_rowset_writer.h /^class BetaRowsetWriter : public RowsetWriter {$/;" c namespace:doris +BfdFindCtx src/util/bfd_parser.cpp /^ BfdFindCtx(bfd_symbol** syms_, bfd_vma pc_)$/;" f struct:doris::BfdFindCtx +BfdFindCtx src/util/bfd_parser.cpp /^struct BfdFindCtx {$/;" s namespace:doris file: +BfdParser src/util/bfd_parser.cpp /^BfdParser::BfdParser(const std::string& file_name) $/;" f class:doris::BfdParser +BfdParser src/util/bfd_parser.h /^class BfdParser {$/;" c namespace:doris +BigEndian src/gutil/endian.h /^class BigEndian {$/;" c +BigIntVal output/udf/include/udf.h /^ BigIntVal() : val(0) { }$/;" f struct:doris_udf::BigIntVal +BigIntVal output/udf/include/udf.h /^ BigIntVal(int64_t val) : val(val) { }$/;" f struct:doris_udf::BigIntVal +BigIntVal output/udf/include/udf.h /^struct BigIntVal : public AnyVal {$/;" s namespace:doris_udf +BigIntVal src/udf/udf.h /^ BigIntVal() : val(0) { }$/;" f struct:doris_udf::BigIntVal +BigIntVal src/udf/udf.h /^ BigIntVal(int64_t val) : val(val) { }$/;" f struct:doris_udf::BigIntVal +BigIntVal src/udf/udf.h /^struct BigIntVal : public AnyVal {$/;" s namespace:doris_udf +BigIntWrapper src/exprs/scalar_fn_call.cpp /^typedef BigIntVal (*BigIntWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +BinaryCompose1 src/gutil/stl_util.h /^BinaryComposeBinary BinaryCompose1(F f, G g) {$/;" f +BinaryCompose2 src/gutil/stl_util.h /^BinaryComposeBinary BinaryCompose2(F f, G1 g1, G2 g2) {$/;" f +BinaryComposeBinary src/gutil/stl_util.h /^ BinaryComposeBinary(F f, G1 g1, G2 g2) : f_(f), g1_(g1), g2_(g2) { }$/;" f class:BinaryComposeBinary +BinaryComposeBinary src/gutil/stl_util.h /^class BinaryComposeBinary : public binary_function BinaryOperate1st(const BinaryOp& f) {$/;" f +BinaryOperate2nd src/gutil/stl_util.h /^BinaryOperateOnSecond BinaryOperate2nd(const BinaryOp& f) {$/;" f +BinaryOperateOnFirst src/gutil/stl_util.h /^ BinaryOperateOnFirst() {$/;" f class:BinaryOperateOnFirst +BinaryOperateOnFirst src/gutil/stl_util.h /^ BinaryOperateOnFirst(const BinaryOp& f) : f_(f) { \/\/ TODO(user): explicit?$/;" f class:BinaryOperateOnFirst +BinaryOperateOnFirst src/gutil/stl_util.h /^class BinaryOperateOnFirst$/;" c +BinaryOperateOnSecond src/gutil/stl_util.h /^ BinaryOperateOnSecond() {$/;" f class:BinaryOperateOnSecond +BinaryOperateOnSecond src/gutil/stl_util.h /^ BinaryOperateOnSecond(const BinaryOp& f) : f_(f) {$/;" f class:BinaryOperateOnSecond +BinaryOperateOnSecond src/gutil/stl_util.h /^class BinaryOperateOnSecond$/;" c +BinaryPlainPageBuilder src/olap/rowset/segment_v2/binary_plain_page.h /^ BinaryPlainPageBuilder(const PageBuilderOptions& options) :$/;" f class:doris::segment_v2::BinaryPlainPageBuilder +BinaryPlainPageBuilder src/olap/rowset/segment_v2/binary_plain_page.h /^class BinaryPlainPageBuilder : public PageBuilder {$/;" c namespace:doris::segment_v2 +BinaryPlainPageDecoder src/olap/rowset/segment_v2/binary_plain_page.h /^ BinaryPlainPageDecoder(Slice data) : BinaryPlainPageDecoder(data, PageDecoderOptions()) { }$/;" f class:doris::segment_v2::BinaryPlainPageDecoder +BinaryPlainPageDecoder src/olap/rowset/segment_v2/binary_plain_page.h /^ BinaryPlainPageDecoder(Slice data, const PageDecoderOptions& options) : _data(data),$/;" f class:doris::segment_v2::BinaryPlainPageDecoder +BinaryPlainPageDecoder src/olap/rowset/segment_v2/binary_plain_page.h /^class BinaryPlainPageDecoder : public PageDecoder {$/;" c namespace:doris::segment_v2 +BinaryPlainPageTest test/olap/rowset/segment_v2/binary_plain_page_test.cpp /^ BinaryPlainPageTest() {}$/;" f class:doris::segment_v2::BinaryPlainPageTest +BinaryPlainPageTest test/olap/rowset/segment_v2/binary_plain_page_test.cpp /^class BinaryPlainPageTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +BinaryPredicate src/exprs/binary_predicate.h /^ BinaryPredicate(const TExprNode& node) : Predicate(node) { $/;" f class:doris::BinaryPredicate +BinaryPredicate src/exprs/binary_predicate.h /^class BinaryPredicate : public Predicate {$/;" c namespace:doris +BinaryPrefixPageBuilder src/olap/rowset/segment_v2/binary_prefix_page.h /^ BinaryPrefixPageBuilder(const PageBuilderOptions& options) :$/;" f class:doris::segment_v2::BinaryPrefixPageBuilder +BinaryPrefixPageBuilder src/olap/rowset/segment_v2/binary_prefix_page.h /^class BinaryPrefixPageBuilder : public PageBuilder {$/;" c namespace:doris::segment_v2 +BinaryPrefixPageDecoder src/olap/rowset/segment_v2/binary_prefix_page.h /^ BinaryPrefixPageDecoder(Slice data, const PageDecoderOptions& options) :$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +BinaryPrefixPageDecoder src/olap/rowset/segment_v2/binary_prefix_page.h /^class BinaryPrefixPageDecoder : public PageDecoder {$/;" c namespace:doris::segment_v2 +BinaryPrefixPageTest test/olap/rowset/segment_v2/binary_prefix_page_test.cpp /^class BinaryPrefixPageTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +BinaryReader src/olap/push_handler.cpp /^BinaryReader::BinaryReader()$/;" f class:doris::BinaryReader +BinaryReader src/olap/push_handler.h /^class BinaryReader: public IBinaryReader {$/;" c namespace:doris +BinarySearchIterator src/olap/utils.h /^ BinarySearchIterator() : _offset(0u) {}$/;" f class:doris::BinarySearchIterator +BinarySearchIterator src/olap/utils.h /^ explicit BinarySearchIterator(iterator_offset_t offset) : _offset(offset) {}$/;" f class:doris::BinarySearchIterator +BinarySearchIterator src/olap/utils.h /^class BinarySearchIterator : public std::iterator {$/;" c namespace:doris +Bit1 src/gutil/utf/rune.c /^ Bit1 = 7,$/;" e enum:__anon8 file: +Bit2 src/gutil/utf/rune.c /^ Bit2 = 5,$/;" e enum:__anon8 file: +Bit3 src/gutil/utf/rune.c /^ Bit3 = 4,$/;" e enum:__anon8 file: +Bit4 src/gutil/utf/rune.c /^ Bit4 = 3,$/;" e enum:__anon8 file: +Bit5 src/gutil/utf/rune.c /^ Bit5 = 2, $/;" e enum:__anon8 file: +BitAndExpr src/exprs/arithmetic_expr.h /^ BitAndExpr(const TExprNode& node) : ArithmeticExpr(node) { }$/;" f class:doris::BitAndExpr +BitAndExpr src/exprs/arithmetic_expr.h /^class BitAndExpr : public ArithmeticExpr {$/;" c namespace:doris +BitFieldReader src/olap/rowset/bit_field_reader.cpp /^BitFieldReader::BitFieldReader(ReadOnlyFileStream* input) : $/;" f class:doris::BitFieldReader +BitFieldReader src/olap/rowset/bit_field_reader.h /^class BitFieldReader {$/;" c namespace:doris +BitFieldWriter src/olap/rowset/bit_field_writer.cpp /^BitFieldWriter::BitFieldWriter(OutStream* output) : $/;" f class:doris::BitFieldWriter +BitFieldWriter src/olap/rowset/bit_field_writer.h /^class BitFieldWriter {$/;" c namespace:doris +BitMapIsAllSet src/util/bitmap.h /^inline bool BitMapIsAllSet(const uint8_t *bitmap, size_t offset, size_t bitmap_size) {$/;" f namespace:doris +BitMapTest test/util/bitmap_test.cpp /^ BitMapTest() { }$/;" f class:doris::BitMapTest +BitMapTest test/util/bitmap_test.cpp /^class BitMapTest : public testing::Test {$/;" c namespace:doris file: +BitMask src/gutil/strings/charset.h /^ static uint64 BitMask(unsigned char c) {$/;" f class:strings::CharSet +BitNotExpr src/exprs/arithmetic_expr.h /^ BitNotExpr(const TExprNode& node) : ArithmeticExpr(node) { }$/;" f class:doris::BitNotExpr +BitNotExpr src/exprs/arithmetic_expr.h /^class BitNotExpr : public ArithmeticExpr {$/;" c namespace:doris +BitOrExpr src/exprs/arithmetic_expr.h /^ BitOrExpr(const TExprNode& node) : ArithmeticExpr(node) { }$/;" f class:doris::BitOrExpr +BitOrExpr src/exprs/arithmetic_expr.h /^class BitOrExpr : public ArithmeticExpr {$/;" c namespace:doris +BitPattern src/gutil/bits.h /^template struct BitPattern {$/;" s +BitReader src/util/bit_stream_utils.h /^ BitReader() : buffer_(NULL), max_bytes_(0) {}$/;" f class:doris::BitReader +BitReader src/util/bit_stream_utils.h /^class BitReader {$/;" c namespace:doris +BitReader src/util/bit_stream_utils.inline.h /^inline BitReader::BitReader(const uint8_t* buffer, int buffer_len)$/;" f class:doris::BitReader +BitRle test/util/rle_encoding_test.cpp /^ BitRle() {$/;" f class:doris::BitRle +BitRle test/util/rle_encoding_test.cpp /^class BitRle : public testing::Test {$/;" c namespace:doris file: +BitSet src/olap/bloom_filter.hpp /^ BitSet() : _data(nullptr), _data_len(0) {}$/;" f class:doris::BitSet +BitSet src/olap/bloom_filter.hpp /^class BitSet {$/;" c namespace:doris +BitShufflePageDecoder src/olap/rowset/segment_v2/bitshuffle_page.h /^ BitShufflePageDecoder(Slice data, const PageDecoderOptions& options) : _data(data),$/;" f class:doris::segment_v2::BitShufflePageDecoder +BitShufflePageDecoder src/olap/rowset/segment_v2/bitshuffle_page.h /^class BitShufflePageDecoder : public PageDecoder {$/;" c namespace:doris::segment_v2 +BitShufflePageTest test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^class BitShufflePageTest : public testing::Test {$/;" c namespace:doris file: +BitUtil src/util/bit_util.h /^class BitUtil {$/;" c namespace:doris +BitWriter src/util/bit_stream_utils.h /^ explicit BitWriter(faststring *buffer)$/;" f class:doris::BitWriter +BitWriter src/util/bit_stream_utils.h /^class BitWriter {$/;" c namespace:doris +BitXorExpr src/exprs/arithmetic_expr.h /^ BitXorExpr(const TExprNode& node) : ArithmeticExpr(node) { }$/;" f class:doris::BitXorExpr +BitXorExpr src/exprs/arithmetic_expr.h /^class BitXorExpr : public ArithmeticExpr {$/;" c namespace:doris +Bitmap src/util/bitmap.h /^ Bitmap(int64_t num_bits) {$/;" f class:doris::Bitmap +Bitmap src/util/bitmap.h /^class Bitmap {$/;" c namespace:doris +BitmapAggField src/olap/field.h /^ explicit BitmapAggField(const TabletColumn& column) : Field(column) {$/;" f class:doris::BitmapAggField +BitmapAggField src/olap/field.h /^class BitmapAggField: public Field {$/;" c namespace:doris +BitmapChange src/util/bitmap.h /^inline void BitmapChange(uint8_t *bitmap, size_t idx, bool value) {$/;" f namespace:doris +BitmapChangeBits src/util/bitmap.cpp /^void BitmapChangeBits(uint8_t *bitmap, size_t offset, size_t num_bits, bool value) {$/;" f namespace:doris +BitmapClear src/util/bitmap.h /^inline void BitmapClear(uint8_t *bitmap, size_t idx) {$/;" f namespace:doris +BitmapDataType src/util/bitmap_value.h /^ enum BitmapDataType {$/;" g class:doris::BitmapValue +BitmapEquals src/util/bitmap.h /^inline bool BitmapEquals(const uint8_t* bm1, const uint8_t* bm2, size_t bitmap_size) {$/;" f namespace:doris +BitmapFindFirst src/util/bitmap.cpp /^bool BitmapFindFirst(const uint8_t *bitmap, size_t offset, size_t bitmap_size,$/;" f namespace:doris +BitmapFindFirstSet src/util/bitmap.h /^inline bool BitmapFindFirstSet(const uint8_t *bitmap, size_t offset,$/;" f namespace:doris +BitmapFindFirstZero src/util/bitmap.h /^inline bool BitmapFindFirstZero(const uint8_t *bitmap, size_t offset,$/;" f namespace:doris +BitmapFunctions src/exprs/bitmap_function.h /^class BitmapFunctions {$/;" c namespace:doris +BitmapFunctionsTest test/exprs/bitmap_function_test.cpp /^class BitmapFunctionsTest : public testing::Test {$/;" c namespace:doris file: +BitmapIndexIterator src/olap/rowset/segment_v2/bitmap_index_reader.h /^ explicit BitmapIndexIterator(BitmapIndexReader* reader)$/;" f class:doris::segment_v2::BitmapIndexIterator +BitmapIndexIterator src/olap/rowset/segment_v2/bitmap_index_reader.h /^class BitmapIndexIterator {$/;" c namespace:doris::segment_v2 +BitmapIndexReader src/olap/rowset/segment_v2/bitmap_index_reader.h /^ explicit BitmapIndexReader(const std::string& file_name,$/;" f class:doris::segment_v2::BitmapIndexReader +BitmapIndexReader src/olap/rowset/segment_v2/bitmap_index_reader.h /^class BitmapIndexReader {$/;" c namespace:doris::segment_v2 +BitmapIndexTest test/olap/rowset/segment_v2/bitmap_index_test.cpp /^ BitmapIndexTest() : _pool(&_tracker) { }$/;" f class:doris::segment_v2::BitmapIndexTest +BitmapIndexTest test/olap/rowset/segment_v2/bitmap_index_test.cpp /^class BitmapIndexTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +BitmapIndexTraits src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^struct BitmapIndexTraits {$/;" s namespace:doris::segment_v2::__anon54 file: +BitmapIndexTraits src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^struct BitmapIndexTraits {$/;" s namespace:doris::segment_v2::__anon54 file: +BitmapIndexWriter src/olap/rowset/segment_v2/bitmap_index_writer.h /^class BitmapIndexWriter {$/;" c namespace:doris::segment_v2 +BitmapIndexWriterImpl src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^ explicit BitmapIndexWriterImpl(const TypeInfo* typeinfo)$/;" f class:doris::segment_v2::__anon54::BitmapIndexWriterImpl +BitmapIndexWriterImpl src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^class BitmapIndexWriterImpl : public BitmapIndexWriter {$/;" c namespace:doris::segment_v2::__anon54 file: +BitmapIntersect src/exprs/bitmap_function.cpp /^ BitmapIntersect() {}$/;" f struct:doris::BitmapIntersect +BitmapIntersect src/exprs/bitmap_function.cpp /^ explicit BitmapIntersect(const char* src) {$/;" f struct:doris::BitmapIntersect +BitmapIntersect src/exprs/bitmap_function.cpp /^struct BitmapIntersect {$/;" s namespace:doris file: +BitmapIsAllZero src/util/bitmap.h /^inline bool BitmapIsAllZero(const uint8_t *bitmap, size_t offset, size_t bitmap_size) {$/;" f namespace:doris +BitmapIterator src/util/bitmap.h /^ BitmapIterator(const uint8_t* map, size_t num_bits)$/;" f class:doris::BitmapIterator +BitmapIterator src/util/bitmap.h /^class BitmapIterator {$/;" c namespace:doris +BitmapMergeOr src/util/bitmap.h /^inline void BitmapMergeOr(uint8_t *dst, const uint8_t *src, size_t n_bits) {$/;" f namespace:doris +BitmapRangeIterator src/olap/rowset/segment_v2/segment_iterator.cpp /^ explicit BitmapRangeIterator(const Roaring& bitmap)$/;" f class:doris::segment_v2::SegmentIterator::BitmapRangeIterator +BitmapRangeIterator src/olap/rowset/segment_v2/segment_iterator.cpp /^class SegmentIterator::BitmapRangeIterator {$/;" c class:doris::segment_v2::SegmentIterator file: +BitmapSet src/util/bitmap.h /^inline void BitmapSet(uint8_t *bitmap, size_t idx) {$/;" f namespace:doris +BitmapSize src/util/bitmap.h /^inline size_t BitmapSize(size_t num_bits) {$/;" f namespace:doris +BitmapTest src/util/bitmap.h /^inline bool BitmapTest(const uint8_t *bitmap, size_t idx) {$/;" f namespace:doris +BitmapToString src/util/bitmap.cpp /^std::string BitmapToString(const uint8_t *bitmap, size_t num_bits) {$/;" f namespace:doris +BitmapTypeCode src/util/bitmap_value.h /^struct BitmapTypeCode {$/;" s namespace:doris +BitmapValue src/util/bitmap_value.h /^ BitmapValue() : _type(EMPTY) {}$/;" f class:doris::BitmapValue +BitmapValue src/util/bitmap_value.h /^ explicit BitmapValue(const char* src) {$/;" f class:doris::BitmapValue +BitmapValue src/util/bitmap_value.h /^ explicit BitmapValue(const std::vector& bits) {$/;" f class:doris::BitmapValue +BitmapValue src/util/bitmap_value.h /^ explicit BitmapValue(uint64_t value) : _sv(value), _type(SINGLE) {}$/;" f class:doris::BitmapValue +BitmapValue src/util/bitmap_value.h /^class BitmapValue {$/;" c namespace:doris +Bits src/gutil/bits.h /^class Bits {$/;" c +BitshufflePageBuilder src/olap/rowset/segment_v2/bitshuffle_page.h /^ BitshufflePageBuilder(const PageBuilderOptions& options) :$/;" f class:doris::segment_v2::BitshufflePageBuilder +BitshufflePageBuilder src/olap/rowset/segment_v2/bitshuffle_page.h /^class BitshufflePageBuilder : public PageBuilder {$/;" c namespace:doris::segment_v2 +Bitx src/gutil/utf/rune.c /^ Bitx = 6,$/;" e enum:__anon8 file: +Block src/olap/fs/block_manager.h /^class Block {$/;" c namespace:doris::fs +Block src/runtime/buffered_block_mgr.cpp /^BufferedBlockMgr::Block::Block()$/;" f class:doris::BufferedBlockMgr::Block +Block src/runtime/buffered_block_mgr.h /^ class Block { \/\/}: public InternalQueue::Node {$/;" c class:doris::BufferedBlockMgr +Block src/runtime/buffered_block_mgr2.cc /^BufferedBlockMgr2::Block::Block(BufferedBlockMgr2* block_mgr) :$/;" f class:doris::BufferedBlockMgr2::Block +Block src/runtime/buffered_block_mgr2.h /^ class Block : public InternalQueue::Node {$/;" c class:doris::BufferedBlockMgr2 +BlockBloomFilterTest test/olap/rowset/segment_v2/block_bloom_filter_test.cpp /^class BlockBloomFilterTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +BlockCompressionCodec src/util/block_compression.h /^class BlockCompressionCodec {$/;" c namespace:doris +BlockCompressionTest test/util/block_compression_test.cpp /^ BlockCompressionTest() { }$/;" f class:doris::BlockCompressionTest +BlockCompressionTest test/util/block_compression_test.cpp /^class BlockCompressionTest : public testing::Test {$/;" c namespace:doris file: +BlockId src/olap/fs/block_id.h /^ BlockId() : _id(kInvalidId) { }$/;" f class:doris::BlockId +BlockId src/olap/fs/block_id.h /^ explicit BlockId(uint64_t id) : _id(id) { }$/;" f class:doris::BlockId +BlockId src/olap/fs/block_id.h /^class BlockId {$/;" c namespace:doris +BlockIdCompare src/olap/fs/block_id.h /^struct BlockIdCompare {$/;" s namespace:doris +BlockIdContainer src/olap/fs/block_id.h /^typedef std::vector BlockIdContainer;$/;" t namespace:doris +BlockIdEqual src/olap/fs/block_id.h /^struct BlockIdEqual {$/;" s namespace:doris +BlockIdHash src/olap/fs/block_id.h /^struct BlockIdHash {$/;" s namespace:doris +BlockIdSet src/olap/fs/block_id.h /^typedef std::unordered_set BlockIdSet;$/;" t namespace:doris +BlockInDir test/runtime/buffered_block_mgr2_test.cpp /^ static bool BlockInDir(BufferedBlockMgr2::Block* block, const string& dir) {$/;" f class:doris::BufferedBlockMgrTest +BlockManager src/olap/fs/block_manager.h /^class BlockManager {$/;" c namespace:doris::fs +BlockManagerMetrics src/olap/fs/block_manager_metrics.cpp /^BlockManagerMetrics::BlockManagerMetrics() {$/;" f class:doris::fs::internal::BlockManagerMetrics +BlockManagerMetrics src/olap/fs/block_manager_metrics.h /^struct BlockManagerMetrics {$/;" s namespace:doris::fs::internal +BlockManagerOptions src/olap/fs/block_manager.cpp /^BlockManagerOptions::BlockManagerOptions() : read_only(false) {}$/;" f class:doris::fs::BlockManagerOptions +BlockManagerOptions src/olap/fs/block_manager.h /^struct BlockManagerOptions {$/;" s namespace:doris::fs +BlockMgrsMap src/runtime/buffered_block_mgr2.h /^ typedef boost::unordered_map > BlockMgrsMap;$/;" t class:doris::BufferedBlockMgr2 +BlockQueueSharedPtr src/runtime/result_queue_mgr.h /^typedef std::shared_ptr BlockQueueSharedPtr;$/;" t namespace:doris +BlockSplitBloomFilter src/olap/rowset/segment_v2/block_split_bloom_filter.h /^class BlockSplitBloomFilter : public BloomFilter {$/;" c namespace:doris::segment_v2 +BlockingJoinNode src/exec/blocking_join_node.cpp /^BlockingJoinNode::BlockingJoinNode(const std::string& node_name,$/;" f class:doris::BlockingJoinNode +BlockingJoinNode src/exec/blocking_join_node.h /^class BlockingJoinNode : public ExecNode {$/;" c namespace:doris +BlockingPriorityQueue src/util/blocking_priority_queue.hpp /^ BlockingPriorityQueue(size_t max_elements) :$/;" f class:doris::BlockingPriorityQueue +BlockingPriorityQueue src/util/blocking_priority_queue.hpp /^class BlockingPriorityQueue {$/;" c namespace:doris +BlockingQueue src/util/blocking_queue.hpp /^ BlockingQueue(size_t max_elements) :$/;" f class:doris::BlockingQueue +BlockingQueue src/util/blocking_queue.hpp /^class BlockingQueue {$/;" c namespace:doris +BloomFilter src/olap/bloom_filter.hpp /^ BloomFilter() : _bit_num(0), _hash_function_num(0) {}$/;" f class:doris::BloomFilter +BloomFilter src/olap/bloom_filter.hpp /^class BloomFilter {$/;" c namespace:doris +BloomFilter src/olap/rowset/segment_v2/bloom_filter.h /^ BloomFilter() : _data(nullptr), _num_bytes(0), _size(0), _has_null(nullptr) { }$/;" f class:doris::segment_v2::BloomFilter +BloomFilter src/olap/rowset/segment_v2/bloom_filter.h /^class BloomFilter {$/;" c namespace:doris::segment_v2 +BloomFilterIndexHeader src/olap/bloom_filter.hpp /^ BloomFilterIndexHeader() :$/;" f struct:doris::BloomFilterIndexHeader +BloomFilterIndexHeader src/olap/bloom_filter.hpp /^struct BloomFilterIndexHeader {$/;" s namespace:doris +BloomFilterIndexIterator src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ explicit BloomFilterIndexIterator(BloomFilterIndexReader* reader)$/;" f class:doris::segment_v2::BloomFilterIndexIterator +BloomFilterIndexIterator src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^class BloomFilterIndexIterator {$/;" c namespace:doris::segment_v2 +BloomFilterIndexReader src/olap/bloom_filter_reader.h /^ BloomFilterIndexReader() {}$/;" f class:doris::BloomFilterIndexReader +BloomFilterIndexReader src/olap/bloom_filter_reader.h /^class BloomFilterIndexReader {$/;" c namespace:doris +BloomFilterIndexReader src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ explicit BloomFilterIndexReader(const std::string& file_name,$/;" f class:doris::segment_v2::BloomFilterIndexReader +BloomFilterIndexReader src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^class BloomFilterIndexReader {$/;" c namespace:doris::segment_v2 +BloomFilterIndexReaderWriterTest test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^ BloomFilterIndexReaderWriterTest() { }$/;" f class:doris::segment_v2::BloomFilterIndexReaderWriterTest +BloomFilterIndexReaderWriterTest test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^class BloomFilterIndexReaderWriterTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +BloomFilterIndexWriter src/olap/bloom_filter_writer.h /^ BloomFilterIndexWriter() {}$/;" f class:doris::BloomFilterIndexWriter +BloomFilterIndexWriter src/olap/bloom_filter_writer.h /^class BloomFilterIndexWriter {$/;" c namespace:doris +BloomFilterIndexWriter src/olap/rowset/segment_v2/bloom_filter_index_writer.h /^class BloomFilterIndexWriter {$/;" c namespace:doris::segment_v2 +BloomFilterIndexWriterImpl src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ explicit BloomFilterIndexWriterImpl(const BloomFilterOptions& bf_options,$/;" f class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl +BloomFilterIndexWriterImpl src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter {$/;" c namespace:doris::segment_v2::__anon52 file: +BloomFilterOptions src/olap/rowset/segment_v2/bloom_filter.h /^struct BloomFilterOptions {$/;" s namespace:doris::segment_v2 +BloomFilterPageTest test/olap/rowset/segment_v2/bloom_filter_page_test.cpp /^class BloomFilterPageTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +BloomFilterTraits src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^struct BloomFilterTraits {$/;" s namespace:doris::segment_v2::__anon52 file: +BloomFilterTraits src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^struct BloomFilterTraits {$/;" s namespace:doris::segment_v2::__anon52 file: +BloomFilterTraits src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^struct BloomFilterTraits {$/;" s namespace:doris::segment_v2::__anon52 file: +BooleanQueryBuilder src/exec/es/es_query_builder.cpp /^BooleanQueryBuilder::BooleanQueryBuilder() {$/;" f class:doris::BooleanQueryBuilder +BooleanQueryBuilder src/exec/es/es_query_builder.cpp /^BooleanQueryBuilder::BooleanQueryBuilder(const std::vector& predicates) {$/;" f class:doris::BooleanQueryBuilder +BooleanQueryBuilder src/exec/es/es_query_builder.h /^class BooleanQueryBuilder : public QueryBuilder {$/;" c namespace:doris +BooleanQueryBuilderTest test/exec/es_query_builder_test.cpp /^ BooleanQueryBuilderTest() { }$/;" f class:doris::BooleanQueryBuilderTest +BooleanQueryBuilderTest test/exec/es_query_builder_test.cpp /^class BooleanQueryBuilderTest : public testing::Test {$/;" c namespace:doris file: +BooleanVal output/udf/include/udf.h /^ BooleanVal() : val(false) {}$/;" f struct:doris_udf::BooleanVal +BooleanVal output/udf/include/udf.h /^ BooleanVal(bool val) : val(val) {}$/;" f struct:doris_udf::BooleanVal +BooleanVal output/udf/include/udf.h /^struct BooleanVal : public AnyVal {$/;" s namespace:doris_udf +BooleanVal src/udf/udf.h /^ BooleanVal() : val(false) {}$/;" f struct:doris_udf::BooleanVal +BooleanVal src/udf/udf.h /^ BooleanVal(bool val) : val(val) {}$/;" f struct:doris_udf::BooleanVal +BooleanVal src/udf/udf.h /^struct BooleanVal : public AnyVal {$/;" s namespace:doris_udf +BooleanWrapper src/exprs/scalar_fn_call.cpp /^typedef BooleanVal (*BooleanWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +BrokerInfo src/util/broker_load_error_hub.h /^ BrokerInfo(const TBrokerErrorHubInfo& t_info,$/;" f struct:doris::BrokerLoadErrorHub::BrokerInfo +BrokerInfo src/util/broker_load_error_hub.h /^ struct BrokerInfo {$/;" s class:doris::BrokerLoadErrorHub +BrokerLoadErrorHub src/util/broker_load_error_hub.cpp /^BrokerLoadErrorHub::BrokerLoadErrorHub($/;" f class:doris::BrokerLoadErrorHub +BrokerLoadErrorHub src/util/broker_load_error_hub.h /^class BrokerLoadErrorHub : public LoadErrorHub {$/;" c namespace:doris +BrokerMgr src/runtime/broker_mgr.cpp /^BrokerMgr::BrokerMgr(ExecEnv* exec_env) : $/;" f class:doris::BrokerMgr +BrokerMgr src/runtime/broker_mgr.h /^class BrokerMgr {$/;" c namespace:doris +BrokerReader src/exec/broker_reader.cpp /^BrokerReader::BrokerReader($/;" f class:doris::BrokerReader +BrokerReader src/exec/broker_reader.h /^class BrokerReader : public FileReader {$/;" c namespace:doris +BrokerReaderTest test/exec/broker_reader_test.cpp /^ BrokerReaderTest() {$/;" f class:doris::BrokerReaderTest +BrokerReaderTest test/exec/broker_reader_test.cpp /^class BrokerReaderTest : public testing::Test {$/;" c namespace:doris file: +BrokerScanNode src/exec/broker_scan_node.cpp /^BrokerScanNode::BrokerScanNode($/;" f class:doris::BrokerScanNode +BrokerScanNode src/exec/broker_scan_node.h /^class BrokerScanNode : public ScanNode {$/;" c namespace:doris +BrokerScanNodeTest test/exec/broker_scan_node_test.cpp /^ BrokerScanNodeTest() : _runtime_state(TQueryGlobals()) {$/;" f class:doris::BrokerScanNodeTest +BrokerScanNodeTest test/exec/broker_scan_node_test.cpp /^class BrokerScanNodeTest : public testing::Test {$/;" c namespace:doris file: +BrokerScanner src/exec/broker_scanner.cpp /^BrokerScanner::BrokerScanner(RuntimeState* state,$/;" f class:doris::BrokerScanner +BrokerScanner src/exec/broker_scanner.h /^class BrokerScanner : public BaseScanner {$/;" c namespace:doris +BrokerScannerTest test/exec/broker_scanner_test.cpp /^ BrokerScannerTest() : _runtime_state(TQueryGlobals()) {$/;" f class:doris::BrokerScannerTest +BrokerScannerTest test/exec/broker_scanner_test.cpp /^class BrokerScannerTest : public testing::Test {$/;" c namespace:doris file: +BrokerServiceClientCache src/runtime/client_cache.h /^typedef ClientCache BrokerServiceClientCache;$/;" t namespace:doris +BrokerServiceConnection src/runtime/client_cache.h /^typedef ClientConnection BrokerServiceConnection;$/;" t namespace:doris +BrokerTableDescriptor src/runtime/descriptors.cpp /^BrokerTableDescriptor::BrokerTableDescriptor(const TTableDescriptor& tdesc)$/;" f class:doris::BrokerTableDescriptor +BrokerTableDescriptor src/runtime/descriptors.h /^class BrokerTableDescriptor : public TableDescriptor {$/;" c namespace:doris +BrokerWriter src/exec/broker_writer.cpp /^BrokerWriter::BrokerWriter($/;" f class:doris::BrokerWriter +BrokerWriter src/exec/broker_writer.h /^class BrokerWriter : public FileWriter {$/;" c namespace:doris +BrpcStubCache src/util/brpc_stub_cache.h /^ BrpcStubCache() {$/;" f class:doris::BrpcStubCache +BrpcStubCache src/util/brpc_stub_cache.h /^class BrpcStubCache {$/;" c namespace:doris +BrpcStubCacheTest test/util/brpc_stub_cache_test.cpp /^ BrpcStubCacheTest() { }$/;" f class:doris::BrpcStubCacheTest +BrpcStubCacheTest test/util/brpc_stub_cache_test.cpp /^class BrpcStubCacheTest : public testing::Test {$/;" c namespace:doris file: +Bucket src/exec/hash_table.h /^ Bucket() {$/;" f struct:doris::HashTable::Bucket +Bucket src/exec/hash_table.h /^ struct Bucket {$/;" s class:doris::HashTable +Bucket src/exec/new_partitioned_hash_table.h /^ struct Bucket {$/;" s class:doris::NewPartitionedHashTable +Bucket src/exec/partitioned_hash_table.h /^ struct Bucket {$/;" s class:doris::PartitionedHashTable +BucketCountersInfo src/util/runtime_profile.h /^ struct BucketCountersInfo {$/;" s class:doris::RuntimeProfile +BucketCountersMap src/util/runtime_profile.h /^ typedef std::map*, BucketCountersInfo> BucketCountersMap;$/;" t struct:doris::RuntimeProfile::PeriodicCounterUpdateState +BucketSize src/exec/new_partitioned_hash_table.h /^ static int64_t BucketSize() { return sizeof(Bucket); }$/;" f class:doris::NewPartitionedHashTable +BufDeleter src/olap/byte_buffer.cpp /^StorageByteBuffer::BufDeleter::BufDeleter() : $/;" f class:doris::StorageByteBuffer::BufDeleter +BufDeleter src/olap/byte_buffer.h /^ class BufDeleter {$/;" c class:doris::StorageByteBuffer +BufferAllocFailed src/common/status.h /^ static Status BufferAllocFailed(const Slice& msg, int16_t precise_code = 1, const Slice& msg2 = Slice()) {$/;" f class:doris::Status +BufferAllocator src/runtime/bufferpool/buffer_allocator.cc /^BufferPool::BufferAllocator::BufferAllocator($/;" f class:doris::BufferPool::BufferAllocator +BufferAllocator src/runtime/bufferpool/buffer_allocator.h /^struct BufferPool::BufferAllocator {$/;" s class:doris::BufferPool +BufferBuilder src/util/buffer_builder.hpp /^ BufferBuilder(char* dst_buffer, int dst_len) : $/;" f class:doris::BufferBuilder +BufferBuilder src/util/buffer_builder.hpp /^ BufferBuilder(uint8_t* dst_buffer, int dst_len) : $/;" f class:doris::BufferBuilder +BufferBuilder src/util/buffer_builder.hpp /^class BufferBuilder {$/;" c namespace:doris +BufferControlBlock src/runtime/buffer_control_block.cpp /^BufferControlBlock::BufferControlBlock(const TUniqueId& id, int buffer_size)$/;" f class:doris::BufferControlBlock +BufferControlBlock src/runtime/buffer_control_block.h /^class BufferControlBlock {$/;" c namespace:doris +BufferControlBlockTest test/runtime/buffer_control_block_test.cpp /^ BufferControlBlockTest() {$/;" f class:doris::BufferControlBlockTest +BufferControlBlockTest test/runtime/buffer_control_block_test.cpp /^class BufferControlBlockTest : public testing::Test {$/;" c namespace:doris file: +BufferDescriptor src/runtime/buffered_block_mgr.h /^ BufferDescriptor(uint8_t* buf, int64_t len)$/;" f struct:doris::BufferedBlockMgr::BufferDescriptor +BufferDescriptor src/runtime/buffered_block_mgr.h /^ struct BufferDescriptor { \/\/}: public InternalQueue::Node {$/;" s class:doris::BufferedBlockMgr +BufferDescriptor src/runtime/buffered_block_mgr2.h /^ BufferDescriptor(uint8_t* buf, int64_t len) : buffer(buf), len(len), block(NULL) {$/;" f struct:doris::BufferedBlockMgr2::BufferDescriptor +BufferDescriptor src/runtime/buffered_block_mgr2.h /^ struct BufferDescriptor : public InternalQueue::Node {$/;" s class:doris::BufferedBlockMgr2 +BufferDescriptor src/runtime/disk_io_mgr.cc /^DiskIoMgr::BufferDescriptor::BufferDescriptor(DiskIoMgr* io_mgr) :$/;" f class:doris::DiskIoMgr::BufferDescriptor +BufferDescriptor src/runtime/disk_io_mgr.h /^ class BufferDescriptor {$/;" c class:doris::DiskIoMgr +BufferHandle src/runtime/bufferpool/buffer_pool.h /^ BufferHandle() { Reset(); }$/;" f class:doris::BufferPool::BufferHandle +BufferHandle src/runtime/bufferpool/buffer_pool.h /^class BufferPool::BufferHandle {$/;" c class:doris::BufferPool +BufferHandle src/runtime/bufferpool/buffer_pool.h /^inline BufferPool::BufferHandle::BufferHandle(BufferHandle&& src) {$/;" f class:doris::BufferPool::BufferHandle +BufferInfo src/runtime/row_batch.h /^ struct BufferInfo {$/;" s class:doris::RowBatch +BufferMap src/runtime/result_buffer_mgr.h /^ typedef boost::unordered_map> BufferMap;$/;" t class:doris::ResultBufferMgr +BufferPool src/runtime/bufferpool/buffer_pool.cc /^BufferPool::BufferPool(int64_t min_buffer_len, int64_t buffer_bytes_limit,$/;" f class:doris::BufferPool +BufferPool src/runtime/bufferpool/buffer_pool.h /^class BufferPool : public CacheLineAligned {$/;" c namespace:doris +BufferPoolClientCounters src/runtime/bufferpool/buffer_pool_counters.h /^struct BufferPoolClientCounters {$/;" s namespace:doris +BufferVal output/udf/include/udf.h /^typedef uint8_t* BufferVal;$/;" t namespace:doris_udf +BufferVal src/udf/udf.h /^typedef uint8_t* BufferVal;$/;" t namespace:doris_udf +BufferValues src/util/bit_stream_utils.inline.h /^inline void BitReader::BufferValues() {$/;" f class:doris::BitReader +BufferedBlockMgr src/runtime/buffered_block_mgr.cpp /^BufferedBlockMgr::BufferedBlockMgr(RuntimeState* state, int64_t block_size)$/;" f class:doris::BufferedBlockMgr +BufferedBlockMgr src/runtime/buffered_block_mgr.h /^class BufferedBlockMgr {$/;" c namespace:doris +BufferedBlockMgr2 src/runtime/buffered_block_mgr2.cc /^BufferedBlockMgr2::BufferedBlockMgr2(RuntimeState* state, TmpFileMgr* tmp_file_mgr,$/;" f class:doris::BufferedBlockMgr2 +BufferedBlockMgr2 src/runtime/buffered_block_mgr2.h /^class BufferedBlockMgr2 {$/;" c namespace:doris +BufferedBlockMgrTest test/runtime/buffered_block_mgr2_test.cpp /^class BufferedBlockMgrTest : public ::testing::Test {$/;" c namespace:doris file: +BufferedTupleStream src/runtime/buffered_tuple_stream.cpp /^BufferedTupleStream::BufferedTupleStream(RuntimeState* state,$/;" f class:doris::BufferedTupleStream +BufferedTupleStream src/runtime/buffered_tuple_stream.h /^class BufferedTupleStream {$/;" c namespace:doris +BufferedTupleStream2 src/runtime/buffered_tuple_stream2.cc /^BufferedTupleStream2::BufferedTupleStream2(RuntimeState* state,$/;" f class:doris::BufferedTupleStream2 +BufferedTupleStream2 src/runtime/buffered_tuple_stream2.h /^class BufferedTupleStream2 {$/;" c namespace:doris +BufferedTupleStream3 src/runtime/buffered_tuple_stream3.cc /^BufferedTupleStream3::BufferedTupleStream3(RuntimeState* state,$/;" f class:BufferedTupleStream3 +BufferedTupleStream3 src/runtime/buffered_tuple_stream3.h /^class BufferedTupleStream3 {$/;" c namespace:doris +BufferedTupleStreamTest test/runtime/buffered_tuple_stream_test.cpp /^ BufferedTupleStreamTest() {$/;" f class:doris::BufferedTupleStreamTest +BufferedTupleStreamTest test/runtime/buffered_tuple_stream_test.cpp /^class BufferedTupleStreamTest : public testing::Test {$/;" c namespace:doris file: +BuildLookupTable src/gutil/strings/stringpiece.cc /^static inline void BuildLookupTable(StringPiece characters_wanted,$/;" f file: +BuildSpilledPartition src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::BuildSpilledPartition(Partition** built_partition) {$/;" f class:doris::NewPartitionedAggregationNode +BuildTupleRowSet src/exec/hash_join_node.h /^ typedef boost::unordered_set BuildTupleRowSet;$/;" t class:doris::HashJoinNode +ByteBuffer src/util/byte_buffer.h /^ ByteBuffer(size_t capacity_)$/;" f struct:doris::ByteBuffer +ByteBuffer src/util/byte_buffer.h /^struct ByteBuffer {$/;" s namespace:doris +ByteBufferTest test/util/byte_buffer_test2.cpp /^ ByteBufferTest() { }$/;" f class:doris::ByteBufferTest +ByteBufferTest test/util/byte_buffer_test2.cpp /^class ByteBufferTest : public testing::Test {$/;" c namespace:doris file: +ByteColumnWriter src/olap/rowset/column_writer.cpp /^ByteColumnWriter::ByteColumnWriter(uint32_t column_id,$/;" f class:doris::ByteColumnWriter +ByteColumnWriter src/olap/rowset/column_writer.h /^class ByteColumnWriter : public ColumnWriter {$/;" c namespace:doris +ByteSize src/exec/new_partitioned_hash_table.h /^ int64_t ByteSize() const {$/;" f class:doris::NewPartitionedHashTable +ByteStringFromAscii src/gutil/strings/escaping.cc /^bool ByteStringFromAscii(string const & hex_string, string * binary_string) {$/;" f namespace:strings +ByteStringToAscii src/gutil/strings/escaping.cc /^void ByteStringToAscii(string const &binary_string, int bytes_to_read,$/;" f namespace:strings +ByteStringToAscii src/gutil/strings/escaping.h /^inline string ByteStringToAscii(const string& binary_string,$/;" f namespace:strings +BytesAllInRange src/gutil/bits.h /^inline bool Bits::BytesAllInRange(T bytes, uint8 lo, uint8 hi) {$/;" f class:Bits +BytesContainByte src/gutil/bits.h /^template inline bool Bits::BytesContainByte(T bytes, uint8 c) {$/;" f class:Bits +BytesContainByteLessThan src/gutil/bits.h /^inline bool Bits::BytesContainByteLessThan(T bytes, uint8 c) {$/;" f class:Bits +BytesPinned src/runtime/buffered_tuple_stream3.h /^ int64_t BytesPinned(bool ignore_current) const {$/;" f class:doris::BufferedTupleStream3 +Bzip2Decompressor src/exec/decompressor.h /^ Bzip2Decompressor() : Decompressor(CompressType::BZIP2) {}$/;" f class:doris::Bzip2Decompressor +Bzip2Decompressor src/exec/decompressor.h /^class Bzip2Decompressor : public Decompressor {$/;" c namespace:doris +CACHELINE_ALIGNED src/gutil/port.h 342;" d +CACHELINE_SIZE src/gutil/port.h 323;" d +CACHE_CONTROL src/http/http_headers.cpp /^const char* HttpHeaders::CACHE_CONTROL = "Cache-Control";$/;" m class:doris::HttpHeaders file: +CACHE_CONTROL src/http/http_headers.h /^ static const char* CACHE_CONTROL;$/;" m class:doris::HttpHeaders +CACHE_LINE_SIZE src/common/compiler_util.h 26;" d +CALLBACK_FUNCTION src/agent/task_worker_pool.h /^ typedef void* (*CALLBACK_FUNCTION)(void*);$/;" t class:doris::TaskWorkerPool +CALL_FN_W_10W src/gutil/valgrind.h 1322;" d +CALL_FN_W_10W src/gutil/valgrind.h 1765;" d +CALL_FN_W_10W src/gutil/valgrind.h 2271;" d +CALL_FN_W_10W src/gutil/valgrind.h 2826;" d +CALL_FN_W_10W src/gutil/valgrind.h 3387;" d +CALL_FN_W_10W src/gutil/valgrind.h 888;" d +CALL_FN_W_11W src/gutil/valgrind.h 1362;" d +CALL_FN_W_11W src/gutil/valgrind.h 1811;" d +CALL_FN_W_11W src/gutil/valgrind.h 2323;" d +CALL_FN_W_11W src/gutil/valgrind.h 2880;" d +CALL_FN_W_11W src/gutil/valgrind.h 3441;" d +CALL_FN_W_11W src/gutil/valgrind.h 926;" d +CALL_FN_W_12W src/gutil/valgrind.h 1404;" d +CALL_FN_W_12W src/gutil/valgrind.h 1861;" d +CALL_FN_W_12W src/gutil/valgrind.h 2379;" d +CALL_FN_W_12W src/gutil/valgrind.h 2938;" d +CALL_FN_W_12W src/gutil/valgrind.h 3499;" d +CALL_FN_W_12W src/gutil/valgrind.h 967;" d +CALL_FN_W_5W src/gutil/valgrind.h 1156;" d +CALL_FN_W_5W src/gutil/valgrind.h 1597;" d +CALL_FN_W_5W src/gutil/valgrind.h 2073;" d +CALL_FN_W_5W src/gutil/valgrind.h 2618;" d +CALL_FN_W_5W src/gutil/valgrind.h 3179;" d +CALL_FN_W_5W src/gutil/valgrind.h 730;" d +CALL_FN_W_6W src/gutil/valgrind.h 1184;" d +CALL_FN_W_6W src/gutil/valgrind.h 1625;" d +CALL_FN_W_6W src/gutil/valgrind.h 2107;" d +CALL_FN_W_6W src/gutil/valgrind.h 2654;" d +CALL_FN_W_6W src/gutil/valgrind.h 3215;" d +CALL_FN_W_6W src/gutil/valgrind.h 757;" d +CALL_FN_W_7W src/gutil/valgrind.h 1214;" d +CALL_FN_W_7W src/gutil/valgrind.h 1655;" d +CALL_FN_W_7W src/gutil/valgrind.h 2143;" d +CALL_FN_W_7W src/gutil/valgrind.h 2692;" d +CALL_FN_W_7W src/gutil/valgrind.h 3253;" d +CALL_FN_W_7W src/gutil/valgrind.h 786;" d +CALL_FN_W_8W src/gutil/valgrind.h 1248;" d +CALL_FN_W_8W src/gutil/valgrind.h 1688;" d +CALL_FN_W_8W src/gutil/valgrind.h 2182;" d +CALL_FN_W_8W src/gutil/valgrind.h 2733;" d +CALL_FN_W_8W src/gutil/valgrind.h 3294;" d +CALL_FN_W_8W src/gutil/valgrind.h 818;" d +CALL_FN_W_9W src/gutil/valgrind.h 1284;" d +CALL_FN_W_9W src/gutil/valgrind.h 1723;" d +CALL_FN_W_9W src/gutil/valgrind.h 2223;" d +CALL_FN_W_9W src/gutil/valgrind.h 2776;" d +CALL_FN_W_9W src/gutil/valgrind.h 3337;" d +CALL_FN_W_9W src/gutil/valgrind.h 852;" d +CALL_FN_W_W src/gutil/valgrind.h 1064;" d +CALL_FN_W_W src/gutil/valgrind.h 1505;" d +CALL_FN_W_W src/gutil/valgrind.h 1957;" d +CALL_FN_W_W src/gutil/valgrind.h 2494;" d +CALL_FN_W_W src/gutil/valgrind.h 3055;" d +CALL_FN_W_W src/gutil/valgrind.h 642;" d +CALL_FN_W_WW src/gutil/valgrind.h 1084;" d +CALL_FN_W_WW src/gutil/valgrind.h 1525;" d +CALL_FN_W_WW src/gutil/valgrind.h 1983;" d +CALL_FN_W_WW src/gutil/valgrind.h 2522;" d +CALL_FN_W_WW src/gutil/valgrind.h 3083;" d +CALL_FN_W_WW src/gutil/valgrind.h 661;" d +CALL_FN_W_WWW src/gutil/valgrind.h 1106;" d +CALL_FN_W_WWW src/gutil/valgrind.h 1547;" d +CALL_FN_W_WWW src/gutil/valgrind.h 2011;" d +CALL_FN_W_WWW src/gutil/valgrind.h 2552;" d +CALL_FN_W_WWW src/gutil/valgrind.h 3113;" d +CALL_FN_W_WWW src/gutil/valgrind.h 682;" d +CALL_FN_W_WWWW src/gutil/valgrind.h 1130;" d +CALL_FN_W_WWWW src/gutil/valgrind.h 1571;" d +CALL_FN_W_WWWW src/gutil/valgrind.h 2041;" d +CALL_FN_W_WWWW src/gutil/valgrind.h 2584;" d +CALL_FN_W_WWWW src/gutil/valgrind.h 3145;" d +CALL_FN_W_WWWW src/gutil/valgrind.h 705;" d +CALL_FN_W_v src/gutil/valgrind.h 1046;" d +CALL_FN_W_v src/gutil/valgrind.h 1487;" d +CALL_FN_W_v src/gutil/valgrind.h 1933;" d +CALL_FN_W_v src/gutil/valgrind.h 2468;" d +CALL_FN_W_v src/gutil/valgrind.h 3029;" d +CALL_FN_W_v src/gutil/valgrind.h 626;" d +CALL_FN_v_W src/gutil/valgrind.h 603;" d +CALL_FN_v_WW src/gutil/valgrind.h 607;" d +CALL_FN_v_WWW src/gutil/valgrind.h 611;" d +CALL_FN_v_v src/gutil/valgrind.h 599;" d +CANT_REWIND src/util/rle_encoding.h /^ CANT_REWIND$/;" e enum:doris::RleDecoder::RewindState +CAPACITY_UC src/olap/options.cpp /^static std::string CAPACITY_UC = "CAPACITY";$/;" m namespace:doris file: +CASESENSITIVITYDEFAULTVALUE src/util/minizip/unzip.c 372;" d file: +CASESENSITIVITYDEFAULTVALUE src/util/minizip/unzip.c 374;" d file: +CASESENSITIVITYDEFAULT_NO src/util/minizip/unzip.c 98;" d file: +CASE_COMPUTE_FN src/exprs/case_expr.cpp 182;" d file: +CASE_COMPUTE_FN_WAPPER src/exprs/case_expr.cpp 223;" d file: +CAST_DECIMAL_TO_INT src/exprs/decimal_operators.cpp 107;" d file: +CAST_DECIMAL_TO_INT src/exprs/decimalv2_operators.cpp 106;" d file: +CAST_EXPR_DEFINE src/exprs/cast_expr.h 33;" d +CAST_FLOAT_TO_STRING src/exprs/cast_functions.cpp 178;" d file: +CAST_FROM_BIGINT src/exprs/cast_expr.cpp 110;" d file: +CAST_FROM_BOOLEAN src/exprs/cast_expr.cpp 62;" d file: +CAST_FROM_DATETIME src/exprs/cast_functions.cpp 254;" d file: +CAST_FROM_DECIMAL src/exprs/decimal_operators.cpp 115;" d file: +CAST_FROM_DECIMAL src/exprs/decimalv2_operators.cpp 114;" d file: +CAST_FROM_DOUBLE src/exprs/cast_expr.cpp 146;" d file: +CAST_FROM_FLOAT src/exprs/cast_expr.cpp 134;" d file: +CAST_FROM_INT src/exprs/cast_expr.cpp 98;" d file: +CAST_FROM_LARGEINT src/exprs/cast_expr.cpp 122;" d file: +CAST_FROM_SMALLINT src/exprs/cast_expr.cpp 86;" d file: +CAST_FROM_STRING src/exprs/cast_functions.cpp 109;" d file: +CAST_FROM_STRINGS src/exprs/cast_functions.cpp 122;" d file: +CAST_FROM_TIME src/exprs/time_operators.cpp 46;" d file: +CAST_FROM_TINYINT src/exprs/cast_expr.cpp 74;" d file: +CAST_FUNCTION src/exprs/cast_expr.cpp 53;" d file: +CAST_FUNCTION src/exprs/cast_functions.cpp 39;" d file: +CAST_INT_TO_DECIMAL src/exprs/decimal_operators.cpp 36;" d file: +CAST_INT_TO_DECIMAL src/exprs/decimalv2_operators.cpp 36;" d file: +CAST_INT_TO_DECIMALS src/exprs/decimal_operators.cpp 46;" d file: +CAST_INT_TO_DECIMALS src/exprs/decimalv2_operators.cpp 46;" d file: +CAST_SAME src/exprs/cast_expr.cpp 48;" d file: +CAST_TIME_TO_INT src/exprs/time_operators.cpp 35;" d file: +CAST_TO_DATE src/exprs/cast_functions.cpp 293;" d file: +CAST_TO_DATES src/exprs/cast_functions.cpp 305;" d file: +CAST_TO_DATETIME src/exprs/cast_functions.cpp 271;" d file: +CAST_TO_DATETIMES src/exprs/cast_functions.cpp 283;" d file: +CAST_TO_STRING src/exprs/cast_functions.cpp 144;" d file: +CEILING src/runtime/decimal_value.h /^ CEILING = 3,$/;" e enum:doris::DecimalRoundMode +CEscape src/gutil/strings/escaping.cc /^string CEscape(const StringPiece& src) {$/;" f namespace:strings +CEscapeInternal src/gutil/strings/escaping.cc /^int CEscapeInternal(const char* src, int src_len, char* dest,$/;" f namespace:strings +CEscapeString src/gutil/strings/escaping.cc /^int CEscapeString(const char* src, int src_len, char* dest, int dest_len) {$/;" f namespace:strings +CFI_ADJUST_CFA_OFFSET src/gutil/linux_syscall_support.h 1045;" d +CFI_ADJUST_CFA_OFFSET src/gutil/linux_syscall_support.h 1048;" d +CGroupUtil src/util/cgroup_util.h /^class CGroupUtil {$/;" c namespace:doris +CGroupUtilTest test/util/cgroup_util_test.cpp /^ CGroupUtilTest() {$/;" f class:doris::CGroupUtilTest +CGroupUtilTest test/util/cgroup_util_test.cpp /^class CGroupUtilTest : public ::testing::Test {$/;" c namespace:doris file: +CHARS_PER_128_BIT_REGISTER src/util/sse_util.hpp /^static const int CHARS_PER_128_BIT_REGISTER = 16;$/;" m namespace:doris::sse_util +CHARS_PER_64_BIT_REGISTER src/util/sse_util.hpp /^static const int CHARS_PER_64_BIT_REGISTER = 8;$/;" m namespace:doris::sse_util +CHECK_ADLER src/exec/decompressor.h /^ CHECK_ADLER$/;" e enum:doris::LzopDecompressor::LzoChecksum +CHECK_CONSISTENCY src/agent/task_worker_pool.h /^ CHECK_CONSISTENCY,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +CHECK_CONSISTENCY_FAST src/runtime/buffered_tuple_stream3.cc 40;" d file: +CHECK_CONSISTENCY_FAST src/runtime/buffered_tuple_stream3.cc 43;" d file: +CHECK_CONSISTENCY_FULL src/runtime/buffered_tuple_stream3.cc 41;" d file: +CHECK_CONSISTENCY_FULL src/runtime/buffered_tuple_stream3.cc 44;" d file: +CHECK_CRC32 src/exec/decompressor.h /^ CHECK_CRC32,$/;" e enum:doris::LzopDecompressor::LzoChecksum +CHECK_NONE src/exec/decompressor.h /^ CHECK_NONE,$/;" e enum:doris::LzopDecompressor::LzoChecksum +CHexEscape src/gutil/strings/escaping.cc /^string CHexEscape(const StringPiece& src) {$/;" f namespace:strings +CHexEscapeString src/gutil/strings/escaping.cc /^int CHexEscapeString(const char* src, int src_len, char* dest, int dest_len) {$/;" f namespace:strings +CIDR src/util/cidr.cpp /^CIDR::CIDR() : _address(0), _netmask(0xffffffff) {$/;" f class:doris::CIDR +CIDR src/util/cidr.h /^class CIDR {$/;" c namespace:doris +CLANG_ANNOTALYSIS_ONLY src/gutil/dynamic_annotations.h 452;" d +CLANG_ANNOTALYSIS_ONLY src/gutil/dynamic_annotations.h 726;" d +CLANG_VERSION src/gutil/atomicops.h 74;" d +CLEAN src/olap/fs/block_manager.h /^ CLEAN,$/;" e enum:doris::fs::WritableBlock::State +CLEAR_ALTER_TASK src/agent/task_worker_pool.h /^ CLEAR_ALTER_TASK,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +CLEAR_TRANSACTION_TASK src/agent/task_worker_pool.h /^ CLEAR_TRANSACTION_TASK,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +CLOCK_MONOTONIC_COARSE src/util/os_info.cpp 44;" d file: +CLOMN_NUMBERS test/exec/parquet_scanner_test.cpp 71;" d file: +CLONE src/agent/task_worker_pool.h /^ CLONE,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +CLONE_PREFIX src/olap/olap_define.h /^static const std::string CLONE_PREFIX = "\/clone";$/;" m namespace:doris +CLOSED src/olap/fs/block_manager.h /^ CLOSED$/;" e enum:doris::fs::WritableBlock::State +CLUSTER_ID_PREFIX src/olap/olap_define.h /^static const std::string CLUSTER_ID_PREFIX = "\/cluster_id";$/;" m namespace:doris +CLUSTER_KEY src/http/action/mini_load.cpp /^const std::string CLUSTER_KEY = "cluster";$/;" m namespace:doris file: +CMAKE_BINARY_DIR build/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/agent/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/common/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/env/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/exec/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/exprs/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/gen_cpp/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/geo/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/gutil/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/http/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/olap/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/olap/fs/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/olap/rowset/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/runtime/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/service/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/testutil/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/tools/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/udf/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/udf_samples/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_BINARY_DIR build/src/util/Makefile /^CMAKE_BINARY_DIR = \/home\/laiyingchun\/ap_doris\/be\/build$/;" m +CMAKE_COMMAND build/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/agent/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/common/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/env/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/exec/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/exprs/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/gen_cpp/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/geo/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/gutil/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/http/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/olap/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/olap/fs/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/olap/rowset/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/runtime/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/service/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/testutil/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/tools/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/udf/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/udf_samples/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_COMMAND build/src/util/Makefile /^CMAKE_COMMAND = \/usr\/local\/bin\/cmake$/;" m +CMAKE_SOURCE_DIR build/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/agent/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/common/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/env/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/exec/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/exprs/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/gen_cpp/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/geo/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/gutil/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/http/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/olap/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/olap/fs/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/olap/rowset/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/runtime/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/service/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/testutil/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/tools/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/udf/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/udf_samples/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMAKE_SOURCE_DIR build/src/util/Makefile /^CMAKE_SOURCE_DIR = \/home\/laiyingchun\/ap_doris\/be$/;" m +CMP128 src/gutil/int128.h 127;" d +CMP128 src/gutil/int128.h 139;" d +COALESCE_COMPUTE_FUNCTION src/exprs/conditional_functions_ir.cpp 98;" d file: +COLUMNS_KEY src/http/action/mini_load.cpp /^const std::string COLUMNS_KEY = "columns";$/;" m namespace:doris file: +COLUMN_SEPARATOR_KEY src/http/action/mini_load.cpp /^const std::string COLUMN_SEPARATOR_KEY = "column_separator";$/;" m namespace:doris file: +COMPARATOR_LARGER src/olap/utils.h /^ COMPARATOR_LARGER = 1,$/;" e enum:doris::ComparatorEnum +COMPARATOR_LESS src/olap/utils.h /^ COMPARATOR_LESS = 0,$/;" e enum:doris::ComparatorEnum +COMPARISON_PREDICATE_CONDITION_VALUE src/olap/reader.cpp 772;" d file: +COMPARISON_PRED_BITMAP_EVALUATE src/olap/comparison_predicate.cpp 196;" d file: +COMPARISON_PRED_BITMAP_EVALUATE_DECLARATION src/olap/comparison_predicate.cpp 290;" d file: +COMPARISON_PRED_CLASS_DEFINE src/olap/comparison_predicate.h 28;" d +COMPARISON_PRED_COLUMN_BLOCK_EVALUATE src/olap/comparison_predicate.cpp 114;" d file: +COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_DECLARATION src/olap/comparison_predicate.cpp 269;" d file: +COMPARISON_PRED_CONSTRUCTOR src/olap/comparison_predicate.cpp 27;" d file: +COMPARISON_PRED_CONSTRUCTOR_DECLARATION src/olap/comparison_predicate.cpp 227;" d file: +COMPARISON_PRED_CONSTRUCTOR_STRING src/olap/comparison_predicate.cpp 41;" d file: +COMPARISON_PRED_EVALUATE src/olap/comparison_predicate.cpp 57;" d file: +COMPARISON_PRED_EVALUATE_DECLARATION src/olap/comparison_predicate.cpp 248;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 101;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 108;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 123;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 138;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 153;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 161;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 166;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 173;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 176;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 179;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 182;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 185;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 19;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 200;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 214;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 224;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 242;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 251;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 260;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 275;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 288;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 306;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 309;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 312;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 42;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 50;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 56;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 62;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 71;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 80;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 94;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 101;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 108;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 123;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 138;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 13;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 153;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 161;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 166;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 173;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 176;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 179;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 194;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 19;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 208;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 222;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 240;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 249;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 258;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 273;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 291;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 294;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 297;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 42;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 50;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 56;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 62;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 71;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 80;" d file: +COMPILER_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 94;" d file: +COMPILER_VERSION_INTERNAL build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 256;" d file: +COMPILER_VERSION_INTERNAL build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 254;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 103;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 110;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 116;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 125;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 131;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 140;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 146;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 154;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 162;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 168;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 189;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 204;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 215;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 226;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 245;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 24;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 253;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 263;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 268;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 277;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 282;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 291;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 296;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 43;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 51;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 58;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 64;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 73;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 83;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 88;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 96;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 103;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 110;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 116;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 125;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 131;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 140;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 146;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 154;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 15;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 162;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 168;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 183;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 198;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 210;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 212;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 224;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 243;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 24;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 251;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 261;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 266;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 276;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 281;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 43;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 51;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 58;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 64;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 73;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 83;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 88;" d file: +COMPILER_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 96;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 104;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 111;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 117;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 126;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 132;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 141;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 147;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 155;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 163;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 169;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 190;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 205;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 217;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 227;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 246;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 254;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 25;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 264;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 269;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 278;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 283;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 292;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 297;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 44;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 52;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 59;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 65;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 74;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 84;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 89;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 97;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 104;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 111;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 117;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 126;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 132;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 141;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 147;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 155;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 163;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 169;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 16;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 184;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 199;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 215;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 225;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 244;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 252;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 25;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 262;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 267;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 277;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 282;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 44;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 52;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 59;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 65;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 74;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 84;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 89;" d file: +COMPILER_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 97;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 105;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 112;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 118;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 127;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 133;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 142;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 148;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 157;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 170;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 191;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 206;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 220;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 231;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 234;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 247;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 255;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 265;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 270;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 279;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 27;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 284;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 293;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 298;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 29;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 46;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 53;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 67;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 76;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 85;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 90;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 98;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 105;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 112;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 118;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 127;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 133;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 142;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 148;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 157;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 170;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 185;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 200;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 218;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 229;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 232;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 245;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 253;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 263;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 268;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 278;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 27;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 283;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 29;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 46;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 53;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 67;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 76;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 85;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 90;" d file: +COMPILER_VERSION_PATCH build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 98;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 113;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 128;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 143;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 197;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 238;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 33;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 113;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 128;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 143;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 191;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 236;" d file: +COMPILER_VERSION_TWEAK build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 33;" d file: +COMPILE_ASSERT src/gutil/macros.h 44;" d +COMPLICATE_BINARY_FOR_NULL_PRED_FN src/exprs/binary_predicate.cpp 429;" d file: +COMPLICATE_BINARY_FOR_NULL_PRED_FNS src/exprs/binary_predicate.cpp 443;" d file: +COMPLICATE_BINARY_PRED_FN src/exprs/binary_predicate.cpp 310;" d file: +COMPLICATE_BINARY_PRED_FNS src/exprs/binary_predicate.cpp 326;" d file: +COMPRESSED src/olap/out_stream.h /^ COMPRESSED = 1$/;" e enum:doris::StreamHead::StreamType +CONCAT_IMPL src/util/runtime_profile.h 42;" d +CONCURRENT src/util/threadpool.h /^ CONCURRENT,$/;" m class:doris::ThreadPool::ExecutionMode +CONFLICT src/http/http_status.h /^ CONFLICT = 409,$/;" e enum:doris::HttpStatus +CONF_Bool src/common/configbase.h 63;" d +CONF_Bool src/common/configbase.h 76;" d +CONF_Bools src/common/configbase.h 69;" d +CONF_Bools src/common/configbase.h 82;" d +CONF_Double src/common/configbase.h 67;" d +CONF_Double src/common/configbase.h 80;" d +CONF_Doubles src/common/configbase.h 73;" d +CONF_Doubles src/common/configbase.h 86;" d +CONF_Int16 src/common/configbase.h 64;" d +CONF_Int16 src/common/configbase.h 77;" d +CONF_Int16s src/common/configbase.h 70;" d +CONF_Int16s src/common/configbase.h 83;" d +CONF_Int32 src/common/configbase.h 65;" d +CONF_Int32 src/common/configbase.h 78;" d +CONF_Int32s src/common/configbase.h 71;" d +CONF_Int32s src/common/configbase.h 84;" d +CONF_Int64 src/common/configbase.h 66;" d +CONF_Int64 src/common/configbase.h 79;" d +CONF_Int64s src/common/configbase.h 72;" d +CONF_Int64s src/common/configbase.h 85;" d +CONF_String src/common/configbase.h 68;" d +CONF_String src/common/configbase.h 81;" d +CONF_Strings src/common/configbase.h 74;" d +CONF_Strings src/common/configbase.h 87;" d +CONNECTION src/http/http_headers.cpp /^const char* HttpHeaders::CONNECTION = "Connection";$/;" m class:doris::HttpHeaders file: +CONNECTION src/http/http_headers.h /^ static const char* CONNECTION;$/;" m class:doris::HttpHeaders +CONTENT_BASE src/http/http_headers.cpp /^const char* HttpHeaders::CONTENT_BASE = "Content-Base";$/;" m class:doris::HttpHeaders file: +CONTENT_BASE src/http/http_headers.h /^ static const char* CONTENT_BASE;$/;" m class:doris::HttpHeaders +CONTENT_ENCODING src/http/http_headers.cpp /^const char* HttpHeaders::CONTENT_ENCODING = "Content-Encoding";$/;" m class:doris::HttpHeaders file: +CONTENT_ENCODING src/http/http_headers.h /^ static const char* CONTENT_ENCODING;$/;" m class:doris::HttpHeaders +CONTENT_LANGUAGE src/http/http_headers.cpp /^const char* HttpHeaders::CONTENT_LANGUAGE = "Content-Language";$/;" m class:doris::HttpHeaders file: +CONTENT_LANGUAGE src/http/http_headers.h /^ static const char* CONTENT_LANGUAGE;$/;" m class:doris::HttpHeaders +CONTENT_LENGTH src/http/http_headers.cpp /^const char* HttpHeaders::CONTENT_LENGTH = "Content-Length";$/;" m class:doris::HttpHeaders file: +CONTENT_LENGTH src/http/http_headers.h /^ static const char* CONTENT_LENGTH;$/;" m class:doris::HttpHeaders +CONTENT_LOCATION src/http/http_headers.cpp /^const char* HttpHeaders::CONTENT_LOCATION = "Content-Location";$/;" m class:doris::HttpHeaders file: +CONTENT_LOCATION src/http/http_headers.h /^ static const char* CONTENT_LOCATION;$/;" m class:doris::HttpHeaders +CONTENT_MD5 src/http/http_headers.cpp /^const char* HttpHeaders::CONTENT_MD5 = "Content-MD5";$/;" m class:doris::HttpHeaders file: +CONTENT_MD5 src/http/http_headers.h /^ static const char* CONTENT_MD5;$/;" m class:doris::HttpHeaders +CONTENT_RANGE src/http/http_headers.cpp /^const char* HttpHeaders::CONTENT_RANGE = "Content-Range";$/;" m class:doris::HttpHeaders file: +CONTENT_RANGE src/http/http_headers.h /^ static const char* CONTENT_RANGE;$/;" m class:doris::HttpHeaders +CONTENT_TRANSFER_ENCODING src/http/http_headers.cpp /^const char* HttpHeaders::CONTENT_TRANSFER_ENCODING = "Content-Transfer-Encoding";$/;" m class:doris::HttpHeaders file: +CONTENT_TRANSFER_ENCODING src/http/http_headers.h /^ static const char* CONTENT_TRANSFER_ENCODING;$/;" m class:doris::HttpHeaders +CONTENT_TYPE src/http/http_headers.cpp /^const char* HttpHeaders::CONTENT_TYPE = "Content-Type";$/;" m class:doris::HttpHeaders file: +CONTENT_TYPE src/http/http_headers.h /^ static const char* CONTENT_TYPE;$/;" m class:doris::HttpHeaders +CONTINUE src/http/http_status.h /^ CONTINUE = 100,$/;" e enum:doris::HttpStatus +CONVERTED_FLAG src/olap/olap_define.h /^static const std::string CONVERTED_FLAG = "true";$/;" m namespace:doris +CONVERT_FROM_TYPE src/olap/schema_change.cpp 156;" d file: +CONVERT_FROM_TYPE src/olap/schema_change.cpp 438;" d file: +COOKIE src/http/http_headers.cpp /^const char* HttpHeaders::COOKIE = "Cookie";$/;" m class:doris::HttpHeaders file: +COOKIE src/http/http_headers.h /^ static const char* COOKIE;$/;" m class:doris::HttpHeaders +COUNT src/exprs/agg_fn.h /^ COUNT,$/;" e enum:doris::AggFn::AggregationOp +COUNT src/exprs/agg_fn_evaluator.h /^ COUNT,$/;" e enum:doris::AggFnEvaluator::AggregationOp +COUNTER src/util/metrics.h /^ COUNTER,$/;" m class:doris::MetricType +COUNTER_NAME src/runtime/mem_tracker.cpp /^const std::string MemTracker::COUNTER_NAME = "PeakMemoryUsage";$/;" m class:doris::MemTracker file: +COUNTER_NAME src/runtime/mem_tracker.h /^ static const std::string COUNTER_NAME;$/;" m class:doris::MemTracker +COUNTER_SET src/util/runtime_profile.h 55;" d +COUNTER_SIZE src/util/perf_counters.cpp 36;" d file: +COUNTER_UPDATE src/util/runtime_profile.h 54;" d +COUNT_DISTINCT src/exprs/agg_fn.h /^ COUNT_DISTINCT,$/;" e enum:doris::AggFn::AggregationOp +COUNT_DISTINCT src/exprs/agg_fn_evaluator.h /^ COUNT_DISTINCT,$/;" e enum:doris::AggFnEvaluator::AggregationOp +CPU src/gutil/cpu.cc /^CPU::CPU()$/;" f class:base::CPU +CR src/http/http_parser.cpp 31;" d file: +CRC test/util/crc32c_test.cpp /^class CRC { };$/;" c namespace:doris::crc32c file: +CRC32_INIT src/olap/utils.h 157;" d +CRC32_INIT_VALUE src/exec/decompressor.h /^ const static uint32_t CRC32_INIT_VALUE;$/;" m class:doris::LzopDecompressor +CRC32_INIT_VALUE src/exec/lzo_decompressor.cpp /^const uint32_t LzopDecompressor::CRC32_INIT_VALUE = 0;$/;" m class:doris::LzopDecompressor file: +CREATED src/http/http_status.h /^ CREATED = 201,$/;" e enum:doris::HttpStatus +CREATE_AND_START_POOL src/agent/agent_server.cpp 59;" d file: +CREATE_AND_START_POOL src/agent/agent_server.cpp 90;" d file: +CREATE_OR_OPEN src/env/env.h /^ CREATE_OR_OPEN,$/;" e enum:doris::Env::OpenMode +CREATE_OR_OPEN_WITH_TRUNCATE src/env/env.h /^ CREATE_OR_OPEN_WITH_TRUNCATE,$/;" e enum:doris::Env::OpenMode +CREATE_TABLE src/agent/task_worker_pool.h /^ CREATE_TABLE,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +CTOR_DCTOR_FUN src/exprs/conditional_functions.cpp 28;" d file: +CUMULATIVE_COMPACTION src/olap/olap_common.h /^ CUMULATIVE_COMPACTION = 2$/;" e enum:doris::CompactionType +CURRENT_COLUMN_DATA_VERSION src/olap/rowset/segment_reader.h /^ static const uint32_t CURRENT_COLUMN_DATA_VERSION = 1;$/;" m class:doris::SegmentReader +CUnescape src/gutil/strings/escaping.cc /^bool CUnescape(const StringPiece& source, char* dest, int* dest_len,$/;" f namespace:strings +CUnescape src/gutil/strings/escaping.cc /^bool CUnescape(const StringPiece& source, string* dest, string* error) {$/;" f namespace:strings +CUnescape src/gutil/strings/escaping.h /^inline bool CUnescape(const StringPiece& source, string* dest) {$/;" f namespace:strings +CUnescapeForNullTerminatedString src/gutil/strings/escaping.cc /^bool CUnescapeForNullTerminatedString(const StringPiece& source,$/;" f namespace:strings +CUnescapeForNullTerminatedString src/gutil/strings/escaping.h /^inline bool CUnescapeForNullTerminatedString(const StringPiece& source,$/;" f namespace:strings +CUnescapeInternal src/gutil/strings/escaping.cc /^bool CUnescapeInternal(const StringPiece& source,$/;" f namespace:strings +CUnescapeInternal src/gutil/strings/escaping.cc /^static bool CUnescapeInternal(const StringPiece& source,$/;" f namespace:strings +CXX_STD build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 558;" d file: +CXX_STD build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 560;" d file: +C_DIALECT build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 575;" d file: +C_DIALECT build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 577;" d file: +C_DIALECT build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 580;" d file: +C_DIALECT build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 582;" d file: +C_DIALECT build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 584;" d file: +Cache src/olap/lru_cache.h /^ Cache() {}$/;" f class:doris::CachePriority::Cache +Cache src/olap/lru_cache.h /^ class Cache {$/;" c class:doris::CachePriority +CacheEntry src/runtime/small_file_mgr.h /^struct CacheEntry {$/;" s namespace:doris +CacheKey src/olap/lru_cache.h /^ CacheKey() : _data(NULL), _size(0) {}$/;" f class:doris::CacheKey +CacheKey src/olap/lru_cache.h /^ CacheKey(const char* d, size_t n) : _data(d), _size(n) {}$/;" f class:doris::CacheKey +CacheKey src/olap/lru_cache.h /^ CacheKey(const char* s) : _data(s), _size(strlen(s)) { }$/;" f class:doris::CacheKey +CacheKey src/olap/lru_cache.h /^ CacheKey(const std::string& s) : _data(s.data()), _size(s.size()) { }$/;" f class:doris::CacheKey +CacheKey src/olap/lru_cache.h /^ class CacheKey {$/;" c namespace:doris +CacheKey src/olap/page_cache.h /^ CacheKey(std::string fname_, int64_t offset_) : fname(std::move(fname_)), offset(offset_) { }$/;" f struct:doris::StoragePageCache::CacheKey +CacheKey src/olap/page_cache.h /^ struct CacheKey {$/;" s class:doris::StoragePageCache +CacheLevel src/util/cpu_info.h /^ enum CacheLevel {$/;" g class:doris::CpuInfo +CachePriority src/olap/lru_cache.h /^ enum class CachePriority {$/;" c namespace:doris +CacheTest test/olap/lru_cache_test.cpp /^ CacheTest() : _cache(new_lru_cache(kCacheSize)) {$/;" f class:doris::CacheTest +CacheTest test/olap/lru_cache_test.cpp /^class CacheTest : public testing::Test {$/;" c namespace:doris file: +CalcBytesPinned src/runtime/buffered_tuple_stream3.cc /^int64_t BufferedTupleStream3::CalcBytesPinned() const {$/;" f class:BufferedTupleStream3 +CalcMaxBufferLen src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::BufferAllocator::CalcMaxBufferLen($/;" f class:doris::BufferPool::BufferAllocator +CalcPageLenForRow src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::CalcPageLenForRow(int64_t row_size, int64_t* page_len) {$/;" f class:BufferedTupleStream3 +CalculateBase32EscapedLen src/gutil/strings/escaping.cc /^int CalculateBase32EscapedLen(size_t input_len) {$/;" f namespace:strings +CalculateBase64EscapedLen src/gutil/strings/escaping.cc /^int CalculateBase64EscapedLen(int input_len) {$/;" f namespace:strings +CalculateBase64EscapedLen src/gutil/strings/escaping.cc /^int CalculateBase64EscapedLen(int input_len, bool do_padding) {$/;" f namespace:strings +CalculateReserveForVector src/gutil/strings/split.cc /^static int CalculateReserveForVector(const string& full, const char* delim) {$/;" f file: +Cancelled src/common/status.h /^ static Status Cancelled(const Slice& msg, int16_t precise_code = 1, const Slice& msg2 = Slice()) {$/;" f class:doris::Status +Cancelled src/runtime/disk_io_mgr_internal.h /^ Cancelled,$/;" e enum:doris::DiskIoMgr::RequestContext::State +CappedDifference src/gutil/bits.cc /^int Bits::CappedDifference(const void *m1, const void *m2,$/;" f class:Bits +CaseExpr src/exprs/case_expr.cpp /^CaseExpr::CaseExpr(const TExprNode& node) : $/;" f class:doris::CaseExpr +CaseExpr src/exprs/case_expr.h /^class CaseExpr: public Expr {$/;" c namespace:doris +CaseExprState src/exprs/case_expr.cpp /^struct CaseExprState {$/;" s namespace:doris file: +CastExpr src/exprs/cast_expr.h /^ CastExpr(const TExprNode& node) : Expr(node) { }$/;" f class:doris::CastExpr +CastExpr src/exprs/cast_expr.h /^class CastExpr : public Expr {$/;" c namespace:doris +CastFunctions src/exprs/cast_functions.h /^class CastFunctions {$/;" c namespace:doris +Ceil src/util/bit_util.h /^ constexpr static inline int64_t Ceil(int64_t value, int64_t divisor) {$/;" f class:doris::BitUtil +Ceil src/util/bit_util.h /^ static inline int Ceil(int value, int divisor) {$/;" f class:doris::BitUtil +Centroid src/util/tdigest.h /^ Centroid() : Centroid(0.0, 0.0) {}$/;" f class:doris::Centroid +Centroid src/util/tdigest.h /^ Centroid(Value mean, Weight weight) : _mean(mean), _weight(weight) {}$/;" f class:doris::Centroid +Centroid src/util/tdigest.h /^class Centroid {$/;" c namespace:doris +CentroidComparator src/util/tdigest.h /^struct CentroidComparator {$/;" s namespace:doris +CentroidList src/util/tdigest.h /^ CentroidList(const std::vector& s) : iter(s.cbegin()), end(s.cend()) {}$/;" f struct:doris::CentroidList +CentroidList src/util/tdigest.h /^struct CentroidList {$/;" s namespace:doris +CentroidListComparator src/util/tdigest.h /^ CentroidListComparator() {}$/;" f class:doris::CentroidListComparator +CentroidListComparator src/util/tdigest.h /^class CentroidListComparator {$/;" c namespace:doris +CgroupsMgr src/agent/cgroups_mgr.cpp /^CgroupsMgr::CgroupsMgr(ExecEnv* exec_env, const string& root_cgroups_path)$/;" f class:doris::CgroupsMgr +CgroupsMgr src/agent/cgroups_mgr.h /^class CgroupsMgr {$/;" c namespace:doris +CgroupsMgrTest test/agent/cgroups_mgr_test.cpp /^class CgroupsMgrTest : public testing::Test {$/;" c namespace:doris file: +Channel src/runtime/data_stream_sender.cpp /^ Channel(DataStreamSender* parent, const RowDescriptor& row_desc,$/;" f class:doris::DataStreamSender::Channel +Channel src/runtime/data_stream_sender.cpp /^class DataStreamSender::Channel {$/;" c class:doris::DataStreamSender file: +CharField src/olap/field.h /^ explicit CharField(const TabletColumn& column) : Field(column) {$/;" f class:doris::CharField +CharField src/olap/field.h /^class CharField: public Field {$/;" c namespace:doris +CharSet src/gutil/strings/charset.cc /^CharSet::CharSet() {$/;" f class:strings::CharSet +CharSet src/gutil/strings/charset.cc /^CharSet::CharSet(const CharSet& other) {$/;" f class:strings::CharSet +CharSet src/gutil/strings/charset.cc /^CharSet::CharSet(const char* characters) {$/;" f class:strings::CharSet +CharSet src/gutil/strings/charset.h /^class CharSet {$/;" c namespace:strings +Charmap src/gutil/charmap.h /^ Charmap(const char* str, int len) {$/;" f class:Charmap +Charmap src/gutil/charmap.h /^ Charmap(uint32 b0, uint32 b1, uint32 b2, uint32 b3,$/;" f class:Charmap +Charmap src/gutil/charmap.h /^ explicit Charmap(const char* str) {$/;" f class:Charmap +Charmap src/gutil/charmap.h /^DECLARE_POD(Charmap);$/;" v +Charmap src/gutil/charmap.h /^class Charmap {$/;" c +CharsetStruct src/exec/schema_scanner/schema_charsets_scanner.h /^ struct CharsetStruct {$/;" s class:doris::SchemaCharsetsScanner +Check src/gutil/threading/thread_collision_warner.h /^ explicit Check(ThreadCollisionWarner* warner)$/;" f class:base::ThreadCollisionWarner::Check +Check src/gutil/threading/thread_collision_warner.h /^ class BASE_EXPORT Check {$/;" c class:base::ThreadCollisionWarner +CheckAndResize src/exec/new_partitioned_hash_table.cc /^Status NewPartitionedHashTable::CheckAndResize($/;" f class:NewPartitionedHashTable +CheckAndResizeHashPartitions src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::CheckAndResizeHashPartitions($/;" f class:doris::NewPartitionedAggregationNode +CheckConsistency src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::CheckConsistency() const {$/;" f class:doris::ReservationTracker +CheckConsistencyFast src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::CheckConsistencyFast() const {$/;" f class:BufferedTupleStream3 +CheckConsistencyFull src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::CheckConsistencyFull() const {$/;" f class:BufferedTupleStream3 +CheckKuduAvailability src/exec/kudu_util.cpp /^Status CheckKuduAvailability() {$/;" f namespace:doris +CheckNaturalAlignment src/gutil/atomicops-internals-x86.h /^inline void CheckNaturalAlignment(const T *ptr) {$/;" f namespace:base::subtle +CheckPageConsistency src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::CheckPageConsistency(const Page* page) const {$/;" f class:BufferedTupleStream3 +ChecksumAction src/http/action/checksum_action.cpp /^ChecksumAction::ChecksumAction(ExecEnv* exec_env) :$/;" f class:doris::ChecksumAction +ChecksumAction src/http/action/checksum_action.h /^class ChecksumAction : public HttpHandler {$/;" c namespace:doris +ChildCounterMap src/util/runtime_profile.h /^ typedef std::map > ChildCounterMap;$/;" t class:doris::RuntimeProfile +ChildCtx src/olap/reader.cpp /^ ChildCtx(RowsetReaderSharedPtr rs_reader, Reader* reader)$/;" f class:doris::CollectIterator::ChildCtx +ChildCtx src/olap/reader.cpp /^ class ChildCtx {$/;" c class:doris::CollectIterator file: +ChildCtxComparator src/olap/reader.cpp /^ class ChildCtxComparator {$/;" c class:doris::CollectIterator file: +ChildMap src/util/runtime_profile.h /^ typedef std::map ChildMap;$/;" t class:doris::RuntimeProfile +ChildReaderContext src/exec/merge_join_node.h /^ ChildReaderContext(const RowDescriptor& desc, int batch_size, MemTracker* mem_tracker) :$/;" f struct:doris::MergeJoinNode::ChildReaderContext +ChildReaderContext src/exec/merge_join_node.h /^ struct ChildReaderContext {$/;" s class:doris::MergeJoinNode +ChildVector src/util/runtime_profile.h /^ typedef std::vector > ChildVector;$/;" t class:doris::RuntimeProfile +Chunk src/runtime/memory/chunk.h /^struct Chunk {$/;" s namespace:doris +ChunkAllocator src/runtime/memory/chunk_allocator.cpp /^ChunkAllocator::ChunkAllocator(size_t reserve_limit)$/;" f class:doris::ChunkAllocator +ChunkAllocator src/runtime/memory/chunk_allocator.h /^class ChunkAllocator {$/;" c namespace:doris +ChunkArena src/runtime/memory/chunk_allocator.cpp /^ ChunkArena() : _chunk_lists(64) { }$/;" f class:doris::ChunkArena +ChunkArena src/runtime/memory/chunk_allocator.cpp /^class ChunkArena {$/;" c namespace:doris file: +ChunkInfo src/runtime/mem_pool.cpp /^MemPool::ChunkInfo::ChunkInfo(const Chunk& chunk_)$/;" f class:doris::MemPool::ChunkInfo +ChunkInfo src/runtime/mem_pool.h /^ ChunkInfo() : allocated_bytes(0) { }$/;" f struct:doris::MemPool::ChunkInfo +ChunkInfo src/runtime/mem_pool.h /^ struct ChunkInfo {$/;" s class:doris::MemPool +CityHash128 src/gutil/hash/city.cc /^uint128 CityHash128(const char *s, size_t len) {$/;" f namespace:util_hash +CityHash128WithSeed src/gutil/hash/city.cc /^uint128 CityHash128WithSeed(const char *s, size_t len, uint128 seed) {$/;" f namespace:util_hash +CityHash64 src/gutil/hash/city.cc /^uint64 CityHash64(const char *s, size_t len) {$/;" f namespace:util_hash +CityHash64WithSeed src/gutil/hash/city.cc /^uint64 CityHash64WithSeed(const char *s, size_t len, uint64 seed) {$/;" f namespace:util_hash +CityHash64WithSeeds src/gutil/hash/city.cc /^uint64 CityHash64WithSeeds(const char *s, size_t len,$/;" f namespace:util_hash +CityMurmur src/gutil/hash/city.cc /^static uint128 CityMurmur(const char *s, size_t len, uint128 seed) {$/;" f namespace:util_hash +Claim src/runtime/initial_reservations.cc /^void InitialReservations::Claim(BufferPool::ClientHandle* dst, int64_t bytes) {$/;" f class:doris::InitialReservations +CleanPages src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::Client::CleanPages(unique_lock* client_lock, int64_t len) {$/;" f class:doris::BufferPool::Client +CleanStringLineEndings src/gutil/strings/escaping.cc /^void CleanStringLineEndings(const string& src, string* dst,$/;" f namespace:strings +CleanStringLineEndings src/gutil/strings/escaping.cc /^void CleanStringLineEndings(string* str, bool auto_end_last_line) {$/;" f namespace:strings +Cleanup src/exec/exec_node.cpp /^int ExecNode::RowBatchQueue::Cleanup() {$/;" f class:doris::ExecNode::RowBatchQueue +CleanupHashTbl src/exec/new_partitioned_aggregation_node.cc /^void NewPartitionedAggregationNode::CleanupHashTbl($/;" f class:doris::NewPartitionedAggregationNode +Clear src/util/bit_stream_utils.h /^ void Clear() {$/;" f class:doris::BitWriter +Clear src/util/rle_encoding.h /^inline void RleEncoder::Clear() {$/;" f class:doris::RleEncoder +Client src/runtime/buffered_block_mgr2.cc /^ Client(BufferedBlockMgr2* mgr, int num_reserved_buffers, MemTracker* tracker,$/;" f class:doris::BufferedBlockMgr2::Client +Client src/runtime/buffered_block_mgr2.cc /^class BufferedBlockMgr2::Client {$/;" c class:doris::BufferedBlockMgr2 file: +Client src/runtime/bufferpool/buffer_pool.cc /^BufferPool::Client::Client(BufferPool* pool, \/\/TmpFileMgr::FileGroup* file_group,$/;" f class:doris::BufferPool::Client +Client src/runtime/bufferpool/buffer_pool_internal.h /^class BufferPool::Client {$/;" c class:doris::BufferPool +Client src/runtime/client_cache.h /^ typedef ThriftClient Client;$/;" t class:doris::ClientCache +ClientCache src/runtime/client_cache.h /^ ClientCache() : _client_cache_helper() {$/;" f class:doris::ClientCache +ClientCache src/runtime/client_cache.h /^ ClientCache(int max_cache_size) : _client_cache_helper(max_cache_size) {$/;" f class:doris::ClientCache +ClientCache src/runtime/client_cache.h /^class ClientCache {$/;" c namespace:doris +ClientCacheHelper src/runtime/client_cache.h /^ ClientCacheHelper() : _metrics_enabled(false), _max_cache_size_per_host(-1) { }$/;" f class:doris::ClientCacheHelper +ClientCacheHelper src/runtime/client_cache.h /^ ClientCacheHelper(int max_cache_size_per_host):$/;" f class:doris::ClientCacheHelper +ClientCacheHelper src/runtime/client_cache.h /^class ClientCacheHelper {$/;" c namespace:doris +ClientCacheMap src/runtime/client_cache.h /^ TNetworkAddress, std::list > ClientCacheMap;$/;" t class:doris::ClientCacheHelper +ClientConnection src/runtime/client_cache.h /^ ClientConnection(ClientCache* client_cache, TNetworkAddress address, Status* status)$/;" f class:doris::ClientConnection +ClientConnection src/runtime/client_cache.h /^ ClientConnection(ClientCache* client_cache, TNetworkAddress address, int timeout_ms,$/;" f class:doris::ClientConnection +ClientConnection src/runtime/client_cache.h /^class ClientConnection {$/;" c namespace:doris +ClientHandle src/runtime/bufferpool/buffer_pool.h /^ ClientHandle() : impl_(NULL) {}$/;" f class:doris::BufferPool::ClientHandle +ClientHandle src/runtime/bufferpool/buffer_pool.h /^class BufferPool::ClientHandle {$/;" c class:doris::BufferPool +ClientMap src/runtime/client_cache.h /^ typedef boost::unordered_map ClientMap;$/;" t class:doris::ClientCacheHelper +ClipString src/gutil/strings/split.cc /^void ClipString(char* str, int max_len) {$/;" f +ClipString src/gutil/strings/split.cc /^void ClipString(string* full_str, int max_len) {$/;" f +ClipStringHelper src/gutil/strings/split.cc /^static int ClipStringHelper(const char* str, int max_len, bool use_ellipsis) {$/;" f file: +Close src/exec/new_partitioned_aggregation_node.cc /^void NewPartitionedAggregationNode::Partition::Close(bool finalize_rows) {$/;" f class:doris::NewPartitionedAggregationNode::Partition +Close src/exec/new_partitioned_hash_table.cc /^void NewPartitionedHashTable::Close() {$/;" f class:NewPartitionedHashTable +Close src/exec/new_partitioned_hash_table.cc /^void NewPartitionedHashTableCtx::Close(RuntimeState* state) {$/;" f class:NewPartitionedHashTableCtx +Close src/exec/new_partitioned_hash_table.cc /^void NewPartitionedHashTableCtx::ExprValuesCache::Close(MemTracker* tracker) {$/;" f class:NewPartitionedHashTableCtx::ExprValuesCache +Close src/exec/parquet_reader.cpp /^arrow::Status ParquetFile::Close() {$/;" f class:doris::ParquetFile +Close src/exprs/agg_fn.cc /^void AggFn::Close() {$/;" f class:doris::AggFn +Close src/exprs/agg_fn.cc /^void AggFn::Close(const vector& exprs) {$/;" f class:doris::AggFn +Close src/exprs/new_agg_fn_evaluator.cc /^void NewAggFnEvaluator::Close($/;" f class:NewAggFnEvaluator +Close src/exprs/new_agg_fn_evaluator.cc /^void NewAggFnEvaluator::Close(RuntimeState* state) {$/;" f class:NewAggFnEvaluator +Close src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::Close(RowBatch* batch, RowBatch::FlushMode flush) {$/;" f class:BufferedTupleStream3 +Close src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::SubReservation::Close() {$/;" f class:doris::BufferPool::SubReservation +Close src/runtime/bufferpool/buffer_pool_internal.h /^ void Close() { reservation_.Close(); }$/;" f class:doris::BufferPool::Client +Close src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::Close() {$/;" f class:doris::ReservationTracker +ClosePartitions src/exec/new_partitioned_aggregation_node.cc /^void NewPartitionedAggregationNode::ClosePartitions() {$/;" f class:doris::NewPartitionedAggregationNode +ClosingSymbolLookup src/gutil/strings/split.cc /^ explicit ClosingSymbolLookup(const char* symbol_pairs)$/;" f class:__anon31::ClosingSymbolLookup +ClosingSymbolLookup src/gutil/strings/split.cc /^class ClosingSymbolLookup {$/;" c namespace:__anon31 file: +CmdlineAction src/http/action/pprof_actions.cpp /^ CmdlineAction() {}$/;" f class:doris::CmdlineAction +CmdlineAction src/http/action/pprof_actions.cpp /^class CmdlineAction : public HttpHandler {$/;" c namespace:doris file: +CoalesceBuddies src/runtime/bufferpool/suballocator.cc /^unique_ptr Suballocator::CoalesceBuddies($/;" f class:doris::Suballocator +CoalesceExpr src/exprs/conditional_functions.h /^class CoalesceExpr : public Expr {$/;" c namespace:doris +CodingTest test/util/coding_test.cpp /^ CodingTest() { }$/;" f class:doris::CodingTest +CodingTest test/util/coding_test.cpp /^class CodingTest : public testing::Test {$/;" c namespace:doris file: +CollationStruct src/exec/schema_scanner/schema_collations_scanner.h /^ struct CollationStruct {$/;" s class:doris::SchemaCollationsScanner +CollectIterator src/olap/reader.cpp /^class CollectIterator {$/;" c namespace:doris file: +ColumnBlock src/olap/column_block.h /^ ColumnBlock(const TypeInfo* type_info, uint8_t* data, uint8_t* null_bitmap,$/;" f class:doris::ColumnBlock +ColumnBlock src/olap/column_block.h /^class ColumnBlock {$/;" c namespace:doris +ColumnBlockCell src/olap/column_block.h /^ ColumnBlockCell(ColumnBlock block, size_t idx) : _block(block), _idx(idx) { }$/;" f struct:doris::ColumnBlockCell +ColumnBlockCell src/olap/column_block.h /^struct ColumnBlockCell {$/;" s namespace:doris +ColumnBlockView src/olap/column_block.h /^ explicit ColumnBlockView(ColumnBlock* block, size_t row_offset = 0)$/;" f class:doris::ColumnBlockView +ColumnBlockView src/olap/column_block.h /^class ColumnBlockView {$/;" c namespace:doris +ColumnData src/olap/rowset/column_data.cpp /^ColumnData::ColumnData(SegmentGroup* segment_group)$/;" f class:doris::ColumnData +ColumnData src/olap/rowset/column_data.h /^class ColumnData {$/;" c namespace:doris +ColumnDataComparator src/olap/rowset/column_data.h /^ ColumnDataComparator($/;" f class:doris::ColumnDataComparator +ColumnDataComparator src/olap/rowset/column_data.h /^class ColumnDataComparator {$/;" c namespace:doris +ColumnDataWriter src/olap/rowset/column_data_writer.cpp /^ColumnDataWriter::ColumnDataWriter(SegmentGroup* segment_group,$/;" f class:doris::ColumnDataWriter +ColumnDataWriter src/olap/rowset/column_data_writer.h /^class ColumnDataWriter {$/;" c namespace:doris +ColumnDesc src/exec/schema_scanner.h /^ struct ColumnDesc {$/;" s class:doris::SchemaScanner +ColumnFamilyIndex src/olap/olap_define.h /^enum ColumnFamilyIndex {$/;" g namespace:doris +ColumnId src/olap/olap_common.h /^typedef uint32_t ColumnId;$/;" t namespace:doris +ColumnIdIterator src/olap/rowset/segment_reader.h /^ typedef std::vector::iterator ColumnIdIterator;$/;" t class:doris::SegmentReader +ColumnIterator src/olap/rowset/segment_v2/column_reader.h /^ ColumnIterator() { }$/;" f class:doris::segment_v2::ColumnIterator +ColumnIterator src/olap/rowset/segment_v2/column_reader.h /^class ColumnIterator {$/;" c namespace:doris::segment_v2 +ColumnIteratorOptions src/olap/rowset/segment_v2/column_reader.h /^struct ColumnIteratorOptions {$/;" s namespace:doris::segment_v2 +ColumnMapping src/olap/column_mapping.h /^ ColumnMapping() : ref_column(-1), default_value(nullptr) {}$/;" f struct:doris::ColumnMapping +ColumnMapping src/olap/column_mapping.h /^struct ColumnMapping {$/;" s namespace:doris +ColumnPredicate src/olap/column_predicate.h /^ explicit ColumnPredicate(uint32_t column_id) : _column_id(column_id) { }$/;" f class:doris::ColumnPredicate +ColumnPredicate src/olap/column_predicate.h /^class ColumnPredicate {$/;" c namespace:doris +ColumnReader src/olap/rowset/column_reader.cpp /^ColumnReader::ColumnReader(uint32_t column_id, uint32_t column_unique_id) :$/;" f class:doris::ColumnReader +ColumnReader src/olap/rowset/column_reader.h /^class ColumnReader {$/;" c namespace:doris +ColumnReader src/olap/rowset/segment_v2/column_reader.cpp /^ColumnReader::ColumnReader(const ColumnReaderOptions& opts,$/;" f class:doris::segment_v2::ColumnReader +ColumnReader src/olap/rowset/segment_v2/column_reader.h /^class ColumnReader {$/;" c namespace:doris::segment_v2 +ColumnReaderOptions src/olap/rowset/segment_v2/column_reader.h /^struct ColumnReaderOptions {$/;" s namespace:doris::segment_v2 +ColumnReaderWriterTest test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^ ColumnReaderWriterTest() : _pool(&_tracker) { }$/;" f class:doris::segment_v2::ColumnReaderWriterTest +ColumnReaderWriterTest test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^class ColumnReaderWriterTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +ColumnStatistics src/olap/stream_index_common.cpp /^ColumnStatistics::ColumnStatistics() : $/;" f class:doris::ColumnStatistics +ColumnStatistics src/olap/stream_index_common.h /^class ColumnStatistics {$/;" c namespace:doris +ColumnValueRange src/exec/olap_common.h /^ColumnValueRange::ColumnValueRange() : _column_type(INVALID_TYPE) {$/;" f class:doris::ColumnValueRange +ColumnValueRange src/exec/olap_common.h /^ColumnValueRange::ColumnValueRange(std::string col_name, PrimitiveType type, T min, T max)$/;" f class:doris::ColumnValueRange +ColumnValueRange src/exec/olap_common.h /^class ColumnValueRange {$/;" c namespace:doris +ColumnValueRangeTest test/exec/olap_common_test.cpp /^class ColumnValueRangeTest : public ::testing::Test {$/;" c namespace:doris file: +ColumnValueRangeType src/exec/olap_common.h /^ ColumnValueRange > ColumnValueRangeType;$/;" t namespace:doris +ColumnVector src/runtime/vectorized_row_batch.h /^ ColumnVector() { }$/;" f class:doris::ColumnVector +ColumnVector src/runtime/vectorized_row_batch.h /^class ColumnVector {$/;" c namespace:doris +ColumnWriter src/olap/rowset/column_writer.cpp /^ColumnWriter::ColumnWriter($/;" f class:doris::ColumnWriter +ColumnWriter src/olap/rowset/column_writer.h /^class ColumnWriter {$/;" c namespace:doris +ColumnWriter src/olap/rowset/segment_v2/column_writer.cpp /^ColumnWriter::ColumnWriter(const ColumnWriterOptions& opts,$/;" f class:doris::segment_v2::ColumnWriter +ColumnWriter src/olap/rowset/segment_v2/column_writer.h /^class ColumnWriter {$/;" c namespace:doris::segment_v2 +ColumnWriterOptions src/olap/rowset/segment_v2/column_writer.h /^struct ColumnWriterOptions {$/;" s namespace:doris::segment_v2 +ColumnZoneMapTest test/olap/rowset/segment_v2/zone_map_index_test.cpp /^class ColumnZoneMapTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +CombineFingerprintHalves src/gutil/hash/hash.h /^inline uint64 CombineFingerprintHalves(uint32 hi, uint32 lo) {$/;" f +ComesBefore src/util/monotime.cpp /^bool MonoTime::ComesBefore(const MonoTime &rhs) const {$/;" f class:doris::MonoTime +CommaMoneypunct src/exprs/string_functions.h /^ struct CommaMoneypunct : std::moneypunct {$/;" s class:doris::StringFunctions +Compaction src/olap/compaction.cpp /^Compaction::Compaction(TabletSharedPtr tablet)$/;" f class:doris::Compaction +Compaction src/olap/compaction.h /^class Compaction {$/;" c namespace:doris +CompactionAction src/http/action/compaction_action.h /^ CompactionAction(CompactionActionType type) : _type(type) {}$/;" f class:doris::CompactionAction +CompactionAction src/http/action/compaction_action.h /^class CompactionAction : public HttpHandler {$/;" c namespace:doris +CompactionActionType src/http/action/compaction_action.h /^enum CompactionActionType {$/;" g namespace:doris +CompactionCandidate src/olap/storage_engine.h /^ CompactionCandidate(uint32_t nicumulative_compaction_, int64_t tablet_id_, uint32_t index_) :$/;" f struct:doris::StorageEngine::CompactionCandidate +CompactionCandidate src/olap/storage_engine.h /^ struct CompactionCandidate {$/;" s class:doris::StorageEngine +CompactionCandidateComparator src/olap/storage_engine.h /^ struct CompactionCandidateComparator {$/;" s class:doris::StorageEngine +CompactionDiskStat src/olap/storage_engine.h /^ CompactionDiskStat(std::string path, uint32_t index, bool used) :$/;" f struct:doris::StorageEngine::CompactionDiskStat +CompactionDiskStat src/olap/storage_engine.h /^ struct CompactionDiskStat {$/;" s class:doris::StorageEngine +CompactionState src/olap/compaction.h /^ enum CompactionState {$/;" g class:doris::Compaction +CompactionType src/olap/olap_common.h /^enum CompactionType {$/;" g namespace:doris +Comparator src/util/slice.h /^ struct Comparator {$/;" s struct:doris::Slice +ComparatorEnum src/olap/utils.h /^enum ComparatorEnum {$/;" g namespace:doris +ComparatorException src/olap/utils.h /^class ComparatorException : public std::exception {$/;" c namespace:doris +CompareFn src/exec/merge_join_node.h /^ typedef int (*CompareFn)(const void*, const void*);$/;" t class:doris::MergeJoinNode +CompareFn src/util/tuple_row_compare.h /^ typedef int (*CompareFn)(ExprContext* const*, ExprContext* const*, TupleRow*, TupleRow*);$/;" t class:doris::TupleRowComparator +CompareLargeFunc src/exec/olap_utils.h /^typedef bool (*CompareLargeFunc)(const void*, const void*);$/;" t namespace:doris +ComparisonOp src/runtime/data_stream_mgr.h /^ struct ComparisonOp {$/;" s class:doris::DataStreamMgr +CompileAssert src/gutil/macros.h /^struct CompileAssert {$/;" s +CompoundPredicate src/exprs/compound_predicate.cpp /^CompoundPredicate::CompoundPredicate(const TExprNode& node) :$/;" f class:doris::CompoundPredicate +CompoundPredicate src/exprs/compound_predicate.h /^class CompoundPredicate: public Predicate {$/;" c namespace:doris +CompressType src/exec/decompressor.h /^enum CompressType {$/;" g namespace:doris +CompressedSizeType src/olap/push_handler.h /^ typedef uint64_t CompressedSizeType;$/;" t class:doris::LzoBinaryReader +Compressor src/olap/compress.h /^typedef OLAPStatus(*Compressor)(StorageByteBuffer* in, StorageByteBuffer* out, bool* smaller);$/;" t namespace:doris +ComputeFn src/exprs/expr.h /^ typedef void* (*ComputeFn)(Expr*, TupleRow*);$/;" t class:doris::Expr +ComputeIfAbsent src/gutil/map-util.h /^ComputeIfAbsent(MapContainer* container,$/;" f class:MapContainer +ComputeIfAbsentReturnAbsense src/gutil/map-util.h /^ComputeIfAbsentReturnAbsense(MapContainer* container,$/;" f +ComputeListIndex src/runtime/bufferpool/suballocator.cc /^int Suballocator::ComputeListIndex(int64_t bytes) const {$/;" f class:doris::Suballocator +ComputePairIfAbsent src/gutil/map-util.h /^ComputePairIfAbsent(MapContainer* container,$/;" f class:MapContainer +ComputePairIfAbsentReturnAbsense src/gutil/map-util.h /^ComputePairIfAbsentReturnAbsense(MapContainer* container,$/;" f +ComputeX86FamilyAndModel src/gutil/cpu.cc /^std::tuple ComputeX86FamilyAndModel($/;" f namespace:base::internal +ConcurrentTest test/olap/skiplist_test.cpp /^ ConcurrentTest():$/;" f class:doris::ConcurrentTest +ConcurrentTest test/olap/skiplist_test.cpp /^class ConcurrentTest {$/;" c namespace:doris file: +Cond src/olap/olap_cond.cpp /^Cond::Cond() : op(OP_NULL), operand_field(nullptr) {$/;" f class:doris::Cond +Cond src/olap/olap_cond.h /^struct Cond {$/;" s namespace:doris +CondColumn src/olap/olap_cond.h /^ CondColumn(const TabletSchema& tablet_schema, int32_t index) : _col_index(index) {$/;" f class:doris::CondColumn +CondColumn src/olap/olap_cond.h /^class CondColumn {$/;" c namespace:doris +CondColumns src/olap/olap_cond.h /^ typedef std::map CondColumns;$/;" t class:doris::Conditions +CondOp src/olap/olap_cond.h /^enum CondOp {$/;" g namespace:doris +ConditionVariable src/util/condition_variable.cpp /^ConditionVariable::ConditionVariable(Mutex* user_lock)$/;" f class:doris::ConditionVariable +ConditionVariable src/util/condition_variable.h /^class ConditionVariable {$/;" c namespace:doris +ConditionalFunctions src/exprs/conditional_functions.h /^class ConditionalFunctions {$/;" c namespace:doris +Conditions src/olap/olap_cond.h /^ Conditions() {}$/;" f class:doris::Conditions +Conditions src/olap/olap_cond.h /^class Conditions {$/;" c namespace:doris +ConstructIntermediateTuple src/exec/new_partitioned_aggregation_node.cc /^Tuple* NewPartitionedAggregationNode::ConstructIntermediateTuple($/;" f class:doris::NewPartitionedAggregationNode +ConstructSingletonOutputTuple src/exec/new_partitioned_aggregation_node.cc /^Tuple* NewPartitionedAggregationNode::ConstructSingletonOutputTuple($/;" f class:doris::NewPartitionedAggregationNode +ConsumeFinishCallback src/runtime/routine_load/data_consumer_group.h /^ typedef std::function ConsumeFinishCallback;$/;" t class:doris::DataConsumerGroup +ConsumeStrayLeadingZeroes src/gutil/strings/numbers.cc /^void ConsumeStrayLeadingZeroes(string *const str) {$/;" f +ConsumerThread test/util/internal_queue_test.cpp /^void ConsumerThread($/;" f namespace:doris +Contains src/olap/skiplist.h /^bool SkipList::Contains(const Key& key) const {$/;" f class:doris::SkipList +ContainsKey src/gutil/map-util.h /^bool ContainsKey(const Collection& collection, const Key& key) {$/;" f +ContainsKeyValuePair src/gutil/map-util.h /^bool ContainsKeyValuePair(const Collection& collection,$/;" f +ContentionAction src/http/action/pprof_actions.cpp /^ ContentionAction() {}$/;" f class:doris::ContentionAction +ContentionAction src/http/action/pprof_actions.cpp /^class ContentionAction : public HttpHandler {$/;" c namespace:doris file: +ContiguousRow src/olap/row.h /^ ContiguousRow(const Schema* schema, const void* row) : _schema(schema), _row((void*)row) { }$/;" f struct:doris::ContiguousRow +ContiguousRow src/olap/row.h /^ ContiguousRow(const Schema* schema, void* row) : _schema(schema), _row(row) { }$/;" f struct:doris::ContiguousRow +ContiguousRow src/olap/row.h /^struct ContiguousRow {$/;" s namespace:doris +ConvertHelper src/gutil/type_traits.h /^struct ConvertHelper {$/;" s namespace:base::internal +CopyGroupingValues src/exec/new_partitioned_aggregation_node.cc /^void NewPartitionedAggregationNode::CopyGroupingValues(Tuple* intermediate_tuple,$/;" f class:doris::NewPartitionedAggregationNode +CopyStringData src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::CopyStringData(const SlotDescriptor& slot_desc,$/;" f class:doris::NewPartitionedAggregationNode +CopyStrings src/runtime/buffered_tuple_stream3.cc /^bool BufferedTupleStream3::CopyStrings(const Tuple* tuple,$/;" f class:BufferedTupleStream3 +CopyToString src/gutil/strings/stringpiece.cc /^void StringPiece::CopyToString(string* target) const {$/;" f class:StringPiece +CoreDataAllocator src/util/core_local.h /^class CoreDataAllocator {$/;" c namespace:doris +CoreDataAllocatorFactory src/util/core_local.h /^ CoreDataAllocatorFactory() { }$/;" f class:doris::CoreDataAllocatorFactory +CoreDataAllocatorFactory src/util/core_local.h /^class CoreDataAllocatorFactory {$/;" c namespace:doris +CoreDataAllocatorImpl src/util/core_local.cpp /^class CoreDataAllocatorImpl : public CoreDataAllocator {$/;" c namespace:doris file: +CoreLocalCounter src/util/metrics.h /^ CoreLocalCounter() :SimpleMetric(MetricType::COUNTER), _value() { }$/;" f class:doris::CoreLocalCounter +CoreLocalCounter src/util/metrics.h /^class CoreLocalCounter : public SimpleMetric {$/;" c namespace:doris +CoreLocalTest test/util/core_local_test.cpp /^ CoreLocalTest() {$/;" f class:doris::CoreLocalTest +CoreLocalTest test/util/core_local_test.cpp /^class CoreLocalTest : public ::testing::Test {$/;" c namespace:doris file: +CoreLocalValue src/util/core_local.h /^ CoreLocalValue(const T init_value = T()) {$/;" f class:doris::CoreLocalValue +CoreLocalValue src/util/core_local.h /^class CoreLocalValue {$/;" c namespace:doris +CoreLocalValueController src/util/core_local.h /^ CoreLocalValueController() {$/;" f class:doris::CoreLocalValueController +CoreLocalValueController src/util/core_local.h /^class CoreLocalValueController {$/;" c namespace:doris +Corruption src/common/status.h /^ static Status Corruption(const Slice& msg, int16_t precise_code = 1, const Slice& msg2 = Slice()) {$/;" f class:doris::Status +Count src/gutil/bits.cc /^int Bits::Count(const void *m, int num_bytes) {$/;" f class:Bits +Count2Update test/udf/uda_test.cpp /^void Count2Update(FunctionContext* context, const IntVal& input1, const IntVal& input2,$/;" f namespace:doris_udf +Count3Update test/udf/uda_test.cpp /^void Count3Update(FunctionContext* context, const IntVal& input1, const IntVal& input2,$/;" f namespace:doris_udf +Count4Update test/udf/uda_test.cpp /^void Count4Update(FunctionContext* context, const IntVal& input1, const IntVal& input2,$/;" f namespace:doris_udf +CountDownLatch src/util/countdown_latch.h /^ explicit CountDownLatch(int count)$/;" f class:doris::CountDownLatch +CountDownLatch src/util/countdown_latch.h /^class CountDownLatch {$/;" c namespace:doris +CountDownOnScopeExit src/util/countdown_latch.h /^ explicit CountDownOnScopeExit(CountDownLatch *latch) : latch_(latch) {}$/;" f class:doris::CountDownOnScopeExit +CountDownOnScopeExit src/util/countdown_latch.h /^class CountDownOnScopeExit {$/;" c namespace:doris +CountFinalize src/udf_samples/uda_sample.cpp /^BigIntVal CountFinalize(FunctionContext* context, const BigIntVal& val) {$/;" f namespace:doris_udf +CountFinalize test/udf/uda_test.cpp /^BigIntVal CountFinalize(FunctionContext* context, const BigIntVal& val) {$/;" f namespace:doris_udf +CountInit src/udf_samples/uda_sample.cpp /^void CountInit(FunctionContext* context, BigIntVal* val) {$/;" f namespace:doris_udf +CountInit test/udf/uda_test.cpp /^void CountInit(FunctionContext* context, BigIntVal* val) {$/;" f namespace:doris_udf +CountMerge src/udf_samples/uda_sample.cpp /^void CountMerge(FunctionContext* context, const BigIntVal& src, BigIntVal* dst) {$/;" f namespace:doris_udf +CountMerge test/udf/uda_test.cpp /^void CountMerge(FunctionContext* context, const BigIntVal& src, BigIntVal* dst) {$/;" f namespace:doris_udf +CountOnes src/gutil/bits.h /^ static int CountOnes(uint32 n) {$/;" f class:Bits +CountOnes64 src/gutil/bits.h /^ static inline int CountOnes64(uint64 n) {$/;" f class:Bits +CountOnes64withPopcount src/gutil/bits.h /^ static inline int CountOnes64withPopcount(uint64 n) {$/;" f class:Bits +CountOnesInByte src/gutil/bits.h /^inline int Bits::CountOnesInByte(unsigned char n) {$/;" f class:Bits +CountSubstituteArgs src/gutil/strings/substitute.cc /^static int CountSubstituteArgs(const SubstituteArg* const* args_array) {$/;" f namespace:strings +CountSubstring src/gutil/strings/util.cc /^int CountSubstring(StringPiece text, StringPiece substring) {$/;" f +CountUpdate src/udf_samples/uda_sample.cpp /^void CountUpdate(FunctionContext* context, const IntVal& input, BigIntVal* val) {$/;" f namespace:doris_udf +CountUpdate test/udf/uda_test.cpp /^void CountUpdate(FunctionContext* context, const IntVal& input, BigIntVal* val) {$/;" f namespace:doris_udf +Counter src/util/perf_counters.h /^ enum Counter {$/;" g class:doris::PerfCounters +Counter src/util/runtime_profile.h /^ Counter(TUnit::type type, int64_t value = 0) :$/;" f class:doris::RuntimeProfile::Counter +Counter src/util/runtime_profile.h /^ class Counter {$/;" c class:doris::RuntimeProfile +CounterCondVariable src/util/counter_cond_variable.hpp /^ explicit CounterCondVariable(int init = 0) : _count(init) {$/;" f class:doris::CounterCondVariable +CounterCondVariable src/util/counter_cond_variable.hpp /^class CounterCondVariable {$/;" c namespace:doris +CounterCondVariableTest test/util/counter_cond_variable_test.cpp /^ CounterCondVariableTest() { }$/;" f class:doris::CounterCondVariableTest +CounterCondVariableTest test/util/counter_cond_variable_test.cpp /^class CounterCondVariableTest : public testing::Test {$/;" c namespace:doris file: +CounterData src/util/perf_counters.h /^ struct CounterData {$/;" s class:doris::PerfCounters +CounterMap src/util/runtime_profile.h /^ typedef std::map CounterMap;$/;" t class:doris::RuntimeProfile +CppType src/olap/aggregate_func.h /^ typedef typename FieldTypeTraits::CppType CppType;$/;" t struct:doris::AggregateFuncTraits +CppType src/olap/aggregate_func.h /^ typedef typename FieldTypeTraits::CppType CppType;$/;" t struct:doris::AggregateFuncTraits +CppType src/olap/rowset/segment_v2/bitshuffle_page.h /^ typedef typename TypeTraits::CppType CppType;$/;" t class:doris::segment_v2::BitShufflePageDecoder +CppType src/olap/rowset/segment_v2/bitshuffle_page.h /^ typedef typename TypeTraits::CppType CppType;$/;" t class:doris::segment_v2::BitshufflePageBuilder +CppType src/olap/rowset/segment_v2/frame_of_reference_page.h /^ typedef typename TypeTraits::CppType CppType;$/;" t class:doris::segment_v2::FrameOfReferencePageBuilder +CppType src/olap/rowset/segment_v2/frame_of_reference_page.h /^ typedef typename TypeTraits::CppType CppType;$/;" t class:doris::segment_v2::FrameOfReferencePageDecoder +CppType src/olap/rowset/segment_v2/plain_page.h /^ typedef typename TypeTraits::CppType CppType;$/;" t class:doris::segment_v2::PlainPageBuilder +CppType src/olap/rowset/segment_v2/plain_page.h /^ typedef typename TypeTraits::CppType CppType;$/;" t class:doris::segment_v2::PlainPageDecoder +CppType src/olap/rowset/segment_v2/rle_page.h /^ typedef typename TypeTraits::CppType CppType;$/;" t class:doris::segment_v2::RlePageBuilder +CppType src/olap/rowset/segment_v2/rle_page.h /^ typedef typename TypeTraits::CppType CppType;$/;" t class:doris::segment_v2::RlePageDecoder +CppTypeTraits src/olap/types.h /^struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CppTypeTraits src/olap/types.h /^template<> struct CppTypeTraits {$/;" s namespace:doris +CpuInfo src/util/cpu_info.h /^class CpuInfo {$/;" c namespace:doris +CpuInfoBrand src/gutil/cpu.cc /^std::string* CpuInfoBrand() {$/;" f namespace:base::__anon16 +CpuMetrics src/util/system_metrics.cpp /^struct CpuMetrics {$/;" s namespace:doris file: +Create src/exec/new_partitioned_hash_table.cc /^NewPartitionedHashTable* NewPartitionedHashTable::Create(Suballocator* allocator, bool stores_duplicates,$/;" f class:NewPartitionedHashTable +Create src/exec/new_partitioned_hash_table.cc /^Status NewPartitionedHashTableCtx::Create(ObjectPool* pool, RuntimeState* state,$/;" f class:NewPartitionedHashTableCtx +Create src/exprs/agg_fn.cc /^Status AggFn::Create(const TExpr& texpr, const RowDescriptor& row_desc,$/;" f class:doris::AggFn +Create src/exprs/new_agg_fn_evaluator.cc /^Status NewAggFnEvaluator::Create(const AggFn& agg_fn, RuntimeState* state, ObjectPool* pool,$/;" f class:NewAggFnEvaluator +Create src/exprs/new_agg_fn_evaluator.cc /^Status NewAggFnEvaluator::Create(const vector& agg_fns, RuntimeState* state,$/;" f class:NewAggFnEvaluator +Create src/runtime/bufferpool/suballocator.cc /^Status Suballocation::Create(unique_ptr* new_suballocation) {$/;" f class:doris::Suballocation +CreateBlockOptions src/olap/fs/block_manager.h /^struct CreateBlockOptions {$/;" s namespace:doris::fs +CreateColumnReader test/olap/column_reader_test.cpp /^ void CreateColumnReader($/;" f class:doris::TestColumn +CreateColumnReader test/olap/column_reader_test.cpp /^ void CreateColumnReader(const TabletSchema& tablet_schema) {$/;" f class:doris::TestColumn +CreateColumnReader test/olap/schema_change_test.cpp /^ void CreateColumnReader($/;" f class:doris::TestColumn +CreateColumnReader test/olap/schema_change_test.cpp /^ void CreateColumnReader(const TabletSchema& tablet_schema) {$/;" f class:doris::TestColumn +CreateColumnWriter test/olap/column_reader_test.cpp /^ void CreateColumnWriter(const TabletSchema& tablet_schema) {$/;" f class:doris::TestColumn +CreateColumnWriter test/olap/schema_change_test.cpp /^ void CreateColumnWriter(const TabletSchema& tablet_schema) {$/;" f class:doris::TestColumn +CreateDestroyMulti test/runtime/buffered_block_mgr2_test.cpp /^ void CreateDestroyMulti() {$/;" f class:doris::BufferedBlockMgrTest +CreateDestroyThread test/runtime/buffered_block_mgr2_test.cpp /^ void CreateDestroyThread(int index, RuntimeState* state) {$/;" f class:doris::BufferedBlockMgrTest +CreateFlags src/util/thread.h /^ enum CreateFlags {$/;" g class:doris::Thread +CreateHashPartitions src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::CreateHashPartitions($/;" f class:doris::NewPartitionedAggregationNode +CreateHashTable test/exec/partitioned_hash_table_test.cpp /^ bool CreateHashTable(bool quadratic, int64_t initial_num_buckets,$/;" f class:doris::PartitionedHashTableTest +CreateIntBatch test/runtime/buffered_tuple_stream2_test.cpp /^ virtual RowBatch* CreateIntBatch(int offset, int num_rows, bool gen_null) {$/;" f class:doris::SimpleTupleStreamTest +CreateKuduClient src/exec/kudu_util.cpp /^Status CreateKuduClient(const std::vector& master_addrs,$/;" f namespace:doris +CreateMgr test/runtime/buffered_block_mgr2_test.cpp /^ BufferedBlockMgr2* CreateMgr(int64_t query_id, int max_buffers, int block_size,$/;" f class:doris::BufferedBlockMgrTest +CreateMgrAndClient test/runtime/buffered_block_mgr2_test.cpp /^ BufferedBlockMgr2* CreateMgrAndClient(int64_t query_id, int max_buffers, int block_size,$/;" f class:doris::BufferedBlockMgrTest +CreateMgrsAndClients test/runtime/buffered_block_mgr2_test.cpp /^ void CreateMgrsAndClients(int64_t start_query_id, int num_mgrs, int buffers_per_mgr,$/;" f class:doris::BufferedBlockMgrTest +CreatePage src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::CreatePage($/;" f class:doris::BufferPool +CreatePinnedPage src/runtime/bufferpool/buffer_pool.cc /^BufferPool::Page* BufferPool::Client::CreatePinnedPage(BufferHandle&& buffer) {$/;" f class:doris::BufferPool::Client +CreateQueryMemTracker src/runtime/mem_tracker.cpp /^MemTracker* MemTracker::CreateQueryMemTracker(const TUniqueId& id,$/;" f class:doris::MemTracker +CreateReader test/olap/bit_field_test.cpp /^ void CreateReader() {$/;" f class:doris::TestBitField +CreateReader test/olap/run_length_byte_test.cpp /^ void CreateReader() {$/;" f class:doris::TestRunLengthByte +CreateReader test/olap/run_length_integer_test.cpp /^ void CreateReader() {$/;" f class:doris::TestRunLengthSignInteger +CreateReader test/olap/run_length_integer_test.cpp /^ void CreateReader() {$/;" f class:doris::TestRunLengthUnsignInteger +CreateRowBatch test/runtime/sorter_test.cpp /^RowBatch* SorterTest::CreateRowBatch(int num_rows) {$/;" f class:doris::SorterTest +CreateStringBatch test/runtime/buffered_tuple_stream2_test.cpp /^ virtual RowBatch* CreateStringBatch(int offset, int num_rows, bool gen_null) {$/;" f class:doris::SimpleTupleStreamTest +CreateTempFile test/runtime/disk_io_mgr_test.cpp /^ int CreateTempFile(const char* filename, int file_size) {$/;" f class:doris::DiskIoMgrTest +CreateTempFile test/runtime/disk_io_mgr_test.cpp /^ void CreateTempFile(const char* filename, const char* data) {$/;" f class:doris::DiskIoMgrTest +CreateTupleRow test/exec/partitioned_hash_table_test.cpp /^ TupleRow* CreateTupleRow(int32_t val) {$/;" f class:doris::PartitionedHashTableTest +CrossJoinNode src/exec/cross_join_node.cpp /^CrossJoinNode::CrossJoinNode($/;" f class:doris::CrossJoinNode +CrossJoinNode src/exec/cross_join_node.h /^class CrossJoinNode : public BlockingJoinNode {$/;" c namespace:doris +CsvScanNode src/exec/csv_scan_node.cpp /^CsvScanNode::CsvScanNode($/;" f class:doris::CsvScanNode +CsvScanNode src/exec/csv_scan_node.h /^class CsvScanNode : public ScanNode {$/;" c namespace:doris +CsvScanNodeBenchTest test/exec/csv_scan_bench_test.cpp /^ CsvScanNodeBenchTest(){}$/;" f class:doris::CsvScanNodeBenchTest +CsvScanNodeBenchTest test/exec/csv_scan_bench_test.cpp /^class CsvScanNodeBenchTest : public testing::Test {$/;" c namespace:doris file: +CsvScanNodeTest test/exec/csv_scan_node_test.cpp /^ CsvScanNodeTest(){}$/;" f class:doris::CsvScanNodeTest +CsvScanNodeTest test/exec/csv_scan_node_test.cpp /^class CsvScanNodeTest : public testing::Test {$/;" c namespace:doris file: +CsvScanner src/exec/csv_scanner.cpp /^ CsvScanner::CsvScanner(const std::vector& csv_file_paths) :$/;" f class:doris::CsvScanner +CsvScanner src/exec/csv_scanner.h /^class CsvScanner {$/;" c namespace:doris +CsvScannerTest test/exec/csv_scanner_test.cpp /^ CsvScannerTest() {$/;" f class:doris::CsvScannerTest +CsvScannerTest test/exec/csv_scanner_test.cpp /^class CsvScannerTest : public testing::Test {$/;" c namespace:doris file: +CumulativeCompaction src/olap/cumulative_compaction.cpp /^CumulativeCompaction::CumulativeCompaction(TabletSharedPtr tablet)$/;" f class:doris::CumulativeCompaction +CumulativeCompaction src/olap/cumulative_compaction.h /^class CumulativeCompaction : public Compaction {$/;" c namespace:doris +CurExprValuesHash src/exec/new_partitioned_hash_table.h /^ uint32_t ALWAYS_INLINE CurExprValuesHash() const { return *cur_expr_values_hash_; }$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +CurIdx src/exec/new_partitioned_hash_table.h /^ int ALWAYS_INLINE CurIdx() const {$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +CurrentMemSize src/exec/new_partitioned_hash_table.inline.h /^inline int64_t NewPartitionedHashTable::CurrentMemSize() const {$/;" f class:doris::NewPartitionedHashTable +CurrentThread src/gutil/threading/thread_collision_warner.cc /^static subtle::Atomic64 CurrentThread() {$/;" f namespace:base +CycleClock src/gutil/walltime.h /^class CycleClock {$/;" c +CyclesPerSecond src/gutil/sysinfo.cc /^double CyclesPerSecond(void) {$/;" f namespace:base +DATA_PAGE_SIZE src/exec/new_partitioned_hash_table.cc /^constexpr int64_t NewPartitionedHashTable::DATA_PAGE_SIZE;$/;" m class:NewPartitionedHashTable file: +DATA_PAGE_SIZE src/exec/new_partitioned_hash_table.h /^ static constexpr int64_t DATA_PAGE_SIZE = 64L * 1024;$/;" m class:doris::NewPartitionedHashTable +DATA_PREFIX src/olap/olap_define.h /^static const std::string DATA_PREFIX = "\/data";$/;" m namespace:doris +DATE src/http/http_headers.cpp /^const char* HttpHeaders::DATE = "Date";$/;" m class:doris::HttpHeaders file: +DATE src/http/http_headers.h /^ static const char* DATE;$/;" m class:doris::HttpHeaders +DATETIME_BINARY_FOR_NULL_PRED_FN src/exprs/binary_predicate.cpp 449;" d file: +DATETIME_BINARY_FOR_NULL_PRED_FNS src/exprs/binary_predicate.cpp 461;" d file: +DATETIME_BINARY_PRED_FN src/exprs/binary_predicate.cpp 337;" d file: +DATETIME_BINARY_PRED_FNS src/exprs/binary_predicate.cpp 350;" d file: +DATETIME_PACKED_TIME_BYTE_SIZE src/exprs/aggregate_functions.cpp /^ const int DATETIME_PACKED_TIME_BYTE_SIZE = 8;$/;" m class:doris::MultiDistinctCountDateState file: +DATETIME_PACKED_TIME_BYTE_SIZE src/exprs/bitmap_function.cpp /^const int DATETIME_PACKED_TIME_BYTE_SIZE = 8;$/;" m namespace:doris::detail file: +DATETIME_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t DATETIME_SIZE = 16;$/;" m class:doris::AggFnEvaluator +DATETIME_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t DATETIME_SIZE = 16; $/;" m class:doris::NewAggFnEvaluator +DATETIME_TYPE_BYTE_SIZE src/exprs/aggregate_functions.cpp /^ const int DATETIME_TYPE_BYTE_SIZE = 4;$/;" m class:doris::MultiDistinctCountDateState file: +DATETIME_TYPE_BYTE_SIZE src/exprs/bitmap_function.cpp /^const int DATETIME_TYPE_BYTE_SIZE = 4;$/;" m namespace:doris::detail file: +DATE_MAX_DAYNR src/runtime/datetime_value.h /^const int DATE_MAX_DAYNR = 3652424;$/;" m namespace:doris +DATE_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t DATE_SIZE = sizeof(boost::gregorian::date);$/;" m class:doris::AggFnEvaluator +DATE_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t DATE_SIZE = sizeof(boost::gregorian::date);$/;" m class:doris::NewAggFnEvaluator +DAY src/runtime/datetime_value.h /^ DAY,$/;" e enum:doris::TimeUnit +DAY_HOUR src/runtime/datetime_value.h /^ DAY_HOUR,$/;" e enum:doris::TimeUnit +DAY_MICROSECOND src/runtime/datetime_value.h /^ DAY_MICROSECOND,$/;" e enum:doris::TimeUnit +DAY_MINUTE src/runtime/datetime_value.h /^ DAY_MINUTE,$/;" e enum:doris::TimeUnit +DAY_SECOND src/runtime/datetime_value.h /^ DAY_SECOND,$/;" e enum:doris::TimeUnit +DB_KEY src/http/action/mini_load.cpp /^const std::string DB_KEY = "db";$/;" m namespace:doris file: +DB_PARAMETER src/http/download_action.cpp /^const std::string DB_PARAMETER = "db";$/;" m namespace:doris file: +DCHECK src/common/logging.h 26;" d +DCHECK_CONSISTENCY src/runtime/bufferpool/buffer_pool_internal.h 33;" d +DCHECK_CONSISTENCY src/runtime/bufferpool/buffer_pool_internal.h 35;" d +DCHECK_EQ src/common/logging.h 27;" d +DCHECK_GE src/common/logging.h 31;" d +DCHECK_GT src/common/logging.h 29;" d +DCHECK_LE src/common/logging.h 32;" d +DCHECK_LT src/common/logging.h 30;" d +DCHECK_NE src/common/logging.h 28;" d +DCHECK_NO_OVERLAP src/gutil/strings/strcat.cc 179;" d file: +DCheckAsserter src/gutil/threading/thread_collision_warner.h /^struct BASE_EXPORT DCheckAsserter : public AsserterBase {$/;" s namespace:base +DCheckConsistency src/runtime/bufferpool/buffer_pool_internal.h /^ void DCheckConsistency() {$/;" f class:doris::BufferPool::Client +DCheckConsistency src/runtime/bufferpool/buffer_pool_internal.h /^ void DCheckConsistency() {$/;" f class:doris::BufferPool::PageList +DCheckHoldsLock src/runtime/bufferpool/buffer_pool_internal.h /^ void DCheckHoldsLock(const boost::unique_lock& client_lock) {$/;" f class:doris::BufferPool::Client +DEBUG_COUNTER_SET src/util/debug_counters.h 53;" d +DEBUG_COUNTER_SET src/util/debug_counters.h 64;" d +DEBUG_COUNTER_UPDATE src/util/debug_counters.h 49;" d +DEBUG_COUNTER_UPDATE src/util/debug_counters.h 63;" d +DEBUG_MODE src/service/brpc.h 45;" d +DEBUG_SCOPED_TIMER src/util/debug_counters.h 45;" d +DEBUG_SCOPED_TIMER src/util/debug_counters.h 62;" d +DEC build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 496;" d file: +DEC build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 481;" d file: +DECIMALV2_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t DECIMALV2_SIZE = sizeof(DecimalV2Value);$/;" m class:doris::AggFnEvaluator +DECIMALV2_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t DECIMALV2_SIZE = sizeof(DecimalV2Value);$/;" m class:doris::NewAggFnEvaluator +DECIMAL_ARITHMETIC_OP src/exprs/decimal_operators.cpp 151;" d file: +DECIMAL_ARITHMETIC_OP src/exprs/decimalv2_operators.cpp 160;" d file: +DECIMAL_ARITHMETIC_OPS src/exprs/decimal_operators.cpp 163;" d file: +DECIMAL_ARITHMETIC_OPS src/exprs/decimalv2_operators.cpp 172;" d file: +DECIMAL_BINARY_PREDICATE_NONNUMERIC_FN src/exprs/decimal_operators.cpp 172;" d file: +DECIMAL_BINARY_PREDICATE_NONNUMERIC_FN src/exprs/decimalv2_operators.cpp 181;" d file: +DECIMAL_BUFFER_BYTE_SIZE src/exprs/aggregate_functions.cpp /^ const int DECIMAL_BUFFER_BYTE_SIZE = 36;$/;" m class:doris::MultiDistinctDecimalState file: +DECIMAL_BUFF_LENGTH src/runtime/decimal_value.h /^static const int32_t DECIMAL_BUFF_LENGTH = 9;$/;" m namespace:doris +DECIMAL_BYTE_SIZE src/exprs/aggregate_functions.cpp /^ const int DECIMAL_BYTE_SIZE = 16;$/;" m class:doris::MultiDistinctDecimalV2State file: +DECIMAL_BYTE_SIZE src/exprs/bitmap_function.cpp /^const int DECIMAL_BYTE_SIZE = 16;$/;" m namespace:doris::detail file: +DECIMAL_FRAC_BYTE_SIZE src/exprs/aggregate_functions.cpp /^ const int DECIMAL_FRAC_BYTE_SIZE = 1;$/;" m class:doris::MultiDistinctDecimalState file: +DECIMAL_INT_LEN_BYTE_SIZE src/exprs/aggregate_functions.cpp /^ const int DECIMAL_INT_LEN_BYTE_SIZE = 1;$/;" m class:doris::MultiDistinctDecimalState file: +DECIMAL_MAX_POSSIBLE_PRECISION src/runtime/decimal_value.h /^static const int32_t DECIMAL_MAX_POSSIBLE_PRECISION = DECIMAL_BUFF_LENGTH * 9;$/;" m namespace:doris +DECIMAL_MAX_PRECISION src/runtime/decimal_value.h /^static const int32_t DECIMAL_MAX_PRECISION = DECIMAL_MAX_POSSIBLE_PRECISION - 8 * 2;$/;" m namespace:doris +DECIMAL_MAX_SCALE src/runtime/decimal_value.h /^static const int32_t DECIMAL_MAX_SCALE = 30;$/;" m namespace:doris +DECIMAL_MAX_STR_LENGTH src/runtime/decimal_value.h /^static const int32_t DECIMAL_MAX_STR_LENGTH = (DECIMAL_MAX_POSSIBLE_PRECISION + 2);$/;" m namespace:doris +DECIMAL_SIGN_BYTE_SIZE src/exprs/aggregate_functions.cpp /^ const int DECIMAL_SIGN_BYTE_SIZE = 1;$/;" m class:doris::MultiDistinctDecimalState file: +DECIMAL_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t DECIMAL_SIZE = sizeof(DecimalValue);$/;" m class:doris::AggFnEvaluator +DECIMAL_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t DECIMAL_SIZE = sizeof(DecimalValue);$/;" m class:doris::NewAggFnEvaluator +DECLARE_FIELD src/common/configbase.h 60;" d +DECLARE_NESTED_POD src/gutil/type_traits.h 358;" d +DECLARE_POD src/gutil/type_traits.h 357;" d +DECLARE_SINGLETON src/olap/olap_define.h 428;" d +DEFAULT_ALIGNMENT src/runtime/mem_pool.cpp /^const int MemPool::DEFAULT_ALIGNMENT;$/;" m class:doris::MemPool file: +DEFAULT_ALIGNMENT src/runtime/mem_pool.h /^ static const int DEFAULT_ALIGNMENT = 8;$/;" m class:doris::MemPool +DEFAULT_BATCH_SIZE src/runtime/runtime_state.h /^ static const int DEFAULT_BATCH_SIZE = 2048;$/;" m class:doris::RuntimeState +DEFAULT_COLUMN_FAMILY src/olap/olap_define.h /^static const std::string DEFAULT_COLUMN_FAMILY = "default";$/;" m namespace:doris +DEFAULT_COLUMN_FAMILY_INDEX src/olap/olap_define.h /^ DEFAULT_COLUMN_FAMILY_INDEX = 0,$/;" e enum:doris::ColumnFamilyIndex +DEFAULT_DOWNLOAD_TIMEOUT src/olap/task/engine_batch_load_task.h /^const uint32_t DEFAULT_DOWNLOAD_TIMEOUT = 3600;$/;" m namespace:doris +DEFAULT_FILE_NAME_SIZE src/util/zip_util.cpp 29;" d file: +DEFAULT_MULTI_DISTINCT_COUNT_STRING_BUFFER_SIZE src/exprs/new_agg_fn_evaluator.cc /^const int DEFAULT_MULTI_DISTINCT_COUNT_STRING_BUFFER_SIZE = 1024;$/;" v +DEFAULT_PAGE_SIZE src/olap/rowset/segment_v2/options.h /^static const size_t DEFAULT_PAGE_SIZE = 1024 * 1024; \/\/ default size: 1M$/;" m namespace:doris::segment_v2 +DEFAULT_QUEUE_CAPACITY src/runtime/disk_io_mgr.cc /^const int DiskIoMgr::DEFAULT_QUEUE_CAPACITY = 2;$/;" m class:doris::DiskIoMgr file: +DEFAULT_QUEUE_CAPACITY src/runtime/disk_io_mgr.h /^ static const int DEFAULT_QUEUE_CAPACITY;$/;" m class:doris::DiskIoMgr +DEFAULT_SEED src/olap/bloom_filter.hpp /^static const uint64_t DEFAULT_SEED = 104729;$/;" m namespace:doris +DEFAULT_SEED src/olap/rowset/segment_v2/bloom_filter.h /^ static const uint32_t DEFAULT_SEED = 1575457558;$/;" m class:doris::segment_v2::BloomFilter +DEFAULT_STRING_LENGTH src/olap/wrapper_field.cpp /^const size_t DEFAULT_STRING_LENGTH = 50;$/;" m namespace:doris file: +DEFAULT_UNZIP_BUFFER src/util/zip_util.cpp /^const static ZPOS64_T DEFAULT_UNZIP_BUFFER = 1048576; \/\/ 1MB$/;" m namespace:doris file: +DEFAULT_WORKER_THREADS src/util/thrift_server.h /^ static const int DEFAULT_WORKER_THREADS = 2;$/;" m class:doris::ThriftServer +DEFINE_FIELD src/common/configbase.h 56;" d +DEFINE_SPLIT_ONE_NUMBER_TOKEN src/gutil/strings/split.cc 639;" d file: +DEFLATE src/exec/decompressor.h /^ DEFLATE,$/;" e enum:doris::CompressType +DELETE src/agent/task_worker_pool.h /^ DELETE,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +DELETE src/http/http_method.h /^ DELETE,$/;" e enum:doris::HttpMethod +DELETE src/olap/delta_writer.h /^ DELETE = 3$/;" e enum:doris::WriteType +DELETE_META src/olap/tablet_sync_service.h /^ DELETE_META$/;" e enum:doris::MetaOpType +DELTA src/olap/rowset/run_length_integer_writer.h /^ DELTA = 3,$/;" e enum:doris::RunLengthIntegerWriter::EncodingType +DEL_NOT_SATISFIED src/olap/olap_common.h /^ DEL_NOT_SATISFIED = 1, \/\/not satisfy delete condition$/;" e enum:doris::DelCondSatisfied +DEL_PARTIAL_SATISFIED src/olap/olap_common.h /^ DEL_PARTIAL_SATISFIED = 2, \/\/partially satisfy delete condition$/;" e enum:doris::DelCondSatisfied +DEL_SATISFIED src/olap/olap_common.h /^ DEL_SATISFIED = 0, \/\/satisfy delete condition$/;" e enum:doris::DelCondSatisfied +DEST_NODE_ID test/runtime/data_stream_test.cpp /^ static const PlanNodeId DEST_NODE_ID = 1;$/;" m class:doris::DataStreamTest file: +DETECT_CODEC src/exec/decompressor.h /^ const static int DETECT_CODEC = 32; \/\/ Determine if this is libz or gzip from header. $/;" m class:doris::GzipDecompressor +DFAKE_MUTEX src/gutil/threading/thread_collision_warner.h 108;" d +DFAKE_MUTEX src/gutil/threading/thread_collision_warner.h 124;" d +DFAKE_SCOPED_LOCK src/gutil/threading/thread_collision_warner.h 112;" d +DFAKE_SCOPED_LOCK src/gutil/threading/thread_collision_warner.h 125;" d +DFAKE_SCOPED_LOCK_THREAD_LOCKED src/gutil/threading/thread_collision_warner.h 119;" d +DFAKE_SCOPED_LOCK_THREAD_LOCKED src/gutil/threading/thread_collision_warner.h 127;" d +DFAKE_SCOPED_RECURSIVE_LOCK src/gutil/threading/thread_collision_warner.h 116;" d +DFAKE_SCOPED_RECURSIVE_LOCK src/gutil/threading/thread_collision_warner.h 126;" d +DFATAL_LEVEL src/gutil/logging-inl.h 30;" d +DFATAL_LEVEL src/gutil/logging-inl.h 32;" d +DIG_BASE src/runtime/decimal_value.h /^static const int32_t DIG_BASE = 1000000000; \/\/ 10^9$/;" m namespace:doris +DIG_MASK src/runtime/decimal_value.h /^static const int32_t DIG_MASK = 100000000; \/\/ 10^8$/;" m namespace:doris +DIG_MAX src/runtime/decimal_value.h /^static const int32_t DIG_MAX = DIG_BASE - 1;$/;" m namespace:doris +DIG_PER_DEC1 src/runtime/decimal_value.h /^static const int32_t DIG_PER_DEC1 = 9;$/;" m namespace:doris +DIRECT src/olap/rowset/run_length_integer_writer.h /^ DIRECT = 1,$/;" e enum:doris::RunLengthIntegerWriter::EncodingType +DIRECT_IO_ALIGNMENT src/olap/data_dir.h /^ static const size_t DIRECT_IO_ALIGNMENT = 512;$/;" m class:doris::DataDir +DIRTY src/olap/fs/block_manager.h /^ DIRTY,$/;" e enum:doris::fs::WritableBlock::State +DISALLOW_COPY_AND_ASSIGN src/gutil/macros.h 100;" d +DISALLOW_COPY_AND_ASSIGN src/olap/olap_define.h 419;" d +DISALLOW_EVIL_CONSTRUCTORS src/gutil/macros.h 107;" d +DISALLOW_IMPLICIT_CONSTRUCTORS src/gutil/macros.h 115;" d +DISALLOW_IMPLICIT_CONSTRUCTORS src/service/brpc.h 28;" d +DIV src/exprs/arithmetic_expr.h /^ DIV,$/;" e enum:doris::ArithmeticExpr::BinaryOpType +DMGL_ANSI src/util/bfd_parser.cpp 234;" d file: +DMGL_PARAMS src/util/bfd_parser.cpp 233;" d file: +DONE test/olap/skiplist_test.cpp /^ DONE$/;" e enum:doris::TestState::ReaderState file: +DORIS_BE_EXEC_QUERY_STATISTICS_H src/runtime/query_statistics.h 19;" d +DORIS_BE_EXPRS_BASE64_H src/exprs/base64.h 19;" d +DORIS_BE_EXPRS_TIMEZONE_DB_H src/exprs/timezone_db.h 19;" d +DORIS_BE_RUNTIME_BUFFERED_TUPLE_STREAM_H src/runtime/buffered_tuple_stream3.h 19;" d +DORIS_BE_RUNTIME_BUFFERED_TUPLE_STREAM_INLINE_H src/runtime/buffered_tuple_stream3.inline.h 19;" d +DORIS_BE_RUNTIME_BUFFERPOOL_FREE_LIST_H src/runtime/bufferpool/free_list.h 19;" d +DORIS_BE_RUNTIME_BUFFERPOOL_RESERVATION_UTIL_H_ src/runtime/bufferpool/reservation_util.h 19;" d +DORIS_BE_RUNTIME_BUFFER_ALLOCATOR_H src/runtime/bufferpool/buffer_allocator.h 19;" d +DORIS_BE_RUNTIME_BUFFER_CONTROL_BLOCK_H src/runtime/buffer_control_block.h 19;" d +DORIS_BE_RUNTIME_BUFFER_POOL_COUNTERS_H src/runtime/bufferpool/buffer_pool_counters.h 19;" d +DORIS_BE_RUNTIME_BUFFER_POOL_H src/runtime/bufferpool/buffer_pool.h 19;" d +DORIS_BE_RUNTIME_BUFFER_POOL_INTERNAL_H src/runtime/bufferpool/buffer_pool_internal.h 19;" d +DORIS_BE_RUNTIME_CLIENT_CACHE_H src/runtime/client_cache.h 19;" d +DORIS_BE_RUNTIME_DATA_SPLITER_H src/runtime/data_spliter.h 19;" d +DORIS_BE_RUNTIME_DATA_STREAM_SENDER_H src/runtime/data_stream_sender.h 19;" d +DORIS_BE_RUNTIME_DATETIME_VALUE_H src/runtime/datetime_value.h 19;" d +DORIS_BE_RUNTIME_DESCRIPTORS_H src/runtime/descriptors.h 19;" d +DORIS_BE_RUNTIME_DPP_SINK_H src/runtime/dpp_sink.h 19;" d +DORIS_BE_RUNTIME_DPP_SINK_INTERNAL_H src/runtime/dpp_sink_internal.h 19;" d +DORIS_BE_RUNTIME_DPP_WRITER_H src/runtime/dpp_writer.h 19;" d +DORIS_BE_RUNTIME_ETL_JOB_MGR_H src/runtime/etl_job_mgr.h 19;" d +DORIS_BE_RUNTIME_EXEC_ENV_H src/runtime/exec_env.h 19;" d +DORIS_BE_RUNTIME_FRAGMENT_MGR_H src/runtime/fragment_mgr.h 19;" d +DORIS_BE_RUNTIME_INITIAL_RESERVATIONS_H src/runtime/initial_reservations.h 19;" d +DORIS_BE_RUNTIME_LARGE_INT_VALUE_H src/runtime/large_int_value.h 19;" d +DORIS_BE_RUNTIME_MEM_POOL_H src/runtime/mem_pool.h 19;" d +DORIS_BE_RUNTIME_MULTI_PRECISION_H src/runtime/multi_precision.h 19;" d +DORIS_BE_RUNTIME_MYSQL_TABLE_SINK_H src/runtime/mysql_table_sink.h 19;" d +DORIS_BE_RUNTIME_MYSQL_TABLE_WRITER_H src/runtime/mysql_table_writer.h 19;" d +DORIS_BE_RUNTIME_PLAN_FRAGMENT_EXECUTOR_H src/runtime/plan_fragment_executor.h 19;" d +DORIS_BE_RUNTIME_PRIMITIVE_TYPE_H src/runtime/primitive_type.h 19;" d +DORIS_BE_RUNTIME_QSORTER_H src/runtime/qsorter.h 19;" d +DORIS_BE_RUNTIME_RAW_VALUE_H src/runtime/raw_value.h 19;" d +DORIS_BE_RUNTIME_RESERVATION_TRACKER_COUNTERS_H src/runtime/bufferpool/reservation_tracker_counters.h 19;" d +DORIS_BE_RUNTIME_RESERVATION_TRACKER_H src/runtime/bufferpool/reservation_tracker.h 19;" d +DORIS_BE_RUNTIME_RESULT_BUFFER_MGR_H src/runtime/result_buffer_mgr.h 19;" d +DORIS_BE_RUNTIME_RESULT_SINK_H src/runtime/result_sink.h 19;" d +DORIS_BE_RUNTIME_RESULT_WRITER_H src/runtime/result_writer.h 19;" d +DORIS_BE_RUNTIME_ROW_BATCH_H src/runtime/row_batch.h 19;" d +DORIS_BE_RUNTIME_SORTER_H src/runtime/sorter.h 19;" d +DORIS_BE_RUNTIME_STRING_VALUE_H src/runtime/string_value.h 19;" d +DORIS_BE_RUNTIME_THREAD_RESOURCE_MGR_H src/runtime/thread_resource_mgr.h 19;" d +DORIS_BE_RUNTIME_TUPLE_H src/runtime/tuple.h 19;" d +DORIS_BE_RUNTIME_TUPLE_ROW_H src/runtime/tuple_row.h 19;" d +DORIS_BE_RUNTIME_TYPES_H src/runtime/types.h 19;" d +DORIS_BE_SERVICE_BACKEND_OPTIONS_H src/service/backend_options.h 19;" d +DORIS_BE_SERVICE_BACKEND_SERVICE_H src/service/backend_service.h 19;" d +DORIS_BE_SRC_AGENT_AGENT_SERVER_H src/agent/agent_server.h 19;" d +DORIS_BE_SRC_AGENT_CGROUPS_MGR_H src/agent/cgroups_mgr.h 19;" d +DORIS_BE_SRC_AGENT_HEARTBEAT_SERVER_H src/agent/heartbeat_server.h 19;" d +DORIS_BE_SRC_AGENT_MOCK_MOCK_PUSHER_H test/agent/mock_pusher.h 19;" d +DORIS_BE_SRC_AGENT_MOCK_MOCK_UTILS_H test/agent/mock_utils.h 19;" d +DORIS_BE_SRC_AGENT_STATUS_H src/agent/status.h 19;" d +DORIS_BE_SRC_AGENT_TOPIC_LISTENER_H src/agent/topic_listener.h 19;" d +DORIS_BE_SRC_AGENT_TOPIC_SUBSCRIBER_H src/agent/topic_subscriber.h 19;" d +DORIS_BE_SRC_AGENT_USER_RESOURCE_LISTENER_H src/agent/user_resource_listener.h 19;" d +DORIS_BE_SRC_AGENT_UTILS_H src/agent/utils.h 19;" d +DORIS_BE_SRC_COMMON_ATOMIC_H src/common/atomic.h 19;" d +DORIS_BE_SRC_COMMON_COMMON_COMPILER_UTIL_H src/common/compiler_util.h 19;" d +DORIS_BE_SRC_COMMON_COMMON_DAEMON_H src/common/daemon.h 19;" d +DORIS_BE_SRC_COMMON_COMMON_GLOBAL_TYPES_H src/common/global_types.h 19;" d +DORIS_BE_SRC_COMMON_COMMON_HDFS_H src/common/hdfs.h 19;" d +DORIS_BE_SRC_COMMON_COMMON_OBJECT_POOL_H src/common/object_pool.h 19;" d +DORIS_BE_SRC_COMMON_COMMON_RESOURCE_TLS_H src/common/resource_tls.h 19;" d +DORIS_BE_SRC_COMMON_CONFIGBASE_H src/common/configbase.h 19;" d +DORIS_BE_SRC_COMMON_CONFIG_H src/common/config.h 19;" d +DORIS_BE_SRC_COMMON_UITL_BITMAP_H src/util/bitmap.h 19;" d +DORIS_BE_SRC_COMMON_UITL_BIT_UTIL_H src/util/bit_util.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_BATCH_PROCESS_THREAD_POOL_HPP src/util/batch_process_thread_pool.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_BLOCKING_PRIORITY_QUEUE_HPP src/util/blocking_priority_queue.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_BLOCKING_QUEUE_HPP src/util/blocking_queue.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_BUFFER_BUILDER_HPP src/util/buffer_builder.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_CIDR_H src/util/cidr.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_DEBUG_COUNTERS_H src/util/debug_counters.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_DEBUG_UTIL_H src/util/debug_util.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_DEFAULT_PATH_HANDLERS_H src/http/default_path_handlers.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_DISK_INFO_H src/util/disk_info.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_DORIS_METRICS_H src/util/doris_metrics.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_DYNAMIC_UTIL_H src/util/dynamic_util.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_HASH_UTIL_HPP src/util/hash_util.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_HTTP_CHANNEL_H src/http/http_channel.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_HTTP_HANDLER_H src/http/http_handler.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_HTTP_HEADERS_H src/http/http_headers.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_HTTP_METHOD_H src/http/http_method.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_HTTP_PARSER_H src/http/http_parser.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_HTTP_REQUEST_H src/http/http_request.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_HTTP_RESPONSE_H src/http/http_response.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_HTTP_STATUS_H src/http/http_status.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_LOGGING_H src/util/logging.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_MEM_INFO_H src/util/mem_info.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_MEM_UTIL_HPP src/util/mem_util.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_MINI_LOAD_H src/http/action/mini_load.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_MONITOR_ACTION_H src/http/monitor_action.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_NETWORK_UTIL_H src/util/network_util.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_PARSE_UTIL_H src/util/parse_util.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_PATH_BUILDER_H src/util/path_builder.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_PERF_COUNTERS_H src/util/perf_counters.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_PRIORITY_THREAD_POOL_HPP src/util/priority_thread_pool.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_PROGRESS_UPDATER_H src/util/progress_updater.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_RANDOM_H src/util/random.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_RUNTIME_PROFILE_H src/util/runtime_profile.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_SSE_UTIL_H src/util/sse_util.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_STAT_UTIL_HPP src/util/stat_util.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_STOPWATCH_HPP src/util/stopwatch.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_STRING_PARSER_H src/util/string_parser.hpp 19;" d +DORIS_BE_SRC_COMMON_UTIL_SYMBOLS_UTIL_H src/util/symbols_util.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_THRIFT_CLIENT_H src/util/thrift_client.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_THRIFT_SERVER_H src/util/thrift_server.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_THRIFT_UTIL_H src/util/thrift_util.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_URL_CODING_H src/util/url_coding.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_URL_PARSER_H src/util/url_parser.h 19;" d +DORIS_BE_SRC_COMMON_UTIL_WEB_PAGE_HANDLER_H src/http/web_page_handler.h 19;" d +DORIS_BE_SRC_COMMON__UTIL_CONTAINER_UTIL_HPP src/util/container_util.hpp 19;" d +DORIS_BE_SRC_DELTA_WRITER_H src/olap/delta_writer.h 19;" d +DORIS_BE_SRC_EXEC_BROKER_WRITER_H src/exec/broker_writer.h 19;" d +DORIS_BE_SRC_EXEC_FILE_WRITER_H src/exec/file_writer.h 19;" d +DORIS_BE_SRC_EXEC_LOCAL_FILE_WRITER_H src/exec/local_file_writer.h 19;" d +DORIS_BE_SRC_EXEC_NEW_PARTITIONED_AGGREGATION_NODE_H src/exec/new_partitioned_aggregation_node.h 19;" d +DORIS_BE_SRC_EXEC_NEW_PARTITIONED_HASH_TABLE_H src/exec/new_partitioned_hash_table.h 19;" d +DORIS_BE_SRC_EXEC_NEW_PARTITIONED_HASH_TABLE_INLINE_H src/exec/new_partitioned_hash_table.inline.h 19;" d +DORIS_BE_SRC_EXEC_PARTITIONED_AGGREGATION_NODE_H src/exec/partitioned_aggregation_node.h 19;" d +DORIS_BE_SRC_EXEC_PARTITIONED_HASH_TABLE_H src/exec/partitioned_hash_table.h 19;" d +DORIS_BE_SRC_EXEC_PARTITIONED_HASH_TABLE_INLINE_H src/exec/partitioned_hash_table.inline.h 19;" d +DORIS_BE_SRC_EXEC_SPILL_SORT_NODE_H src/exec/spill_sort_node.h 19;" d +DORIS_BE_SRC_EXPRS_ARITHMETIC_EXPR_H src/exprs/arithmetic_expr.h 19;" d +DORIS_BE_SRC_EXPRS_CAST_EXPR_H src/exprs/cast_expr.h 19;" d +DORIS_BE_SRC_EXPRS_DECIMAL_OPERATORS_H src/exprs/decimalv2_operators.h 19;" d +DORIS_BE_SRC_EXPRS_TIME_OPERATORS_H src/exprs/time_operators.h 19;" d +DORIS_BE_SRC_HTTP_ACTION_HEALTH_ACTION_H src/http/action/health_action.h 19;" d +DORIS_BE_SRC_HTTP_ACTION_META_ACTION_H src/http/action/meta_action.h 19;" d +DORIS_BE_SRC_HTTP_ACTION_PPROF_ACTIONS_H src/http/action/pprof_actions.h 19;" d +DORIS_BE_SRC_HTTP_CHECKSUM_ACTION_H src/http/action/checksum_action.h 19;" d +DORIS_BE_SRC_HTTP_DOWNLOAD_ACTION_H src/http/download_action.h 19;" d +DORIS_BE_SRC_HTTP_RELOAD_TABLET_ACTION_H src/http/action/reload_tablet_action.h 19;" d +DORIS_BE_SRC_HTTP_RESTORE_TABLET_ACTION_H src/http/action/restore_tablet_action.h 19;" d +DORIS_BE_SRC_HTTP_SNAPSHOT_ACTION_H src/http/action/snapshot_action.h 19;" d +DORIS_BE_SRC_MOCK_MOCK_TASK_WORKER_POOL_H test/agent/mock_task_worker_pool.h 19;" d +DORIS_BE_SRC_OLAP_BASE_COMPACTION_H src/olap/base_compaction.h 19;" d +DORIS_BE_SRC_OLAP_BHP_LIB_H src/olap/bhp_lib.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_BLOOM_FILTER_HPP src/olap/bloom_filter.hpp 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_BLOOM_FILTER_READER_H src/olap/bloom_filter_reader.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_BLOOM_FILTER_WRITER_H src/olap/bloom_filter_writer.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_BYTE_BUFFER_H src/olap/byte_buffer.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_COMPRESS_H src/olap/compress.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_FILE_STREAM_H src/olap/file_stream.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_IN_STREAM_H src/olap/in_stream.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_OUT_STREAM_H src/olap/out_stream.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_SERIALIZE_H src/olap/serialize.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_STREAM_INDEX_COMMON_H src/olap/stream_index_common.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_STREAM_INDEX_READER_H src/olap/stream_index_reader.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_STREAM_INDEX_WRITER_H src/olap/stream_index_writer.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_FILE_STREAM_NAME_H src/olap/stream_name.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_MAPPING_H src/olap/column_mapping.h 19;" d +DORIS_BE_SRC_OLAP_COLUMN_PREDICATE_H src/olap/column_predicate.h 19;" d +DORIS_BE_SRC_OLAP_COMPACTION_H src/olap/compaction.h 19;" d +DORIS_BE_SRC_OLAP_COMPARISON_PREDICATE_H src/olap/comparison_predicate.h 19;" d +DORIS_BE_SRC_OLAP_CUMULATIVE_COMPACTION_H src/olap/cumulative_compaction.h 19;" d +DORIS_BE_SRC_OLAP_DELETE_HANDLER_H src/olap/delete_handler.h 19;" d +DORIS_BE_SRC_OLAP_FIELD_H src/olap/field.h 19;" d +DORIS_BE_SRC_OLAP_FILE_HELPER_H src/olap/file_helper.h 19;" d +DORIS_BE_SRC_OLAP_HLL_H src/olap/hll.h 19;" d +DORIS_BE_SRC_OLAP_IN_LIST_PREDICATE_H src/olap/in_list_predicate.h 19;" d +DORIS_BE_SRC_OLAP_LRU_CACHE_H src/olap/lru_cache.h 6;" d +DORIS_BE_SRC_OLAP_MEMTABLE_H src/olap/memtable.h 19;" d +DORIS_BE_SRC_OLAP_MERGER_H src/olap/merger.h 19;" d +DORIS_BE_SRC_OLAP_MOCK_MOCK_COMMAND_EXECUTOR_H test/olap/mock_command_executor.h 19;" d +DORIS_BE_SRC_OLAP_NULL_PREDICATE_H src/olap/null_predicate.h 19;" d +DORIS_BE_SRC_OLAP_OLAP_COMMON_H src/olap/olap_common.h 19;" d +DORIS_BE_SRC_OLAP_OLAP_COND_H src/olap/olap_cond.h 19;" d +DORIS_BE_SRC_OLAP_OLAP_DEFINE_H src/olap/olap_define.h 19;" d +DORIS_BE_SRC_OLAP_OLAP_INDEX_H src/olap/olap_index.h 19;" d +DORIS_BE_SRC_OLAP_OLAP_OLAP_META_H src/olap/olap_meta.h 19;" d +DORIS_BE_SRC_OLAP_OLAP_SNAPSHOT_CONVERTER_H src/olap/olap_snapshot_converter.h 19;" d +DORIS_BE_SRC_OLAP_PUSH_HANDLER_H src/olap/push_handler.h 19;" d +DORIS_BE_SRC_OLAP_READER_H src/olap/reader.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_H src/olap/rowset/alpha_rowset.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_META_H src/olap/rowset/alpha_rowset_meta.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_READER_H src/olap/rowset/alpha_rowset_reader.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_WRITER_H src/olap/rowset/alpha_rowset_writer.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_BETA_ROWSET_READER_H src/olap/rowset/beta_rowset_reader.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_BETA_ROWSET_WRITER_H src/olap/rowset/beta_rowset_writer.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_BIT_FIELD_READER_H src/olap/rowset/bit_field_reader.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_BIT_FIELD_WRITER_H src/olap/rowset/bit_field_writer.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_COLUMN_DATA_H src/olap/rowset/column_data.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_COLUMN_DATA_WRITER_H src/olap/rowset/column_data_writer.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_COLUMN_READER_H src/olap/rowset/column_reader.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_COLUMN_WRITER_H src/olap/rowset/column_writer.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_FACTORY_H src/olap/rowset/rowset_factory.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_GRAPH_H src/olap/rowset_graph.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ROWSET_H src/olap/rowset/rowset.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ROWSET_META_H src/olap/rowset/rowset_meta.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ROWSET_META_MANAGER_H src/olap/rowset/rowset_meta_manager.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ROWSET_READER_CONTEXT_H src/olap/rowset/rowset_reader_context.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ROWSET_READER_H src/olap/rowset/rowset_reader.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ROWSET_WRITER_CONTEXT_H src/olap/rowset/rowset_writer_context.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_ROWSET_WRITER_H src/olap/rowset/rowset_writer.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_RUN_LENGTH_BYTE_READER_H src/olap/rowset/run_length_byte_reader.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_RUN_LENGTH_BYTE_WRITER_H src/olap/rowset/run_length_byte_writer.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_RUN_LENGTH_INTEGER_READER_H src/olap/rowset/run_length_integer_reader.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_RUN_LENGTH_INTEGER_WRITER_H src/olap/rowset/run_length_integer_writer.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_SEGMENT_GROUP_H src/olap/rowset/segment_group.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_SEGMENT_READER_H src/olap/rowset/segment_reader.h 19;" d +DORIS_BE_SRC_OLAP_ROWSET_SEGMENT_WRITER_H src/olap/rowset/segment_writer.h 19;" d +DORIS_BE_SRC_OLAP_ROW_BLOCK_H src/olap/row_block.h 19;" d +DORIS_BE_SRC_OLAP_ROW_CURSOR_H src/olap/row_cursor.h 19;" d +DORIS_BE_SRC_OLAP_SCHEMA_CHANGE_H src/olap/schema_change.h 19;" d +DORIS_BE_SRC_OLAP_SKIPLIST_H src/olap/skiplist.h 6;" d +DORIS_BE_SRC_OLAP_SNAPSHOT_MANAGER_H src/olap/snapshot_manager.h 19;" d +DORIS_BE_SRC_OLAP_STORAGE_ENGINE_H src/olap/storage_engine.h 19;" d +DORIS_BE_SRC_OLAP_STRING_SLICE_H src/util/slice.h 19;" d +DORIS_BE_SRC_OLAP_TABLET_H src/olap/tablet.h 19;" d +DORIS_BE_SRC_OLAP_TABLET_MANAGER_H src/olap/tablet_manager.h 19;" d +DORIS_BE_SRC_OLAP_TABLET_META_H src/olap/tablet_meta.h 19;" d +DORIS_BE_SRC_OLAP_TABLET_META_MANAGER_H src/olap/tablet_meta_manager.h 19;" d +DORIS_BE_SRC_OLAP_TABLET_SCHEMA_H src/olap/tablet_schema.h 19;" d +DORIS_BE_SRC_OLAP_TABLET_SYNC_SERVICE_H src/olap/tablet_sync_service.h 19;" d +DORIS_BE_SRC_OLAP_TASK_ENGINE_ALTER_TABLET_TASK_H src/olap/task/engine_alter_tablet_task.h 19;" d +DORIS_BE_SRC_OLAP_TASK_ENGINE_BATCH_LOAD_TASK_H src/olap/task/engine_batch_load_task.h 19;" d +DORIS_BE_SRC_OLAP_TASK_ENGINE_CHECKSUM_TASK_H src/olap/task/engine_checksum_task.h 19;" d +DORIS_BE_SRC_OLAP_TASK_ENGINE_CLONE_TASK_H src/olap/task/engine_clone_task.h 19;" d +DORIS_BE_SRC_OLAP_TASK_ENGINE_PUBLISH_VERSION_TASK_H src/olap/task/engine_publish_version_task.h 19;" d +DORIS_BE_SRC_OLAP_TASK_ENGINE_STORAGE_MIGRATION_TASK_H src/olap/task/engine_storage_migration_task.h 19;" d +DORIS_BE_SRC_OLAP_TASK_ENGINE_TASK_H src/olap/task/engine_task.h 19;" d +DORIS_BE_SRC_OLAP_TXN_MANAGER_H src/olap/txn_manager.h 19;" d +DORIS_BE_SRC_OLAP_TYPES_H src/olap/types.h 19;" d +DORIS_BE_SRC_OLAP_UTILS_H src/olap/utils.h 19;" d +DORIS_BE_SRC_OLAP_WRAPPER_FIELD_H src/olap/wrapper_field.h 19;" d +DORIS_BE_SRC_QUERY_BE_RUNTIME_DATA_STREAM_RECVR_H src/runtime/data_stream_recvr.hpp 19;" d +DORIS_BE_SRC_QUERY_BE_RUNTIME_FREE_LIST_H src/runtime/free_list.hpp 19;" d +DORIS_BE_SRC_QUERY_BE_RUNTIME_FREE_POOL_H src/runtime/free_pool.hpp 19;" d +DORIS_BE_SRC_QUERY_BE_RUNTIME_MEM_LIMIT_H src/runtime/mem_tracker.h 19;" d +DORIS_BE_SRC_QUERY_BE_RUNTIME_ROW_BATCH_INTERFACE_H src/runtime/row_batch_interface.hpp 19;" d +DORIS_BE_SRC_QUERY_BE_RUNTIME_STRING_BUFFER_H src/runtime/string_buffer.hpp 19;" d +DORIS_BE_SRC_QUERY_BE_RUNTIME_STRING_SEARCH_H src/runtime/string_search.hpp 55;" d +DORIS_BE_SRC_QUERY_BE_RUNTIME_STRING_VALUE_INLINE_H src/runtime/string_value.hpp 19;" d +DORIS_BE_SRC_QUERY_CODGEN_DORIS_IR_H src/codegen/doris_ir.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_AGGREGATION_NODE_H src/exec/aggregation_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_BLOCKING_JOIN_NODE_H src/exec/blocking_join_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_CROSS_JOIN_NODE_H src/exec/cross_join_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_CSV_SCANNER_H src/exec/csv_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_CSV_SCAN_NODE_H src/exec/csv_scan_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_DATA_SINK_H src/exec/data_sink.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_EXCEPT_NODE_H src/exec/except_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_EXCHANGE_NODE_H src/exec/exchange_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_EXEC_NODE_H src/exec/exec_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_HASH_JOIN_NODE_H src/exec/hash_join_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_HASH_TABLE_H src/exec/hash_table.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_HASH_TABLE_HPP src/exec/hash_table.hpp 19;" d +DORIS_BE_SRC_QUERY_EXEC_INTERSECT_NODE_H src/exec/intersect_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_MERGE_JOIN_NODE_H src/exec/merge_join_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_MERGE_NODE_H src/exec/merge_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_MYSQL_SCANNER_H src/exec/mysql_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_MYSQL_SCAN_NODE_H src/exec/mysql_scan_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_OLAP_COMMON_H src/exec/olap_common.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_OLAP_REWRITE_NODE_H src/exec/olap_rewrite_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_OLAP_SCANNER_H src/exec/olap_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_OLAP_SCAN_NODE_H src/exec/olap_scan_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_OLAP_UTILS_H src/exec/olap_utils.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_PRE_AGGREGATION_NODE_H src/exec/pre_aggregation_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_READ_WRITE_UTIL_H src/exec/read_write_util.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_ROW_BATCH_LIST_H src/exec/row_batch_list.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SCAN_NODE_H src/exec/scan_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_H src/exec/schema_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_CHARSETS_SCANNER_H src/exec/schema_scanner/schema_charsets_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_COLLATIONS_SCANNER_H src/exec/schema_scanner/schema_collations_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_COLUMNS_SCANNER_H src/exec/schema_scanner/schema_columns_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_DUMMY_SCANNER_H src/exec/schema_scanner/schema_dummy_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_SCHEMATA_SCANNER_H src/exec/schema_scanner/schema_schemata_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_TABLES_SCANNER_H src/exec/schema_scanner/schema_tables_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_VARIABLES_SCANNER_H src/exec/schema_scanner/schema_variables_scanner.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCAN_NODE_H src/exec/schema_scan_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_SELECT_NODE_H src/exec/select_node.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_TEXT_CONVERTER_H src/exec/text_converter.h 19;" d +DORIS_BE_SRC_QUERY_EXEC_TEXT_CONVERTER_HPP src/exec/text_converter.hpp 19;" d +DORIS_BE_SRC_QUERY_EXEC_TOPN_NODE_H src/exec/topn_node.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_AGGREGATE_FUNCTIONS_H src/exprs/aggregate_functions.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_AGG_FN_EVALUATOR_H src/exprs/agg_fn_evaluator.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_ANYVAL_UTIL_H src/exprs/anyval_util.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_BINARY_PREDICATE_H src/exprs/binary_predicate.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_BITMAP_FUNCTION_H src/exprs/bitmap_function.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_CASE_EXPR_H src/exprs/case_expr.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_CAST_FUNCTIONS_H src/exprs/cast_functions.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_COMPOUND_PREDICATE_H src/exprs/compound_predicate.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_CONDITIONAL_FUNCTIONS_H src/exprs/conditional_functions.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_DECIMAL_OPERATORS_H src/exprs/decimal_operators.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_ENCRYPTION_FUNCTIONS_H src/exprs/encryption_functions.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_ES_FUNCTIONS_H src/exprs/es_functions.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_EXPR_CONTEXT_H src/exprs/expr_context.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_EXPR_H src/exprs/expr.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_EXPR_VALUE_H src/exprs/expr_value.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_HLL_FUNCTION_H src/exprs/hll_function.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_HLL_HASH_FUNCTION_H src/exprs/hll_hash_function.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_HYBIRD_MAP_H src/exprs/hybird_map.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_HYBIRD_SET_H src/exprs/hybird_set.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_INFO_FUNC_H src/exprs/info_func.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_IN_PREDICATE_H src/exprs/in_predicate.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_IS_NULL_PREDICATE_H src/exprs/is_null_predicate.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_JSON_FUNCTIONS_H src/exprs/json_functions.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_LIKE_PREDICATE_H src/exprs/like_predicate.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_LITERAL_H src/exprs/literal.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_MATH_FUNCTIONS_H src/exprs/math_functions.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_NEW_IN_PREDICATE_H src/exprs/new_in_predicate.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_NULL_LITERAL_H src/exprs/null_literal.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_OPERATORS_H src/exprs/operators.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_PREDICATE_H src/exprs/predicate.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_SCALAR_FN_CALL_H src/exprs/scalar_fn_call.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_SLOT_REF_H src/exprs/slot_ref.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_STRING_FUNCTIONS_H src/exprs/string_functions.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_TIMESTAMP_FUNCTIONS_H src/exprs/timestamp_functions.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_TUPLE_IS_NULL_PREDICATE_H src/exprs/tuple_is_null_predicate.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_UDF_BUILTINS_H src/exprs/udf_builtins.h 19;" d +DORIS_BE_SRC_QUERY_EXPRS_UTILITY_FUNCTIONS_H src/exprs/utility_functions.h 19;" d +DORIS_BE_SRC_QUERY_MYSQL_MYSQL_ROW_BUFFER_H src/util/mysql_row_buffer.h 19;" d +DORIS_BE_SRC_QUERY_NEW_EXPRS_AGG_FN_H src/exprs/agg_fn.h 19;" d +DORIS_BE_SRC_QUERY_RUNTIME_DECIMAL_VALUE_H src/runtime/decimal_value.h 19;" d +DORIS_BE_SRC_QUERY_RUNTIME_DISK_IO_MGR_H src/runtime/disk_io_mgr.h 19;" d +DORIS_BE_SRC_QUERY_RUNTIME_DISK_IO_MGR_INTERNAL_H src/runtime/disk_io_mgr_internal.h 19;" d +DORIS_BE_SRC_QUERY_RUNTIME_RUNTIME_STATE_H src/runtime/runtime_state.h 19;" d +DORIS_BE_SRC_QUERY_RUNTIME_SORTER_H src/runtime/merge_sorter.h 19;" d +DORIS_BE_SRC_QUERY_RUNTIME_TMP_FILE_MGR_H src/runtime/tmp_file_mgr.h 19;" d +DORIS_BE_SRC_RUNTIME_BUFFERED_BLOCK_MGR2_H src/runtime/buffered_block_mgr2.h 19;" d +DORIS_BE_SRC_RUNTIME_BUFFERED_TUPLE_STREAM2_H src/runtime/buffered_tuple_stream2.h 19;" d +DORIS_BE_SRC_RUNTIME_BUFFERED_TUPLE_STREAM2_INLINE_H src/runtime/buffered_tuple_stream2.inline.h 19;" d +DORIS_BE_SRC_RUNTIME_DATA_STREAM_MGR_H src/runtime/data_stream_mgr.h 19;" d +DORIS_BE_SRC_RUNTIME_DATA_STREAM_RECVR_H src/runtime/data_stream_recvr.h 19;" d +DORIS_BE_SRC_RUNTIME_DECIMALV2_VALUE_H src/runtime/decimalv2_value.h 19;" d +DORIS_BE_SRC_RUNTIME_EXPORT_SINK_H src/runtime/export_sink.h 19;" d +DORIS_BE_SRC_RUNTIME_EXPORT_TASK_MGR_H src/runtime/export_task_mgr.h 19;" d +DORIS_BE_SRC_RUNTIME_LOAD_PATH_MGR_H src/runtime/load_path_mgr.h 19;" d +DORIS_BE_SRC_RUNTIME_SMALL_FILE_MGR_H src/runtime/small_file_mgr.h 19;" d +DORIS_BE_SRC_RUNTIME_SNAPSHOT_LOADER_H src/runtime/snapshot_loader.h 19;" d +DORIS_BE_SRC_RUNTIME_SORTED_RUN_MERGER_H src/runtime/sorted_run_merger.h 19;" d +DORIS_BE_SRC_RUNTIME_SPILL_SORTER_H src/runtime/spill_sorter.h 19;" d +DORIS_BE_SRC_RUNTIME_VECTORIZED_ROW_BATCH_H src/runtime/vectorized_row_batch.h 19;" d +DORIS_BE_SRC_TASK_WORKER_POOL_H src/agent/task_worker_pool.h 19;" d +DORIS_BE_SRC_TESTUTIL_DESC_TBL_BUILDER_H src/testutil/desc_tbl_builder.h 19;" d +DORIS_BE_SRC_UTIL_ALIGNED_NEW_H_ src/util/aligned_new.h 19;" d +DORIS_BE_SRC_UTIL_BARRIER_H src/util/barrier.h 19;" d +DORIS_BE_SRC_UTIL_BFD_PARSER_H src/util/bfd_parser.h 19;" d +DORIS_BE_SRC_UTIL_BITMAP_VALUE_H src/util/bitmap_value.h 19;" d +DORIS_BE_SRC_UTIL_BROKER_LOAD_ERROR_HUB_H src/util/broker_load_error_hub.h 19;" d +DORIS_BE_SRC_UTIL_CONDITION_VARIABLE_H src/util/condition_variable.h 66;" d +DORIS_BE_SRC_UTIL_COUNTDOWN_LATCH_H src/util/countdown_latch.h 19;" d +DORIS_BE_SRC_UTIL_CPU_INFO_H src/util/cpu_info.h 19;" d +DORIS_BE_SRC_UTIL_CRC32C_H src/util/crc32c.h 22;" d +DORIS_BE_SRC_UTIL_DEBUG_RUNTIME_PROFILE_H src/util/dummy_runtime_profile.h 19;" d +DORIS_BE_SRC_UTIL_ERROR_UTIL_H src/util/error_util.h 19;" d +DORIS_BE_SRC_UTIL_FAKE_LOCK_H src/util/fake_lock.h 19;" d +DORIS_BE_SRC_UTIL_FILESYSTEM_UTIL_H src/util/filesystem_util.h 19;" d +DORIS_BE_SRC_UTIL_INTERNAL_QUEUE_H src/util/internal_queue.h 19;" d +DORIS_BE_SRC_UTIL_JSON_UTIL_H src/util/json_util.h 19;" d +DORIS_BE_SRC_UTIL_LOAD_ERROR_HUB_H src/util/load_error_hub.h 19;" d +DORIS_BE_SRC_UTIL_MEM_RANGE_H src/util/mem_range.h 19;" d +DORIS_BE_SRC_UTIL_MONOTIME_H src/util/monotime.h 19;" d +DORIS_BE_SRC_UTIL_MURMUR_HASH3_H src/util/murmur_hash3.h 6;" d +DORIS_BE_SRC_UTIL_MUTEX_H src/util/mutex.h 19;" d +DORIS_BE_SRC_UTIL_MYSQL_LOAD_ERROR_HUB_H src/util/mysql_load_error_hub.h 19;" d +DORIS_BE_SRC_UTIL_NULL_LOAD_ERROR_HUB_H src/util/null_load_error_hub.h 19;" d +DORIS_BE_SRC_UTIL_ONCE_H src/util/once.h 19;" d +DORIS_BE_SRC_UTIL_SCOPED_CLEANUP_H src/util/scoped_cleanup.h 19;" d +DORIS_BE_SRC_UTIL_SPINLOCK_H src/util/spinlock.h 19;" d +DORIS_BE_SRC_UTIL_STREAMING_SAMPLER_H src/util/streaming_sampler.h 19;" d +DORIS_BE_SRC_UTIL_THREAD_H src/util/thread.h 19;" d +DORIS_BE_SRC_UTIL_THREAD_POOL_H src/util/threadpool.h 19;" d +DORIS_BE_SRC_UTIL_UID_UTIL_H src/util/uid_util.h 19;" d +DORIS_BE_SRC_UTIL_UTF8_CHECK_H src/util/utf8_check.h 20;" d +DORIS_BE_SRC_UTIL_ZIP_UTIL_H src/util/zip_util.h 19;" d +DORIS_BE_TEST_QUERY_RUNTIME_TEST_ENV_H src/runtime/test_env.h 19;" d +DORIS_BE_UDF_UDA_TEST_HARNESS_H output/udf/include/uda_test_harness.h 19;" d +DORIS_BE_UDF_UDA_TEST_HARNESS_H src/udf/uda_test_harness.h 19;" d +DORIS_BE_UDF_UDF_DEBUG_H src/udf/udf_debug.h 19;" d +DORIS_BE_UDF_UDF_H output/udf/include/udf.h 19;" d +DORIS_BE_UDF_UDF_H src/udf/udf.h 19;" d +DORIS_BE_UDF_UDF_INTERNAL_H src/udf/udf_internal.h 19;" d +DORIS_BE_UTIL_DEFER_OP_H src/util/defer_op.h 19;" d +DORIS_BE_UTIL_FILE_UTILS_H src/util/file_utils.h 19;" d +DORIS_BE_UTIL_LRU_CACHE_HPP src/util/lru_cache.hpp 19;" d +DORIS_BE_UTIL_OS_INFO_H src/util/os_info.h 19;" d +DORIS_BE_UTIL_REST_MONITOR_IFACE_H src/http/rest_monitor_iface.h 19;" d +DORIS_BE_UTIL_TIME_H src/util/time.h 19;" d +DORIS_COLUMN_FAMILY src/olap/olap_define.h /^static const std::string DORIS_COLUMN_FAMILY = "doris";$/;" m namespace:doris +DORIS_COLUMN_FAMILY_INDEX src/olap/olap_define.h /^ DORIS_COLUMN_FAMILY_INDEX,$/;" e enum:doris::ColumnFamilyIndex +DORIS_CREATE_TABLE_DIFF_SCHEMA_EXIST src/agent/status.h /^ DORIS_CREATE_TABLE_DIFF_SCHEMA_EXIST = -302,$/;" e enum:doris::AgentStatus +DORIS_CREATE_TABLE_EXIST src/agent/status.h /^ DORIS_CREATE_TABLE_EXIST = -301,$/;" e enum:doris::AgentStatus +DORIS_CREATE_TABLE_NOT_EXIST src/agent/status.h /^ DORIS_CREATE_TABLE_NOT_EXIST = -303,$/;" e enum:doris::AgentStatus +DORIS_DISK_REACH_CAPACITY_LIMIT src/agent/status.h /^ DORIS_DISK_REACH_CAPACITY_LIMIT = -903,$/;" e enum:doris::AgentStatus +DORIS_DROP_TABLE_NOT_EXIST src/agent/status.h /^ DORIS_DROP_TABLE_NOT_EXIST = -401,$/;" e enum:doris::AgentStatus +DORIS_ERROR src/agent/status.h /^ DORIS_ERROR = -1,$/;" e enum:doris::AgentStatus +DORIS_EXEC_KUDU_SCANNER_H src/exec/kudu_scanner.h 19;" d +DORIS_FILE_DOWNLOAD_CURL_INIT_FAILED src/agent/status.h /^ DORIS_FILE_DOWNLOAD_CURL_INIT_FAILED = -203,$/;" e enum:doris::AgentStatus +DORIS_FILE_DOWNLOAD_FAILED src/agent/status.h /^ DORIS_FILE_DOWNLOAD_FAILED = -204,$/;" e enum:doris::AgentStatus +DORIS_FILE_DOWNLOAD_GET_LENGTH_FAILED src/agent/status.h /^ DORIS_FILE_DOWNLOAD_GET_LENGTH_FAILED = -205,$/;" e enum:doris::AgentStatus +DORIS_FILE_DOWNLOAD_INSTALL_OPT_FAILED src/agent/status.h /^ DORIS_FILE_DOWNLOAD_INSTALL_OPT_FAILED = -202,$/;" e enum:doris::AgentStatus +DORIS_FILE_DOWNLOAD_INVALID_PARAM src/agent/status.h /^ DORIS_FILE_DOWNLOAD_INVALID_PARAM = -201,$/;" e enum:doris::AgentStatus +DORIS_FILE_DOWNLOAD_LIST_DIR_FAIL src/agent/status.h /^ DORIS_FILE_DOWNLOAD_LIST_DIR_FAIL = -207,$/;" e enum:doris::AgentStatus +DORIS_FILE_DOWNLOAD_NOT_EXIST src/agent/status.h /^ DORIS_FILE_DOWNLOAD_NOT_EXIST = -206,$/;" e enum:doris::AgentStatus +DORIS_FRAME_OF_REFERENCE_CODING_H src/util/frame_of_reference_coding.h 19;" d +DORIS_INTERNAL_ERROR src/agent/status.h /^ DORIS_INTERNAL_ERROR = -902,$/;" e enum:doris::AgentStatus +DORIS_LZ4F_VERSION src/exec/decompressor.cpp /^const unsigned Lz4FrameDecompressor::DORIS_LZ4F_VERSION = 100;$/;" m class:doris::Lz4FrameDecompressor file: +DORIS_LZ4F_VERSION src/exec/decompressor.h /^ const static unsigned DORIS_LZ4F_VERSION;$/;" m class:doris::Lz4FrameDecompressor +DORIS_PLUGIN_VERSION src/plugin/plugin.h 24;" d +DORIS_PUSH_HAD_LOADED src/agent/status.h /^ DORIS_PUSH_HAD_LOADED = -504,$/;" e enum:doris::AgentStatus +DORIS_PUSH_INVALID_TABLE src/agent/status.h /^ DORIS_PUSH_INVALID_TABLE = -501,$/;" e enum:doris::AgentStatus +DORIS_PUSH_INVALID_VERSION src/agent/status.h /^ DORIS_PUSH_INVALID_VERSION = -502,$/;" e enum:doris::AgentStatus +DORIS_PUSH_TIME_OUT src/agent/status.h /^ DORIS_PUSH_TIME_OUT = -503,$/;" e enum:doris::AgentStatus +DORIS_RECORD_BATCH_QUEUE_H src/runtime/record_batch_queue.h 19;" d +DORIS_SRC_OLAP_ROWSET_BETA_ROWSET_H_ src/olap/rowset/beta_rowset.h 19;" d +DORIS_SUCCESS src/agent/status.h /^ DORIS_SUCCESS = 0,$/;" e enum:doris::AgentStatus +DORIS_TASK_REQUEST_ERROR src/agent/status.h /^ DORIS_TASK_REQUEST_ERROR = -101,$/;" e enum:doris::AgentStatus +DORIS_TIMEOUT src/agent/status.h /^ DORIS_TIMEOUT = -901,$/;" e enum:doris::AgentStatus +DORIS_V1 src/olap/olap_define.h /^ DORIS_V1 = 1,$/;" e enum:doris::OLAPDataVersion +DOUBLE_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t DOUBLE_SIZE = sizeof(double);$/;" m class:doris::AggFnEvaluator +DOUBLE_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t DOUBLE_SIZE = sizeof(double);$/;" m class:doris::NewAggFnEvaluator +DOUBLE_TRUNCATE src/util/pretty_printer.h 31;" d +DOWNLOAD src/agent/task_worker_pool.h /^ DOWNLOAD,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +DOWNLOAD_FILE_MAX_RETRY src/olap/task/engine_clone_task.cpp /^const uint32_t DOWNLOAD_FILE_MAX_RETRY = 3;$/;" m namespace:doris file: +DOWNLOAD_FILE_MAX_RETRY test/agent/mock_task_worker_pool.h /^const uint32_t DOWNLOAD_FILE_MAX_RETRY = 3;$/;" m namespace:doris +DOWNLOAD_TYPE src/http/download_action.h /^ enum DOWNLOAD_TYPE {$/;" g class:doris::DownloadAction +DO_NOT_TAKE_OWNERSHIP src/gutil/basictypes.h /^ DO_NOT_TAKE_OWNERSHIP,$/;" e enum:Ownership +DPP_ABNORMAL_ALL src/runtime/etl_job_mgr.cpp /^const std::string DPP_ABNORMAL_ALL = "dpp.abnorm.ALL";$/;" m namespace:doris file: +DPP_NORMAL_ALL src/runtime/etl_job_mgr.cpp /^const std::string DPP_NORMAL_ALL = "dpp.norm.ALL";$/;" m namespace:doris file: +DPP_PREFIX src/olap/olap_define.h /^static const std::string DPP_PREFIX = "\/dpp_download";$/;" m namespace:doris +DROP_TABLE src/agent/task_worker_pool.h /^ DROP_TABLE,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +DST_TUPLE_SLOT_ID_START test/exec/parquet_scanner_test.cpp 72;" d file: +DS_SUCCESS src/exec/olap_scan_node.cpp 46;" d file: +DURABLE src/olap/lru_cache.h /^ DURABLE = 1$/;" m class:doris::CachePriority +DYNAMIC_ANNOTATIONS_ENABLED src/gutil/dynamic_annotations.h 61;" d +DYNAMIC_ANNOTATIONS_EXTERNAL_IMPL src/gutil/dynamic_annotations.c 53;" d file: +Data src/gutil/gscoped_ptr.h /^ Data(T* ptr_in, D other) : D(std::move(other)), ptr(ptr_in) {}$/;" f struct:doris::internal::gscoped_ptr_impl::Data +Data src/gutil/gscoped_ptr.h /^ explicit Data(T* ptr_in) : ptr(ptr_in) {}$/;" f struct:doris::internal::gscoped_ptr_impl::Data +Data src/gutil/gscoped_ptr.h /^ struct Data : public D {$/;" s class:doris::internal::gscoped_ptr_impl +DataConsumer src/runtime/routine_load/data_consumer.h /^ DataConsumer(StreamLoadContext* ctx):$/;" f class:doris::DataConsumer +DataConsumer src/runtime/routine_load/data_consumer.h /^class DataConsumer {$/;" c namespace:doris +DataConsumerGroup src/runtime/routine_load/data_consumer_group.h /^ DataConsumerGroup():$/;" f class:doris::DataConsumerGroup +DataConsumerGroup src/runtime/routine_load/data_consumer_group.h /^class DataConsumerGroup {$/;" c namespace:doris +DataConsumerPool src/runtime/routine_load/data_consumer_pool.h /^ DataConsumerPool(int64_t max_pool_size):$/;" f class:doris::DataConsumerPool +DataConsumerPool src/runtime/routine_load/data_consumer_pool.h /^class DataConsumerPool {$/;" c namespace:doris +DataDir src/olap/data_dir.cpp /^DataDir::DataDir(const std::string& path, int64_t capacity_bytes,$/;" f class:doris::DataDir +DataDir src/olap/data_dir.h /^class DataDir {$/;" c namespace:doris +DataDirInfo src/olap/olap_common.h /^ DataDirInfo():$/;" f struct:doris::DataDirInfo +DataDirInfo src/olap/olap_common.h /^struct DataDirInfo {$/;" s namespace:doris +DataSink src/exec/data_sink.h /^ DataSink() : _closed(false) {}$/;" f class:doris::DataSink +DataSink src/exec/data_sink.h /^class DataSink {$/;" c namespace:doris +DataSource src/util/perf_counters.h /^ enum DataSource {$/;" g class:doris::PerfCounters +DataSplitTest test/runtime/data_spliter_test.cpp /^ DataSplitTest() {$/;" f class:doris::DataSplitTest +DataSplitTest test/runtime/data_spliter_test.cpp /^class DataSplitTest : public testing::Test {$/;" c namespace:doris file: +DataSpliter src/runtime/data_spliter.cpp /^DataSpliter::DataSpliter(const RowDescriptor& row_desc) :$/;" f class:doris::DataSpliter +DataSpliter src/runtime/data_spliter.h /^class DataSpliter : public DataSink {$/;" c namespace:doris +DataStreamMgr src/runtime/data_stream_mgr.h /^ DataStreamMgr() {}$/;" f class:doris::DataStreamMgr +DataStreamMgr src/runtime/data_stream_mgr.h /^class DataStreamMgr {$/;" c namespace:doris +DataStreamRecvr src/runtime/data_stream_recvr.cc /^DataStreamRecvr::DataStreamRecvr($/;" f class:doris::DataStreamRecvr +DataStreamRecvr src/runtime/data_stream_recvr.h /^class DataStreamRecvr {$/;" c namespace:doris +DataStreamRecvr src/runtime/data_stream_recvr.hpp /^ DataStreamRecvr(DataStreamMgr* mgr,$/;" f class:doris::DataStreamRecvr +DataStreamRecvr src/runtime/data_stream_recvr.hpp /^class DataStreamRecvr {$/;" c namespace:doris +DataStreamSender src/runtime/data_stream_sender.cpp /^DataStreamSender::DataStreamSender($/;" f class:doris::DataStreamSender +DataStreamSender src/runtime/data_stream_sender.h /^class DataStreamSender : public DataSink {$/;" c namespace:doris +DataStreamTest test/runtime/data_stream_test.cpp /^ DataStreamTest() :$/;" f class:doris::DataStreamTest +DataStreamTest test/runtime/data_stream_test.cpp /^class DataStreamTest : public testing::Test {$/;" c namespace:doris file: +DateColumnReader src/olap/rowset/column_reader.h /^typedef IntegerColumnReaderWrapper DateColumnReader;$/;" t namespace:doris +DateColumnWriter src/olap/rowset/column_writer.h /^typedef IntegerColumnWriterWrapper DateColumnWriter;$/;" t namespace:doris +DateTimeColumnReader src/olap/rowset/column_reader.h /^typedef IntegerColumnReaderWrapper DateTimeColumnReader;$/;" t namespace:doris +DateTimeColumnWriter src/olap/rowset/column_writer.h /^typedef IntegerColumnWriterWrapper DateTimeColumnWriter;$/;" t namespace:doris +DateTimeHashHelper src/exprs/aggregate_functions.cpp /^ class DateTimeHashHelper {$/;" c class:doris::MultiDistinctCountDateState file: +DateTimeVal output/udf/include/udf.h /^ DateTimeVal() : packed_time(0), type(3) { }$/;" f struct:doris_udf::DateTimeVal +DateTimeVal output/udf/include/udf.h /^struct DateTimeVal : public AnyVal {$/;" s namespace:doris_udf +DateTimeVal src/udf/udf.h /^ DateTimeVal() : packed_time(0), type(3) { }$/;" f struct:doris_udf::DateTimeVal +DateTimeVal src/udf/udf.h /^struct DateTimeVal : public AnyVal {$/;" s namespace:doris_udf +DateTimeValue src/runtime/datetime_value.h /^ DateTimeValue() :$/;" f class:doris::DateTimeValue +DateTimeValue src/runtime/datetime_value.h /^ DateTimeValue(int64_t t) {$/;" f class:doris::DateTimeValue +DateTimeValue src/runtime/datetime_value.h /^ DateTimeValue(uint8_t neg, uint8_t type, uint8_t hour, $/;" f class:doris::DateTimeValue +DateTimeValue src/runtime/datetime_value.h /^class DateTimeValue {$/;" c namespace:doris +DateTimeValueTest test/runtime/datetime_value_test.cpp /^ DateTimeValueTest() {$/;" f class:doris::DateTimeValueTest +DateTimeValueTest test/runtime/datetime_value_test.cpp /^class DateTimeValueTest : public testing::Test {$/;" c namespace:doris file: +DatetimeWrapper src/exprs/scalar_fn_call.cpp /^typedef DateTimeVal (*DatetimeWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +DebugRuntimeProfile src/util/debug_counters.h /^class DebugRuntimeProfile {$/;" c namespace:doris +DebugString src/exec/new_partitioned_aggregation_node.cc /^string NewPartitionedAggregationNode::DebugString(int indentation_level) const {$/;" f class:doris::NewPartitionedAggregationNode +DebugString src/exec/new_partitioned_aggregation_node.cc /^void NewPartitionedAggregationNode::DebugString(int indentation_level,$/;" f class:doris::NewPartitionedAggregationNode +DebugString src/exec/new_partitioned_hash_table.cc /^string NewPartitionedHashTable::DebugString(bool skip_empty, bool show_match,$/;" f class:NewPartitionedHashTable +DebugString src/exprs/agg_fn.cc /^string AggFn::DebugString() const {$/;" f class:doris::AggFn +DebugString src/exprs/agg_fn.cc /^string AggFn::DebugString(const vector& agg_fns) {$/;" f class:doris::AggFn +DebugString src/runtime/buffered_tuple_stream3.cc /^string BufferedTupleStream3::DebugString() const {$/;" f class:BufferedTupleStream3 +DebugString src/runtime/buffered_tuple_stream3.cc /^string BufferedTupleStream3::Page::DebugString() const {$/;" f class:BufferedTupleStream3::Page +DebugString src/runtime/bufferpool/buffer_allocator.cc /^string BufferPool::BufferAllocator::DebugString() {$/;" f class:doris::BufferPool::BufferAllocator +DebugString src/runtime/bufferpool/buffer_allocator.cc /^string BufferPool::FreeBufferArena::DebugString() {$/;" f class:doris::BufferPool::FreeBufferArena +DebugString src/runtime/bufferpool/buffer_pool.cc /^string BufferPool::BufferHandle::DebugString() const {$/;" f class:doris::BufferPool::BufferHandle +DebugString src/runtime/bufferpool/buffer_pool.cc /^string BufferPool::Client::DebugString() {$/;" f class:doris::BufferPool::Client +DebugString src/runtime/bufferpool/buffer_pool.cc /^string BufferPool::ClientHandle::DebugString() const {$/;" f class:doris::BufferPool::ClientHandle +DebugString src/runtime/bufferpool/buffer_pool.cc /^string BufferPool::DebugString() {$/;" f class:doris::BufferPool +DebugString src/runtime/bufferpool/buffer_pool.cc /^string BufferPool::Page::DebugString() {$/;" f class:doris::BufferPool::Page +DebugString src/runtime/bufferpool/reservation_tracker.cc /^string ReservationTracker::DebugString() {$/;" f class:doris::ReservationTracker +DebugString src/runtime/descriptors.cpp /^std::string KuduTableDescriptor::DebugString() const {$/;" f class:doris::KuduTableDescriptor +DebugString src/util/bitmap.cpp /^std::string Bitmap::DebugString(bool print_bits) const {$/;" f class:doris::Bitmap +DebugString src/util/internal_queue.h /^ std::string DebugString() {$/;" f class:doris::InternalQueueBase +DebugString src/util/os_info.cpp /^string OsInfo::DebugString() {$/;" f class:doris::OsInfo +DebugString src/util/streaming_sampler.h /^ std::string DebugString(const std::string& prefix="") const {$/;" f class:doris::StreamingSampler +DebugStringCallback src/runtime/bufferpool/buffer_pool.cc /^bool BufferPool::Page::DebugStringCallback(stringstream* ss, BufferPool::Page* page) {$/;" f class:doris::BufferPool::Page +DebugStringTuple src/exec/new_partitioned_hash_table.cc /^void NewPartitionedHashTable::DebugStringTuple(std::stringstream& ss, HtData& htdata,$/;" f class:NewPartitionedHashTable +DecimalAvgState src/exprs/aggregate_functions.cpp /^struct DecimalAvgState {$/;" s namespace:doris file: +DecimalColumnReader src/olap/rowset/column_reader.cpp /^DecimalColumnReader::DecimalColumnReader(uint32_t column_id, uint32_t column_unique_id) :$/;" f class:doris::DecimalColumnReader +DecimalColumnReader src/olap/rowset/column_reader.h /^class DecimalColumnReader : public ColumnReader {$/;" c namespace:doris +DecimalColumnWriter src/olap/rowset/column_writer.cpp /^DecimalColumnWriter::DecimalColumnWriter(uint32_t column_id,$/;" f class:doris::DecimalColumnWriter +DecimalColumnWriter src/olap/rowset/column_writer.h /^class DecimalColumnWriter : public ColumnWriter {$/;" c namespace:doris +DecimalError src/runtime/decimal_value.h /^enum DecimalError {$/;" g namespace:doris +DecimalOperators src/exprs/decimal_operators.h /^class DecimalOperators {$/;" c namespace:doris +DecimalRoundMode src/runtime/decimal_value.h /^enum DecimalRoundMode {$/;" g namespace:doris +DecimalV2AvgState src/exprs/aggregate_functions.cpp /^struct DecimalV2AvgState {$/;" s namespace:doris file: +DecimalV2Operators src/exprs/decimalv2_operators.h /^class DecimalV2Operators {$/;" c namespace:doris +DecimalV2Val output/udf/include/udf.h /^ DecimalV2Val() : val(0) {}$/;" f struct:doris_udf::DecimalV2Val +DecimalV2Val output/udf/include/udf.h /^ DecimalV2Val(__int128 value) : val(value) {}$/;" f struct:doris_udf::DecimalV2Val +DecimalV2Val output/udf/include/udf.h /^struct DecimalV2Val : public AnyVal {$/;" s namespace:doris_udf +DecimalV2Val src/udf/udf.h /^ DecimalV2Val() : val(0) {}$/;" f struct:doris_udf::DecimalV2Val +DecimalV2Val src/udf/udf.h /^ DecimalV2Val(__int128 value) : val(value) {}$/;" f struct:doris_udf::DecimalV2Val +DecimalV2Val src/udf/udf.h /^struct DecimalV2Val : public AnyVal {$/;" s namespace:doris_udf +DecimalV2Value src/runtime/decimalv2_value.h /^ DecimalV2Value() : _value(0){}$/;" f class:doris::DecimalV2Value +DecimalV2Value src/runtime/decimalv2_value.h /^ DecimalV2Value(const std::string& decimal_str) {$/;" f class:doris::DecimalV2Value +DecimalV2Value src/runtime/decimalv2_value.h /^ DecimalV2Value(int64_t int_value, int64_t frac_value) {$/;" f class:doris::DecimalV2Value +DecimalV2Value src/runtime/decimalv2_value.h /^ explicit DecimalV2Value(int128_t int_value) {$/;" f class:doris::DecimalV2Value +DecimalV2Value src/runtime/decimalv2_value.h /^class DecimalV2Value {$/;" c namespace:doris +DecimalV2ValueTest test/runtime/decimalv2_value_test.cpp /^ DecimalV2ValueTest() {$/;" f class:doris::DecimalV2ValueTest +DecimalV2ValueTest test/runtime/decimalv2_value_test.cpp /^class DecimalV2ValueTest : public testing::Test {$/;" c namespace:doris file: +DecimalV2Wrapper src/exprs/scalar_fn_call.cpp /^typedef DecimalV2Val (*DecimalV2Wrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +DecimalVal output/udf/include/udf.h /^ DecimalVal() : int_len(0), frac_len(0), sign(false) {$/;" f struct:doris_udf::DecimalVal +DecimalVal output/udf/include/udf.h /^struct DecimalVal : public AnyVal {$/;" s namespace:doris_udf +DecimalVal src/udf/udf.h /^ DecimalVal() : int_len(0), frac_len(0), sign(false) {$/;" f struct:doris_udf::DecimalVal +DecimalVal src/udf/udf.h /^struct DecimalVal : public AnyVal {$/;" s namespace:doris_udf +DecimalValue src/runtime/decimal_value.h /^ DecimalValue() : _buffer_length(DECIMAL_BUFF_LENGTH){$/;" f class:doris::DecimalValue +DecimalValue src/runtime/decimal_value.h /^ DecimalValue(const std::string& decimal_str) : _buffer_length(DECIMAL_BUFF_LENGTH) {$/;" f class:doris::DecimalValue +DecimalValue src/runtime/decimal_value.h /^ DecimalValue(int64_t int_value) : _buffer_length(DECIMAL_BUFF_LENGTH){$/;" f class:doris::DecimalValue +DecimalValue src/runtime/decimal_value.h /^ DecimalValue(int64_t int_value, int64_t frac_value) : _buffer_length(DECIMAL_BUFF_LENGTH) {$/;" f class:doris::DecimalValue +DecimalValue src/runtime/decimal_value.h /^class DecimalValue {$/;" c namespace:doris +DecimalValueTest test/runtime/decimal_value_test.cpp /^ DecimalValueTest() {$/;" f class:doris::DecimalValueTest +DecimalValueTest test/runtime/decimal_value_test.cpp /^class DecimalValueTest : public testing::Test {$/;" c namespace:doris file: +DecimalWrapper src/exprs/scalar_fn_call.cpp /^typedef DecimalVal (*DecimalWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +DecodeFixed32 test/olap/lru_cache_test.cpp /^uint32_t DecodeFixed32(const char* ptr) {$/;" f namespace:doris +DecodeKey test/olap/lru_cache_test.cpp /^static int DecodeKey(const CacheKey& k) {$/;" f namespace:doris +DecodeValue test/olap/lru_cache_test.cpp /^static int DecodeValue(void* v) {$/;" f namespace:doris +Decompressor src/exec/decompressor.h /^ Decompressor(CompressType ctype):_ctype(ctype) {}$/;" f class:doris::Decompressor +Decompressor src/exec/decompressor.h /^class Decompressor {$/;" c namespace:doris +Decompressor src/olap/compress.h /^typedef OLAPStatus(*Decompressor)(StorageByteBuffer* in, StorageByteBuffer* out);$/;" t namespace:doris +DecompressorTest test/util/decompress_test.cpp /^ DecompressorTest() {$/;" f class:doris::DecompressorTest +DecompressorTest test/util/decompress_test.cpp /^class DecompressorTest : public ::testing::Test {$/;" c namespace:doris file: +DecreaseBytesRemaining src/runtime/bufferpool/buffer_allocator.cc /^int64_t DecreaseBytesRemaining($/;" f namespace:doris +DecreaseReservation src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::DecreaseReservation(int64_t bytes, bool is_child_reservation) {$/;" f class:doris::ReservationTracker +DecreaseReservation src/runtime/bufferpool/reservation_tracker.h /^ void DecreaseReservation(int64_t bytes) { DecreaseReservation(bytes, false); }$/;" f class:doris::ReservationTracker +DecreaseReservationLocked src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::DecreaseReservationLocked($/;" f class:doris::ReservationTracker +DecreaseReservationTo src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::Client::DecreaseReservationTo(int64_t target_bytes) {$/;" f class:doris::BufferPool::Client +DecreaseReservationTo src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::ClientHandle::DecreaseReservationTo(int64_t target_bytes) {$/;" f class:doris::BufferPool::ClientHandle +Default src/env/env_posix.cpp /^Env* Env::Default() {$/;" f class:doris::Env +DefaultDeleter src/gutil/gscoped_ptr.h /^ DefaultDeleter() {}$/;" f struct:doris::DefaultDeleter +DefaultDeleter src/gutil/gscoped_ptr.h /^ template DefaultDeleter(const DefaultDeleter& other) {$/;" f struct:doris::DefaultDeleter +DefaultDeleter src/gutil/gscoped_ptr.h /^struct DefaultDeleter {$/;" s namespace:doris +DefaultDeleter src/gutil/gscoped_ptr.h /^struct DefaultDeleter {$/;" s namespace:doris +DefaultDeleter src/gutil/gscoped_ptr.h /^struct DefaultDeleter {$/;" s namespace:doris +DefaultIntGenerator test/olap/rowset/segment_v2/segment_test.cpp /^static void DefaultIntGenerator(size_t rid, int cid, int block_id, RowCursorCell& cell) {$/;" f namespace:doris::segment_v2 +DefaultRefCountedThreadSafeTraits src/gutil/ref_counted.h /^struct DefaultRefCountedThreadSafeTraits {$/;" s namespace:doris +DefaultValueColumnIterator src/olap/rowset/segment_v2/column_reader.h /^ DefaultValueColumnIterator(bool has_default_value, const std::string& default_value,$/;" f class:doris::segment_v2::DefaultValueColumnIterator +DefaultValueColumnIterator src/olap/rowset/segment_v2/column_reader.h /^class DefaultValueColumnIterator : public ColumnIterator {$/;" c namespace:doris::segment_v2 +DefaultValueReader src/olap/rowset/column_reader.h /^ DefaultValueReader(uint32_t column_id, uint32_t column_unique_id,$/;" f class:doris::DefaultValueReader +DefaultValueReader src/olap/rowset/column_reader.h /^class DefaultValueReader : public ColumnReader {$/;" c namespace:doris +DeferFunction src/util/defer_op.h /^ typedef std::function DeferFunction;$/;" t class:doris::DeferOp +DeferOp src/util/defer_op.h /^ DeferOp(const DeferFunction& func) : _func(func) {$/;" f class:doris::DeferOp +DeferOp src/util/defer_op.h /^class DeferOp {$/;" c namespace:doris +DelCondSatisfied src/olap/olap_common.h /^enum DelCondSatisfied {$/;" g namespace:doris +DelPredicateArray src/olap/delete_handler.h /^typedef google::protobuf::RepeatedPtrField DelPredicateArray;$/;" t namespace:doris +DeleteBackingFile test/runtime/buffered_block_mgr2_test.cpp /^ static void DeleteBackingFile(BufferedBlockMgr2::Block* block) {$/;" f class:doris::BufferedBlockMgrTest +DeleteConditionHandler src/olap/delete_handler.h /^ DeleteConditionHandler() {}$/;" f class:doris::DeleteConditionHandler +DeleteConditionHandler src/olap/delete_handler.h /^class DeleteConditionHandler {$/;" c namespace:doris +DeleteConditions src/olap/delete_handler.h /^ DeleteConditions() : filter_version(0), del_cond(NULL) {}$/;" f struct:doris::DeleteConditions +DeleteConditions src/olap/delete_handler.h /^struct DeleteConditions {$/;" s namespace:doris +DeleteHandler src/olap/delete_handler.h /^ DeleteHandler() : _is_inited(false) {}$/;" f class:doris::DeleteHandler +DeleteHandler src/olap/delete_handler.h /^class DeleteHandler {$/;" c namespace:doris +DeleteInternal src/gutil/ref_counted.h /^ static void DeleteInternal(const T* x) { delete x; }$/;" f class:doris::RefCountedThreadSafe +Deleter test/olap/lru_cache_test.cpp /^ static void Deleter(const CacheKey& key, void* v) {$/;" f class:doris::CacheTest +DeltaHead src/olap/rowset/run_length_integer_writer.h /^ struct DeltaHead {$/;" s class:doris::RunLengthIntegerWriter +DeltaWriter src/olap/delta_writer.cpp /^DeltaWriter::DeltaWriter(WriteRequest* req, MemTracker* parent, StorageEngine* storage_engine) :$/;" f class:doris::DeltaWriter +DeltaWriter src/olap/delta_writer.h /^class DeltaWriter {$/;" c namespace:doris +DeltaWriter test/runtime/load_channel_mgr_test.cpp /^DeltaWriter::DeltaWriter(WriteRequest* req, MemTracker* mem_tracker,$/;" f class:doris::DeltaWriter +DeregisterClient src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::DeregisterClient(ClientHandle* client) {$/;" f class:doris::BufferPool +DerivedCounter src/util/runtime_profile.h /^ DerivedCounter(TUnit::type type, const DerivedCounterFunction& counter_fn) :$/;" f class:doris::RuntimeProfile::DerivedCounter +DerivedCounter src/util/runtime_profile.h /^ class DerivedCounter : public Counter {$/;" c class:doris::RuntimeProfile +DerivedCounterFunction src/util/runtime_profile.h /^ typedef boost::function DerivedCounterFunction;$/;" t class:doris::RuntimeProfile +DescriptorTbl src/runtime/descriptors.h /^ DescriptorTbl(): _tbl_desc_map(), _tuple_desc_map(), _slot_desc_map() {}$/;" f class:doris::DescriptorTbl +DescriptorTbl src/runtime/descriptors.h /^class DescriptorTbl {$/;" c namespace:doris +DescriptorTblBuilder src/testutil/desc_tbl_builder.cc /^DescriptorTblBuilder::DescriptorTblBuilder(ObjectPool* obj_pool) : _obj_pool(obj_pool) {$/;" f class:doris::DescriptorTblBuilder +DescriptorTblBuilder src/testutil/desc_tbl_builder.h /^class DescriptorTblBuilder {$/;" c namespace:doris +DestroyPage src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::DestroyPage(ClientHandle* client, PageHandle* handle) {$/;" f class:doris::BufferPool +DestroyPageInternal src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::Client::DestroyPageInternal($/;" f class:doris::BufferPool::Client +Destruct src/gutil/ref_counted.h /^ static void Destruct(const T* x) {$/;" f struct:doris::DefaultRefCountedThreadSafeTraits +DeviceId src/runtime/tmp_file_mgr.h /^ typedef int DeviceId;$/;" t class:doris::TmpFileMgr +DictKey src/olap/rowset/column_writer.h /^ explicit DictKey(const std::string& str_ref) : _str_ref(str_ref) {}$/;" f class:doris::VarStringColumnWriter::DictKey +DictKey src/olap/rowset/column_writer.h /^ class DictKey {$/;" c class:doris::VarStringColumnWriter +Difference src/gutil/bits.cc /^int Bits::Difference(const void *m1, const void *m2, int num_bytes) {$/;" f class:Bits +Dir src/runtime/tmp_file_mgr.h /^ Dir(const std::string& path, bool blacklisted) :$/;" f class:doris::TmpFileMgr::Dir +Dir src/runtime/tmp_file_mgr.h /^ class Dir {$/;" c class:doris::TmpFileMgr +DirectHead src/olap/rowset/run_length_integer_writer.h /^ struct DirectHead {$/;" s class:doris::RunLengthIntegerWriter +DiscreteDoubleColumnReader src/olap/rowset/column_reader.h /^typedef IntegerColumnReaderWrapper DiscreteDoubleColumnReader;$/;" t namespace:doris +DiscreteDoubleColumnWriter src/olap/rowset/column_writer.h /^typedef IntegerColumnWriterWrapper DiscreteDoubleColumnWriter;$/;" t namespace:doris +Disk src/util/disk_info.h /^ Disk() : name(""), id(0) {}$/;" f struct:doris::DiskInfo::Disk +Disk src/util/disk_info.h /^ Disk(const std::string& name) : name(name), id(0), is_rotational(true) {}$/;" f struct:doris::DiskInfo::Disk +Disk src/util/disk_info.h /^ Disk(const std::string& name, int id) : name(name), id(id), is_rotational(true) {}$/;" f struct:doris::DiskInfo::Disk +Disk src/util/disk_info.h /^ Disk(const std::string& name, int id, bool is_rotational) :$/;" f struct:doris::DiskInfo::Disk +Disk src/util/disk_info.h /^ struct Disk {$/;" s class:doris::DiskInfo +DiskInfo src/util/disk_info.h /^class DiskInfo {$/;" c namespace:doris +DiskIoMgr src/runtime/disk_io_mgr.cc /^DiskIoMgr::DiskIoMgr() :$/;" f class:doris::DiskIoMgr +DiskIoMgr src/runtime/disk_io_mgr.cc /^DiskIoMgr::DiskIoMgr(int num_local_disks, int threads_per_disk, int min_buffer_size,$/;" f class:doris::DiskIoMgr +DiskIoMgr src/runtime/disk_io_mgr.h /^class DiskIoMgr {$/;" c namespace:doris +DiskIoMgrTest test/runtime/disk_io_mgr_test.cpp /^class DiskIoMgrTest : public testing::Test {$/;" c namespace:doris file: +DiskMetrics src/util/system_metrics.cpp /^struct DiskMetrics {$/;" s namespace:doris file: +DiskQueue src/runtime/disk_io_mgr_internal.h /^ DiskQueue(int id) : disk_id(id) { }$/;" f struct:doris::DiskIoMgr::DiskQueue +DiskQueue src/runtime/disk_io_mgr_internal.h /^struct DiskIoMgr::DiskQueue {$/;" s class:doris::DiskIoMgr +DiskRange src/olap/rowset/segment_reader.h /^ DiskRange() : offset(0), end(0) {}$/;" f struct:doris::SegmentReader::DiskRange +DiskRange src/olap/rowset/segment_reader.h /^ struct DiskRange {$/;" s class:doris::SegmentReader +DistinctEstimatSerialize test/udf/uda_test.cpp /^StringVal DistinctEstimatSerialize(FunctionContext* context,$/;" f namespace:doris_udf +DistinctEstimatUpdate test/udf/uda_test.cpp /^void DistinctEstimatUpdate(FunctionContext* context,$/;" f namespace:doris_udf +DistinctEstimateFinalize test/udf/uda_test.cpp /^BigIntVal DistinctEstimateFinalize(FunctionContext* context, const StringVal& val) {$/;" f namespace:doris_udf +DistinctEstimateInit test/udf/uda_test.cpp /^void DistinctEstimateInit(FunctionContext* context, StringVal* val) {$/;" f namespace:doris_udf +DistinctEstimateMerge test/udf/uda_test.cpp /^void DistinctEstimateMerge(FunctionContext* context, const StringVal& src, StringVal* dst) {$/;" f namespace:doris_udf +DivExpr src/exprs/arithmetic_expr.h /^ DivExpr(const TExprNode& node) : ArithmeticExpr(node) { }$/;" f class:doris::DivExpr +DivExpr src/exprs/arithmetic_expr.h /^class DivExpr : public ArithmeticExpr {$/;" c namespace:doris +DoTestMonoTimePerf test/util/monotime_test.cpp /^static void DoTestMonoTimePerf() {$/;" f namespace:doris +DorisCallOnce src/util/once.h /^ DorisCallOnce()$/;" f class:doris::DorisCallOnce +DorisCallOnce src/util/once.h /^class DorisCallOnce {$/;" c namespace:doris +DorisMetrics src/util/doris_metrics.cpp /^DorisMetrics::DorisMetrics() : _metrics(nullptr), _system_metrics(nullptr) {$/;" f class:doris::DorisMetrics +DorisMetrics src/util/doris_metrics.h /^class DorisMetrics {$/;" c namespace:doris +DorisMetricsTest test/util/doris_metrics_test.cpp /^ DorisMetricsTest() { }$/;" f class:doris::DorisMetricsTest +DorisMetricsTest test/util/doris_metrics_test.cpp /^class DorisMetricsTest : public testing::Test {$/;" c namespace:doris file: +DorisNodesInfo src/exec/tablet_info.h /^ DorisNodesInfo(const TPaloNodesInfo& t_nodes) {$/;" f class:doris::DorisNodesInfo +DorisNodesInfo src/exec/tablet_info.h /^class DorisNodesInfo {$/;" c namespace:doris +DorisScanRangeTest test/exec/olap_common_test.cpp /^class DorisScanRangeTest : public ::testing::Test {$/;" c namespace:doris file: +DorisTestBackend test/runtime/data_stream_test.cpp /^ DorisTestBackend(DataStreamMgr* stream_mgr) : _mgr(stream_mgr) {}$/;" f class:doris::DorisTestBackend +DorisTestBackend test/runtime/data_stream_test.cpp /^class DorisTestBackend : public BackendServiceIf {$/;" c namespace:doris file: +DorisVersion output/udf/include/udf.h /^ enum DorisVersion {$/;" g class:doris_udf::FunctionContext +DorisVersion src/udf/udf.h /^ enum DorisVersion {$/;" g class:doris_udf::FunctionContext +DoubleColumnReader src/olap/rowset/column_reader.h /^typedef FloatintPointColumnReader DoubleColumnReader;$/;" t namespace:doris +DoubleColumnWriter src/olap/rowset/column_writer.h /^typedef DoubleColumnWriterBase DoubleColumnWriter;$/;" t namespace:doris +DoubleColumnWriterBase src/olap/rowset/column_writer.h /^ DoubleColumnWriterBase($/;" f class:doris::DoubleColumnWriterBase +DoubleColumnWriterBase src/olap/rowset/column_writer.h /^class DoubleColumnWriterBase: public ColumnWriter {$/;" c namespace:doris +DoubleRangeOptions src/gutil/strings/numbers.h /^struct DoubleRangeOptions {$/;" s +DoubleToBuffer src/gutil/strings/numbers.cc /^char* DoubleToBuffer(double value, char* buffer) {$/;" f +DoubleToBuffer src/gutil/strings/numbers.cc /^int DoubleToBuffer(double value, int width, char *buffer) {$/;" f +DoubleVal output/udf/include/udf.h /^ DoubleVal() : val(0.0) { }$/;" f struct:doris_udf::DoubleVal +DoubleVal output/udf/include/udf.h /^ DoubleVal(double val) : val(val) { }$/;" f struct:doris_udf::DoubleVal +DoubleVal output/udf/include/udf.h /^struct DoubleVal : public AnyVal {$/;" s namespace:doris_udf +DoubleVal src/udf/udf.h /^ DoubleVal() : val(0.0) { }$/;" f struct:doris_udf::DoubleVal +DoubleVal src/udf/udf.h /^ DoubleVal(double val) : val(val) { }$/;" f struct:doris_udf::DoubleVal +DoubleVal src/udf/udf.h /^struct DoubleVal : public AnyVal {$/;" s namespace:doris_udf +DoubleWrapper src/exprs/scalar_fn_call.cpp /^typedef DoubleVal (*DoubleWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +DownloadAction src/http/download_action.cpp /^DownloadAction::DownloadAction(ExecEnv* exec_env, const std::string& error_log_root_dir) :$/;" f class:doris::DownloadAction +DownloadAction src/http/download_action.cpp /^DownloadAction::DownloadAction(ExecEnv* exec_env, const std::vector& allow_dirs) :$/;" f class:doris::DownloadAction +DownloadAction src/http/download_action.h /^class DownloadAction : public HttpHandler {$/;" c namespace:doris +DppSink src/runtime/dpp_sink.h /^ DppSink(const RowDescriptor& row_desc,$/;" f class:doris::DppSink +DppSink src/runtime/dpp_sink.h /^class DppSink {$/;" c namespace:doris +DppSinkInternalTest test/runtime/dpp_sink_internal_test.cpp /^ DppSinkInternalTest() {$/;" f class:doris::DppSinkInternalTest +DppSinkInternalTest test/runtime/dpp_sink_internal_test.cpp /^class DppSinkInternalTest : public testing::Test {$/;" c namespace:doris file: +DppSinkTest test/runtime/dpp_sink_test.cpp /^ DppSinkTest() {$/;" f class:doris::DppSinkTest +DppSinkTest test/runtime/dpp_sink_test.cpp /^class DppSinkTest : public testing::Test {$/;" c namespace:doris file: +DppWriter src/runtime/dpp_writer.cpp /^DppWriter::DppWriter($/;" f class:doris::DppWriter +DppWriter src/runtime/dpp_writer.h /^class DppWriter {$/;" c namespace:doris +DppWriterTest test/runtime/dpp_writer_test.cpp /^ DppWriterTest() :$/;" f class:doris::DppWriterTest +DppWriterTest test/runtime/dpp_writer_test.cpp /^class DppWriterTest : public testing::Test {$/;" c namespace:doris file: +DummyProfile src/util/dummy_runtime_profile.h /^ DummyProfile() : _pool(), _profile(new RuntimeProfile(&_pool, "dummy", false)) {}$/;" f class:doris::DummyProfile +DummyProfile src/util/dummy_runtime_profile.h /^class DummyProfile {$/;" c namespace:doris +DuplicateNode src/exec/new_partitioned_hash_table.h /^ struct DuplicateNode {$/;" s class:doris::NewPartitionedHashTable +DuplicateNode src/exec/partitioned_hash_table.h /^ struct DuplicateNode {$/;" s class:doris::PartitionedHashTable +ECONNRESET src/gutil/port.h 946;" d +EIGHT src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +EIGHTEEN src/olap/serialize.h /^ THIRTEEN, FOURTEEN, FIFTEEN, SIXTEEN, SEVENTEEN, EIGHTEEN, NINETEEN,$/;" e enum:doris::ser::FixedBitSize +EINPROGRESS src/gutil/port.h 945;" d +ELEMENTS_PER_BLOCK src/util/core_local.cpp /^ static constexpr int ELEMENTS_PER_BLOCK = BLOCK_SIZE \/ ELEMENT_BYTES;$/;" m class:doris::CoreDataAllocatorImpl file: +ELEVEN src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +EMPTY src/util/bitmap_value.h /^ EMPTY = 0,$/;" e enum:doris::BitmapTypeCode::type +EMPTY src/util/bitmap_value.h /^ EMPTY = 0,$/;" e enum:doris::BitmapValue::BitmapDataType +ENABLE_COUNTERS src/util/runtime_profile.h 39;" d +ENABLE_DEBUG_COUNTERS src/util/debug_counters.h 23;" d +ENABLE_IF_ARITHMETIC src/util/template_util.h 35;" d +ENABLE_IF_FLOAT src/util/template_util.h 49;" d +ENABLE_IF_INTEGRAL src/util/template_util.h 44;" d +ENABLE_IF_NOT_ARITHMETIC src/util/template_util.h 40;" d +ENDSWITH src/olap/utils.h 262;" d +END_ROWSET_ID src/olap/olap_define.h /^static const std::string END_ROWSET_ID = "end_rowset_id";$/;" m namespace:doris +ENFORCE_POD src/gutil/type_traits.h 361;" d +ENOTSOCK src/gutil/port.h 944;" d +EOS src/gutil/strings/split.cc 693;" d file: +EPSILON src/exprs/math_functions.cpp /^const double EPSILON = 1e-9;$/;" m namespace:doris file: +EQUALS build/Makefile /^EQUALS = =$/;" m +EQUALS build/src/agent/Makefile /^EQUALS = =$/;" m +EQUALS build/src/common/Makefile /^EQUALS = =$/;" m +EQUALS build/src/env/Makefile /^EQUALS = =$/;" m +EQUALS build/src/exec/Makefile /^EQUALS = =$/;" m +EQUALS build/src/exprs/Makefile /^EQUALS = =$/;" m +EQUALS build/src/gen_cpp/Makefile /^EQUALS = =$/;" m +EQUALS build/src/geo/Makefile /^EQUALS = =$/;" m +EQUALS build/src/gutil/Makefile /^EQUALS = =$/;" m +EQUALS build/src/http/Makefile /^EQUALS = =$/;" m +EQUALS build/src/olap/Makefile /^EQUALS = =$/;" m +EQUALS build/src/olap/fs/Makefile /^EQUALS = =$/;" m +EQUALS build/src/olap/rowset/Makefile /^EQUALS = =$/;" m +EQUALS build/src/runtime/Makefile /^EQUALS = =$/;" m +EQUALS build/src/service/Makefile /^EQUALS = =$/;" m +EQUALS build/src/testutil/Makefile /^EQUALS = =$/;" m +EQUALS build/src/tools/Makefile /^EQUALS = =$/;" m +EQUALS build/src/udf/Makefile /^EQUALS = =$/;" m +EQUALS build/src/udf_samples/Makefile /^EQUALS = =$/;" m +EQUALS build/src/util/Makefile /^EQUALS = =$/;" m +ERROR src/exec/olap_scan_node.h /^ ERROR = 7$/;" e enum:doris::TransferStatus +ERROR src/gutil/port.h 779;" d +ERROR_COL_DATA_IS_ARRAY src/exec/es/es_scroll_parser.cpp /^static const string ERROR_COL_DATA_IS_ARRAY = "Data source returned an array for the type $0"$/;" m namespace:doris file: +ERROR_FILE_NAME src/runtime/load_path_mgr.cpp /^const std::string ERROR_FILE_NAME = "error_log";$/;" m namespace:doris file: +ERROR_FILE_NAME src/runtime/runtime_state.cpp /^const std::string ERROR_FILE_NAME = "error_log";$/;" m namespace:doris file: +ERROR_FILE_PREFIX src/runtime/etl_job_mgr.cpp /^const std::string ERROR_FILE_PREFIX = "error_log";$/;" m namespace:doris file: +ERROR_INVALID_COL_DATA src/exec/es/es_scroll_parser.cpp /^static const string ERROR_INVALID_COL_DATA = "Data source returned inconsistent column data. "$/;" m namespace:doris file: +ERROR_INVALID_COL_DATA src/exec/es_scan_node.cpp /^const string ERROR_INVALID_COL_DATA = "Data source returned inconsistent column data. "$/;" m namespace:doris file: +ERROR_LOG src/http/download_action.h /^ ERROR_LOG = 2,$/;" e enum:doris::DownloadAction::DOWNLOAD_TYPE +ERROR_LOG_PREFIX src/olap/olap_define.h /^static const std::string ERROR_LOG_PREFIX = "\/error_log";$/;" m namespace:doris +ERROR_MEM_LIMIT_EXCEEDED src/exec/es/es_scroll_parser.cpp /^static const string ERROR_MEM_LIMIT_EXCEEDED = "DataSourceScanNode::$0() failed to allocate "$/;" m namespace:doris file: +ERROR_MEM_LIMIT_EXCEEDED src/exec/es_scan_node.cpp /^const string ERROR_MEM_LIMIT_EXCEEDED = "DataSourceScanNode::$0() failed to allocate "$/;" m namespace:doris file: +ESFunctions src/exprs/es_functions.h /^class ESFunctions {$/;" c namespace:doris +ESQueryBuilder src/exec/es/es_query_builder.cpp /^ESQueryBuilder::ESQueryBuilder(const ExtFunction& es_query) {$/;" f class:doris::ESQueryBuilder +ESQueryBuilder src/exec/es/es_query_builder.cpp /^ESQueryBuilder::ESQueryBuilder(const std::string& es_query_str) : _es_query_str(es_query_str) {$/;" f class:doris::ESQueryBuilder +ESQueryBuilder src/exec/es/es_query_builder.h /^class ESQueryBuilder : public QueryBuilder {$/;" c namespace:doris +ESScanReader src/exec/es/es_scan_reader.cpp /^ESScanReader::ESScanReader(const std::string& target, const std::map& props) : _scroll_keep_alive(config::es_scroll_keepalive), _http_timeout_ms(config::es_http_timeout_ms) {$/;" f class:doris::ESScanReader +ESScanReader src/exec/es/es_scan_reader.h /^class ESScanReader {$/;" c namespace:doris +ESScrollQueryBuilder src/exec/es/es_scroll_query.cpp /^ESScrollQueryBuilder::ESScrollQueryBuilder() {$/;" f class:doris::ESScrollQueryBuilder +ESScrollQueryBuilder src/exec/es/es_scroll_query.h /^class ESScrollQueryBuilder {$/;" c namespace:doris +ETAG src/http/http_headers.cpp /^const char* HttpHeaders::ETAG = "ETag";$/;" m class:doris::HttpHeaders file: +ETAG src/http/http_headers.h /^ static const char* ETAG;$/;" m class:doris::HttpHeaders +ETIMEDOUT src/gutil/port.h 942;" d +EWOULDBLOCK src/gutil/port.h 940;" d +EXFULL src/gutil/port.h 224;" d +EXFULL src/gutil/port.h 937;" d +EXIT_IF_ERROR src/common/status.h 267;" d +EXPECT src/http/http_headers.cpp /^const char* HttpHeaders::EXPECT = "Expect";$/;" m class:doris::HttpHeaders file: +EXPECT src/http/http_headers.h /^ static const char* EXPECT;$/;" m class:doris::HttpHeaders +EXPECTATION_FAILED src/http/http_status.h /^ EXPECTATION_FAILED = 417,$/;" e enum:doris::HttpStatus +EXPIRES src/http/http_headers.cpp /^const char* HttpHeaders::EXPIRES = "Expires";$/;" m class:doris::HttpHeaders file: +EXPIRES src/http/http_headers.h /^ static const char* EXPIRES;$/;" m class:doris::HttpHeaders +EXPORTER_MAX_ERROR_NUM src/util/mysql_load_error_hub.h /^ static const int32_t EXPORTER_MAX_ERROR_NUM = 50;$/;" m class:doris::MysqlLoadErrorHub +EXPORTER_MAX_LINE_SIZE src/util/mysql_load_error_hub.h /^ static const int32_t EXPORTER_MAX_LINE_SIZE = 500;$/;" m class:doris::MysqlLoadErrorHub +EXPORTER_THRESHOLD src/util/broker_load_error_hub.h /^ static const int32_t EXPORTER_THRESHOLD = 20;$/;" m class:doris::BrokerLoadErrorHub +EXPORTER_THRESHOLD src/util/mysql_load_error_hub.h /^ static const int32_t EXPORTER_THRESHOLD = 100;$/;" m class:doris::MysqlLoadErrorHub +E_DEC_BAD_NUM src/runtime/decimal_value.h /^ E_DEC_BAD_NUM = 8,$/;" e enum:doris::DecimalError +E_DEC_DIV_ZERO src/runtime/decimal_value.h /^ E_DEC_DIV_ZERO = 4,$/;" e enum:doris::DecimalError +E_DEC_ERROR src/runtime/decimal_value.h /^ E_DEC_ERROR = 31,$/;" e enum:doris::DecimalError +E_DEC_FATAL_ERROR src/runtime/decimal_value.h /^ E_DEC_FATAL_ERROR = 30$/;" e enum:doris::DecimalError +E_DEC_OK src/runtime/decimal_value.h /^ E_DEC_OK = 0,$/;" e enum:doris::DecimalError +E_DEC_OOM src/runtime/decimal_value.h /^ E_DEC_OOM = 16,$/;" e enum:doris::DecimalError +E_DEC_OVERFLOW src/runtime/decimal_value.h /^ E_DEC_OVERFLOW = 2,$/;" e enum:doris::DecimalError +E_DEC_TRUNCATED src/runtime/decimal_value.h /^ E_DEC_TRUNCATED = 1,$/;" e enum:doris::DecimalError +Earliest src/util/monotime.cpp /^const MonoTime& MonoTime::Earliest(const MonoTime& a, const MonoTime& b) {$/;" f class:doris::MonoTime +EatAChar src/gutil/strings/numbers.cc /^static inline char EatAChar(const char** text, int* len,$/;" f file: +EatADouble src/gutil/strings/numbers.cc /^static inline bool EatADouble(const char** text, int* len, bool allow_question,$/;" f file: +EatSameChars src/gutil/strings/util.cc /^static void EatSameChars(const CHAR** pattern, const CHAR* pattern_end,$/;" f file: +EatWildcard src/gutil/strings/util.cc /^static void EatWildcard(const CHAR** pattern, const CHAR* end, NEXT next) {$/;" f file: +EightBase32DigitsToFiveBytes src/gutil/strings/escaping.cc /^void EightBase32DigitsToFiveBytes(const char *in, unsigned char *bytes_out) {$/;" f namespace:strings +EightBase32DigitsToTenHexDigits src/gutil/strings/escaping.cc /^void EightBase32DigitsToTenHexDigits(const char *in, char *out) {$/;" f namespace:strings +ElementDeleter src/gutil/stl_util.h /^ explicit ElementDeleter(STLContainer *ptr)$/;" f class:ElementDeleter +ElementDeleter src/gutil/stl_util.h /^class ElementDeleter {$/;" c +ElementVector src/common/object_pool.h /^ typedef std::vector ElementVector;$/;" t class:doris::ObjectPool +EmplaceIfNotPresent src/gutil/map-util.h /^bool EmplaceIfNotPresent(Collection* const collection,$/;" f +EmplaceOrDie src/gutil/map-util.h /^void EmplaceOrDie(Collection* const collection,$/;" f +EmplaceOrUpdate src/gutil/map-util.h /^bool EmplaceOrUpdate(Collection* const collection,$/;" f +EmplaceValuesFromMap src/gutil/map-util.h /^void EmplaceValuesFromMap(MapContainer&& map_container,$/;" f +EmptyBuckets src/exec/new_partitioned_hash_table.h /^ int64_t EmptyBuckets() const { return num_buckets_ - num_filled_buckets_; }$/;" f class:doris::NewPartitionedHashTable +EmptyLabels src/util/metrics.cpp /^MetricLabels MetricLabels::EmptyLabels;$/;" m class:doris::MetricLabels file: +EmptyLabels src/util/metrics.h /^ static MetricLabels EmptyLabels;$/;" m struct:doris::MetricLabels +EmptySegmentIterator src/olap/rowset/segment_v2/empty_segment_iterator.cpp /^EmptySegmentIterator::EmptySegmentIterator(const doris::Schema &schema): _schema(schema) {}$/;" f class:doris::segment_v2::EmptySegmentIterator +EmptySegmentIterator src/olap/rowset/segment_v2/empty_segment_iterator.h /^class EmptySegmentIterator : public RowwiseIterator {$/;" c namespace:doris::segment_v2 +EmptySetNode src/exec/empty_set_node.cpp /^EmptySetNode::EmptySetNode(ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::EmptySetNode +EmptySetNode src/exec/empty_set_node.h /^class EmptySetNode : public ExecNode {$/;" c namespace:doris +EncodeKey test/olap/lru_cache_test.cpp /^const CacheKey EncodeKey(std::string* result, int k) {$/;" f namespace:doris +EncodeValue test/olap/lru_cache_test.cpp /^static void* EncodeValue(uintptr_t v) {$/;" f namespace:doris +EncodingInfo src/olap/rowset/segment_v2/encoding_info.cpp /^EncodingInfo::EncodingInfo(TraitsClass traits)$/;" f class:doris::segment_v2::EncodingInfo +EncodingInfo src/olap/rowset/segment_v2/encoding_info.h /^class EncodingInfo {$/;" c namespace:doris::segment_v2 +EncodingInfoResolver src/olap/rowset/segment_v2/encoding_info.cpp /^EncodingInfoResolver::EncodingInfoResolver() {$/;" f class:doris::segment_v2::EncodingInfoResolver +EncodingInfoResolver src/olap/rowset/segment_v2/encoding_info.cpp /^class EncodingInfoResolver {$/;" c namespace:doris::segment_v2 file: +EncodingInfoTest test/olap/rowset/segment_v2/encoding_info_test.cpp /^ EncodingInfoTest() { }$/;" f class:doris::segment_v2::EncodingInfoTest +EncodingInfoTest test/olap/rowset/segment_v2/encoding_info_test.cpp /^class EncodingInfoTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +EncodingMapHash src/olap/rowset/segment_v2/encoding_info.cpp /^struct EncodingMapHash {$/;" s namespace:doris::segment_v2 file: +EncodingTraits src/olap/rowset/segment_v2/encoding_info.cpp /^struct EncodingTraits : TypeEncodingTraits::CppType> {$/;" s namespace:doris::segment_v2 file: +EncodingType src/olap/rowset/run_length_integer_writer.h /^ enum EncodingType {$/;" g class:doris::RunLengthIntegerWriter +EncryptionFunctions src/exprs/encryption_functions.h /^class EncryptionFunctions {$/;" c namespace:doris +End src/exec/new_partitioned_hash_table.h /^ Iterator End() { return Iterator(); }$/;" f class:doris::NewPartitionedHashTable +End src/exec/partitioned_hash_table.h /^ Iterator End() { return Iterator(); }$/;" f class:doris::PartitionedHashTable +EndOfFile src/common/status.h /^ static Status EndOfFile(const Slice& msg,$/;" f class:doris::Status +EngineAlterTabletTask src/olap/task/engine_alter_tablet_task.cpp /^EngineAlterTabletTask::EngineAlterTabletTask(const TAlterTabletReqV2& request,$/;" f class:doris::EngineAlterTabletTask +EngineAlterTabletTask src/olap/task/engine_alter_tablet_task.h /^class EngineAlterTabletTask : public EngineTask {$/;" c namespace:doris +EngineBatchLoadTask src/olap/task/engine_batch_load_task.cpp /^EngineBatchLoadTask::EngineBatchLoadTask(TPushReq& push_req, $/;" f class:doris::EngineBatchLoadTask +EngineBatchLoadTask src/olap/task/engine_batch_load_task.h /^class EngineBatchLoadTask : public EngineTask {$/;" c namespace:doris +EngineChecksumTask src/olap/task/engine_checksum_task.cpp /^EngineChecksumTask::EngineChecksumTask(TTabletId tablet_id, TSchemaHash schema_hash, $/;" f class:doris::EngineChecksumTask +EngineChecksumTask src/olap/task/engine_checksum_task.h /^class EngineChecksumTask : public EngineTask {$/;" c namespace:doris +EngineCloneTask src/olap/task/engine_clone_task.cpp /^EngineCloneTask::EngineCloneTask(const TCloneReq& clone_req,$/;" f class:doris::EngineCloneTask +EngineCloneTask src/olap/task/engine_clone_task.h /^class EngineCloneTask : public EngineTask {$/;" c namespace:doris +EngineOptions src/olap/options.h /^struct EngineOptions {$/;" s namespace:doris +EnginePublishVersionTask src/olap/task/engine_publish_version_task.cpp /^EnginePublishVersionTask::EnginePublishVersionTask(TPublishVersionRequest& publish_version_req,$/;" f class:doris::EnginePublishVersionTask +EnginePublishVersionTask src/olap/task/engine_publish_version_task.h /^class EnginePublishVersionTask : public EngineTask {$/;" c namespace:doris +EngineStorageMigrationTask src/olap/task/engine_storage_migration_task.cpp /^EngineStorageMigrationTask::EngineStorageMigrationTask(TStorageMediumMigrateReq& storage_medium_migrate_req) :$/;" f class:doris::EngineStorageMigrationTask +EngineStorageMigrationTask src/olap/task/engine_storage_migration_task.h /^class EngineStorageMigrationTask : public EngineTask {$/;" c namespace:doris +EngineTask src/olap/task/engine_task.h /^class EngineTask {$/;" c namespace:doris +EnsureRoomForAppend src/util/faststring.h /^ void EnsureRoomForAppend(size_t count) {$/;" f class:doris::faststring +Enter src/gutil/threading/thread_collision_warner.cc /^void ThreadCollisionWarner::Enter() {$/;" f class:base::ThreadCollisionWarner +EnterSelf src/gutil/threading/thread_collision_warner.cc /^void ThreadCollisionWarner::EnterSelf() {$/;" f class:base::ThreadCollisionWarner +EntrySlice src/olap/olap_index.h /^ EntrySlice() : data(nullptr), length(0) {}$/;" f struct:doris::EntrySlice +EntrySlice src/olap/olap_index.h /^struct EntrySlice {$/;" s namespace:doris +EnumClassHash src/olap/key_coder.cpp /^struct EnumClassHash {$/;" s namespace:doris file: +EnumToString src/olap/utils.h 344;" d +Env src/env/env.h /^ Env() { }$/;" f class:doris::Env +Env src/env/env.h /^class Env {$/;" c namespace:doris +EnvPosixTest test/env/env_posix_test.cpp /^ EnvPosixTest() { }$/;" f class:doris::EnvPosixTest +EnvPosixTest test/env/env_posix_test.cpp /^class EnvPosixTest : public testing::Test {$/;" c namespace:doris file: +Equal src/olap/skiplist.h /^ bool Equal(const Key& a, const Key& b) const { return (compare_(a, b) == 0); }$/;" f class:doris::SkipList +Equals src/exec/new_partitioned_hash_table.h /^ bool ALWAYS_INLINE Equals(TupleRow* build_row) const {$/;" f class:doris::NewPartitionedHashTableCtx +Equals src/util/monotime.cpp /^bool MonoDelta::Equals(const MonoDelta &rhs) const {$/;" f class:doris::MonoDelta +Equals src/util/monotime.cpp /^bool MonoTime::Equals(const MonoTime& other) const {$/;" f class:doris::MonoTime +Erase test/olap/lru_cache_test.cpp /^ void Erase(int key) {$/;" f class:doris::CacheTest +EraseKeyReturnValuePtr src/gutil/map-util.h /^typename Collection::mapped_type EraseKeyReturnValuePtr($/;" f +Errno src/olap/utils.h /^class Errno {$/;" c namespace:doris +ErrorMsg src/util/load_error_hub.h /^ ErrorMsg(int64_t id, const std::string& message) :$/;" f struct:doris::LoadErrorHub::ErrorMsg +ErrorMsg src/util/load_error_hub.h /^ struct ErrorMsg {$/;" s class:doris::LoadErrorHub +EsHttpScanNode src/exec/es_http_scan_node.cpp /^EsHttpScanNode::EsHttpScanNode($/;" f class:doris::EsHttpScanNode +EsHttpScanNode src/exec/es_http_scan_node.h /^class EsHttpScanNode : public ScanNode {$/;" c namespace:doris +EsHttpScanNodeTest test/exec/es_http_scan_node_test.cpp /^ EsHttpScanNodeTest() : _runtime_state(TQueryGlobals()) {$/;" f class:doris::EsHttpScanNodeTest +EsHttpScanNodeTest test/exec/es_http_scan_node_test.cpp /^class EsHttpScanNodeTest : public testing::Test {$/;" c namespace:doris file: +EsHttpScanner src/exec/es_http_scanner.cpp /^EsHttpScanner::EsHttpScanner($/;" f class:doris::EsHttpScanner +EsHttpScanner src/exec/es_http_scanner.h /^class EsHttpScanner {$/;" c namespace:doris +EsPredicate src/exec/es/es_predicate.cpp /^EsPredicate::EsPredicate(ExprContext* context,$/;" f class:doris::EsPredicate +EsPredicate src/exec/es/es_predicate.h /^ EsPredicate(const std::vector& all_predicates) {$/;" f class:doris::EsPredicate +EsPredicate src/exec/es/es_predicate.h /^class EsPredicate {$/;" c namespace:doris +EsPredicateTest test/exec/es_predicate_test.cpp /^ EsPredicateTest() : _runtime_state(TQueryGlobals()) {$/;" f class:doris::EsPredicateTest +EsPredicateTest test/exec/es_predicate_test.cpp /^class EsPredicateTest : public testing::Test {$/;" c namespace:doris file: +EsScanCounter src/exec/es_http_scanner.h /^ EsScanCounter() : num_rows_returned(0), num_rows_filtered(0) {$/;" f struct:doris::EsScanCounter +EsScanCounter src/exec/es_http_scanner.h /^struct EsScanCounter {$/;" s namespace:doris +EsScanNode src/exec/es_scan_node.cpp /^EsScanNode::EsScanNode($/;" f class:doris::EsScanNode +EsScanNode src/exec/es_scan_node.h /^class EsScanNode : public ScanNode {$/;" c namespace:doris +EsScanNodeTest test/exec/es_scan_node_test.cpp /^ EsScanNodeTest() : _runtime_state(TQueryGlobals()) {$/;" f class:doris::EsScanNodeTest +EsScanNodeTest test/exec/es_scan_node_test.cpp /^class EsScanNodeTest : public testing::Test {$/;" c namespace:doris file: +EsTableDescriptor src/runtime/descriptors.cpp /^EsTableDescriptor::EsTableDescriptor(const TTableDescriptor& tdesc)$/;" f class:doris::EsTableDescriptor +EsTableDescriptor src/runtime/descriptors.h /^class EsTableDescriptor : public TableDescriptor {$/;" c namespace:doris +EscapeFileName src/gutil/strings/escaping.cc /^void EscapeFileName(const StringPiece& src, string* dst) {$/;" f namespace:strings +EscapeFileName src/gutil/strings/escaping.h /^inline string EscapeFileName(const StringPiece& src) {$/;" f namespace:strings +EscapeStrForCSV src/gutil/strings/escaping.cc /^int EscapeStrForCSV(const char* src, char* dest, int dest_len) {$/;" f namespace:strings +EstimateCyclesPerSecond src/gutil/sysinfo.cc /^static int64 EstimateCyclesPerSecond(const int estimate_time_ms) {$/;" f namespace:base +EstimateNumBuckets src/exec/new_partitioned_hash_table.h /^ static int64_t EstimateNumBuckets(int64_t num_rows) {$/;" f class:doris::NewPartitionedHashTable +EstimateNumBuckets src/exec/partitioned_hash_table.h /^ static int64_t EstimateNumBuckets(int64_t num_rows) {$/;" f class:doris::PartitionedHashTable +EstimateSize src/exec/new_partitioned_hash_table.h /^ static int64_t EstimateSize(int64_t num_rows) {$/;" f class:doris::NewPartitionedHashTable +EstimateSize src/exec/partitioned_hash_table.h /^ static int64_t EstimateSize(int64_t num_rows) {$/;" f class:doris::PartitionedHashTable +EtlJobCtx src/runtime/etl_job_mgr.h /^struct EtlJobCtx {$/;" s namespace:doris +EtlJobMgr src/runtime/etl_job_mgr.cpp /^EtlJobMgr::EtlJobMgr(ExecEnv* exec_env) :$/;" f class:doris::EtlJobMgr +EtlJobMgr src/runtime/etl_job_mgr.h /^class EtlJobMgr : public RestMonitorIface {$/;" c namespace:doris +EtlJobMgrTest test/runtime/etl_job_mgr_test.cpp /^ EtlJobMgrTest() {$/;" f class:doris::EtlJobMgrTest +EtlJobMgrTest test/runtime/etl_job_mgr_test.cpp /^class EtlJobMgrTest : public testing::Test {$/;" c namespace:doris file: +EtlJobResult src/runtime/etl_job_mgr.h /^ EtlJobResult() :$/;" f struct:doris::EtlJobResult +EtlJobResult src/runtime/etl_job_mgr.h /^struct EtlJobResult {$/;" s namespace:doris +EvHttpServer src/http/ev_http_server.cpp /^EvHttpServer::EvHttpServer(const std::string& host, int port, int num_workers)$/;" f class:doris::EvHttpServer +EvHttpServer src/http/ev_http_server.cpp /^EvHttpServer::EvHttpServer(int port, int num_workers)$/;" f class:doris::EvHttpServer +EvHttpServer src/http/ev_http_server.h /^class EvHttpServer {$/;" c namespace:doris +EvalAndHashBuild src/exec/new_partitioned_hash_table.inline.h /^inline bool NewPartitionedHashTableCtx::EvalAndHashBuild(TupleRow* row) {$/;" f class:doris::NewPartitionedHashTableCtx +EvalAndHashPrefetchGroup src/exec/new_partitioned_aggregation_node_ir.cc /^void IR_ALWAYS_INLINE NewPartitionedAggregationNode::EvalAndHashPrefetchGroup($/;" f class:NewPartitionedAggregationNode +EvalAndHashProbe src/exec/new_partitioned_hash_table.inline.h /^inline bool NewPartitionedHashTableCtx::EvalAndHashProbe(TupleRow* row) {$/;" f class:doris::NewPartitionedHashTableCtx +EvalBuildRow src/exec/partitioned_hash_table.h /^ bool IR_NO_INLINE EvalBuildRow(TupleRow* row) {$/;" f class:doris::PartitionedHashTableCtx +EvalConjunctsFn src/exec/exec_node.h /^ typedef bool (*EvalConjunctsFn)(ExprContext* const* ctxs, int num_ctxs, TupleRow* row);$/;" t class:doris::ExecNode +EvalProbeRow src/exec/partitioned_hash_table.h /^ bool IR_NO_INLINE EvalProbeRow(TupleRow* row) {$/;" f class:doris::PartitionedHashTableCtx +Event src/util/runtime_profile.h /^ typedef std::pair Event;$/;" t class:doris::RuntimeProfile::EventSequence +EventList src/util/runtime_profile.h /^ typedef std::vector EventList;$/;" t class:doris::RuntimeProfile::EventSequence +EventSequence src/util/runtime_profile.h /^ EventSequence() { }$/;" f class:doris::RuntimeProfile::EventSequence +EventSequence src/util/runtime_profile.h /^ class EventSequence {$/;" c class:doris::RuntimeProfile +EventSequenceMap src/util/runtime_profile.h /^ typedef std::map EventSequenceMap;$/;" t class:doris::RuntimeProfile +ExceptMaterializeBatchFn src/exec/except_node.h /^ typedef void (*ExceptMaterializeBatchFn)(ExceptNode*, RowBatch*, uint8_t**);$/;" t class:doris::ExceptNode +ExceptNode src/exec/except_node.cpp /^ExceptNode::ExceptNode(ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::ExceptNode +ExceptNode src/exec/except_node.h /^class ExceptNode : public ExecNode {$/;" c namespace:doris +ExchangeNode src/exec/exchange_node.cpp /^ExchangeNode::ExchangeNode($/;" f class:doris::ExchangeNode +ExchangeNode src/exec/exchange_node.h /^class ExchangeNode : public ExecNode {$/;" c namespace:doris +ExchangeNode src/exec/pl_task_root.cpp /^ExchangeNode::ExchangeNode($/;" f class:doris::ExchangeNode +ExecEnv src/runtime/exec_env.cpp /^ExecEnv::ExecEnv() {$/;" f class:doris::ExecEnv +ExecEnv src/runtime/exec_env.h /^class ExecEnv {$/;" c namespace:doris +ExecFinishCallback src/runtime/routine_load/routine_load_task_executor.h /^ typedef std::function ExecFinishCallback; $/;" t class:doris::RoutineLoadTaskExecutor +ExecNode src/exec/exec_node.cpp /^ExecNode::ExecNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) :$/;" f class:doris::ExecNode +ExecNode src/exec/exec_node.h /^class ExecNode {$/;" c namespace:doris +ExecutionMode src/util/threadpool.h /^ enum class ExecutionMode {$/;" c class:doris::ThreadPool +ExistsQueryBuilder src/exec/es/es_query_builder.cpp /^ExistsQueryBuilder::ExistsQueryBuilder(const ExtIsNullPredicate& is_null_predicate) : _field(is_null_predicate.col.name) {$/;" f class:doris::ExistsQueryBuilder +ExistsQueryBuilder src/exec/es/es_query_builder.h /^class ExistsQueryBuilder : public QueryBuilder {$/;" c namespace:doris +ExpectedPinCount src/runtime/buffered_tuple_stream3.cc /^int BufferedTupleStream3::ExpectedPinCount(bool stream_pinned, const Page* page) const {$/;" f class:BufferedTupleStream3 +ExpliclitLengthValueType src/olap/hll.h /^ typedef uint8_t ExpliclitLengthValueType;$/;" t class:doris::HllSetResolver +ExportSink src/runtime/export_sink.cpp /^ExportSink::ExportSink(ObjectPool* pool,$/;" f class:doris::ExportSink +ExportSink src/runtime/export_sink.h /^class ExportSink : public DataSink {$/;" c namespace:doris +ExportTaskCtx src/runtime/export_task_mgr.h /^struct ExportTaskCtx {$/;" s namespace:doris +ExportTaskMgr src/runtime/export_task_mgr.cpp /^ExportTaskMgr::ExportTaskMgr(ExecEnv* exec_env) :$/;" f class:doris::ExportTaskMgr +ExportTaskMgr src/runtime/export_task_mgr.h /^class ExportTaskMgr {$/;" c namespace:doris +ExportTaskMgrTest test/runtime/export_task_mgr_test.cpp /^ ExportTaskMgrTest() {$/;" f class:doris::ExportTaskMgrTest +ExportTaskMgrTest test/runtime/export_task_mgr_test.cpp /^class ExportTaskMgrTest : public testing::Test {$/;" c namespace:doris file: +ExportTaskResult src/runtime/export_task_mgr.h /^struct ExportTaskResult {$/;" s namespace:doris +Expr src/exprs/expr.cpp /^Expr::Expr(const Expr& expr) $/;" f class:doris::Expr +Expr src/exprs/expr.cpp /^Expr::Expr(const TExprNode& node) :$/;" f class:doris::Expr +Expr src/exprs/expr.cpp /^Expr::Expr(const TExprNode& node, bool is_slotref) :$/;" f class:doris::Expr +Expr src/exprs/expr.cpp /^Expr::Expr(const TypeDescriptor& type) :$/;" f class:doris::Expr +Expr src/exprs/expr.cpp /^Expr::Expr(const TypeDescriptor& type, bool is_slotref) :$/;" f class:doris::Expr +Expr src/exprs/expr.h /^class Expr {$/;" c namespace:doris +ExprConstant src/exprs/expr.h /^ enum ExprConstant {$/;" g class:doris::Expr +ExprContext src/exprs/expr_context.cpp /^ExprContext::ExprContext(Expr* root) :$/;" f class:doris::ExprContext +ExprContext src/exprs/expr_context.h /^class ExprContext {$/;" c namespace:doris +ExprValue src/exec/new_partitioned_hash_table.h /^ void* ALWAYS_INLINE ExprValue(int expr_idx) const {$/;" f class:doris::NewPartitionedHashTableCtx +ExprValue src/exprs/expr_value.h /^ ExprValue() : $/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^ ExprValue(__int128 value) : large_int_val(value) {}$/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^ ExprValue(bool v): bool_val(v) {}$/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^ ExprValue(const std::string& str) : $/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^ ExprValue(double v): double_val(v) {}$/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^ ExprValue(float v): float_val(v) {}$/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^ ExprValue(int16_t v): smallint_val(v) {}$/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^ ExprValue(int32_t v): int_val(v) {}$/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^ ExprValue(int64_t i, int32_t f) : decimal_val(i, f), decimalv2_val(i, f) {}$/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^ ExprValue(int64_t v): bigint_val(v) {}$/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^ ExprValue(int8_t v): tinyint_val(v) {}$/;" f struct:doris::ExprValue +ExprValue src/exprs/expr_value.h /^struct ExprValue {$/;" s namespace:doris +ExprValueNull src/exec/new_partitioned_hash_table.h /^ bool ALWAYS_INLINE ExprValueNull(int expr_idx) const {$/;" f class:doris::NewPartitionedHashTableCtx +ExprValuePtr src/exec/new_partitioned_hash_table.cc /^const uint8_t* NewPartitionedHashTableCtx::ExprValuesCache::ExprValuePtr($/;" f class:NewPartitionedHashTableCtx::ExprValuesCache +ExprValuePtr src/exec/new_partitioned_hash_table.cc /^uint8_t* NewPartitionedHashTableCtx::ExprValuesCache::ExprValuePtr($/;" f class:NewPartitionedHashTableCtx::ExprValuesCache +ExprValuesCache src/exec/new_partitioned_hash_table.cc /^NewPartitionedHashTableCtx::ExprValuesCache::ExprValuesCache()$/;" f class:NewPartitionedHashTableCtx::ExprValuesCache +ExprValuesCache src/exec/new_partitioned_hash_table.h /^ class ExprValuesCache {$/;" c class:doris::NewPartitionedHashTableCtx +ExtBinaryPredicate src/exec/es/es_predicate.h /^ ExtBinaryPredicate($/;" f struct:doris::ExtBinaryPredicate +ExtBinaryPredicate src/exec/es/es_predicate.h /^struct ExtBinaryPredicate : public ExtPredicate {$/;" s namespace:doris +ExtColumnDesc src/exec/es/es_predicate.h /^ ExtColumnDesc(const std::string& name, const TypeDescriptor& type) :$/;" f struct:doris::ExtColumnDesc +ExtColumnDesc src/exec/es/es_predicate.h /^struct ExtColumnDesc {$/;" s namespace:doris +ExtDataSourceServiceClientCache src/runtime/client_cache.h /^typedef ClientCache ExtDataSourceServiceClientCache;$/;" t namespace:doris +ExtDataSourceServiceConnection src/runtime/client_cache.h /^typedef ClientConnection ExtDataSourceServiceConnection;$/;" t namespace:doris +ExtFunction src/exec/es/es_predicate.h /^ ExtFunction(TExprNodeType::type node_type,$/;" f struct:doris::ExtFunction +ExtFunction src/exec/es/es_predicate.h /^struct ExtFunction : public ExtPredicate {$/;" s namespace:doris +ExtInPredicate src/exec/es/es_predicate.h /^ ExtInPredicate($/;" f struct:doris::ExtInPredicate +ExtInPredicate src/exec/es/es_predicate.h /^struct ExtInPredicate : public ExtPredicate {$/;" s namespace:doris +ExtIsNullPredicate src/exec/es/es_predicate.h /^ ExtIsNullPredicate($/;" f struct:doris::ExtIsNullPredicate +ExtIsNullPredicate src/exec/es/es_predicate.h /^struct ExtIsNullPredicate : public ExtPredicate {$/;" s namespace:doris +ExtLikePredicate src/exec/es/es_predicate.h /^ ExtLikePredicate($/;" f struct:doris::ExtLikePredicate +ExtLikePredicate src/exec/es/es_predicate.h /^struct ExtLikePredicate : public ExtPredicate {$/;" s namespace:doris +ExtLiteral src/exec/es/es_predicate.h /^ ExtLiteral(PrimitiveType type, void *value) : $/;" f class:doris::ExtLiteral +ExtLiteral src/exec/es/es_predicate.h /^class ExtLiteral {$/;" c namespace:doris +ExtPredicate src/exec/es/es_predicate.h /^ ExtPredicate(TExprNodeType::type node_type) : node_type(node_type) {$/;" f struct:doris::ExtPredicate +ExtPredicate src/exec/es/es_predicate.h /^struct ExtPredicate {$/;" s namespace:doris +Extend src/util/crc32c.cpp /^uint32_t Extend(uint32_t crc, const char* buf, size_t size) {$/;" f namespace:doris::crc32c +ExtendImpl src/util/crc32c.cpp /^uint32_t ExtendImpl(uint32_t crc, const char* buf, size_t size) {$/;" f namespace:doris::crc32c +ExtendScanKeyVisitor src/exec/olap_scan_node.h /^ ExtendScanKeyVisitor(OlapScanKeys& scan_keys) : _scan_keys(scan_keys) { }$/;" f class:doris::OlapScanNode::ExtendScanKeyVisitor +ExtendScanKeyVisitor src/exec/olap_scan_node.h /^ class ExtendScanKeyVisitor : public boost::static_visitor {$/;" c class:doris::OlapScanNode +ExternalScanContextMgr src/runtime/external_scan_context_mgr.cpp /^ExternalScanContextMgr::ExternalScanContextMgr(ExecEnv* exec_env) : _exec_env(exec_env), _is_stop(false), _scan_context_gc_interval_min(doris::config::scan_context_gc_interval_min) {$/;" f class:doris::ExternalScanContextMgr +ExternalScanContextMgr src/runtime/external_scan_context_mgr.h /^class ExternalScanContextMgr {$/;" c namespace:doris +ExternalScanContextMgrTest test/runtime/external_scan_context_mgr_test.cpp /^ ExternalScanContextMgrTest() {$/;" f class:doris::ExternalScanContextMgrTest +ExternalScanContextMgrTest test/runtime/external_scan_context_mgr_test.cpp /^class ExternalScanContextMgrTest : public testing::Test {$/;" c namespace:doris file: +ExtractBuffer src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::ExtractBuffer($/;" f class:doris::BufferPool +FACTORIAL_FN src/exprs/operators.cpp 52;" d file: +FALLTHROUGH_INTENDED src/gutil/macros.h 257;" d +FALLTHROUGH_INTENDED src/gutil/macros.h 262;" d +FETCH_DATA src/olap/tablet_sync_service.h 28;" d +FIELD_HITS src/exec/es/es_scroll_parser.cpp /^static const char* FIELD_HITS = "hits";$/;" m namespace:doris file: +FIELD_INNER_HITS src/exec/es/es_scroll_parser.cpp /^static const char* FIELD_INNER_HITS = "hits";$/;" m namespace:doris file: +FIELD_SCROLL_ID src/exec/es/es_scroll_parser.cpp /^static const char* FIELD_SCROLL_ID = "_scroll_id";$/;" m namespace:doris file: +FIELD_SOURCE src/exec/es/es_scroll_parser.cpp /^static const char* FIELD_SOURCE = "_source";$/;" m namespace:doris file: +FIELD_TOTAL src/exec/es/es_scroll_parser.cpp /^static const char* FIELD_TOTAL = "total";$/;" m namespace:doris file: +FIFTEEN src/olap/serialize.h /^ THIRTEEN, FOURTEEN, FIFTEEN, SIXTEEN, SEVENTEEN, EIGHTEEN, NINETEEN,$/;" e enum:doris::ser::FixedBitSize +FIFTYSIX src/olap/serialize.h /^ TWENTYEIGHT, THIRTY, THIRTYTWO, FORTY, FORTYEIGHT, FIFTYSIX, SIXTYFOUR$/;" e enum:doris::ser::FixedBitSize +FILE src/util/url_parser.h /^ FILE,$/;" e enum:doris::UrlParser::UrlPart +FILE_PARAMETER src/http/download_action.cpp /^const std::string FILE_PARAMETER = "file";$/;" m namespace:doris file: +FILE_PATH_KEY src/http/action/mini_load.cpp /^const std::string FILE_PATH_KEY = "file_path";$/;" m namespace:doris file: +FILTER_IN src/exec/olap_utils.h /^ FILTER_IN = 4,$/;" e enum:doris::SQLFilterOp +FILTER_LARGER src/exec/olap_utils.h /^ FILTER_LARGER = 0,$/;" e enum:doris::SQLFilterOp +FILTER_LARGER_OR_EQUAL src/exec/olap_utils.h /^ FILTER_LARGER_OR_EQUAL = 1,$/;" e enum:doris::SQLFilterOp +FILTER_LESS src/exec/olap_utils.h /^ FILTER_LESS = 2,$/;" e enum:doris::SQLFilterOp +FILTER_LESS_OR_EQUAL src/exec/olap_utils.h /^ FILTER_LESS_OR_EQUAL = 3,$/;" e enum:doris::SQLFilterOp +FILTER_NOT_IN src/exec/olap_utils.h /^ FILTER_NOT_IN = 5$/;" e enum:doris::SQLFilterOp +FINAL src/gutil/port.h 494;" d +FINAL src/service/brpc.h 36;" d +FINALIZED src/olap/fs/block_manager.h /^ FINALIZED,$/;" e enum:doris::fs::WritableBlock::State +FININSH src/exec/olap_scan_node.h /^ FININSH = 5,$/;" e enum:doris::TransferStatus +FIVE src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +FIXED_LEN_BUFFER_LIMIT src/runtime/row_batch.cpp /^const int RowBatch::FIXED_LEN_BUFFER_LIMIT = AT_CAPACITY_MEM_USAGE \/ 2;$/;" m class:doris::RowBatch file: +FIXED_LEN_BUFFER_LIMIT src/runtime/row_batch.h /^ static const int FIXED_LEN_BUFFER_LIMIT;$/;" m class:doris::RowBatch +FLOAT_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t FLOAT_SIZE = sizeof(float);$/;" m class:doris::AggFnEvaluator +FLOAT_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t FLOAT_SIZE = sizeof(float);$/;" m class:doris::NewAggFnEvaluator +FLOOR src/runtime/decimal_value.h /^ FLOOR = 4,$/;" e enum:doris::DecimalRoundMode +FLUSH_ASYNC src/env/env.h /^ FLUSH_ASYNC$/;" e enum:doris::RandomRWFile::FlushMode +FLUSH_ASYNC src/env/env.h /^ FLUSH_ASYNC$/;" e enum:doris::WritableFile::FlushMode +FLUSH_RESOURCES src/runtime/row_batch.h /^ FLUSH_RESOURCES,$/;" m class:doris::RowBatch::FlushMode +FLUSH_SYNC src/env/env.h /^ FLUSH_SYNC,$/;" e enum:doris::RandomRWFile::FlushMode +FLUSH_SYNC src/env/env.h /^ FLUSH_SYNC,$/;" e enum:doris::WritableFile::FlushMode +FNV64_PRIME src/util/hash_util.hpp /^ static const uint64_t FNV64_PRIME = 1099511628211UL;$/;" m class:doris::HashUtil +FNV64_SEED src/util/hash_util.hpp /^ static const uint64_t FNV64_SEED = 14695981039346656037UL;$/;" m class:doris::HashUtil +FNV_PRIME src/util/hash_util.hpp /^ static const uint32_t FNV_PRIME = 0x01000193; \/\/ 16777619$/;" m class:doris::HashUtil +FNV_SEED src/util/hash_util.hpp /^ static const uint32_t FNV_SEED = 0x811C9DC5; \/\/ 2166136261$/;" m class:doris::HashUtil +FOPEN_FUNC src/util/minizip/ioapi.c 19;" d file: +FOPEN_FUNC src/util/minizip/ioapi.c 23;" d file: +FORBIDDEN src/http/http_status.h /^ FORBIDDEN = 403,$/;" e enum:doris::HttpStatus +FORCE_INLINE src/util/murmur_hash3.cpp 19;" d file: +FORCE_INLINE src/util/murmur_hash3.cpp 32;" d file: +FOREACH_ROW src/runtime/row_batch.h 522;" d +FOREACH_ROW_LIMIT src/runtime/row_batch.h 526;" d +FORTY src/olap/serialize.h /^ TWENTYEIGHT, THIRTY, THIRTYTWO, FORTY, FORTYEIGHT, FIFTYSIX, SIXTYFOUR$/;" e enum:doris::ser::FixedBitSize +FORTYEIGHT src/olap/serialize.h /^ TWENTYEIGHT, THIRTY, THIRTYTWO, FORTY, FORTYEIGHT, FIFTYSIX, SIXTYFOUR$/;" e enum:doris::ser::FixedBitSize +FOUND src/http/http_status.h /^ FOUND = 302,$/;" e enum:doris::HttpStatus +FOUR src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +FOURTEEN src/olap/serialize.h /^ THIRTEEN, FOURTEEN, FIFTEEN, SIXTEEN, SEVENTEEN, EIGHTEEN, NINETEEN,$/;" e enum:doris::ser::FixedBitSize +FP_INFINITE src/gutil/port.h /^ FP_INFINITE, \/\/ is either plus or minus infinity.$/;" e enum:__anon4 +FP_NAN src/gutil/port.h /^ FP_NAN, \/\/ is "Not a Number"$/;" e enum:__anon4 +FP_NORMAL src/gutil/port.h /^ FP_NORMAL \/\/ if nothing of the above is correct that it must be a$/;" e enum:__anon4 +FP_SUBNORMAL src/gutil/port.h /^ FP_SUBNORMAL, \/\/ is too small to be represented in normalized format.$/;" e enum:__anon4 +FP_ZERO src/gutil/port.h /^ FP_ZERO,$/;" e enum:__anon4 +FRAC_RATIO src/olap/decimal12.h /^ static const int32_t FRAC_RATIO = 1000000000;$/;" m struct:doris::decimal12_t +FRAGMENT_LOCAL output/udf/include/udf.h /^ FRAGMENT_LOCAL,$/;" e enum:doris_udf::FunctionContext::FunctionStateScope +FRAGMENT_LOCAL src/udf/udf.h /^ FRAGMENT_LOCAL,$/;" e enum:doris_udf::FunctionContext::FunctionStateScope +FRAME_MIN_SIZE src/gutil/linux_syscall_support.h 2050;" d +FRAME_TOC_SAVE src/gutil/linux_syscall_support.h 2051;" d +FRAME_VALUE_NUM src/util/frame_of_reference_coding.h /^ static const uint8_t FRAME_VALUE_NUM = 128;$/;" m class:doris::ForEncoder +FROM src/http/http_headers.cpp /^const char* HttpHeaders::FROM = "From";$/;" m class:doris::HttpHeaders file: +FROM src/http/http_headers.h /^ static const char* FROM;$/;" m class:doris::HttpHeaders +FSEEKO src/gutil/port.h 617;" d +FSEEKO_FUNC src/util/minizip/ioapi.c 21;" d file: +FSEEKO_FUNC src/util/minizip/ioapi.c 25;" d file: +FTELLO src/gutil/port.h 616;" d +FTELLO_FUNC src/util/minizip/ioapi.c 20;" d file: +FTELLO_FUNC src/util/minizip/ioapi.c 24;" d file: +FUNC_PTR_TO_CHAR_PTR src/gutil/port.h 1215;" d +FUNC_PTR_TO_CHAR_PTR src/gutil/port.h 1218;" d +FUTEX_PRIVATE_FLAG src/gutil/spinlock_linux-inl.h 43;" d +FUTEX_WAIT src/gutil/spinlock_linux-inl.h 41;" d +FUTEX_WAKE src/gutil/spinlock_linux-inl.h 42;" d +F_ADLER32_C src/exec/decompressor.h /^ const static uint64_t F_ADLER32_C;$/;" m class:doris::LzopDecompressor +F_ADLER32_C src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_ADLER32_C = 0x00000002L;$/;" m class:doris::LzopDecompressor file: +F_ADLER32_D src/exec/decompressor.h /^ const static uint64_t F_ADLER32_D;$/;" m class:doris::LzopDecompressor +F_ADLER32_D src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_ADLER32_D = 0x00000001L;$/;" m class:doris::LzopDecompressor file: +F_CRC32_C src/exec/decompressor.h /^ const static uint64_t F_CRC32_C;$/;" m class:doris::LzopDecompressor +F_CRC32_C src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_CRC32_C = 0x00000200L;$/;" m class:doris::LzopDecompressor file: +F_CRC32_D src/exec/decompressor.h /^ const static uint64_t F_CRC32_D;$/;" m class:doris::LzopDecompressor +F_CRC32_D src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_CRC32_D = 0x00000100L;$/;" m class:doris::LzopDecompressor file: +F_CS_MASK src/exec/decompressor.h /^ const static uint64_t F_CS_MASK;$/;" m class:doris::LzopDecompressor +F_CS_MASK src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_CS_MASK = 0x00f00000L;$/;" m class:doris::LzopDecompressor file: +F_H_CRC32 src/exec/decompressor.h /^ const static uint64_t F_H_CRC32;$/;" m class:doris::LzopDecompressor +F_H_CRC32 src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_H_CRC32 = 0x00001000L;$/;" m class:doris::LzopDecompressor file: +F_H_EXTRA_FIELD src/exec/decompressor.h /^ const static uint64_t F_H_EXTRA_FIELD;$/;" m class:doris::LzopDecompressor +F_H_EXTRA_FIELD src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_H_EXTRA_FIELD = 0x00000040L;$/;" m class:doris::LzopDecompressor file: +F_H_FILTER src/exec/decompressor.h /^ const static uint64_t F_H_FILTER;$/;" m class:doris::LzopDecompressor +F_H_FILTER src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_H_FILTER = 0x00000800L;$/;" m class:doris::LzopDecompressor file: +F_MASK src/exec/decompressor.h /^ const static uint64_t F_MASK;$/;" m class:doris::LzopDecompressor +F_MASK src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_MASK = 0x00003FFFL;$/;" m class:doris::LzopDecompressor file: +F_MULTIPART src/exec/decompressor.h /^ const static uint64_t F_MULTIPART;$/;" m class:doris::LzopDecompressor +F_MULTIPART src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_MULTIPART = 0x00000400L;$/;" m class:doris::LzopDecompressor file: +F_OS_MASK src/exec/decompressor.h /^ const static uint64_t F_OS_MASK;$/;" m class:doris::LzopDecompressor +F_OS_MASK src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_OS_MASK = 0xff000000L;$/;" m class:doris::LzopDecompressor file: +F_RESERVED src/exec/decompressor.h /^ const static uint64_t F_RESERVED;$/;" m class:doris::LzopDecompressor +F_RESERVED src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::F_RESERVED = ((F_MASK | F_OS_MASK | F_CS_MASK) ^ 0xffffffffL);$/;" m class:doris::LzopDecompressor file: +FakeLock src/util/fake_lock.h /^ FakeLock() {}$/;" f class:doris::FakeLock +FakeLock src/util/fake_lock.h /^class FakeLock {$/;" c namespace:doris +FastHex32ToBuffer src/gutil/strings/numbers.cc /^char *FastHex32ToBuffer(uint32 value, char* buffer) {$/;" f +FastHex64ToBuffer src/gutil/strings/numbers.cc /^char *FastHex64ToBuffer(uint64 value, char* buffer) {$/;" f +FastHexToBuffer src/gutil/strings/numbers.cc /^char *FastHexToBuffer(int i, char* buffer) {$/;" f +FastInt32ToBuffer src/gutil/strings/numbers.cc /^char *FastInt32ToBuffer(int32 i, char* buffer) {$/;" f +FastInt32ToBufferLeft src/gutil/strings/numbers.cc /^char* FastInt32ToBufferLeft(int32 i, char* buffer) {$/;" f +FastInt64ToBuffer src/gutil/strings/numbers.cc /^char *FastInt64ToBuffer(int64 i, char* buffer) {$/;" f +FastInt64ToBufferLeft src/gutil/strings/numbers.cc /^char* FastInt64ToBufferLeft(int64 i, char* buffer) {$/;" f +FastIntToBuffer src/gutil/strings/numbers.h /^inline char* FastIntToBuffer(int i, char* buffer) {$/;" f +FastStringAppend src/gutil/strings/util.cc /^void FastStringAppend(string* s, const char* data, int len) {$/;" f +FastTimeToBuffer src/gutil/strings/util.cc /^char* FastTimeToBuffer(time_t s, char* buffer) {$/;" f +FastUInt32ToBuffer src/gutil/strings/numbers.h /^inline char* FastUInt32ToBuffer(uint32 i, char* buffer) {$/;" f +FastUInt32ToBufferLeft src/gutil/strings/numbers.cc /^char* FastUInt32ToBufferLeft(uint32 u, char* buffer) {$/;" f +FastUInt64ToBuffer src/gutil/strings/numbers.h /^inline char* FastUInt64ToBuffer(uint64 i, char* buffer) {$/;" f +FastUInt64ToBufferLeft src/gutil/strings/numbers.cc /^char* FastUInt64ToBufferLeft(uint64 u64, char* buffer) {$/;" f +FastUIntToBuffer src/gutil/strings/numbers.h /^inline char* FastUIntToBuffer(unsigned int i, char* buffer) {$/;" f +Fast_CRC32 src/util/crc32c.cpp /^static inline void Fast_CRC32(uint64_t* l, uint8_t const **p) {$/;" f namespace:doris::crc32c +FaststringTest test/util/faststring_test.cpp /^class FaststringTest : public ::testing::Test {};$/;" c namespace:doris file: +FetchRowsetMetaTask src/olap/tablet_sync_service.h /^struct FetchRowsetMetaTask {$/;" s namespace:doris +FetchTabletMetaTask src/olap/tablet_sync_service.h /^struct FetchTabletMetaTask {$/;" s namespace:doris +Field src/common/configbase.h /^ Field(const char* ftype, const char* fname, void* fstorage, const char* fdefval) :$/;" f struct:doris::config::Register::Field +Field src/common/configbase.h /^ struct Field {$/;" s class:doris::config::Register +Field src/olap/field.h /^ explicit Field(const TabletColumn& column)$/;" f class:doris::Field +Field src/olap/field.h /^class Field {$/;" c namespace:doris +FieldAggregationMethod src/olap/olap_common.h /^enum FieldAggregationMethod {$/;" g namespace:doris +FieldEqual src/olap/olap_cond.h /^struct FieldEqual {$/;" s namespace:doris +FieldFactory src/olap/field.h /^class FieldFactory {$/;" c namespace:doris +FieldHash src/olap/olap_cond.h /^struct FieldHash {$/;" s namespace:doris +FieldSet src/olap/olap_cond.h /^ typedef std::unordered_set FieldSet;$/;" t struct:doris::Cond +FieldType src/olap/olap_common.h /^enum FieldType {$/;" g namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits { };$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public FieldTypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public FieldTypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public BaseFieldtypeTraits {$/;" s namespace:doris +FieldTypeTraits src/olap/types.h /^struct FieldTypeTraits : public FieldTypeTraits {$/;" s namespace:doris +File src/runtime/tmp_file_mgr.cc /^TmpFileMgr::File::File(TmpFileMgr* mgr, DeviceId device_id, const string& path) :$/;" f class:doris::TmpFileMgr::File +File src/runtime/tmp_file_mgr.h /^ class File {$/;" c class:doris::TmpFileMgr +FileBlockManager src/olap/fs/file_block_manager.cpp /^FileBlockManager::FileBlockManager(Env* env, BlockManagerOptions opts) :$/;" f class:doris::fs::FileBlockManager +FileBlockManager src/olap/fs/file_block_manager.h /^class FileBlockManager : public BlockManager {$/;" c namespace:doris::fs +FileBlockManagerTest test/olap/fs/file_block_manager_test.cpp /^class FileBlockManagerTest : public testing::Test {$/;" c namespace:doris file: +FileCache src/util/file_cache.cpp /^FileCache::FileCache(const std::string& cache_name, int max_open_files) :$/;" f class:doris::FileCache +FileCache src/util/file_cache.h /^class FileCache {$/;" c namespace:doris +FileCacheTest test/util/file_cache_test.cpp /^ FileCacheTest() { }$/;" f class:doris::FileCacheTest +FileCacheTest test/util/file_cache_test.cpp /^class FileCacheTest : public testing::Test {$/;" c namespace:doris file: +FileColumnIterator src/olap/rowset/segment_v2/column_reader.cpp /^FileColumnIterator::FileColumnIterator(ColumnReader* reader) : _reader(reader) {$/;" f class:doris::segment_v2::FileColumnIterator +FileColumnIterator src/olap/rowset/segment_v2/column_reader.h /^class FileColumnIterator : public ColumnIterator {$/;" c namespace:doris::segment_v2 +FileCursor src/olap/file_stream.h /^ FileCursor(FileHandler* file_handler,$/;" f class:doris::ReadOnlyFileStream::FileCursor +FileCursor src/olap/file_stream.h /^ class FileCursor {$/;" c class:doris::ReadOnlyFileStream +FileDescriptor src/olap/file_helper.h /^ FileDescriptor(int fd) : fd(fd) {}$/;" f struct:doris::FileDescriptor +FileDescriptor src/olap/file_helper.h /^typedef struct FileDescriptor {$/;" s namespace:doris +FileDescriptor src/olap/file_helper.h /^} FileDescriptor;$/;" t namespace:doris typeref:struct:doris::FileDescriptor +FileDescriptorMetrics src/util/system_metrics.cpp /^struct FileDescriptorMetrics {$/;" s namespace:doris file: +FileHandler src/olap/file_helper.cpp /^FileHandler::FileHandler() :$/;" f class:doris::FileHandler +FileHandler src/olap/file_helper.h /^class FileHandler {$/;" c namespace:doris +FileHandlerTest test/olap/file_helper_test.cpp /^class FileHandlerTest : public testing::Test {$/;" c namespace:doris file: +FileHandlerWithBuf src/olap/file_helper.cpp /^FileHandlerWithBuf::FileHandlerWithBuf() :$/;" f class:doris::FileHandlerWithBuf +FileHandlerWithBuf src/olap/file_helper.h /^class FileHandlerWithBuf {$/;" c namespace:doris +FileHeader src/olap/file_helper.h /^ FileHeader() {$/;" f class:doris::FileHeader +FileHeader src/olap/file_helper.h /^class FileHeader {$/;" c namespace:doris +FileManager src/util/file_manager.h /^ FileManager(Env* env) : _env(env),$/;" f class:doris::FileManager +FileManager src/util/file_manager.h /^class FileManager {$/;" c namespace:doris +FileManagerTest test/util/file_manager_test.cpp /^ FileManagerTest() { }$/;" f class:doris::FileManagerTest +FileManagerTest test/util/file_manager_test.cpp /^class FileManagerTest : public testing::Test {$/;" c namespace:doris file: +FileReadableBlock src/olap/fs/file_block_manager.cpp /^FileReadableBlock::FileReadableBlock(FileBlockManager* block_manager,$/;" f class:doris::fs::internal::FileReadableBlock +FileReadableBlock src/olap/fs/file_block_manager.cpp /^class FileReadableBlock : public ReadableBlock {$/;" c namespace:doris::fs::internal file: +FileReader src/exec/file_reader.h /^class FileReader {$/;" c namespace:doris +FileStat src/runtime/snapshot_loader.h /^struct FileStat {$/;" s namespace:doris +FileSystemUtil src/util/filesystem_util.h /^class FileSystemUtil {$/;" c namespace:doris +FileUtils src/util/file_utils.h /^class FileUtils {$/;" c namespace:doris +FileUtilsTest test/olap/file_utils_test.cpp /^class FileUtilsTest : public testing::Test {$/;" c namespace:doris file: +FileWritableBlock src/olap/fs/file_block_manager.cpp /^FileWritableBlock::FileWritableBlock(FileBlockManager* block_manager,$/;" f class:doris::fs::internal::FileWritableBlock +FileWritableBlock src/olap/fs/file_block_manager.cpp /^class FileWritableBlock : public WritableBlock {$/;" c namespace:doris::fs::internal file: +FileWriter src/exec/file_writer.h /^class FileWriter {$/;" c namespace:doris +Finalize src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::Finalize(Tuple* agg_val, Tuple* output_val) {$/;" f class:doris::NewAggFnEvaluator +Finalize src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::Finalize(const std::vector& evals,$/;" f class:doris::NewAggFnEvaluator +FinalizeFn output/udf/include/uda_test_harness.h /^ typedef RESULT(*FinalizeFn)(FunctionContext* context, const INTERMEDIATE& value);$/;" t class:doris_udf::UdaTestHarnessBase +FinalizeFn src/exprs/agg_fn_evaluator.cpp /^typedef AnyVal(*FinalizeFn)(FunctionContext*, const AnyVal&);$/;" t namespace:doris file: +FinalizeFn src/exprs/new_agg_fn_evaluator.cc /^typedef AnyVal (*FinalizeFn)(FunctionContext*, const AnyVal&);$/;" t file: +FinalizeFn src/udf/uda_test_harness.h /^ typedef RESULT(*FinalizeFn)(FunctionContext* context, const INTERMEDIATE& value);$/;" t class:doris_udf::UdaTestHarnessBase +Find src/gutil/strings/split.cc /^ int Find(StringPiece text, StringPiece delimiter) {$/;" f struct:strings::delimiter::__anon29::LiteralPolicy +Find src/gutil/strings/split.cc /^ size_t Find(StringPiece text, StringPiece delimiter) {$/;" f struct:strings::delimiter::__anon29::AnyOfPolicy +Find src/gutil/strings/split.cc /^StringPiece AnyOf::Find(StringPiece text) const {$/;" f class:strings::delimiter::AnyOf +Find src/gutil/strings/split.cc /^StringPiece Literal::Find(StringPiece text) const {$/;" f class:strings::delimiter::Literal +Find src/gutil/strings/split.h /^ StringPiece Find(StringPiece text) {$/;" f class:strings::delimiter::LimitImpl +Find src/olap/skiplist.h /^bool SkipList::Find(const Key& key, Hint* hint) const {$/;" f class:doris::SkipList +FindBlockForDir test/runtime/buffered_block_mgr2_test.cpp /^ static BufferedBlockMgr2::Block* FindBlockForDir($/;" f class:doris::BufferedBlockMgrTest +FindBuildRowBucket src/exec/new_partitioned_hash_table.inline.h /^inline NewPartitionedHashTable::Iterator NewPartitionedHashTable::FindBuildRowBucket($/;" f class:doris::NewPartitionedHashTable +FindCopy src/gutil/map-util.h /^bool FindCopy(const Collection& collection,$/;" f +FindEol src/gutil/strings/util.cc /^StringPiece FindEol(StringPiece s) {$/;" f namespace:strings +FindFloorOrDie src/gutil/map-util.h /^FindFloorOrDie(Collection& collection,$/;" f +FindFloorOrDie src/gutil/map-util.h /^FindFloorOrDie(const Collection& collection,$/;" f +FindFloorOrNull src/gutil/map-util.h /^FindFloorOrNull(Collection& collection, \/\/ NOLINT$/;" f +FindFloorOrNull src/gutil/map-util.h /^FindFloorOrNull(const Collection& collection,$/;" f +FindGreaterOrEqual src/olap/skiplist.h /^SkipList::FindGreaterOrEqual(const Key& key, Node** prev) const {$/;" f class:doris::SkipList +FindLSBSetNonZero src/gutil/bits.h /^inline int Bits::FindLSBSetNonZero(uint32 n) {$/;" f class:Bits +FindLSBSetNonZero64 src/gutil/bits.h /^inline int Bits::FindLSBSetNonZero64(uint64 n) {$/;" f class:Bits +FindLSBSetNonZero64_Portable src/gutil/bits.h /^inline int Bits::FindLSBSetNonZero64_Portable(uint64 n) {$/;" f class:Bits +FindLSBSetNonZero_Portable src/gutil/bits.cc /^int Bits::FindLSBSetNonZero_Portable(uint32 n) {$/;" f class:Bits +FindLast src/olap/skiplist.h /^SkipList::FindLast() const {$/;" f class:doris::SkipList +FindLessThan src/olap/skiplist.h /^SkipList::FindLessThan(const Key& key) const {$/;" f class:doris::SkipList +FindMSBSetNonZero src/gutil/bits.h /^ static int FindMSBSetNonZero(uint32 n) { return Log2FloorNonZero(n); }$/;" f class:Bits +FindMSBSetNonZero64 src/gutil/bits.h /^ static int FindMSBSetNonZero64(uint64 n) { return Log2FloorNonZero64(n); }$/;" f class:Bits +FindNth src/gutil/strings/util.cc /^int FindNth(StringPiece s, char c, int n) {$/;" f +FindOrDie src/gutil/map-util.h /^FindOrDie(Collection& collection, \/\/ NOLINT$/;" f +FindOrDie src/gutil/map-util.h /^FindOrDie(const Collection& collection,$/;" f +FindOrDieNoPrint src/gutil/map-util.h /^FindOrDieNoPrint(Collection& collection, \/\/ NOLINT$/;" f +FindOrDieNoPrint src/gutil/map-util.h /^FindOrDieNoPrint(const Collection& collection,$/;" f +FindOrNull src/gutil/map-util.h /^FindOrNull(Collection& collection, \/\/ NOLINT$/;" f +FindOrNull src/gutil/map-util.h /^FindOrNull(const Collection& collection,$/;" f +FindPathToRoot src/runtime/bufferpool/reservation_tracker.cc /^vector ReservationTracker::FindPathToRoot() {$/;" f class:doris::ReservationTracker +FindPointeeOrNull src/gutil/map-util.h /^FindPointeeOrNull(const Collection& collection, \/\/ NOLINT,$/;" f +FindProbeRow src/exec/new_partitioned_hash_table.inline.h /^inline NewPartitionedHashTable::Iterator NewPartitionedHashTable::FindProbeRow($/;" f class:doris::NewPartitionedHashTable +FindPtrOrNull src/gutil/map-util.h /^FindPtrOrNull(Collection& collection, \/\/ NOLINT$/;" f +FindPtrOrNull src/gutil/map-util.h /^FindPtrOrNull(const Collection& collection,$/;" f +FindShortestSeparator src/gutil/strings/util.cc /^void FindShortestSeparator(const StringPiece& start,$/;" f +FindTagValuePair src/gutil/strings/util.cc /^bool FindTagValuePair(const char* arg_str, char tag_value_separator,$/;" f +FindWithDefault src/gutil/map-util.h /^FindWithDefault(const Collection& collection,$/;" f +Fingerprint src/gutil/hash/hash.h /^inline uint64 Fingerprint(char c) {$/;" f +Fingerprint src/gutil/hash/hash.h /^inline uint64 Fingerprint(const char *s, uint32 len) {$/;" f +Fingerprint src/gutil/hash/hash.h /^inline uint64 Fingerprint(const std::string& s) {$/;" f +Fingerprint src/gutil/hash/hash.h /^inline uint64 Fingerprint(int16 c) {$/;" f +Fingerprint src/gutil/hash/hash.h /^inline uint64 Fingerprint(int32 c) {$/;" f +Fingerprint src/gutil/hash/hash.h /^inline uint64 Fingerprint(int64 c) {$/;" f +Fingerprint src/gutil/hash/hash.h /^inline uint64 Fingerprint(schar c) {$/;" f +Fingerprint src/gutil/hash/hash.h /^inline uint64 Fingerprint(uint16 c) {$/;" f +Fingerprint src/gutil/hash/hash.h /^inline uint64 Fingerprint(uint32 c) {$/;" f +Fingerprint src/gutil/hash/hash.h /^inline uint64 Fingerprint(uint64 c) {$/;" f +FingerprintCat src/gutil/hash/hash.h /^inline uint64 FingerprintCat(uint64 fp1, uint64 fp2) {$/;" f +FingerprintInterleavedImplementation src/gutil/hash/hash.cc /^uint64 FingerprintInterleavedImplementation(const char *s, uint32 len) {$/;" f +FingerprintReferenceImplementation src/gutil/hash/hash.cc /^uint64 FingerprintReferenceImplementation(const char *s, uint32 len) {$/;" f +FinishCallback src/runtime/fragment_mgr.h /^ typedef std::function FinishCallback;$/;" t class:doris::FragmentMgr +FirstUnmatched src/exec/new_partitioned_hash_table.inline.h /^inline NewPartitionedHashTable::Iterator NewPartitionedHashTable::FirstUnmatched($/;" f class:doris::NewPartitionedHashTable +FiveBytesToEightBase32Digits src/gutil/strings/escaping.cc /^void FiveBytesToEightBase32Digits(const unsigned char *in_bytes, char *out) {$/;" f namespace:strings +FixLengthStringColumnReader src/olap/rowset/column_reader.h /^ FixLengthStringColumnReader(uint32_t column_id,$/;" f class:doris::FixLengthStringColumnReader +FixLengthStringColumnReader src/olap/rowset/column_reader.h /^class FixLengthStringColumnReader : public ColumnReader {$/;" c namespace:doris +FixLengthStringColumnWriter src/olap/rowset/column_writer.cpp /^FixLengthStringColumnWriter::FixLengthStringColumnWriter($/;" f class:doris::FixLengthStringColumnWriter +FixLengthStringColumnWriter src/olap/rowset/column_writer.h /^class FixLengthStringColumnWriter : public VarStringColumnWriter {$/;" c namespace:doris +FixUpStringsForRead src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::FixUpStringsForRead($/;" f class:BufferedTupleStream3 +FixedBitSize src/olap/serialize.h /^enum FixedBitSize {$/;" g namespace:doris::ser +FixedFileHeader src/olap/file_helper.h /^} __attribute__((packed)) FixedFileHeader;$/;" t namespace:doris typeref:struct:doris::_FixedFileHeader +FixedFileHeaderV2 src/olap/file_helper.h /^} __attribute__((packed)) FixedFileHeaderV2;$/;" t namespace:doris typeref:struct:doris::_FixedFileHeaderV2 +FlatRowPtr src/runtime/buffered_tuple_stream3.h /^ typedef uint8_t* FlatRowPtr;$/;" t class:doris::BufferedTupleStream3 +FloatColumnReader src/olap/rowset/column_reader.h /^typedef FloatintPointColumnReader FloatColumnReader;$/;" t namespace:doris +FloatColumnWriter src/olap/rowset/column_writer.h /^typedef DoubleColumnWriterBase FloatColumnWriter;$/;" t namespace:doris +FloatToBuffer src/gutil/strings/numbers.cc /^char* FloatToBuffer(float value, char* buffer) {$/;" f +FloatToBuffer src/gutil/strings/numbers.cc /^int FloatToBuffer(float value, int width, char *buffer) {$/;" f +FloatToString src/gutil/strings/numbers.cc /^string FloatToString(float f, const char* format) {$/;" f +FloatVal output/udf/include/udf.h /^ FloatVal() : val(0.0) { }$/;" f struct:doris_udf::FloatVal +FloatVal output/udf/include/udf.h /^ FloatVal(float val) : val(val) { }$/;" f struct:doris_udf::FloatVal +FloatVal output/udf/include/udf.h /^struct FloatVal : public AnyVal {$/;" s namespace:doris_udf +FloatVal src/udf/udf.h /^ FloatVal() : val(0.0) { }$/;" f struct:doris_udf::FloatVal +FloatVal src/udf/udf.h /^ FloatVal(float val) : val(val) { }$/;" f struct:doris_udf::FloatVal +FloatVal src/udf/udf.h /^struct FloatVal : public AnyVal {$/;" s namespace:doris_udf +FloatWrapper src/exprs/scalar_fn_call.cpp /^typedef FloatVal (*FloatWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +FloatintPointColumnReader src/olap/rowset/column_reader.h /^ FloatintPointColumnReader(uint32_t column_id, uint32_t column_unique_id) :$/;" f class:doris::FloatintPointColumnReader +FloatintPointColumnReader src/olap/rowset/column_reader.h /^class FloatintPointColumnReader : public ColumnReader {$/;" c namespace:doris +Flush src/util/bit_stream_utils.inline.h /^inline void BitWriter::Flush(bool align) {$/;" f class:doris::BitWriter +Flush src/util/rle_encoding.h /^inline int RleEncoder::Flush() {$/;" f class:doris::RleEncoder +FlushBufferedValues src/util/rle_encoding.h /^inline void RleEncoder::FlushBufferedValues(bool done) {$/;" f class:doris::RleEncoder +FlushLiteralRun src/util/rle_encoding.h /^inline void RleEncoder::FlushLiteralRun(bool update_indicator_byte) {$/;" f class:doris::RleEncoder +FlushMode src/env/env.h /^ enum FlushMode {$/;" g class:doris::RandomRWFile +FlushMode src/env/env.h /^ enum FlushMode {$/;" g class:doris::WritableFile +FlushMode src/runtime/row_batch.h /^ enum class FlushMode {$/;" c class:doris::RowBatch +FlushRepeatedRun src/util/rle_encoding.h /^inline void RleEncoder::FlushRepeatedRun() {$/;" f class:doris::RleEncoder +FlushStatistic src/olap/memtable_flush_executor.h /^struct FlushStatistic {$/;" s namespace:doris +FlushToken src/olap/memtable_flush_executor.h /^ explicit FlushToken(std::unique_ptr flush_pool_token) :$/;" f class:doris::FlushToken +FlushToken src/olap/memtable_flush_executor.h /^class FlushToken {$/;" c namespace:doris +Foo test/util/lru_cache_util_test.cpp /^ Foo(int num_param): num(num_param) { }$/;" f struct:doris::Foo +Foo test/util/lru_cache_util_test.cpp /^struct Foo {$/;" s namespace:doris file: +ForDecoder src/util/frame_of_reference_coding.h /^ explicit ForDecoder(const uint8_t* in_buffer, size_t buffer_len)$/;" f class:doris::ForDecoder +ForDecoder src/util/frame_of_reference_coding.h /^class ForDecoder {$/;" c namespace:doris +ForEncoder src/util/frame_of_reference_coding.h /^ explicit ForEncoder(faststring* buffer): _buffer(buffer) {}$/;" f class:doris::ForEncoder +ForEncoder src/util/frame_of_reference_coding.h /^class ForEncoder {$/;" c namespace:doris +FormatSubSecond src/util/time.cpp /^static string FormatSubSecond(const chrono::system_clock::time_point& t,$/;" f file: +FpToString src/gutil/strings/numbers.cc /^string FpToString(Fprint fp) {$/;" f +Fprint src/gutil/integral_types.h /^typedef uint64 Fprint;$/;" t +FragmentExecState src/runtime/fragment_mgr.cpp /^FragmentExecState::FragmentExecState($/;" f class:doris::FragmentExecState +FragmentExecState src/runtime/fragment_mgr.cpp /^class FragmentExecState {$/;" c namespace:doris file: +FragmentMgr src/runtime/fragment_mgr.cpp /^FragmentMgr::FragmentMgr(ExecEnv* exec_env) :$/;" f class:doris::FragmentMgr +FragmentMgr src/runtime/fragment_mgr.h /^class FragmentMgr : public RestMonitorIface {$/;" c namespace:doris +FragmentMgr test/runtime/etl_job_mgr_test.cpp /^FragmentMgr::FragmentMgr(ExecEnv* exec_env) :$/;" f class:doris::FragmentMgr +FragmentMgr test/runtime/export_task_mgr_test.cpp /^FragmentMgr::FragmentMgr(ExecEnv* exec_env) :$/;" f class:doris::FragmentMgr +FragmentMgrTest test/runtime/fragment_mgr_test.cpp /^ FragmentMgrTest() {$/;" f class:doris::FragmentMgrTest +FragmentMgrTest test/runtime/fragment_mgr_test.cpp /^class FragmentMgrTest : public testing::Test {$/;" c namespace:doris file: +FragmentStreamSet src/runtime/data_stream_mgr.h /^ typedef std::set, ComparisonOp > FragmentStreamSet;$/;" t class:doris::DataStreamMgr +FrameOfReferencePageBuilder src/olap/rowset/segment_v2/frame_of_reference_page.h /^ explicit FrameOfReferencePageBuilder(const PageBuilderOptions& options) :$/;" f class:doris::segment_v2::FrameOfReferencePageBuilder +FrameOfReferencePageBuilder src/olap/rowset/segment_v2/frame_of_reference_page.h /^class FrameOfReferencePageBuilder : public PageBuilder {$/;" c namespace:doris::segment_v2 +FrameOfReferencePageDecoder src/olap/rowset/segment_v2/frame_of_reference_page.h /^ FrameOfReferencePageDecoder(Slice slice, const PageDecoderOptions& options) :$/;" f class:doris::segment_v2::FrameOfReferencePageDecoder +FrameOfReferencePageDecoder src/olap/rowset/segment_v2/frame_of_reference_page.h /^class FrameOfReferencePageDecoder : public PageDecoder {$/;" c namespace:doris::segment_v2 +FrameOfReferencePageTest test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^class FrameOfReferencePageTest : public testing::Test {$/;" c namespace:doris file: +Free src/runtime/bufferpool/buffer_allocator.cc /^void BufferPool::BufferAllocator::Free(BufferHandle&& handle) {$/;" f class:doris::BufferPool::BufferAllocator +Free src/runtime/bufferpool/suballocator.cc /^void Suballocator::Free(unique_ptr allocation) {$/;" f class:doris::Suballocator +Free src/runtime/bufferpool/system_allocator.cc /^void SystemAllocator::Free(BufferPool::BufferHandle&& buffer) {$/;" f class:doris::SystemAllocator +FreeBuffer src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::FreeBuffer(ClientHandle* client, BufferHandle* handle) {$/;" f class:doris::BufferPool +FreeBufferArena src/runtime/bufferpool/buffer_allocator.cc /^BufferPool::FreeBufferArena::FreeBufferArena(BufferAllocator* parent) : parent_(parent) {}$/;" f class:doris::BufferPool::FreeBufferArena +FreeBufferArena src/runtime/bufferpool/buffer_allocator.cc /^class BufferPool::FreeBufferArena : public CacheLineAligned {$/;" c class:doris::BufferPool file: +FreeBuildLocalAllocations src/exec/new_partitioned_hash_table.cc /^void NewPartitionedHashTableCtx::FreeBuildLocalAllocations() {$/;" f class:NewPartitionedHashTableCtx +FreeDeleter src/gutil/gscoped_ptr.h /^struct FreeDeleter {$/;" s namespace:doris +FreeList src/runtime/bufferpool/free_list.h /^ FreeList() {}$/;" f class:doris::FreeList +FreeList src/runtime/bufferpool/free_list.h /^class FreeList {$/;" c namespace:doris +FreeList src/runtime/free_list.hpp /^ FreeList() {$/;" f class:doris::FreeList +FreeList src/runtime/free_list.hpp /^class FreeList {$/;" c namespace:doris +FreeListNode src/runtime/free_list.hpp /^ struct FreeListNode {$/;" s class:doris::FreeList +FreeListNode src/runtime/free_pool.hpp /^ struct FreeListNode {$/;" s class:doris::FreePool +FreeLocalAllocations src/exec/new_partitioned_hash_table.cc /^void NewPartitionedHashTableCtx::FreeLocalAllocations() {$/;" f class:NewPartitionedHashTableCtx +FreePool src/runtime/free_pool.hpp /^ FreePool(MemPool* mem_pool) : _mem_pool(mem_pool) {$/;" f class:doris::FreePool +FreePool src/runtime/free_pool.hpp /^class FreePool {$/;" c namespace:doris +FreePool src/udf/udf.cpp /^ FreePool(MemPool*) { }$/;" f class:doris::FreePool +FreePool src/udf/udf.cpp /^class FreePool {$/;" c namespace:doris file: +FreeProbeLocalAllocations src/exec/new_partitioned_hash_table.cc /^void NewPartitionedHashTableCtx::FreeProbeLocalAllocations() {$/;" f class:NewPartitionedHashTableCtx +FreeSystemMemory src/runtime/bufferpool/buffer_allocator.cc /^std::pair BufferPool::FreeBufferArena::FreeSystemMemory($/;" f class:doris::BufferPool::FreeBufferArena +FreeToSystem src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::BufferAllocator::FreeToSystem(vector&& buffers) {$/;" f class:doris::BufferPool::BufferAllocator +FreedBuffer src/runtime/bufferpool/buffer_pool_internal.h /^ void FreedBuffer(int64_t len) {$/;" f class:doris::BufferPool::Client +FromHost128 src/gutil/endian.h /^ static unsigned __int128 FromHost128(unsigned __int128 x) { return gbswap_128(x); }$/;" f class:BigEndian +FromHost128 src/gutil/endian.h /^ static unsigned __int128 FromHost128(unsigned __int128 x) { return x; }$/;" f class:LittleEndian +FromHost16 src/gutil/endian.h /^ static uint16 FromHost16(uint16 x) { return bswap_16(x); }$/;" f class:BigEndian +FromHost16 src/gutil/endian.h /^ static uint16 FromHost16(uint16 x) { return x; }$/;" f class:LittleEndian +FromHost24 src/gutil/endian.h /^ static uint32 FromHost24(uint32 x) { return bswap_24(x); }$/;" f class:BigEndian +FromHost32 src/gutil/endian.h /^ static uint32 FromHost32(uint32 x) { return bswap_32(x); }$/;" f class:BigEndian +FromHost32 src/gutil/endian.h /^ static uint32 FromHost32(uint32 x) { return x; }$/;" f class:LittleEndian +FromHost64 src/gutil/endian.h /^ static uint64 FromHost64(uint64 x) { return gbswap_64(x); }$/;" f class:BigEndian +FromHost64 src/gutil/endian.h /^ static uint64 FromHost64(uint64 x) { return x; }$/;" f class:LittleEndian +FromMicroseconds src/util/monotime.cpp /^MonoDelta MonoDelta::FromMicroseconds(int64_t us) {$/;" f class:doris::MonoDelta +FromMilliseconds src/util/monotime.cpp /^MonoDelta MonoDelta::FromMilliseconds(int64_t ms) {$/;" f class:doris::MonoDelta +FromNanoseconds src/util/monotime.cpp /^MonoDelta MonoDelta::FromNanoseconds(int64_t ns) {$/;" f class:doris::MonoDelta +FromRowBatchConverter src/util/arrow/row_batch.cpp /^ FromRowBatchConverter(const RowBatch& batch,$/;" f class:doris::FromRowBatchConverter +FromRowBatchConverter src/util/arrow/row_batch.cpp /^class FromRowBatchConverter : public arrow::TypeVisitor {$/;" c namespace:doris file: +FromRowBlockConverter src/util/arrow/row_block.cpp /^ FromRowBlockConverter(const RowBlockV2& block,$/;" f class:doris::FromRowBlockConverter +FromRowBlockConverter src/util/arrow/row_block.cpp /^class FromRowBlockConverter : public arrow::TypeVisitor {$/;" c namespace:doris file: +FromSeconds src/util/monotime.cpp /^MonoDelta MonoDelta::FromSeconds(double seconds) {$/;" f class:doris::MonoDelta +FromStdString test/runtime/string_value_test.cpp /^StringValue FromStdString(const string& str) {$/;" f namespace:doris +FrontendServiceClientCache src/runtime/client_cache.h /^typedef ClientCache FrontendServiceClientCache;$/;" t namespace:doris +FrontendServiceConnection src/runtime/client_cache.h /^typedef ClientConnection FrontendServiceConnection;$/;" t namespace:doris +FullScan test/exec/partitioned_hash_table_test.cpp /^ void FullScan(PartitionedHashTable* table, PartitionedHashTableCtx* ht_ctx, int min, int max,$/;" f class:doris::PartitionedHashTableTest +FunctionContext output/udf/include/udf.h /^class FunctionContext {$/;" c namespace:doris_udf +FunctionContext src/udf/udf.cpp /^FunctionContext::FunctionContext() : _impl(new doris::FunctionContextImpl(this)) {$/;" f class:doris_udf::FunctionContext +FunctionContext src/udf/udf.h /^class FunctionContext {$/;" c namespace:doris_udf +FunctionContextImpl src/udf/udf.cpp /^FunctionContextImpl::FunctionContextImpl(doris_udf::FunctionContext* parent) : $/;" f class:doris::FunctionContextImpl +FunctionContextImpl src/udf/udf_internal.h /^class FunctionContextImpl {$/;" c namespace:doris +FunctionRunnable src/util/threadpool.cpp /^ explicit FunctionRunnable(std::function func) : _func(std::move(func)) {}$/;" f class:doris::FunctionRunnable +FunctionRunnable src/util/threadpool.cpp /^class FunctionRunnable : public Runnable {$/;" c namespace:doris file: +FunctionStateScope output/udf/include/udf.h /^ enum FunctionStateScope {$/;" g class:doris_udf::FunctionContext +FunctionStateScope src/udf/udf.h /^ enum FunctionStateScope {$/;" g class:doris_udf::FunctionContext +FunctionUtils src/testutil/function_utils.cpp /^FunctionUtils::FunctionUtils() {$/;" f class:doris::FunctionUtils +FunctionUtils src/testutil/function_utils.cpp /^FunctionUtils::FunctionUtils(RuntimeState* state) { $/;" f class:doris::FunctionUtils +FunctionUtils src/testutil/function_utils.h /^class FunctionUtils {$/;" c namespace:doris +FuzzyCompare test/udf/uda_test.cpp /^bool FuzzyCompare(const BigIntVal& r1, const BigIntVal& r2) {$/;" f namespace:doris_udf +GATEWAY_TIMEOUT src/http/http_status.h /^ GATEWAY_TIMEOUT = 504,$/;" e enum:doris::HttpStatus +GAUGE src/util/metrics.h /^ GAUGE,$/;" m class:doris::MetricType +GB_EXCHANGE_BYTE src/olap/olap_define.h /^static const uint64_t GB_EXCHANGE_BYTE = 1024 * 1024 * 1024;$/;" m namespace:doris +GCC_VERSION src/gutil/atomicops.h 70;" d +GE src/olap/olap_common.h /^ GE = 1, \/\/ greater or equal$/;" e enum:doris::RangeCondition +GEN_SAFE_STRTO src/gutil/strings/numbers.cc 757;" d file: +GEN_SAFE_STRTO src/gutil/strings/numbers.cc 771;" d file: +GEO_PARSE_CIRCLE_INVALID src/geo/geo_common.h /^ GEO_PARSE_CIRCLE_INVALID = 8,$/;" e enum:doris::GeoParseStatus +GEO_PARSE_COORD_INVALID src/geo/geo_common.h /^ GEO_PARSE_COORD_INVALID = 1,$/;" e enum:doris::GeoParseStatus +GEO_PARSE_LOOP_INVALID src/geo/geo_common.h /^ GEO_PARSE_LOOP_INVALID = 4,$/;" e enum:doris::GeoParseStatus +GEO_PARSE_LOOP_LACK_VERTICES src/geo/geo_common.h /^ GEO_PARSE_LOOP_LACK_VERTICES = 3,$/;" e enum:doris::GeoParseStatus +GEO_PARSE_LOOP_NOT_CLOSED src/geo/geo_common.h /^ GEO_PARSE_LOOP_NOT_CLOSED = 2,$/;" e enum:doris::GeoParseStatus +GEO_PARSE_OK src/geo/geo_common.h /^ GEO_PARSE_OK = 0,$/;" e enum:doris::GeoParseStatus +GEO_PARSE_POLYGON_NOT_HOLE src/geo/geo_common.h /^ GEO_PARSE_POLYGON_NOT_HOLE = 5,$/;" e enum:doris::GeoParseStatus +GEO_PARSE_POLYLINE_INVALID src/geo/geo_common.h /^ GEO_PARSE_POLYLINE_INVALID = 7,$/;" e enum:doris::GeoParseStatus +GEO_PARSE_POLYLINE_LACK_VERTICES src/geo/geo_common.h /^ GEO_PARSE_POLYLINE_LACK_VERTICES = 6,$/;" e enum:doris::GeoParseStatus +GEO_PARSE_WKT_SYNTAX_ERROR src/geo/geo_common.h /^ GEO_PARSE_WKT_SYNTAX_ERROR = 9,$/;" e enum:doris::GeoParseStatus +GEO_SHAPE_ANY src/geo/geo_common.h /^ GEO_SHAPE_ANY = 0,$/;" e enum:doris::GeoShapeType +GEO_SHAPE_CIRCLE src/geo/geo_common.h /^ GEO_SHAPE_CIRCLE = 7,$/;" e enum:doris::GeoShapeType +GEO_SHAPE_LINE_STRING src/geo/geo_common.h /^ GEO_SHAPE_LINE_STRING = 2,$/;" e enum:doris::GeoShapeType +GEO_SHAPE_MULTI_LINE_STRING src/geo/geo_common.h /^ GEO_SHAPE_MULTI_LINE_STRING = 5,$/;" e enum:doris::GeoShapeType +GEO_SHAPE_MULTI_POINT src/geo/geo_common.h /^ GEO_SHAPE_MULTI_POINT = 4,$/;" e enum:doris::GeoShapeType +GEO_SHAPE_MULTI_POLYGON src/geo/geo_common.h /^ GEO_SHAPE_MULTI_POLYGON = 6,$/;" e enum:doris::GeoShapeType +GEO_SHAPE_POINT src/geo/geo_common.h /^ GEO_SHAPE_POINT = 1,$/;" e enum:doris::GeoShapeType +GEO_SHAPE_POLYGON src/geo/geo_common.h /^ GEO_SHAPE_POLYGON = 3,$/;" e enum:doris::GeoShapeType +GET src/http/http_method.h /^ GET,$/;" e enum:doris::HttpMethod +GETDENTS src/gutil/linux_syscall_support.h 2494;" d +GET_INPUT src/gutil/strings/escaping.cc 864;" d file: +GET_INPUT src/gutil/strings/escaping.cc 958;" d file: +GET_LENGTH_TIMEOUT src/olap/task/engine_clone_task.cpp /^const uint32_t GET_LENGTH_TIMEOUT = 10;$/;" m namespace:doris file: +GG_LL_FORMAT src/gutil/integral_types.h 69;" d +GG_LONGLONG src/gutil/integral_types.h 67;" d +GG_LONGLONG src/gutil/integral_types.h 74;" d +GG_LONGLONG src/gutil/integral_types.h 79;" d +GG_ULONGLONG src/gutil/integral_types.h 68;" d +GG_ULONGLONG src/gutil/integral_types.h 75;" d +GG_ULONGLONG src/gutil/integral_types.h 80;" d +GIGABYTE src/util/debug_util.cpp 34;" d file: +GIGABYTE src/util/pretty_printer.h /^ static const int64_t GIGABYTE = MEGABYTE * 1024;$/;" m class:doris::PrettyPrinter +GONE src/http/http_status.h /^ GONE = 410,$/;" e enum:doris::HttpStatus +GOOGLE_OBSCURE_SIGNAL src/gutil/port.h 241;" d +GOOGLE_OBSCURE_SIGNAL src/gutil/port.h 242;" d +GOOGLE_OBSCURE_SIGNAL src/gutil/port.h 35;" d +GOOGLE_ONCE_INIT src/gutil/once.h 46;" d +GOOGLE_ONCE_INTERNAL_DONE src/gutil/once.h /^ GOOGLE_ONCE_INTERNAL_DONE = 0x3F2D8AB0, \/\/ yet another improbable value$/;" e enum:__anon19 +GOOGLE_ONCE_INTERNAL_INIT src/gutil/once.h /^ GOOGLE_ONCE_INTERNAL_INIT = 0,$/;" e enum:__anon19 +GOOGLE_ONCE_INTERNAL_RUNNING src/gutil/once.h /^ GOOGLE_ONCE_INTERNAL_RUNNING = 0x65C2937B, \/\/ an improbable 32-bit value$/;" e enum:__anon19 +GOOGLE_ONCE_INTERNAL_WAITER src/gutil/once.h /^ GOOGLE_ONCE_INTERNAL_WAITER = 0x05A308D2, \/\/ a different improbable value$/;" e enum:__anon19 +GPRIuPTHREAD src/gutil/port.h 1172;" d +GPRIxPTHREAD src/gutil/port.h 1173;" d +GREATEST_FN src/exprs/math_functions.cpp 625;" d file: +GREATEST_FNS src/exprs/math_functions.cpp 637;" d file: +GREATEST_NONNUMERIC_FN src/exprs/math_functions.cpp 648;" d file: +GREATEST_NONNUMERIC_FNS src/exprs/math_functions.cpp 663;" d file: +GT src/olap/olap_common.h /^ GT = 0, \/\/ greater than$/;" e enum:doris::RangeCondition +GUTIL_ATOMICOPS_INTERNALS_X86_H_ src/gutil/atomicops-internals-x86.h 28;" d +GUTIL_CYCLECLOCK_INL_H_ src/gutil/cycleclock-inl.h 35;" d +GUTIL_DEBUG_MODE src/gutil/logging-inl.h /^const bool GUTIL_DEBUG_MODE = false;$/;" v +GUTIL_DEBUG_MODE src/gutil/logging-inl.h /^const bool GUTIL_DEBUG_MODE = true;$/;" v +GUTIL_GSCOPED_PTR_H_ src/gutil/gscoped_ptr.h 95;" d +GUTIL_WALLTIME_H_ src/gutil/walltime.h 20;" d +GZIP src/exec/decompressor.h /^ GZIP,$/;" e enum:doris::CompressType +GcFunction src/runtime/mem_tracker.h /^ typedef std::function GcFunction;$/;" t class:doris::MemTracker +GcMemory src/runtime/mem_tracker.cpp /^bool MemTracker::GcMemory(int64_t max_consumption) {$/;" f class:doris::MemTracker +GenBoolValue test/runtime/buffered_tuple_stream2_test.cpp /^ bool GenBoolValue(int i) {$/;" f class:doris::SimpleTupleStreamTest +GenIntValue test/runtime/buffered_tuple_stream2_test.cpp /^ int GenIntValue(int i) {$/;" f class:doris::SimpleTupleStreamTest +GeneralBase32Escape src/gutil/strings/escaping.cc /^static bool GeneralBase32Escape(const string& src, string* dest,$/;" f namespace:strings +GeneralBase32Escape src/gutil/strings/escaping.cc /^static int GeneralBase32Escape(const unsigned char *src, size_t szsrc,$/;" f namespace:strings +GeneralFiveBytesToEightBase32Digits src/gutil/strings/escaping.cc /^void GeneralFiveBytesToEightBase32Digits(const unsigned char *in_bytes,$/;" f namespace:strings +GenerateRandomBitString test/util/rle_encoding_test.cpp /^static size_t GenerateRandomBitString(int num_runs, faststring* enc_buf, string* string_rep) {$/;" f namespace:doris +GenericElement src/common/object_pool.h /^ struct GenericElement {$/;" s class:doris::ObjectPool +GenericFind src/gutil/strings/split.cc /^StringPiece GenericFind($/;" f namespace:strings::delimiter::__anon29 +GenericIteratorsTest test/olap/generic_iterators_test.cpp /^ GenericIteratorsTest() { }$/;" f class:doris::GenericIteratorsTest +GenericIteratorsTest test/olap/generic_iterators_test.cpp /^class GenericIteratorsTest : public testing::Test {$/;" c namespace:doris file: +GeoCircle src/geo/geo_types.h /^ GeoCircle() { }$/;" f class:doris::GeoCircle +GeoCircle src/geo/geo_types.h /^class GeoCircle : public GeoShape {$/;" c namespace:doris +GeoCoordinate src/geo/wkt_parse_type.h /^struct GeoCoordinate {$/;" s namespace:doris +GeoCoordinateList src/geo/wkt_parse_type.h /^struct GeoCoordinateList {$/;" s namespace:doris +GeoCoordinateListList src/geo/wkt_parse_type.h /^struct GeoCoordinateListList {$/;" s namespace:doris +GeoFunctions src/geo/geo_functions.h /^class GeoFunctions {$/;" c namespace:doris +GeoFunctionsTest test/geo/geo_functions_test.cpp /^ GeoFunctionsTest() { }$/;" f class:doris::GeoFunctionsTest +GeoFunctionsTest test/geo/geo_functions_test.cpp /^class GeoFunctionsTest : public testing::Test {$/;" c namespace:doris file: +GeoLine src/geo/geo_types.h /^ GeoLine() { }$/;" f class:doris::GeoLine +GeoLine src/geo/geo_types.h /^class GeoLine : public GeoShape {$/;" c namespace:doris +GeoParseStatus src/geo/geo_common.h /^enum GeoParseStatus {$/;" g namespace:doris +GeoPoint src/geo/geo_types.h /^ GeoPoint() { }$/;" f class:doris::GeoPoint +GeoPoint src/geo/geo_types.h /^class GeoPoint : public GeoShape {$/;" c namespace:doris +GeoPolygon src/geo/geo_types.h /^ GeoPolygon() { }$/;" f class:doris::GeoPolygon +GeoPolygon src/geo/geo_types.h /^class GeoPolygon : public GeoShape {$/;" c namespace:doris +GeoShape src/geo/geo_types.h /^class GeoShape {$/;" c namespace:doris +GeoShapeType src/geo/geo_common.h /^enum GeoShapeType {$/;" g namespace:doris +GeoTypesTest test/geo/geo_types_test.cpp /^ GeoTypesTest() { }$/;" f class:doris::GeoTypesTest +GeoTypesTest test/geo/geo_types_test.cpp /^class GeoTypesTest : public testing::Test {$/;" c namespace:doris file: +Get src/util/bitmap.h /^ bool Get(int64_t bit_index) const {$/;" f class:doris::Bitmap +Get src/util/rle_encoding.h /^inline bool RleDecoder::Get(T* val) {$/;" f class:doris::RleDecoder +GetAligned src/util/bit_stream_utils.inline.h /^inline bool BitReader::GetAligned(int num_bytes, T* v) {$/;" f class:doris::BitReader +GetBatch src/exec/exec_node.cpp /^RowBatch* ExecNode::RowBatchQueue::GetBatch() {$/;" f class:doris::ExecNode::RowBatchQueue +GetBit src/util/bit_util.h /^ static inline int8_t GetBit(T v, int bitpos) {$/;" f class:doris::BitUtil +GetBuffer src/runtime/buffered_tuple_stream3.h /^ Status GetBuffer(const BufferPool::BufferHandle** buffer) {$/;" f struct:doris::BufferedTupleStream3::Page +GetBuffer src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::PageHandle::GetBuffer(const BufferHandle** buffer) const {$/;" f class:doris::BufferPool::PageHandle +GetBuffersToFree src/runtime/bufferpool/free_list.h /^ std::vector GetBuffersToFree(int64_t num_buffers) {$/;" f class:doris::FreeList +GetByteIndexAndAdvance src/util/bit_stream_utils.h /^ size_t GetByteIndexAndAdvance(int num_bytes) {$/;" f class:doris::BitWriter +GetChildReservations src/runtime/bufferpool/reservation_tracker.cc /^int64_t ReservationTracker::GetChildReservations() {$/;" f class:doris::ReservationTracker +GetCleanPageBytes src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::BufferAllocator::GetCleanPageBytes() const {$/;" f class:doris::BufferPool::BufferAllocator +GetCleanPageBytes src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::GetCleanPageBytes() const {$/;" f class:doris::BufferPool +GetCleanPageBytesLimit src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::BufferAllocator::GetCleanPageBytesLimit() const {$/;" f class:doris::BufferPool::BufferAllocator +GetCleanPageBytesLimit src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::GetCleanPageBytesLimit() const {$/;" f class:doris::BufferPool +GetClosingChar src/gutil/strings/split.cc /^ char GetClosingChar(char opening) const {$/;" f class:__anon31::ClosingSymbolLookup +GetCurrentTime src/gutil/walltime.h /^inline void GetCurrentTime(mach_timespec_t* ts) {$/;" f namespace:walltime_internal +GetCurrentTimeMicros src/gutil/walltime.h /^inline MicrosecondsInt64 GetCurrentTimeMicros() {$/;" f +GetCurrentTimeMicros src/gutil/walltime.h /^inline MicrosecondsInt64 GetCurrentTimeMicros() {$/;" f namespace:walltime_internal +GetCurrentTimeMicros src/util/time.h /^inline int64_t GetCurrentTimeMicros() {$/;" f namespace:doris +GetDeltaSince src/util/monotime.cpp /^MonoDelta MonoTime::GetDeltaSince(const MonoTime &rhs) const {$/;" f class:doris::MonoTime +GetFixedValueSizeVisitor src/exec/olap_scan_node.h /^ class GetFixedValueSizeVisitor : public boost::static_visitor {$/;" c class:doris::OlapScanNode +GetFreeBufferBytes src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::BufferAllocator::GetFreeBufferBytes() const {$/;" f class:doris::BufferPool::BufferAllocator +GetFreeBufferBytes src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::FreeBufferArena::GetFreeBufferBytes() {$/;" f class:doris::BufferPool::FreeBufferArena +GetFreeBufferBytes src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::GetFreeBufferBytes() const {$/;" f class:doris::BufferPool +GetFreeListSize src/runtime/bufferpool/buffer_allocator.cc /^int BufferPool::BufferAllocator::GetFreeListSize(int core, int64_t len) {$/;" f class:doris::BufferPool::BufferAllocator +GetFreeListSize src/runtime/bufferpool/buffer_allocator.cc /^int BufferPool::FreeBufferArena::GetFreeListSize(int64_t len) {$/;" f class:doris::BufferPool::FreeBufferArena +GetHashSeed src/exec/new_partitioned_hash_table_ir.cc /^uint32_t NewPartitionedHashTableCtx::GetHashSeed() const { return seeds_[level_]; }$/;" f class:NewPartitionedHashTableCtx +GetHashTable src/exec/new_partitioned_aggregation_node.h /^ NewPartitionedHashTable* ALWAYS_INLINE GetHashTable(int partition_idx) {$/;" f class:doris::NewPartitionedAggregationNode +GetInstance src/runtime/exec_env.h /^ static ExecEnv* GetInstance() {$/;" f class:doris::ExecEnv +GetIntelMicroArchitecture src/gutil/cpu.cc /^CPU::IntelMicroArchitecture CPU::GetIntelMicroArchitecture() const {$/;" f class:base::CPU +GetIntermediateTypeDesc src/exprs/agg_fn.cc /^FunctionContext::TypeDesc AggFn::GetIntermediateTypeDesc() const {$/;" f class:doris::AggFn +GetListsForSize src/runtime/bufferpool/buffer_allocator.cc /^ PerSizeLists* GetListsForSize(int64_t buffer_len) {$/;" f class:doris::BufferPool::FreeBufferArena file: +GetMaxHeight src/olap/skiplist.h /^ inline int GetMaxHeight() const {$/;" f class:doris::SkipList +GetMinMemLimitFromReservation src/runtime/bufferpool/reservation_util.cc /^int64_t ReservationUtil::GetMinMemLimitFromReservation(int64_t buffer_reservation) {$/;" f class:doris::ReservationUtil +GetMonoTimeMicros src/gutil/walltime.h /^inline MicrosecondsInt64 GetMonoTimeMicros() {$/;" f +GetMonoTimeMicros src/gutil/walltime.h /^inline MicrosecondsInt64 GetMonoTimeMicros() {$/;" f namespace:walltime_internal +GetMonoTimeMicros src/util/time.h /^inline int64_t GetMonoTimeMicros() {$/;" f namespace:doris +GetMonoTimeMicrosRaw src/gutil/walltime.h /^inline MicrosecondsInt64 GetMonoTimeMicrosRaw() {$/;" f +GetMonoTimeNanos src/gutil/walltime.h /^inline int64_t GetMonoTimeNanos() {$/;" f namespace:walltime_internal +GetNext src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::GetNext($/;" f class:BufferedTupleStream3 +GetNext src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::GetNext(RowBatch* batch, bool* eos) {$/;" f class:BufferedTupleStream3 +GetNextBytePtr src/util/bit_stream_utils.inline.h /^inline uint8_t* BitWriter::GetNextBytePtr(int num_bytes) {$/;" f class:doris::BitWriter +GetNextInternal src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::GetNextInternal(RuntimeState* state,$/;" f class:doris::NewPartitionedAggregationNode +GetNextInternal src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::GetNextInternal($/;" f class:BufferedTupleStream3 +GetNextRun src/util/rle_encoding.h /^inline size_t RleDecoder::GetNextRun(T* val, size_t max_run) {$/;" f class:doris::RleDecoder +GetNumCleanPages src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::BufferAllocator::GetNumCleanPages() const {$/;" f class:doris::BufferPool::BufferAllocator +GetNumCleanPages src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::FreeBufferArena::GetNumCleanPages() {$/;" f class:doris::BufferPool::FreeBufferArena +GetNumCleanPages src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::GetNumCleanPages() const {$/;" f class:doris::BufferPool +GetNumFreeBuffers src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::BufferAllocator::GetNumFreeBuffers() const {$/;" f class:doris::BufferPool::BufferAllocator +GetNumFreeBuffers src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::FreeBufferArena::GetNumFreeBuffers() {$/;" f class:doris::BufferPool::FreeBufferArena +GetNumFreeBuffers src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::GetNumFreeBuffers() const {$/;" f class:doris::BufferPool +GetOutputTuple src/exec/new_partitioned_aggregation_node.cc /^Tuple* NewPartitionedAggregationNode::GetOutputTuple($/;" f class:doris::NewPartitionedAggregationNode +GetOutputTypeDesc src/exprs/agg_fn.cc /^FunctionContext::TypeDesc AggFn::GetOutputTypeDesc() const {$/;" f class:doris::AggFn +GetParentMemTracker src/runtime/bufferpool/reservation_tracker.h /^ MemTracker* GetParentMemTracker() const {$/;" f class:doris::ReservationTracker +GetPoolMemReserved src/runtime/mem_tracker.cpp /^int64_t MemTracker::GetPoolMemReserved() const {$/;" f class:doris::MemTracker +GetPrintableString src/gutil/strings/util.h /^inline const char* GetPrintableString(const char* const in) {$/;" f +GetRequestPoolMemTracker src/runtime/mem_tracker.cpp /^MemTracker* PoolMemTrackerRegistry::GetRequestPoolMemTracker($/;" f class:doris::PoolMemTrackerRegistry +GetReservation src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::ClientHandle::GetReservation() const {$/;" f class:doris::BufferPool::ClientHandle +GetReservation src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::SubReservation::GetReservation() const {$/;" f class:doris::BufferPool::SubReservation +GetReservation src/runtime/bufferpool/reservation_tracker.cc /^int64_t ReservationTracker::GetReservation() {$/;" f class:doris::ReservationTracker +GetReservationLimitFromMemLimit src/runtime/bufferpool/reservation_util.cc /^int64_t ReservationUtil::GetReservationLimitFromMemLimit(int64_t mem_limit) {$/;" f class:doris::ReservationUtil +GetResultBatchCtx src/runtime/buffer_control_block.h /^ GetResultBatchCtx(brpc::Controller* cntl_,$/;" f struct:doris::GetResultBatchCtx +GetResultBatchCtx src/runtime/buffer_control_block.h /^struct GetResultBatchCtx {$/;" s namespace:doris +GetRow src/exec/new_partitioned_hash_table.inline.h /^inline TupleRow* IR_ALWAYS_INLINE NewPartitionedHashTable::GetRow(Bucket* bucket, TupleRow* row) const {$/;" f class:doris::NewPartitionedHashTable +GetRow src/exec/new_partitioned_hash_table.inline.h /^inline TupleRow* IR_ALWAYS_INLINE NewPartitionedHashTable::GetRow(HtData& htdata, TupleRow* row) const {$/;" f class:doris::NewPartitionedHashTable +GetRow src/exec/new_partitioned_hash_table.inline.h /^inline TupleRow* IR_ALWAYS_INLINE NewPartitionedHashTable::Iterator::GetRow() const {$/;" f class:doris::NewPartitionedHashTable::Iterator +GetRows src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::GetRows($/;" f class:BufferedTupleStream3 +GetRowsFromPartition src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::GetRowsFromPartition(RuntimeState* state,$/;" f class:doris::NewPartitionedAggregationNode +GetRowsStreaming src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::GetRowsStreaming(RuntimeState* state,$/;" f class:doris::NewPartitionedAggregationNode +GetRunningOnValgrind src/gutil/dynamic_annotations.c /^static int GetRunningOnValgrind(void) {$/;" f file: +GetSamples src/util/streaming_sampler.h /^ const T* GetSamples(int* num_samples, int* period, SpinLock** lock = NULL) const {$/;" f class:doris::StreamingSampler +GetSingletonOutput src/exec/new_partitioned_aggregation_node.cc /^void NewPartitionedAggregationNode::GetSingletonOutput(RowBatch* row_batch) {$/;" f class:doris::NewPartitionedAggregationNode +GetSize src/exec/parquet_reader.cpp /^arrow::Status ParquetFile::GetSize(int64_t* size) {$/;" f class:doris::ParquetFile +GetSystemBytesAllocated src/runtime/bufferpool/buffer_allocator.h /^ int64_t GetSystemBytesAllocated() const {$/;" f struct:doris::BufferPool::BufferAllocator +GetSystemBytesAllocated src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::GetSystemBytesAllocated() const {$/;" f class:doris::BufferPool +GetSystemBytesLimit src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::GetSystemBytesLimit() const {$/;" f class:doris::BufferPool +GetThreadCpuTimeMicros src/gutil/walltime.h /^inline MicrosecondsInt64 GetThreadCpuTimeMicros() {$/;" f +GetThreadCpuTimeMicros src/gutil/walltime.h /^inline MicrosecondsInt64 GetThreadCpuTimeMicros() {$/;" f namespace:walltime_internal +GetTuple src/exec/new_partitioned_hash_table.inline.h /^inline Tuple* IR_ALWAYS_INLINE NewPartitionedHashTable::Iterator::GetTuple() const {$/;" f class:doris::NewPartitionedHashTable::Iterator +GetTupleRow src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::GetTupleRow(FlatRowPtr flat_row, TupleRow* row) const {$/;" f class:BufferedTupleStream3 +GetUnusedReservation src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::ClientHandle::GetUnusedReservation() const {$/;" f class:doris::BufferPool::ClientHandle +GetUnusedReservation src/runtime/bufferpool/reservation_tracker.cc /^int64_t ReservationTracker::GetUnusedReservation() {$/;" f class:doris::ReservationTracker +GetUsedReservation src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::ClientHandle::GetUsedReservation() const {$/;" f class:doris::BufferPool::ClientHandle +GetUsedReservation src/runtime/bufferpool/reservation_tracker.cc /^int64_t ReservationTracker::GetUsedReservation() {$/;" f class:doris::ReservationTracker +GetValue src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::GetValue(Tuple* src, Tuple* dst) {$/;" f class:doris::NewAggFnEvaluator +GetValue src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::GetValue(const std::vector& evals,$/;" f class:doris::NewAggFnEvaluator +GetValue src/util/bit_stream_utils.inline.h /^inline bool BitReader::GetValue(int num_bits, T* v) {$/;" f class:doris::BitReader +GetValueFn src/exprs/agg_fn_evaluator.cpp /^typedef AnyVal(*GetValueFn)(FunctionContext*, const AnyVal&);$/;" t namespace:doris file: +GetValueFn src/exprs/new_agg_fn_evaluator.cc /^typedef AnyVal (*GetValueFn)(FunctionContext*, const AnyVal&);$/;" t file: +GetVlqInt src/util/bit_stream_utils.inline.h /^inline bool BitReader::GetVlqInt(int32_t* v) {$/;" f class:doris::BitReader +GetlineFromStdioFile src/gutil/strings/util.cc /^bool GetlineFromStdioFile(FILE* file, string* str, char delim) {$/;" f +GlobalReplaceSubstring src/gutil/strings/util.cc /^int GlobalReplaceSubstring(const StringPiece& substring,$/;" f +GoodFastHash src/gutil/hash/hash.h /^struct GoodFastHash > {$/;" s +GoodFastHash src/gutil/hash/hash.h /^struct GoodFastHash > {$/;" s +GoodFastHash src/gutil/hash/hash.h /^template<> struct GoodFastHash {$/;" s +GoodFastHash src/gutil/hash/hash.h /^template<> struct GoodFastHash {$/;" s +GoodFastHash src/gutil/strings/stringpiece.h /^template<> struct GoodFastHash {$/;" s +Google1At src/gutil/hash/jenkins_lookup2.h /^static inline uint32 Google1At(const char *ptr) {$/;" f +Google1At src/gutil/hash/jenkins_lookup2.h /^static inline uint32 Google1At(const char *ptr2) {$/;" f +GoogleOnceDynamic src/gutil/once.h /^ GoogleOnceDynamic() : state_(GOOGLE_ONCE_INTERNAL_INIT) { }$/;" f class:GoogleOnceDynamic +GoogleOnceDynamic src/gutil/once.h /^class GoogleOnceDynamic {$/;" c +GoogleOnceInit src/gutil/once.h /^inline void GoogleOnceInit(GoogleOnceType* state, void (*func)()) {$/;" f +GoogleOnceInitArg src/gutil/once.h /^inline void GoogleOnceInitArg(GoogleOnceType* state,$/;" f +GoogleOnceInternalInit src/gutil/once.cc /^void GoogleOnceInternalInit(Atomic32 *control, void (*func)(),$/;" f +GoogleOnceType src/gutil/once.h /^struct GoogleOnceType {$/;" s +GroupingExprsVarlenSize src/exec/new_partitioned_aggregation_node.cc /^int NewPartitionedAggregationNode::GroupingExprsVarlenSize() {$/;" f class:doris::NewPartitionedAggregationNode +GroupingSetsFunctions src/exprs/grouping_sets_functions.h /^class GroupingSetsFunctions {$/;" c namespace:doris +GrowArray src/util/faststring.cc /^void faststring::GrowArray(size_t newcapacity) {$/;" f class:doris::faststring +GrowNodeArray src/exec/new_partitioned_hash_table.cc /^bool NewPartitionedHashTable::GrowNodeArray(Status* status) {$/;" f class:NewPartitionedHashTable +GrowTableTest test/exec/partitioned_hash_table_test.cpp /^ void GrowTableTest(bool quadratic) {$/;" f class:doris::PartitionedHashTableTest +GrowToAtLeast src/util/faststring.cc /^void faststring::GrowToAtLeast(size_t newcapacity) {$/;" f class:doris::faststring +GrowthAction src/http/action/pprof_actions.cpp /^ GrowthAction() {}$/;" f class:doris::GrowthAction +GrowthAction src/http/action/pprof_actions.cpp /^class GrowthAction : public HttpHandler {$/;" c namespace:doris file: +GutilAtomicOps_Internalx86CPUFeatures src/gutil/atomicops-internals-x86.cc /^struct GutilAtomicOps_x86CPUFeatureStruct GutilAtomicOps_Internalx86CPUFeatures = {$/;" v typeref:struct:GutilAtomicOps_x86CPUFeatureStruct +GutilAtomicOps_x86CPUFeatureStruct src/gutil/atomicops-internals-x86.h /^struct GutilAtomicOps_x86CPUFeatureStruct {$/;" s +GzipDecompressor src/exec/decompressor.cpp /^GzipDecompressor::GzipDecompressor(bool is_deflate):$/;" f class:doris::GzipDecompressor +GzipDecompressor src/exec/decompressor.h /^class GzipDecompressor : public Decompressor {$/;" c namespace:doris +HALF_EVEN src/runtime/decimal_value.h /^ HALF_EVEN = 2,$/;" e enum:doris::DecimalRoundMode +HALF_UP src/runtime/decimal_value.h /^ HALF_UP = 1,$/;" e enum:doris::DecimalRoundMode +HANDLE_ERROR src/runtime/routine_load/routine_load_task_executor.cpp 165;" d file: +HANDLE_TYPE src/agent/agent_server.cpp 120;" d file: +HANDLE_TYPE src/agent/agent_server.cpp 186;" d file: +HASH_NAMESPACE_DECLARATION_END src/gutil/port.h 1001;" d +HASH_NAMESPACE_DECLARATION_END src/gutil/port.h 1004;" d +HASH_NAMESPACE_DECLARATION_START src/gutil/port.h 1000;" d +HASH_NAMESPACE_DECLARATION_START src/gutil/port.h 1003;" d +HASH_TO src/gutil/hash/legacy_hash.h 52;" d +HASH_TO src/gutil/hash/legacy_hash.h 77;" d +HAVE_ATTRIBUTE_ALWAYS_INLINE src/gutil/port.h 368;" d +HAVE_ATTRIBUTE_NOINLINE src/gutil/port.h 370;" d +HAVE_ATTRIBUTE_SECTION src/gutil/port.h 427;" d +HAVE_ATTRIBUTE_WEAK src/gutil/port.h 375;" d +HAVE_CLOCK_MONOTONIC_COARSE src/util/os_info.cpp 41;" d file: +HAVE_CLOCK_MONOTONIC_COARSE src/util/os_info.cpp 43;" d file: +HAVE_DEFINED_HASH_FOR_POINTERS src/gutil/hash/hash.h 248;" d +HAVE_SSIZET src/gutil/port.h 799;" d +HAVE_TLS src/gutil/port.h 68;" d +HAVE_UINT src/gutil/port.h 50;" d +HAVE_UINT src/gutil/port.h 793;" d +HAVE_ULONG src/gutil/port.h 58;" d +HAVE_USHORT src/gutil/port.h 54;" d +HDD_UC src/olap/options.cpp /^static std::string HDD_UC = "HDD";$/;" m namespace:doris file: +HEAD src/http/http_method.h /^ HEAD,$/;" e enum:doris::HttpMethod +HEADER src/http/action/meta_action.h /^ HEADER = 1,$/;" e enum:doris::META_TYPE +HEADER_JSON src/http/action/compaction_action.cpp /^const static std::string HEADER_JSON = "application\/json";$/;" m namespace:doris file: +HEADER_JSON src/http/action/health_action.cpp /^const static std::string HEADER_JSON = "application\/json";$/;" m namespace:doris file: +HEADER_JSON src/http/action/meta_action.cpp /^const static std::string HEADER_JSON = "application\/json";$/;" m namespace:doris file: +HEADER_PREFIX src/olap/tablet_meta_manager.h /^const std::string HEADER_PREFIX = "tabletmeta_";$/;" m namespace:doris +HEADER_PREFIX src/tools/meta_tool.cpp /^const std::string HEADER_PREFIX = "tabletmeta_";$/;" v +HEARTBEAT_INTERVAL src/agent/heartbeat_server.h /^const uint32_t HEARTBEAT_INTERVAL = 10;$/;" m namespace:doris +HEX build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 507;" d file: +HEX build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 492;" d file: +HINIS_KEY_GROUP_SEPARATOR src/olap/olap_define.h /^static const char* const HINIS_KEY_GROUP_SEPARATOR = "&";$/;" m namespace:doris +HINIS_KEY_PAIR_SEPARATOR src/olap/olap_define.h /^static const char* const HINIS_KEY_PAIR_SEPARATOR = "|";$/;" m namespace:doris +HINIS_KEY_SEPARATOR src/olap/olap_define.h /^static const char* const HINIS_KEY_SEPARATOR = ";";$/;" m namespace:doris +HISTOGRAM src/util/metrics.h /^ HISTOGRAM,$/;" m class:doris::MetricType +HISTOGRAM_SIZE src/util/radix_sort.h /^ static constexpr size_t HISTOGRAM_SIZE = 1 << Traits::PART_SIZE_BITS;$/;" m struct:doris::RadixSort +HLL_COLUMN_DEFAULT_LEN src/olap/hll.h /^const static int HLL_COLUMN_DEFAULT_LEN = HLL_REGISTERS_COUNT + 1;$/;" m namespace:doris +HLL_COLUMN_PRECISION src/olap/hll.h /^const static int HLL_COLUMN_PRECISION = 14;$/;" m namespace:doris +HLL_DATA_EMPTY src/olap/hll.h /^ HLL_DATA_EMPTY = 0,$/;" e enum:doris::HllDataType +HLL_DATA_EXPLICIT src/olap/hll.h /^ HLL_DATA_EXPLICIT = 1,$/;" e enum:doris::HllDataType +HLL_DATA_FULL src/olap/hll.h /^ HLL_DATA_FULL = 3,$/;" e enum:doris::HllDataType +HLL_DATA_SPRASE src/olap/hll.h /^ HLL_DATA_SPRASE = 2,$/;" e enum:doris::HllDataType +HLL_EMPTY_SIZE src/olap/hll.h /^const static int HLL_EMPTY_SIZE = 1;$/;" m namespace:doris +HLL_EXPLICLIT_INT64_NUM src/olap/hll.h /^const static int HLL_EXPLICLIT_INT64_NUM = 160;$/;" m namespace:doris +HLL_KEY src/http/action/mini_load.cpp /^const std::string HLL_KEY = "hll";$/;" m namespace:doris file: +HLL_REGISTERS_COUNT src/olap/hll.h /^const static int HLL_REGISTERS_COUNT = 16 * 1024;$/;" m namespace:doris +HLL_SINGLE_VALUE_SIZE src/olap/hll.h /^const static int HLL_SINGLE_VALUE_SIZE = 10;$/;" m namespace:doris +HLL_SPARSE_THRESHOLD src/olap/hll.h /^const static int HLL_SPARSE_THRESHOLD = 4096;$/;" m namespace:doris +HLL_UNION_AGG src/exprs/agg_fn.h /^ HLL_UNION_AGG,$/;" e enum:doris::AggFn::AggregationOp +HLL_UNION_AGG src/exprs/agg_fn_evaluator.h /^ HLL_UNION_AGG,$/;" e enum:doris::AggFnEvaluator::AggregationOp +HLL_ZERO_COUNT_BITS src/olap/hll.h /^const static int HLL_ZERO_COUNT_BITS = (64 - HLL_COLUMN_PRECISION);$/;" m namespace:doris +HOST src/http/http_headers.cpp /^const char* HttpHeaders::HOST = "Host";$/;" m class:doris::HttpHeaders file: +HOST src/http/http_headers.h /^ static const char* HOST;$/;" m class:doris::HttpHeaders +HOST src/util/url_parser.h /^ HOST,$/;" e enum:doris::UrlParser::UrlPart +HOUR src/runtime/datetime_value.h /^ HOUR,$/;" e enum:doris::TimeUnit +HOUR src/util/debug_util.cpp 38;" d file: +HOUR src/util/pretty_printer.h /^ static const int64_t HOUR = MINUTE * 60;$/;" m class:doris::PrettyPrinter +HOUR_MICROSECOND src/runtime/datetime_value.h /^ HOUR_MICROSECOND,$/;" e enum:doris::TimeUnit +HOUR_MINUTE src/runtime/datetime_value.h /^ HOUR_MINUTE,$/;" e enum:doris::TimeUnit +HOUR_SECOND src/runtime/datetime_value.h /^ HOUR_SECOND,$/;" e enum:doris::TimeUnit +HTTP_100_CONTINUE src/http/http_common.h /^static const std::string HTTP_100_CONTINUE = "100-continue";$/;" m namespace:doris +HTTP_COLUMNS src/http/http_common.h /^static const std::string HTTP_COLUMNS = "columns";$/;" m namespace:doris +HTTP_COLUMN_SEPARATOR src/http/http_common.h /^static const std::string HTTP_COLUMN_SEPARATOR = "column_separator";$/;" m namespace:doris +HTTP_DB_KEY src/http/http_common.h /^static const std::string HTTP_DB_KEY = "db";$/;" m namespace:doris +HTTP_EXEC_MEM_LIMIT src/http/http_common.h /^static const std::string HTTP_EXEC_MEM_LIMIT = "exec_mem_limit";$/;" m namespace:doris +HTTP_FORMAT_KEY src/http/http_common.h /^static const std::string HTTP_FORMAT_KEY = "format";$/;" m namespace:doris +HTTP_LABEL_KEY src/http/http_common.h /^static const std::string HTTP_LABEL_KEY = "label";$/;" m namespace:doris +HTTP_MAX_FILTER_RATIO src/http/http_common.h /^static const std::string HTTP_MAX_FILTER_RATIO = "max_filter_ratio";$/;" m namespace:doris +HTTP_NEGATIVE src/http/http_common.h /^static const std::string HTTP_NEGATIVE = "negative";$/;" m namespace:doris +HTTP_PARTITIONS src/http/http_common.h /^static const std::string HTTP_PARTITIONS = "partitions";$/;" m namespace:doris +HTTP_REQUEST_FILE_PARAM src/olap/task/engine_clone_task.cpp /^const std::string HTTP_REQUEST_FILE_PARAM = "&file=";$/;" m namespace:doris file: +HTTP_REQUEST_PREFIX src/olap/task/engine_clone_task.cpp /^const std::string HTTP_REQUEST_PREFIX = "\/api\/_tablet\/_download?";$/;" m namespace:doris file: +HTTP_REQUEST_TOKEN_PARAM src/olap/task/engine_clone_task.cpp /^const std::string HTTP_REQUEST_TOKEN_PARAM = "token=";$/;" m namespace:doris file: +HTTP_STRICT_MODE src/http/http_common.h /^static const std::string HTTP_STRICT_MODE = "strict_mode";$/;" m namespace:doris +HTTP_TABLE_KEY src/http/http_common.h /^static const std::string HTTP_TABLE_KEY = "table";$/;" m namespace:doris +HTTP_TIMEOUT src/http/http_common.h /^static const std::string HTTP_TIMEOUT = "timeout";$/;" m namespace:doris +HTTP_TIMEZONE src/http/http_common.h /^static const std::string HTTP_TIMEZONE = "timezone";$/;" m namespace:doris +HTTP_VERSION_NOT_SUPPORTED src/http/http_status.h /^ HTTP_VERSION_NOT_SUPPORTED = 505$/;" e enum:doris::HttpStatus +HTTP_WHERE src/http/http_common.h /^static const std::string HTTP_WHERE = "where";$/;" m namespace:doris +HUB_MAX_ERROR_NUM src/runtime/runtime_state.cpp /^const int64_t HUB_MAX_ERROR_NUM = 10;$/;" m namespace:doris file: +HUGE_PAGE_SIZE src/runtime/bufferpool/system_allocator.cc /^static int64_t HUGE_PAGE_SIZE = 2LL * 1024 * 1024;$/;" m namespace:doris file: +HUGE_VALF src/gutil/port.h 785;" d +Handle src/olap/lru_cache.h /^ struct Handle {$/;" s class:doris::CachePriority::Cache +HandleOutputStrings src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::HandleOutputStrings(RowBatch* row_batch,$/;" f class:doris::NewPartitionedAggregationNode +HandleTable src/olap/lru_cache.h /^ HandleTable() : _length(0), _elems(0), _list(NULL) {$/;" f class:doris::CachePriority::HandleTable +HandleTable src/olap/lru_cache.h /^ class HandleTable {$/;" c class:doris::CachePriority +HasMatches src/exec/new_partitioned_hash_table.h /^ bool HasMatches() const { return has_matches_; }$/;" f class:doris::NewPartitionedHashTable +HasMatches src/exec/partitioned_hash_table.h /^ bool HasMatches() const { return _has_matches; }$/;" f class:doris::PartitionedHashTable +HasOneRef src/gutil/ref_counted.cc /^bool RefCountedThreadSafeBase::HasOneRef() const {$/;" f class:doris::subtle::RefCountedThreadSafeBase +HasOneRef src/gutil/ref_counted.h /^ bool HasOneRef() const { return ref_count_ == 1; }$/;" f class:doris::subtle::RefCountedBase +HasPrefixString src/gutil/strings/util.h /^inline bool HasPrefixString(const StringPiece& str,$/;" f +HasSuffixString src/gutil/strings/util.h /^inline bool HasSuffixString(const StringPiece& str,$/;" f +Hash src/exec/new_partitioned_hash_table.cc /^uint32_t NewPartitionedHashTableCtx::Hash(const void* input, int len, uint32_t hash) const {$/;" f class:NewPartitionedHashTableCtx +Hash128to64 src/gutil/hash/hash128to64.h /^inline uint64 Hash128to64(const uint128& x) {$/;" f +Hash16 src/gutil/hash/legacy_hash.h /^typedef uint16 Hash16;$/;" t +Hash32 src/gutil/hash/legacy_hash.h /^typedef uint32 Hash32;$/;" t +Hash32NumWithSeed src/gutil/hash/builtin_type_hash.h /^inline uint32 Hash32NumWithSeed(uint32 num, uint32 c) {$/;" f +Hash32PointerWithSeed src/gutil/hash/builtin_type_hash.h /^inline uint32 Hash32PointerWithSeed(const void* p, uint32 seed) {$/;" f +Hash32StringWithSeed src/gutil/hash/jenkins.cc /^uint32 Hash32StringWithSeed(const char *s, uint32 len, uint32 c) {$/;" f +Hash32StringWithSeedReferenceImplementation src/gutil/hash/jenkins.cc /^uint32 Hash32StringWithSeedReferenceImplementation(const char *s, uint32 len,$/;" f +Hash64DoubleWithSeed src/gutil/hash/builtin_type_hash.h /^inline uint64 Hash64DoubleWithSeed(double num, uint64 seed) {$/;" f +Hash64FloatWithSeed src/gutil/hash/builtin_type_hash.h /^inline uint64 Hash64FloatWithSeed(float num, uint64 seed) {$/;" f +Hash64NumWithSeed src/gutil/hash/builtin_type_hash.h /^inline uint64 Hash64NumWithSeed(uint64 num, uint64 c) {$/;" f +Hash64StringWithSeed src/gutil/hash/hash.h /^inline uint64 Hash64StringWithSeed(const std::string& s, uint64 c) {$/;" f +Hash64StringWithSeed src/gutil/hash/jenkins.cc /^uint64 Hash64StringWithSeed(const char *s, uint32 len, uint64 c) {$/;" f +Hash8 src/gutil/hash/legacy_hash.h /^typedef uint8 Hash8;$/;" t +HashCurrentRow src/exec/partitioned_hash_table.h /^ uint32_t IR_NO_INLINE HashCurrentRow() {$/;" f class:doris::PartitionedHashTableCtx +HashFunctions src/exprs/hash_functions.h /^class HashFunctions {$/;" c namespace:doris +HashJoinNode src/exec/hash_join_node.cpp /^HashJoinNode::HashJoinNode($/;" f class:doris::HashJoinNode +HashJoinNode src/exec/hash_join_node.h /^class HashJoinNode : public ExecNode {$/;" c namespace:doris +HashLen0to16 src/gutil/hash/city.cc /^static uint64 HashLen0to16(const char *s, size_t len) {$/;" f namespace:util_hash +HashLen16 src/gutil/hash/city.cc /^static uint64 HashLen16(uint64 u, uint64 v) {$/;" f namespace:util_hash +HashLen17to32 src/gutil/hash/city.cc /^static uint64 HashLen17to32(const char *s, size_t len) {$/;" f namespace:util_hash +HashLen33to64 src/gutil/hash/city.cc /^static uint64 HashLen33to64(const char *s, size_t len) {$/;" f namespace:util_hash +HashMapEquality src/gutil/stl_util.h /^HashMapEquality(const HashMap& map_a,$/;" f +HashOfSlice src/olap/rowset/segment_v2/binary_dict_page.h /^ struct HashOfSlice {$/;" s class:doris::segment_v2::BinaryDictPageBuilder +HashOfVersion src/olap/olap_common.h /^struct HashOfVersion {$/;" s namespace:doris +HashSetEquality src/gutil/stl_util.h /^HashSetEquality(const HashSet& set_a,$/;" f +HashStringThoroughly src/gutil/hash/string_hash.h /^inline size_t HashStringThoroughly(const char* s, size_t len) {$/;" f +HashStringThoroughlyWithSeed src/gutil/hash/string_hash.h /^inline size_t HashStringThoroughlyWithSeed(const char* s, size_t len,$/;" f +HashStringThoroughlyWithSeeds src/gutil/hash/string_hash.h /^inline size_t HashStringThoroughlyWithSeeds(const char* s, size_t len,$/;" f +HashTNetworkAddressPtr src/util/container_util.hpp /^struct HashTNetworkAddressPtr : public std::unary_function {$/;" s namespace:doris +HashTable src/exec/hash_table.cpp /^HashTable::HashTable(const vector& build_expr_ctxs,$/;" f class:doris::HashTable +HashTable src/exec/hash_table.h /^class HashTable {$/;" c namespace:doris +HashTableReplacedConstants src/exec/new_partitioned_hash_table.h /^ struct HashTableReplacedConstants {$/;" s class:doris::NewPartitionedHashTableCtx +HashTableTest test/exec/hash_table_test.cpp /^ HashTableTest() : _mem_pool() {}$/;" f class:doris::HashTableTest +HashTableTest test/exec/hash_table_test.cpp /^class HashTableTest : public testing::Test {$/;" c namespace:doris file: +HashTo16 src/gutil/hash/legacy_hash.h /^inline uint16 HashTo16(const char *s, uint32 slen) {$/;" f +HashUtil src/util/hash_util.hpp /^class HashUtil {$/;" c namespace:doris +HashVariableLenRow src/exec/new_partitioned_hash_table.cc /^uint32_t NewPartitionedHashTableCtx::HashVariableLenRow(const uint8_t* expr_values,$/;" f class:NewPartitionedHashTableCtx +HdfsCachedFileHandle src/runtime/disk_io_mgr.h /^ class HdfsCachedFileHandle {$/;" c class:doris::DiskIoMgr +HeaderInfo src/exec/decompressor.h /^ struct HeaderInfo {$/;" s class:doris::LzopDecompressor +HealthAction src/http/action/health_action.cpp /^HealthAction::HealthAction(ExecEnv* exec_env) :$/;" f class:doris::HealthAction +HealthAction src/http/action/health_action.h /^class HealthAction : public HttpHandler {$/;" c namespace:doris +Heap src/exec/olap_scan_node.h /^ typedef std::priority_queue, MergeComparison> Heap;$/;" t class:doris::OlapScanNode +HeapAction src/http/action/pprof_actions.cpp /^ HeapAction() {}$/;" f class:doris::HeapAction +HeapAction src/http/action/pprof_actions.cpp /^class HeapAction : public HttpHandler {$/;" c namespace:doris file: +HeapCompare src/runtime/bufferpool/free_list.h /^ inline static bool HeapCompare(const BufferHandle& b1, const BufferHandle& b2) {$/;" f class:doris::FreeList +HeapType src/exec/olap_scan_node.h /^ } HeapType;$/;" t class:doris::OlapScanNode typeref:struct:doris::OlapScanNode::__anon40 +HeartbeatFlags src/runtime/heartbeat_flags.h /^ HeartbeatFlags() : HeartbeatFlags(0) { }$/;" f class:doris::HeartbeatFlags +HeartbeatFlags src/runtime/heartbeat_flags.h /^ HeartbeatFlags(uint64_t origin_flags) : _flags(origin_flags) { }$/;" f class:doris::HeartbeatFlags +HeartbeatFlags src/runtime/heartbeat_flags.h /^class HeartbeatFlags {$/;" c namespace:doris +HeartbeatFlagsTest test/runtime/heartbeat_flags_test.cpp /^class HeartbeatFlagsTest : public testing::Test {$/;" c namespace:doris file: +HeartbeatServer src/agent/heartbeat_server.cpp /^HeartbeatServer::HeartbeatServer(TMasterInfo* master_info) :$/;" f class:doris::HeartbeatServer +HeartbeatServer src/agent/heartbeat_server.h /^class HeartbeatServer : public HeartbeatServiceIf {$/;" c namespace:doris +HexDigitsPrefix src/gutil/strings/numbers.cc /^int HexDigitsPrefix(const char* buf, int num_digits) {$/;" f +HighWaterMarkCounter src/util/runtime_profile.h /^ HighWaterMarkCounter(TUnit::type unit) : Counter(unit) {}$/;" f class:doris::RuntimeProfile::HighWaterMarkCounter +HighWaterMarkCounter src/util/runtime_profile.h /^ class HighWaterMarkCounter : public Counter {$/;" c class:doris::RuntimeProfile +Hint src/olap/skiplist.h /^ struct Hint {$/;" s class:doris::SkipList +HllAggField src/olap/field.h /^ explicit HllAggField(const TabletColumn& column) : Field(column) {$/;" f class:doris::HllAggField +HllAggField src/olap/field.h /^class HllAggField: public Field {$/;" c namespace:doris +HllDataType src/olap/hll.h /^enum HllDataType {$/;" g namespace:doris +HllDppSinkMerge src/runtime/dpp_sink.cpp /^class HllDppSinkMerge {$/;" c namespace:doris file: +HllFunctions src/exprs/hll_function.h /^class HllFunctions {$/;" c namespace:doris +HllFunctionsTest test/exprs/hll_function_test.cpp /^class HllFunctionsTest : public testing::Test {$/;" c namespace:doris file: +HllHashFunctions src/exprs/hll_hash_function.h /^class HllHashFunctions {$/;" c namespace:doris +HllMergeValue src/runtime/dpp_sink.cpp /^ struct HllMergeValue {$/;" s class:doris::HllDppSinkMerge file: +HllSetHelper src/olap/hll.h /^class HllSetHelper {$/;" c namespace:doris +HllSetResolver src/olap/hll.h /^ HllSetResolver() : _buf_ref(nullptr),$/;" f class:doris::HllSetResolver +HllSetResolver src/olap/hll.h /^class HllSetResolver {$/;" c namespace:doris +HllVal output/udf/include/udf.h /^ HllVal() : StringVal() { }$/;" f struct:doris_udf::HllVal +HllVal output/udf/include/udf.h /^struct HllVal : public StringVal {$/;" s namespace:doris_udf +HllVal src/udf/udf.h /^ HllVal() : StringVal() { }$/;" f struct:doris_udf::HllVal +HllVal src/udf/udf.h /^struct HllVal : public StringVal {$/;" s namespace:doris_udf +HtData src/exec/new_partitioned_hash_table.h /^ union HtData {$/;" u class:doris::NewPartitionedHashTable +HtData src/exec/partitioned_hash_table.h /^ union HtData {$/;" u class:doris::PartitionedHashTable +HttpChannel src/http/http_channel.h /^class HttpChannel {$/;" c namespace:doris +HttpChunkParseCtx src/http/http_parser.h /^ HttpChunkParseCtx() : state(0), size(0), length(0) { }$/;" f struct:doris::HttpChunkParseCtx +HttpChunkParseCtx src/http/http_parser.h /^struct HttpChunkParseCtx {$/;" s namespace:doris +HttpClient src/http/http_client.cpp /^HttpClient::HttpClient() {$/;" f class:doris::HttpClient +HttpClient src/http/http_client.h /^class HttpClient {$/;" c namespace:doris +HttpClientTest test/http/http_client_test.cpp /^ HttpClientTest() { }$/;" f class:doris::HttpClientTest +HttpClientTest test/http/http_client_test.cpp /^class HttpClientTest : public testing::Test {$/;" c namespace:doris file: +HttpClientTestSimpleGetHandler test/http/http_client_test.cpp /^class HttpClientTestSimpleGetHandler : public HttpHandler {$/;" c namespace:doris file: +HttpClientTestSimplePostHandler test/http/http_client_test.cpp /^class HttpClientTestSimplePostHandler : public HttpHandler {$/;" c namespace:doris file: +HttpHandler src/http/http_handler.h /^class HttpHandler {$/;" c namespace:doris +HttpHeaders src/http/http_headers.h /^class HttpHeaders {$/;" c namespace:doris +HttpMethod src/http/http_method.h /^enum HttpMethod {$/;" g namespace:doris +HttpParser src/http/http_parser.h /^class HttpParser {$/;" c namespace:doris +HttpRequest src/http/http_request.cpp /^HttpRequest::HttpRequest(evhttp_request* evhttp_request)$/;" f class:doris::HttpRequest +HttpRequest src/http/http_request.h /^class HttpRequest {$/;" c namespace:doris +HttpResponse src/http/http_response.cpp /^HttpResponse::HttpResponse($/;" f class:doris::HttpResponse +HttpResponse src/http/http_response.cpp /^HttpResponse::HttpResponse(const HttpStatus& status) :$/;" f class:doris::HttpResponse +HttpResponse src/http/http_response.cpp /^HttpResponse::HttpResponse(const HttpStatus& status, const std::string* content) :$/;" f class:doris::HttpResponse +HttpResponse src/http/http_response.h /^class HttpResponse {$/;" c namespace:doris +HttpService src/service/http_service.cpp /^HttpService::HttpService(ExecEnv* env, int port, int num_threads)$/;" f class:doris::HttpService +HttpService src/service/http_service.h /^class HttpService {$/;" c namespace:doris +HttpStatus src/http/http_status.h /^enum HttpStatus {$/;" g namespace:doris +HttpUtilsTest test/http/http_utils_test.cpp /^ HttpUtilsTest() { }$/;" f class:doris::HttpUtilsTest +HttpUtilsTest test/http/http_utils_test.cpp /^class HttpUtilsTest : public testing::Test {$/;" c namespace:doris file: +HybirdMap src/exprs/hybird_map.h /^ HybirdMap(PrimitiveType type) : _type(type) {$/;" f class:doris::HybirdMap +HybirdMap src/exprs/hybird_map.h /^class HybirdMap {$/;" c namespace:doris +HybirdSet src/exprs/hybird_set.h /^ HybirdSet() {$/;" f class:doris::HybirdSet +HybirdSet src/exprs/hybird_set.h /^class HybirdSet : public HybirdSetBase {$/;" c namespace:doris +HybirdSetBase src/exprs/hybird_set.h /^ HybirdSetBase() {$/;" f class:doris::HybirdSetBase +HybirdSetBase src/exprs/hybird_set.h /^class HybirdSetBase {$/;" c namespace:doris +HybirdSetTest test/exprs/hybird_set_test.cpp /^ HybirdSetTest() {$/;" f class:doris::HybirdSetTest +HybirdSetTest test/exprs/hybird_set_test.cpp /^class HybirdSetTest : public testing::Test {$/;" c namespace:doris file: +HyperLogLog src/olap/hll.cpp /^HyperLogLog::HyperLogLog(const Slice& src) {$/;" f class:doris::HyperLogLog +HyperLogLog src/olap/hll.h /^ explicit HyperLogLog(uint64_t hash_value): _type(HLL_DATA_EXPLICIT) {$/;" f class:doris::HyperLogLog +HyperLogLog src/olap/hll.h /^class HyperLogLog {$/;" c namespace:doris +IBinaryReader src/olap/push_handler.h /^ IBinaryReader()$/;" f class:doris::IBinaryReader +IBinaryReader src/olap/push_handler.h /^class IBinaryReader {$/;" c namespace:doris +IDLE src/util/threadpool.h /^ IDLE,$/;" m class:doris::ThreadPoolToken::State +IDRange src/olap/olap_index.h /^struct IDRange {$/;" s namespace:doris +IDX_MASK src/runtime/buffered_tuple_stream.h /^ static const uint64_t IDX_MASK = 0xFFFFFF0000000000;$/;" m struct:doris::BufferedTupleStream::RowIdx +IDX_MASK src/runtime/buffered_tuple_stream2.h /^ static const uint64_t IDX_MASK = 0xFFFFFF0000000000;$/;" m struct:doris::BufferedTupleStream2::RowIdx +IDX_SHIFT src/runtime/buffered_tuple_stream.h /^ static const uint64_t IDX_SHIFT = 40;$/;" m struct:doris::BufferedTupleStream::RowIdx +IDX_SHIFT src/runtime/buffered_tuple_stream2.h /^ static const uint64_t IDX_SHIFT = 40;$/;" m struct:doris::BufferedTupleStream2::RowIdx +ID_VOID_MAIN build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 6;" d file: +IF_COMPUTE_FUNCTION src/exprs/conditional_functions_ir.cpp 75;" d file: +IF_MATCH src/http/http_headers.cpp /^const char* HttpHeaders::IF_MATCH = "If-Match";$/;" m class:doris::HttpHeaders file: +IF_MATCH src/http/http_headers.h /^ static const char* IF_MATCH;$/;" m class:doris::HttpHeaders +IF_MODIFIED_SINCE src/http/http_headers.cpp /^const char* HttpHeaders::IF_MODIFIED_SINCE = "If-Modified-Since";$/;" m class:doris::HttpHeaders file: +IF_MODIFIED_SINCE src/http/http_headers.h /^ static const char* IF_MODIFIED_SINCE;$/;" m class:doris::HttpHeaders +IF_NONE_MATCH src/http/http_headers.cpp /^const char* HttpHeaders::IF_NONE_MATCH = "If-None-Match";$/;" m class:doris::HttpHeaders file: +IF_NONE_MATCH src/http/http_headers.h /^ static const char* IF_NONE_MATCH;$/;" m class:doris::HttpHeaders +IF_NULL_COMPUTE_FUNCTION src/exprs/conditional_functions_ir.cpp 24;" d file: +IF_RANGE src/http/http_headers.cpp /^const char* HttpHeaders::IF_RANGE = "If-Range";$/;" m class:doris::HttpHeaders file: +IF_RANGE src/http/http_headers.h /^ static const char* IF_RANGE;$/;" m class:doris::HttpHeaders +IF_UNMODIFIED_SINCE src/http/http_headers.cpp /^const char* HttpHeaders::IF_UNMODIFIED_SINCE = "If-Unmodified-Since";$/;" m class:doris::HttpHeaders file: +IF_UNMODIFIED_SINCE src/http/http_headers.h /^ static const char* IF_UNMODIFIED_SINCE;$/;" m class:doris::HttpHeaders +IMPALA_COMMON_LOGGING_H src/common/logging.h 19;" d +IMPALA_EXEC_KUDU_SCAN_NODE_H_ src/exec/kudu_scan_node.h 19;" d +IMPALA_EXPRS_AGG_FN_EVALUATOR_H src/exprs/new_agg_fn_evaluator.h 19;" d +IMPALA_RUNTIME_BUFFERPOOL_SUBALLOCATOR_H src/runtime/bufferpool/suballocator.h 19;" d +IMPALA_RUNTIME_SYSTEM_ALLOCATOR_H src/runtime/bufferpool/system_allocator.h 19;" d +IMPALA_UTIL_BIT_STREAM_UTILS_INLINE_H src/util/bit_stream_utils.inline.h 18;" d +IMPALA_UTIL_KUDU_UTIL_H_ src/exec/kudu_util.h 19;" d +IMPALA_UTIL_PRETTY_PRINTER_H src/util/pretty_printer.h 19;" d +IMPALA_UTIL_TEMPLATE_UTIL_H src/util/template_util.h 19;" d +INCREMENTAL_DELTA_PREFIX src/olap/olap_define.h /^static const std::string INCREMENTAL_DELTA_PREFIX = "\/incremental_delta";$/;" m namespace:doris +INF_DORIS_BE_SRC_EXEC_ANALYTIC_EVAL_NODE_H src/exec/analytic_eval_node.h 19;" d +INF_DORIS_BE_SRC_UTIL_TUPLE_ROW_COMPARE_H src/util/tuple_row_compare.h 19;" d +INF_DORIS_QE_SRC_BE_EXEC_SORT_EXEC_EXPRS_H src/exec/sort_exec_exprs.h 19;" d +INF_DORIS_QE_SRC_BE_EXEC_SORT_NODE_H src/exec/sort_node.h 19;" d +INF_DORIS_QE_SRC_BE_RUNTIME_BUFFERED_TUPLE_STREAM_H src/runtime/buffered_tuple_stream.h 19;" d +INF_DORIS_QE_SRC_BE_SRC_RUNTIME_BUFFERED_BLOCK_MGR_H src/runtime/buffered_block_mgr.h 19;" d +INITED src/olap/compaction.h /^ INITED = 0,$/;" e enum:doris::Compaction::CompactionState +INITIAL_BLOCK_SIZES src/runtime/buffered_tuple_stream.cpp /^static const int64_t INITIAL_BLOCK_SIZES[] =$/;" m namespace:doris file: +INITIAL_BLOCK_SIZES src/runtime/buffered_tuple_stream2.cc /^static const int64_t INITIAL_BLOCK_SIZES[] = { 64 * 1024, 512 * 1024 };$/;" m namespace:doris file: +INITIAL_CHUNK_SIZE src/runtime/mem_pool.cpp /^const int MemPool::INITIAL_CHUNK_SIZE;$/;" m class:doris::MemPool file: +INITIAL_CHUNK_SIZE src/runtime/mem_pool.h /^ static const int INITIAL_CHUNK_SIZE = 4 * 1024;$/;" m class:doris::MemPool +INITIAL_DATA_PAGE_SIZES src/exec/partitioned_hash_table.cc /^static const int64_t INITIAL_DATA_PAGE_SIZES[] = { 64 * 1024, 512 * 1024 };$/;" m namespace:doris file: +INIT_HEAP src/exec/olap_scan_node.h /^ INIT_HEAP = 2,$/;" e enum:doris::TransferStatus +INPUT_CHUNK src/exec/plain_text_line_reader.cpp 27;" d file: +INSERTION_SORT_THRESHOLD src/util/radix_sort.h /^ static constexpr size_t INSERTION_SORT_THRESHOLD = 64;$/;" m struct:doris::RadixSort +INSERTION_THRESHOLD src/runtime/merge_sorter.cpp /^ static const int INSERTION_THRESHOLD = 16;$/;" m class:doris::MergeSorter::TupleSorter file: +INSERTION_THRESHOLD src/runtime/spill_sorter.cc /^ static const int INSERTION_THRESHOLD = 16;$/;" m class:doris::SpillSorter::TupleSorter file: +INTERNAL_SERVER_ERROR src/http/http_status.h /^ INTERNAL_SERVER_ERROR = 500,$/;" e enum:doris::HttpStatus +INT_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t INT_SIZE = sizeof(int32_t);$/;" m class:doris::AggFnEvaluator +INT_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t INT_SIZE = sizeof(int32_t);$/;" m class:doris::NewAggFnEvaluator +INVALID src/util/url_parser.h /^ INVALID,$/;" e enum:doris::UrlParser::UrlPart +INVALID_CHILD_IDX src/exec/merge_node.h /^ const static int INVALID_CHILD_IDX = -1;$/;" m class:doris::MergeNode +INVALID_IDX src/runtime/descriptors.cpp /^const int RowDescriptor::INVALID_IDX = -1;$/;" m class:doris::RowDescriptor file: +INVALID_IDX src/runtime/descriptors.h /^ static const int INVALID_IDX;$/;" m class:doris::RowDescriptor +INVALID_ROW_INDEX src/runtime/row_batch.h /^ static const int INVALID_ROW_INDEX = -1;$/;" m class:doris::RowBatch +INVALID_TID src/util/thread.h /^ INVALID_TID = -1,$/;" e enum:doris::Thread::__anon38 +INVALID_TYPE output/udf/include/udf.h /^ INVALID_TYPE = 0,$/;" e enum:doris_udf::FunctionContext::Type +INVALID_TYPE src/runtime/primitive_type.h /^ INVALID_TYPE = 0,$/;" e enum:doris::PrimitiveType +INVALID_TYPE src/udf/udf.h /^ INVALID_TYPE = 0,$/;" e enum:doris_udf::FunctionContext::Type +IN_FUNCTIONS src/exprs/new_in_predicate.cpp 149;" d file: +IN_FUNCTIONS_STMT src/exprs/new_in_predicate.h 26;" d +IN_LIST_PRED_BITMAP_EVALUATE src/olap/in_list_predicate.cpp 112;" d file: +IN_LIST_PRED_BITMAP_EVALUATE_DECLARATION src/olap/in_list_predicate.cpp 193;" d file: +IN_LIST_PRED_CLASS_DEFINE src/olap/in_list_predicate.h 31;" d +IN_LIST_PRED_COLUMN_BLOCK_EVALUATE src/olap/in_list_predicate.cpp 87;" d file: +IN_LIST_PRED_COLUMN_BLOCK_EVALUATE_DECLARATION src/olap/in_list_predicate.cpp 177;" d file: +IN_LIST_PRED_CONSTRUCTOR src/olap/in_list_predicate.cpp 25;" d file: +IN_LIST_PRED_CONSTRUCTOR_DECLARATION src/olap/in_list_predicate.cpp 145;" d file: +IN_LIST_PRED_EVALUATE src/olap/in_list_predicate.cpp 34;" d file: +IN_LIST_PRED_EVALUATE_DECLARATION src/olap/in_list_predicate.cpp 161;" d file: +IOError src/common/status.h /^ static Status IOError(const Slice& msg,$/;" f class:doris::Status +IR_ALWAYS_INLINE src/codegen/doris_ir.h 29;" d +IR_ALWAYS_INLINE src/codegen/doris_ir.h 32;" d +IR_NO_INLINE src/codegen/doris_ir.h 28;" d +IR_NO_INLINE src/codegen/doris_ir.h 31;" d +IS_BIG_ENDIAN src/gutil/port.h 159;" d +IS_BIG_ENDIAN src/gutil/port.h 167;" d +IS_LITTLE_ENDIAN src/gutil/port.h 155;" d +IS_LITTLE_ENDIAN src/gutil/port.h 165;" d +IS_OCTAL_DIGIT src/gutil/strings/escaping.cc 75;" d file: +IS__MSC_VER src/gutil/stringprintf.cc /^enum { IS__MSC_VER = 1 };$/;" e enum:__anon22 file: +ITERATE src/exprs/new_in_predicate.h /^ ITERATE$/;" e enum:doris::InPredicate::Strategy +I_WRAP_SONAME_FNNAME_ZU src/gutil/valgrind.h 584;" d +I_WRAP_SONAME_FNNAME_ZZ src/gutil/valgrind.h 587;" d +IdleThread src/util/threadpool.h /^ explicit IdleThread(Mutex* m)$/;" f struct:doris::ThreadPool::IdleThread +IdleThread src/util/threadpool.h /^ struct IdleThread : public boost::intrusive::list_base_hook<> {$/;" s class:doris::ThreadPool +IfExpr src/exprs/conditional_functions.h /^class IfExpr : public Expr {$/;" c namespace:doris +IfNullExpr src/exprs/conditional_functions.h /^class IfNullExpr : public Expr {$/;" c namespace:doris +ImmediateSuccessor src/gutil/strings/util.cc /^string ImmediateSuccessor(const StringPiece& s) {$/;" f +InOpTest test/exprs/in_op_test.cpp /^ InOpTest() : $/;" f class:doris::InOpTest +InOpTest test/exprs/in_op_test.cpp /^class InOpTest : public ::testing::Test {$/;" c namespace:doris file: +InPredicate src/exprs/in_predicate.cpp /^InPredicate::InPredicate(const TExprNode& node) : $/;" f class:doris::InPredicate +InPredicate src/exprs/in_predicate.h /^class InPredicate : public Predicate {$/;" c namespace:doris +InPredicate src/exprs/new_in_predicate.h /^class InPredicate {$/;" c namespace:doris +InPredicateTest test/exprs/in_predicate_test.cpp /^ InPredicateTest() : _runtime_stat("abc") {$/;" f class:doris::InPredicateTest +InPredicateTest test/exprs/in_predicate_test.cpp /^class InPredicateTest : public testing::Test {$/;" c namespace:doris file: +InStream src/olap/in_stream.cpp /^InStream::InStream($/;" f class:doris::InStream +InStream src/olap/in_stream.h /^class InStream {$/;" c namespace:doris +InStreamBufferWrapper src/olap/in_stream.h /^ InStreamBufferWrapper(InStream* input) : $/;" f class:doris::InStreamBufferWrapper +InStreamBufferWrapper src/olap/in_stream.h /^class InStreamBufferWrapper : public std::streambuf {$/;" c namespace:doris +Inactive src/runtime/disk_io_mgr_internal.h /^ Inactive,$/;" e enum:doris::DiskIoMgr::RequestContext::State +IncreaseReservation src/runtime/bufferpool/buffer_pool.cc /^bool BufferPool::ClientHandle::IncreaseReservation(int64_t bytes) {$/;" f class:doris::BufferPool::ClientHandle +IncreaseReservation src/runtime/bufferpool/reservation_tracker.cc /^bool ReservationTracker::IncreaseReservation(int64_t bytes) {$/;" f class:doris::ReservationTracker +IncreaseReservationInternalLocked src/runtime/bufferpool/reservation_tracker.cc /^bool ReservationTracker::IncreaseReservationInternalLocked($/;" f class:doris::ReservationTracker +IncreaseReservationToFit src/runtime/bufferpool/buffer_pool.cc /^bool BufferPool::ClientHandle::IncreaseReservationToFit(int64_t bytes) {$/;" f class:doris::BufferPool::ClientHandle +IncreaseReservationToFit src/runtime/bufferpool/reservation_tracker.cc /^bool ReservationTracker::IncreaseReservationToFit(int64_t bytes) {$/;" f class:doris::ReservationTracker +IndexChannel src/exec/tablet_sink.h /^ IndexChannel(OlapTableSink* parent, int64_t index_id, int32_t schema_hash)$/;" f class:doris::stream_load::IndexChannel +IndexChannel src/exec/tablet_sink.h /^class IndexChannel {$/;" c namespace:doris::stream_load +IndexComparator src/olap/olap_index.h /^ IndexComparator(const MemIndex* index, RowCursor* cursor) :$/;" f class:doris::IndexComparator +IndexComparator src/olap/olap_index.h /^class IndexComparator {$/;" c namespace:doris +IndexPageBuilder src/olap/rowset/segment_v2/index_page.h /^ explicit IndexPageBuilder(size_t index_page_size, bool is_leaf)$/;" f class:doris::segment_v2::IndexPageBuilder +IndexPageBuilder src/olap/rowset/segment_v2/index_page.h /^class IndexPageBuilder {$/;" c namespace:doris::segment_v2 +IndexPageIterator src/olap/rowset/segment_v2/index_page.h /^ explicit IndexPageIterator(const IndexPageReader* reader)$/;" f class:doris::segment_v2::IndexPageIterator +IndexPageIterator src/olap/rowset/segment_v2/index_page.h /^class IndexPageIterator {$/;" c namespace:doris::segment_v2 +IndexPageReader src/olap/rowset/segment_v2/index_page.h /^ IndexPageReader() : _parsed(false) {}$/;" f class:doris::segment_v2::IndexPageReader +IndexPageReader src/olap/rowset/segment_v2/index_page.h /^class IndexPageReader {$/;" c namespace:doris::segment_v2 +IndexedColumnIterator src/olap/rowset/segment_v2/indexed_column_reader.h /^ explicit IndexedColumnIterator(const IndexedColumnReader* reader)$/;" f class:doris::segment_v2::IndexedColumnIterator +IndexedColumnIterator src/olap/rowset/segment_v2/indexed_column_reader.h /^class IndexedColumnIterator {$/;" c namespace:doris::segment_v2 +IndexedColumnReader src/olap/rowset/segment_v2/indexed_column_reader.h /^ explicit IndexedColumnReader(const std::string& file_name,$/;" f class:doris::segment_v2::IndexedColumnReader +IndexedColumnReader src/olap/rowset/segment_v2/indexed_column_reader.h /^class IndexedColumnReader {$/;" c namespace:doris::segment_v2 +IndexedColumnWriter src/olap/rowset/segment_v2/indexed_column_writer.cpp /^IndexedColumnWriter::IndexedColumnWriter(const IndexedColumnWriterOptions& options,$/;" f class:doris::segment_v2::IndexedColumnWriter +IndexedColumnWriter src/olap/rowset/segment_v2/indexed_column_writer.h /^class IndexedColumnWriter {$/;" c namespace:doris::segment_v2 +IndexedColumnWriterOptions src/olap/rowset/segment_v2/indexed_column_writer.h /^struct IndexedColumnWriterOptions {$/;" s namespace:doris::segment_v2 +InetAddress src/util/network_util.cpp /^InetAddress::InetAddress(struct sockaddr* addr) {$/;" f class:doris::InetAddress +InetAddress src/util/network_util.h /^class InetAddress {$/;" c namespace:doris +InfoFunc src/exprs/info_func.cpp /^InfoFunc::InfoFunc(const TExprNode& node) :$/;" f class:doris::InfoFunc +InfoFunc src/exprs/info_func.h /^class InfoFunc : public Expr {$/;" c namespace:doris +InfoStrings src/util/runtime_profile.h /^ typedef std::map InfoStrings;$/;" t class:doris::RuntimeProfile +InfoStringsDisplayOrder src/util/runtime_profile.h /^ typedef std::vector InfoStringsDisplayOrder;$/;" t class:doris::RuntimeProfile +Init src/exec/new_partitioned_hash_table.cc /^Status NewPartitionedHashTable::Init(bool* got_memory) {$/;" f class:NewPartitionedHashTable +Init src/exec/new_partitioned_hash_table.cc /^Status NewPartitionedHashTableCtx::ExprValuesCache::Init(RuntimeState* state,$/;" f class:NewPartitionedHashTableCtx::ExprValuesCache +Init src/exec/new_partitioned_hash_table.cc /^Status NewPartitionedHashTableCtx::Init(ObjectPool* pool, RuntimeState* state, int num_build_tuples,$/;" f class:NewPartitionedHashTableCtx +Init src/exprs/agg_fn.cc /^Status AggFn::Init(const RowDescriptor& row_desc, RuntimeState* state) {$/;" f class:doris::AggFn +Init src/exprs/new_agg_fn_evaluator.cc /^void NewAggFnEvaluator::Init(Tuple* dst) {$/;" f class:NewAggFnEvaluator +Init src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::Init(const std::vector& evals, Tuple* dst) {$/;" f class:doris::NewAggFnEvaluator +Init src/gutil/charmap.h /^ void Init(const char* str, int len) {$/;" f class:Charmap +Init src/gutil/once.h /^ void Init(void (*func_with_arg)(T*), T* arg) {$/;" f class:GoogleOnceDynamic +Init src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::Init(int node_id, bool pinned) {$/;" f class:BufferedTupleStream3 +Init src/runtime/initial_reservations.cc /^Status InitialReservations::Init($/;" f class:doris::InitialReservations +Init src/runtime/mem_tracker.cpp /^void MemTracker::Init() {$/;" f class:doris::MemTracker +Init src/util/os_info.cpp /^void OsInfo::Init() {$/;" f class:doris::OsInfo +InitAggSlots src/exec/new_partitioned_aggregation_node.cc /^void NewPartitionedAggregationNode::InitAggSlots($/;" f class:doris::NewPartitionedAggregationNode +InitBlockMgr test/runtime/buffered_tuple_stream2_test.cpp /^ void InitBlockMgr(int64_t limit, int block_size) {$/;" f class:doris::SimpleTupleStreamTest +InitChildTracker src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::InitChildTracker(RuntimeProfile* profile,$/;" f class:doris::ReservationTracker +InitCounters src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::InitCounters($/;" f class:doris::ReservationTracker +InitFn output/udf/include/uda_test_harness.h /^ typedef void (*InitFn)(FunctionContext* context, INTERMEDIATE* result);$/;" t class:doris_udf::UdaTestHarnessBase +InitFn src/exprs/agg_fn_evaluator.cpp /^typedef void (*InitFn)(FunctionContext*, AnyVal*);$/;" t namespace:doris file: +InitFn src/exprs/new_agg_fn_evaluator.cc /^typedef void (*InitFn)(FunctionContext*, AnyVal*);$/;" t file: +InitFn src/udf/uda_test_harness.h /^ typedef void (*InitFn)(FunctionContext* context, INTERMEDIATE* result);$/;" t class:doris_udf::UdaTestHarnessBase +InitHashTable src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::Partition::InitHashTable(bool* got_memory) {$/;" f class:doris::NewPartitionedAggregationNode::Partition +InitKuduLogging src/exec/kudu_util.cpp /^void InitKuduLogging() {$/;" f namespace:doris +InitModule src/gutil/spinlock_linux-inl.h /^ InitModule() {$/;" f struct:__anon20::InitModule +InitModule src/gutil/spinlock_linux-inl.h /^static struct InitModule {$/;" s namespace:__anon20 +InitMultipleTmpDirs test/runtime/buffered_block_mgr2_test.cpp /^ vector InitMultipleTmpDirs(int num_dirs) {$/;" f class:doris::BufferedBlockMgrTest +InitRootTracker src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::InitRootTracker($/;" f class:doris::ReservationTracker +InitStreams src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::Partition::InitStreams() {$/;" f class:doris::NewPartitionedAggregationNode::Partition +InitVectorizedBatch test/olap/in_list_predicate_test.cpp /^ void InitVectorizedBatch(const TabletSchema* tablet_schema,$/;" f class:doris::TestInListPredicate +InitVectorizedBatch test/olap/null_predicate_test.cpp /^ void InitVectorizedBatch(const TabletSchema* tablet_schema,$/;" f class:doris::TestNullPredicate +InitialReservations src/runtime/initial_reservations.cc /^InitialReservations::InitialReservations(ObjectPool* obj_pool,$/;" f class:doris::InitialReservations +InitialReservations src/runtime/initial_reservations.h /^class InitialReservations {$/;" c namespace:doris +Initialize src/gutil/cpu.cc /^void CPU::Initialize() {$/;" f class:base::CPU +Initialize src/gutil/int128.h /^inline void uint128::Initialize(uint64 top, uint64 bottom) {$/;" f class:uint128 +InitializeSystemInfo src/gutil/sysinfo.cc /^static void InitializeSystemInfo() {$/;" f namespace:base +Initialized src/util/monotime.cpp /^bool MonoDelta::Initialized() const {$/;" f class:doris::MonoDelta +Initialized src/util/monotime.cpp /^bool MonoTime::Initialized() const {$/;" f class:doris::MonoTime +InputType output/udf/include/udf.h /^typedef AnyVal InputType;$/;" t namespace:doris_udf +InputType src/udf/udf.h /^typedef AnyVal InputType;$/;" t namespace:doris_udf +InputType2 output/udf/include/udf.h /^typedef AnyVal InputType2;$/;" t namespace:doris_udf +InputType2 src/udf/udf.h /^typedef AnyVal InputType2;$/;" t namespace:doris_udf +Insert src/exec/new_partitioned_hash_table.inline.h /^inline bool NewPartitionedHashTable::Insert(NewPartitionedHashTableCtx* ht_ctx,$/;" f class:doris::NewPartitionedHashTable +Insert src/olap/skiplist.h /^void SkipList::Insert(const Key& key, bool* overwritten) {$/;" f class:doris::SkipList +Insert test/olap/lru_cache_test.cpp /^ void Insert(int key, int value, int charge) {$/;" f class:doris::CacheTest +InsertAndDeleteExisting src/gutil/map-util.h /^bool InsertAndDeleteExisting($/;" f +InsertDuplicateNode src/exec/new_partitioned_hash_table.inline.h /^inline NewPartitionedHashTable::DuplicateNode* NewPartitionedHashTable::InsertDuplicateNode($/;" f class:doris::NewPartitionedHashTable +InsertDurable test/olap/lru_cache_test.cpp /^ void InsertDurable(int key, int value, int charge) {$/;" f class:doris::CacheTest +InsertFullTest test/exec/partitioned_hash_table_test.cpp /^ void InsertFullTest(bool quadratic, int table_size) {$/;" f class:doris::PartitionedHashTableTest +InsertIfNotPresent src/gutil/map-util.h /^bool InsertIfNotPresent($/;" f +InsertIfNotPresent src/gutil/map-util.h /^bool InsertIfNotPresent(Collection* const collection,$/;" f +InsertInMap src/gutil/strings/split_internal.h /^ typename Map::iterator InsertInMap($/;" f class:strings::internal::Splitter +InsertInMap src/gutil/strings/split_internal.h /^ typename std::multimap::iterator InsertInMap($/;" f class:strings::internal::Splitter +InsertInternal src/exec/new_partitioned_hash_table.inline.h /^inline NewPartitionedHashTable::HtData* NewPartitionedHashTable::InsertInternal($/;" f class:doris::NewPartitionedHashTable +InsertKeyOrDie src/gutil/map-util.h /^typename Collection::mapped_type& InsertKeyOrDie($/;" f +InsertKeysFromMap src/gutil/map-util.h /^void InsertKeysFromMap(const MapContainer& map_container,$/;" f +InsertOrDie src/gutil/map-util.h /^void InsertOrDie(Collection* const collection,$/;" f +InsertOrDieNoPrint src/gutil/map-util.h /^void InsertOrDieNoPrint($/;" f +InsertOrDieNoPrint src/gutil/map-util.h /^void InsertOrDieNoPrint(Collection* const collection,$/;" f +InsertOrReturnExisting src/gutil/map-util.h /^InsertOrReturnExisting($/;" f class:Collection +InsertOrReturnExisting src/gutil/map-util.h /^InsertOrReturnExisting(Collection* const collection,$/;" f class:Collection +InsertOrUpdate src/gutil/map-util.h /^bool InsertOrUpdate(Collection* const collection,$/;" f +InsertOrUpdateMany src/gutil/map-util.h /^void InsertOrUpdateMany(Collection* const collection,$/;" f +InsertString src/gutil/strings/util.cc /^void InsertString(string *const s,$/;" f +InsertWithHint src/olap/skiplist.h /^void SkipList::InsertWithHint(const Key& key, bool is_exist, Hint* hint) {$/;" f class:doris::SkipList +Int128Comparator src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^struct Int128Comparator {$/;" s namespace:doris::segment_v2::__anon52 file: +Int64ToString src/gutil/strings/numbers.cc /^string Int64ToString(int64 i64, const char* format) {$/;" f +IntGaugeMetricsMap src/util/doris_metrics.h /^class IntGaugeMetricsMap {$/;" c namespace:doris +IntNode test/util/internal_queue_test.cpp /^ IntNode() : value() {}$/;" f struct:doris::IntNode +IntNode test/util/internal_queue_test.cpp /^ IntNode(int value) : value(value) {}$/;" f struct:doris::IntNode +IntNode test/util/internal_queue_test.cpp /^struct IntNode : public InternalQueue::Node {$/;" s namespace:doris file: +IntToString src/gutil/strings/numbers.cc /^string IntToString(int i, const char* format) {$/;" f +IntVal output/udf/include/udf.h /^ IntVal() : val(0) { }$/;" f struct:doris_udf::IntVal +IntVal output/udf/include/udf.h /^ IntVal(int32_t val) : val(val) { }$/;" f struct:doris_udf::IntVal +IntVal output/udf/include/udf.h /^struct IntVal : public AnyVal {$/;" s namespace:doris_udf +IntVal src/udf/udf.h /^ IntVal() : val(0) { }$/;" f struct:doris_udf::IntVal +IntVal src/udf/udf.h /^ IntVal(int32_t val) : val(val) { }$/;" f struct:doris_udf::IntVal +IntVal src/udf/udf.h /^struct IntVal : public AnyVal {$/;" s namespace:doris_udf +IntWrapper src/exprs/scalar_fn_call.cpp /^typedef IntVal (*IntWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +IntegerColumnReader src/olap/rowset/column_reader.cpp /^IntegerColumnReader::IntegerColumnReader(uint32_t column_unique_id):$/;" f class:doris::IntegerColumnReader +IntegerColumnReader src/olap/rowset/column_reader.h /^class IntegerColumnReader {$/;" c namespace:doris +IntegerColumnReaderWrapper src/olap/rowset/column_reader.h /^ IntegerColumnReaderWrapper(uint32_t column_id, uint32_t column_unique_id) :$/;" f class:doris::IntegerColumnReaderWrapper +IntegerColumnReaderWrapper src/olap/rowset/column_reader.h /^class IntegerColumnReaderWrapper : public ColumnReader {$/;" c namespace:doris +IntegerColumnWriter src/olap/rowset/column_writer.cpp /^IntegerColumnWriter::IntegerColumnWriter($/;" f class:doris::IntegerColumnWriter +IntegerColumnWriter src/olap/rowset/column_writer.h /^class IntegerColumnWriter {$/;" c namespace:doris +IntegerColumnWriterWrapper src/olap/rowset/column_writer.h /^ IntegerColumnWriterWrapper($/;" f class:doris::IntegerColumnWriterWrapper +IntegerColumnWriterWrapper src/olap/rowset/column_writer.h /^class IntegerColumnWriterWrapper : public ColumnWriter {$/;" c namespace:doris +IntelMicroArchitecture src/gutil/cpu.h /^ enum IntelMicroArchitecture {$/;" g class:base::final +IntermediateType output/udf/include/udf.h /^typedef AnyVal IntermediateType;$/;" t namespace:doris_udf +IntermediateType src/udf/udf.h /^typedef AnyVal IntermediateType;$/;" t namespace:doris_udf +InternalError src/common/status.h /^ static Status InternalError(const Slice& msg,$/;" f class:doris::Status +InternalFastHexToBuffer src/gutil/strings/numbers.cc /^char *InternalFastHexToBuffer(uint64 value, char* buffer, int num_byte) {$/;" f +InternalList src/util/internal_queue.h /^class InternalList : public InternalQueueBase {};$/;" c namespace:doris +InternalQueue src/util/internal_queue.h /^class InternalQueue : public InternalQueueBase {};$/;" c namespace:doris +InternalQueueBase src/util/internal_queue.h /^ InternalQueueBase() : head_(NULL), tail_(NULL), size_(0) {}$/;" f class:doris::InternalQueueBase +InternalQueueBase src/util/internal_queue.h /^class InternalQueueBase {$/;" c namespace:doris +InternalStringRepGCC4 src/gutil/stl_util.h /^struct InternalStringRepGCC4 {$/;" s +IntersectNode src/exec/intersect_node.cpp /^IntersectNode::IntersectNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)$/;" f class:doris::IntersectNode +IntersectNode src/exec/intersect_node.h /^class IntersectNode : public ExecNode {$/;" c namespace:doris +IntersectsWith src/gutil/charmap.h /^ bool IntersectsWith(const Charmap & c) const {$/;" f class:Charmap +InvalidArgument src/common/status.h /^ static Status InvalidArgument(const Slice& msg, int16_t precise_code = 1, const Slice& msg2 = Slice()) {$/;" f class:doris::Status +InvalidateReadIterator src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::InvalidateReadIterator() {$/;" f class:BufferedTupleStream3 +InvalidateWriteIterator src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::InvalidateWriteIterator() {$/;" f class:BufferedTupleStream3 +IsAggFn src/exprs/agg_fn.h /^ virtual bool IsAggFn() const { return true; }$/;" f class:doris::AggFn +IsAscii src/gutil/strings/util.cc /^bool IsAscii(const char* str, int len) {$/;" f +IsAscii src/gutil/strings/util.h /^inline bool IsAscii(const StringPiece& str) {$/;" f +IsClassHelper src/gutil/template_util.h /^struct IsClassHelper {$/;" s namespace:base +IsClosing src/gutil/strings/split.cc /^ bool IsClosing(char c) const {$/;" f class:__anon31::ClosingSymbolLookup +IsFixedValueRangeVisitor src/exec/olap_scan_node.h /^ class IsFixedValueRangeVisitor : public boost::static_visitor {$/;" c class:doris::OlapScanNode +IsIdentifier src/gutil/strings/util.cc /^bool IsIdentifier(const char *str) {$/;" f +IsLittleEndian src/gutil/endian.h /^ static bool IsLittleEndian() { return true; }$/;" f class:BigEndian +IsLittleEndian src/gutil/endian.h /^ static bool IsLittleEndian() { return true; }$/;" f class:LittleEndian +IsMatched src/exec/new_partitioned_hash_table.inline.h /^inline bool NewPartitionedHashTable::Iterator::IsMatched() const {$/;" f class:doris::NewPartitionedHashTable::Iterator +IsNotInitializerList src/gutil/strings/split_internal.h /^struct IsNotInitializerList {$/;" s namespace:strings::internal +IsNotInitializerList src/gutil/strings/split_internal.h /^struct IsNotInitializerList > {};$/;" s namespace:strings::internal +IsNotRefCounted src/gutil/gscoped_ptr.h /^template struct IsNotRefCounted {$/;" s namespace:doris::internal +IsNullPredicate src/exprs/is_null_predicate.h /^class IsNullPredicate {$/;" c namespace:doris +IsPowerOf2 src/util/bit_util.h /^ constexpr static inline bool IsPowerOf2(int64_t value) {$/;" f class:doris::BitUtil +IsRowNull src/exec/new_partitioned_hash_table.h /^ bool ALWAYS_INLINE IsRowNull() const { return null_bitmap_.Get(CurIdx()); }$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +IsWildcard src/gutil/strings/util.cc /^static bool IsWildcard(Rune character) {$/;" f file: +IsZero src/gutil/charmap.h /^ bool IsZero() const {$/;" f class:Charmap +Iterator src/exec/hash_table.h /^ Iterator() : _table(NULL), _bucket_idx(-1), _node_idx(-1) {$/;" f class:doris::HashTable::Iterator +Iterator src/exec/hash_table.h /^ Iterator(HashTable* table, int bucket_idx, int64_t node, uint32_t hash) :$/;" f class:doris::HashTable::Iterator +Iterator src/exec/hash_table.h /^ class Iterator {$/;" c class:doris::HashTable +Iterator src/exec/new_partitioned_hash_table.h /^ IR_ALWAYS_INLINE Iterator() :$/;" f class:doris::NewPartitionedHashTable::Iterator +Iterator src/exec/new_partitioned_hash_table.h /^ Iterator(NewPartitionedHashTable* table, TupleRow* row, int bucket_idx, DuplicateNode* node)$/;" f class:doris::NewPartitionedHashTable::Iterator +Iterator src/exec/new_partitioned_hash_table.h /^ class Iterator {$/;" c class:doris::NewPartitionedHashTable +Iterator src/exec/partitioned_hash_table.h /^ Iterator() : _table(NULL), _row(NULL), _bucket_idx(BUCKET_NOT_FOUND), _node(NULL) { }$/;" f class:doris::PartitionedHashTable::Iterator +Iterator src/exec/partitioned_hash_table.h /^ Iterator(PartitionedHashTable* table, TupleRow* row, int bucket_idx, DuplicateNode* node)$/;" f class:doris::PartitionedHashTable::Iterator +Iterator src/exec/partitioned_hash_table.h /^ class Iterator {$/;" c class:doris::PartitionedHashTable +Iterator src/exprs/hybird_set.h /^ Iterator(std::unordered_set::iterator begin,$/;" f class:doris::StringValueSet::Iterator +Iterator src/exprs/hybird_set.h /^ Iterator(typename std::unordered_set<_iT>::iterator begin,$/;" f class:doris::HybirdSet::Iterator +Iterator src/exprs/hybird_set.h /^ class Iterator : public IteratorBase {$/;" c class:doris::HybirdSet +Iterator src/exprs/hybird_set.h /^ class Iterator : public IteratorBase {$/;" c class:doris::StringValueSet +Iterator src/gutil/strings/split_internal.h /^ typedef internal::SplitIterator Iterator;$/;" t class:strings::internal::Splitter +Iterator src/olap/skiplist.h /^ class Iterator {$/;" c class:doris::SkipList +Iterator src/olap/skiplist.h /^inline SkipList::Iterator::Iterator(const SkipList* list) {$/;" f class:doris::SkipList::Iterator +Iterator src/runtime/row_batch.h /^ Iterator(RowBatch* parent, int row_idx, int limit = -1) :$/;" f class:doris::RowBatch::Iterator +Iterator src/runtime/row_batch.h /^ class Iterator {$/;" c class:doris::RowBatch +Iterator src/util/lru_cache.hpp /^ Iterator(typename std::unordered_map::iterator it) : _it(it) { }$/;" f class:doris::LruCache::Iterator +Iterator src/util/lru_cache.hpp /^ class Iterator : public std::iterator {$/;" c class:doris::LruCache +IteratorBase src/exprs/hybird_set.h /^ IteratorBase() {$/;" f class:doris::HybirdSetBase::IteratorBase +IteratorBase src/exprs/hybird_set.h /^ class IteratorBase {$/;" c class:doris::HybirdSetBase +ItoaKMGT src/gutil/strings/numbers.cc /^string ItoaKMGT(int64 i) {$/;" f +JSON_FUN_DOUBLE src/exprs/json_functions.h /^ JSON_FUN_INT = 0, JSON_FUN_DOUBLE, JSON_FUN_STRING$/;" e enum:doris::JsonFunctionType +JSON_FUN_INT src/exprs/json_functions.h /^ JSON_FUN_INT = 0, JSON_FUN_DOUBLE, JSON_FUN_STRING$/;" e enum:doris::JsonFunctionType +JSON_FUN_STRING src/exprs/json_functions.h /^ JSON_FUN_INT = 0, JSON_FUN_DOUBLE, JSON_FUN_STRING$/;" e enum:doris::JsonFunctionType +JoinCSVLine src/gutil/strings/join.cc /^string JoinCSVLine(const vector& cols) {$/;" f +JoinCSVLine src/gutil/strings/join.cc /^void JoinCSVLine(const vector& cols, string* output) {$/;" f +JoinCSVLineWithDelimiter src/gutil/strings/join.cc /^void JoinCSVLineWithDelimiter(const vector& cols, char delimiter,$/;" f +JoinElements src/gutil/strings/join.h /^inline string JoinElements(const CONTAINER& components, StringPiece delim) {$/;" f +JoinElements src/gutil/strings/join.h /^inline void JoinElements(const CONTAINER& components,$/;" f +JoinElementsIterator src/gutil/strings/join.h /^string JoinElementsIterator(ITERATOR first,$/;" f +JoinElementsIterator src/gutil/strings/join.h /^void JoinElementsIterator(ITERATOR first,$/;" f +JoinHashMapKeysAndValues src/gutil/strings/join.h /^void JoinHashMapKeysAndValues(const T& container,$/;" f +JoinInts src/gutil/strings/join.h /^inline string JoinInts(const CONTAINER& components,$/;" f +JoinInts src/gutil/strings/join.h /^void JoinInts(const CONTAINER& components,$/;" f +JoinKeysAndValuesIterator src/gutil/strings/join.h /^string JoinKeysAndValuesIterator(const ITERATOR& start,$/;" f +JoinKeysAndValuesIterator src/gutil/strings/join.h /^void JoinKeysAndValuesIterator(const ITERATOR& start,$/;" f +JoinKeysIterator src/gutil/strings/join.h /^string JoinKeysIterator(const ITERATOR& start,$/;" f +JoinKeysIterator src/gutil/strings/join.h /^void JoinKeysIterator(const ITERATOR& start,$/;" f +JoinMapKeysAndValues src/gutil/strings/join.cc /^void JoinMapKeysAndValues(const map& components,$/;" f +JoinMapped src/gutil/strings/join.h /^string JoinMapped(const CONTAINER& components,$/;" f +JoinStrings src/gutil/strings/join.h /^inline string JoinStrings(const CONTAINER& components,$/;" f +JoinStrings src/gutil/strings/join.h /^inline void JoinStrings(const CONTAINER& components,$/;" f +JoinStringsInArray src/gutil/strings/join.cc /^void JoinStringsInArray(string const *components,$/;" f +JoinStringsInArray src/gutil/strings/join.cc /^void JoinStringsInArray(string const* const* components,$/;" f +JoinStringsInArray src/gutil/strings/join.h /^inline string JoinStringsInArray(string const* components,$/;" f +JoinStringsInArray src/gutil/strings/join.h /^inline string JoinStringsInArray(string const* const* components,$/;" f +JoinStringsIterator src/gutil/strings/join.h /^inline string JoinStringsIterator(const ITERATOR& start,$/;" f +JoinStringsIterator src/gutil/strings/join.h /^void JoinStringsIterator(const ITERATOR& start,$/;" f +JoinUsing src/gutil/strings/join.cc /^char* JoinUsing(const vector& components,$/;" f +JoinUsingToBuffer src/gutil/strings/join.cc /^char* JoinUsingToBuffer(const vector& components,$/;" f +JoinVectorKeysAndValues src/gutil/strings/join.cc /^void JoinVectorKeysAndValues(const vector< pair >& components,$/;" f +JsonFunctionTest test/exprs/json_function_test.cpp /^ JsonFunctionTest() {$/;" f class:doris::JsonFunctionTest +JsonFunctionTest test/exprs/json_function_test.cpp /^class JsonFunctionTest : public testing::Test {$/;" c namespace:doris file: +JsonFunctionType src/exprs/json_functions.h /^enum JsonFunctionType {$/;" g namespace:doris +JsonFunctions src/exprs/json_functions.h /^class JsonFunctions {$/;" c namespace:doris +JsonPath src/exprs/json_functions.h /^ JsonPath(const std::string& key_, int idx_, bool is_valid_):$/;" f struct:doris::JsonPath +JsonPath src/exprs/json_functions.h /^struct JsonPath {$/;" s namespace:doris +JsonType src/http/http_headers.cpp /^const std::string HttpHeaders::JsonType = "application\/json";$/;" m class:doris::HttpHeaders file: +JsonType src/http/http_headers.h /^ static const std::string JsonType;$/;" m class:doris::HttpHeaders +JsonUtilTest test/util/json_util_test.cpp /^ JsonUtilTest() { }$/;" f class:doris::JsonUtilTest +JsonUtilTest test/util/json_util_test.cpp /^class JsonUtilTest : public testing::Test {$/;" c namespace:doris file: +K test/olap/skiplist_test.cpp /^ static const uint32_t K = 4;$/;" m class:doris::ConcurrentTest file: +K test/olap/skiplist_test.cpp /^const uint32_t ConcurrentTest::K;$/;" m class:doris::ConcurrentTest file: +KERNEL_DIRENT src/gutil/linux_syscall_support.h 2493;" d +KERNEL_NSIG src/gutil/linux_syscall_support.h 281;" d +KERNEL_NSIG src/gutil/linux_syscall_support.h 283;" d +KEY_BATCH_SIZE src/exec/es/es_scan_reader.h /^ static constexpr const char* KEY_BATCH_SIZE = "batch_size";$/;" m class:doris::ESScanReader +KEY_BITS src/util/radix_sort.h /^ static constexpr size_t KEY_BITS = sizeof(Key) * 8;$/;" m struct:doris::RadixSort +KEY_HOST_PORT src/exec/es/es_scan_reader.h /^ static constexpr const char* KEY_HOST_PORT = "host_port";$/;" m class:doris::ESScanReader +KEY_INDEX src/exec/es/es_scan_reader.h /^ static constexpr const char* KEY_INDEX = "index";$/;" m class:doris::ESScanReader +KEY_MAXIMAL_MARKER src/olap/short_key_index.h /^constexpr uint8_t KEY_MAXIMAL_MARKER = 0xFF;$/;" m namespace:doris +KEY_MINIMAL_MARKER src/olap/short_key_index.h /^constexpr uint8_t KEY_MINIMAL_MARKER = 0x00;$/;" m namespace:doris +KEY_NORMAL_MARKER src/olap/short_key_index.h /^constexpr uint8_t KEY_NORMAL_MARKER = 0x02;$/;" m namespace:doris +KEY_NULL_FIRST_MARKER src/olap/short_key_index.h /^constexpr uint8_t KEY_NULL_FIRST_MARKER = 0x01;$/;" m namespace:doris +KEY_NULL_LAST_MARKER src/olap/short_key_index.h /^constexpr uint8_t KEY_NULL_LAST_MARKER = 0xFE;$/;" m namespace:doris +KEY_PASS_WORD src/exec/es/es_scan_reader.h /^ static constexpr const char* KEY_PASS_WORD = "password";$/;" m class:doris::ESScanReader +KEY_QUERY src/exec/es/es_scan_reader.h /^ static constexpr const char* KEY_QUERY = "query";$/;" m class:doris::ESScanReader +KEY_SHARD src/exec/es/es_scan_reader.h /^ static constexpr const char* KEY_SHARD = "shard_id";$/;" m class:doris::ESScanReader +KEY_TERMINATE_AFTER src/exec/es/es_scan_reader.h /^ static constexpr const char* KEY_TERMINATE_AFTER = "limit";$/;" m class:doris::ESScanReader +KEY_TYPE src/exec/es/es_scan_reader.h /^ static constexpr const char* KEY_TYPE = "type";$/;" m class:doris::ESScanReader +KEY_USER_NAME src/exec/es/es_scan_reader.h /^ static constexpr const char* KEY_USER_NAME = "user";$/;" m class:doris::ESScanReader +KILOBYTE src/util/debug_util.cpp 32;" d file: +KILOBYTE src/util/pretty_printer.h /^ static const int64_t KILOBYTE = 1024;$/;" m class:doris::PrettyPrinter +KUDU_REMOTE_TOKENS src/exec/kudu_scan_node.cpp /^const string KuduScanNode::KUDU_REMOTE_TOKENS = "KuduRemoteScanTokens";$/;" m class:doris::KuduScanNode file: +KUDU_REMOTE_TOKENS src/exec/kudu_scan_node.h /^ static const std::string KUDU_REMOTE_TOKENS;$/;" m class:doris::KuduScanNode +KUDU_RETURN_IF_ERROR src/exec/kudu_util.h 59;" d +KUDU_ROUND_TRIPS src/exec/kudu_scan_node.cpp /^const string KuduScanNode::KUDU_ROUND_TRIPS = "TotalKuduScanRoundTrips";$/;" m class:doris::KuduScanNode file: +KUDU_ROUND_TRIPS src/exec/kudu_scan_node.h /^ static const std::string KUDU_ROUND_TRIPS;$/;" m class:doris::KuduScanNode +KafkaConsumerPipe src/runtime/routine_load/kafka_consumer_pipe.h /^ KafkaConsumerPipe(size_t max_buffered_bytes = 1024 * 1024,$/;" f class:doris::KafkaConsumerPipe +KafkaConsumerPipe src/runtime/routine_load/kafka_consumer_pipe.h /^class KafkaConsumerPipe : public StreamLoadPipe {$/;" c namespace:doris +KafkaConsumerPipeTest test/runtime/kafka_consumer_pipe_test.cpp /^ KafkaConsumerPipeTest() { }$/;" f class:doris::KafkaConsumerPipeTest +KafkaConsumerPipeTest test/runtime/kafka_consumer_pipe_test.cpp /^class KafkaConsumerPipeTest : public testing::Test {$/;" c namespace:doris file: +KafkaDataConsumer src/runtime/routine_load/data_consumer.h /^ KafkaDataConsumer(StreamLoadContext* ctx):$/;" f class:doris::KafkaDataConsumer +KafkaDataConsumer src/runtime/routine_load/data_consumer.h /^class KafkaDataConsumer : public DataConsumer {$/;" c namespace:doris +KafkaDataConsumerGroup src/runtime/routine_load/data_consumer_group.h /^ KafkaDataConsumerGroup():$/;" f class:doris::KafkaDataConsumerGroup +KafkaDataConsumerGroup src/runtime/routine_load/data_consumer_group.h /^class KafkaDataConsumerGroup : public DataConsumerGroup {$/;" c namespace:doris +KafkaEventCb src/runtime/routine_load/data_consumer.h /^class KafkaEventCb : public RdKafka::EventCb {$/;" c namespace:doris +KafkaLoadInfo src/runtime/stream_load/stream_load_context.h /^ KafkaLoadInfo(const TKafkaLoadInfo& t_info):$/;" f class:doris::KafkaLoadInfo +KafkaLoadInfo src/runtime/stream_load/stream_load_context.h /^class KafkaLoadInfo {$/;" c namespace:doris +Key test/olap/skiplist_test.cpp /^typedef uint64_t Key;$/;" t namespace:doris file: +KeyCoder src/olap/key_coder.cpp /^KeyCoder::KeyCoder(TraitsType traits)$/;" f class:doris::KeyCoder +KeyCoder src/olap/key_coder.h /^class KeyCoder {$/;" c namespace:doris +KeyCoderResolver src/olap/key_coder.cpp /^ KeyCoderResolver() {$/;" f class:doris::KeyCoderResolver file: +KeyCoderResolver src/olap/key_coder.cpp /^class KeyCoderResolver {$/;" c namespace:doris file: +KeyCoderTest test/olap/key_coder_test.cpp /^ KeyCoderTest() : _pool(&_tracker) { }$/;" f class:doris::KeyCoderTest +KeyCoderTest test/olap/key_coder_test.cpp /^class KeyCoderTest : public testing::Test {$/;" c namespace:doris file: +KeyCoderTraits src/olap/key_coder.h /^class KeyCoderTraits {$/;" c namespace:doris +KeyCoderTraits src/olap/key_coder.h /^class KeyCoderTraits {$/;" c namespace:doris +KeyCoderTraits src/olap/key_coder.h /^class KeyCoderTraits {$/;" c namespace:doris +KeyCoderTraits src/olap/key_coder.h /^class KeyCoderTraits {$/;" c namespace:doris +KeyCoderTraits src/olap/key_coder.h /^class KeyCoderTraits {$/;" c namespace:doris +KeyCoderTraits src/olap/key_coder.h /^class KeyCoderTraits::KeyIsAfterNode(const Key& key, Node* n) const {$/;" f class:doris::SkipList +KeyRange src/olap/iterators.h /^ KeyRange()$/;" f struct:doris::StorageReadOptions::KeyRange +KeyRange src/olap/iterators.h /^ KeyRange(const RowCursor* lower_key_,$/;" f struct:doris::StorageReadOptions::KeyRange +KeyRange src/olap/iterators.h /^ struct KeyRange {$/;" s class:doris::StorageReadOptions +KeyValuePair src/util/lru_cache.hpp /^ typedef typename std::pair KeyValuePair;$/;" t class:doris::LruCache +KeysParam src/olap/reader.h /^ struct KeysParam {$/;" s class:doris::Reader +KnuthVarianceState src/exprs/aggregate_functions.cpp /^struct KnuthVarianceState {$/;" s namespace:doris file: +KuduClientIsSupported src/exec/kudu_util.cpp /^bool KuduClientIsSupported() {$/;" f namespace:doris +KuduIsAvailable src/exec/kudu_util.cpp /^bool KuduIsAvailable() { return CheckKuduAvailability().ok(); }$/;" f namespace:doris +KuduScanNode src/exec/kudu_scan_node.cpp /^KuduScanNode::KuduScanNode(ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::KuduScanNode +KuduScanNode src/exec/kudu_scan_node.h /^class KuduScanNode : public ScanNode {$/;" c namespace:doris +KuduScanner src/exec/kudu_scanner.cpp /^KuduScanner::KuduScanner(KuduScanNode* scan_node, RuntimeState* state)$/;" f class:doris::KuduScanner +KuduScanner src/exec/kudu_scanner.h /^class KuduScanner {$/;" c namespace:doris +KuduSchemaDebugString src/exec/kudu_util.cpp /^std::string KuduSchemaDebugString(const KuduSchema& schema) {$/;" f namespace:doris +KuduTableDescriptor src/runtime/descriptors.cpp /^KuduTableDescriptor::KuduTableDescriptor(const TTableDescriptor& tdesc)$/;" f class:doris::KuduTableDescriptor +KuduTableDescriptor src/runtime/descriptors.h /^class KuduTableDescriptor : public TableDescriptor {$/;" c namespace:doris +L1_CACHE src/util/cpu_info.h /^ L1_CACHE = 0,$/;" e enum:doris::CpuInfo::CacheLevel +L2_CACHE src/util/cpu_info.h /^ L2_CACHE = 1,$/;" e enum:doris::CpuInfo::CacheLevel +L3_CACHE src/util/cpu_info.h /^ L3_CACHE = 2,$/;" e enum:doris::CpuInfo::CacheLevel +LABEL_KEY src/http/action/mini_load.cpp /^const std::string LABEL_KEY = "label";$/;" m namespace:doris file: +LABEL_PARAMETER src/http/download_action.cpp /^const std::string LABEL_PARAMETER = "label";$/;" m namespace:doris file: +LANG_CXX11 src/gutil/port.h 1205;" d +LARGEINT_REINTERPRET_CAST src/olap/schema_change.cpp 141;" d file: +LARGEINT_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t LARGEINT_SIZE = sizeof(__int128);$/;" m class:doris::AggFnEvaluator +LARGEINT_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t LARGEINT_SIZE = sizeof(__int128);$/;" m class:doris::NewAggFnEvaluator +LARGE_ALLOC_SIZE test/runtime/mem_pool_test.cpp 129;" d file: +LARGE_MEM_LIMIT test/runtime/disk_io_mgr_test.cpp /^const int LARGE_MEM_LIMIT = 1024 * 1024 * 1024;$/;" m namespace:doris file: +LAST_MODIFIED src/http/http_headers.cpp /^const char* HttpHeaders::LAST_MODIFIED = "Last-Modified";$/;" m class:doris::HttpHeaders file: +LAST_MODIFIED src/http/http_headers.h /^ static const char* LAST_MODIFIED;$/;" m class:doris::HttpHeaders +LE src/olap/olap_common.h /^ LE = 3, \/\/ less or equal$/;" e enum:doris::RangeCondition +LEAST_FN src/exprs/math_functions.cpp 579;" d file: +LEAST_FNS src/exprs/math_functions.cpp 591;" d file: +LEAST_NONNUMERIC_FN src/exprs/math_functions.cpp 602;" d file: +LEAST_NONNUMERIC_FNS src/exprs/math_functions.cpp 617;" d file: +LENGTH_REQUIRED src/http/http_status.h /^ LENGTH_REQUIRED = 411,$/;" e enum:doris::HttpStatus +LE_LOAD32 src/util/crc32c.cpp /^static inline uint32_t LE_LOAD32(const uint8_t *p) {$/;" f namespace:doris::crc32c +LE_LOAD64 src/util/crc32c.cpp /^static inline uint64_t LE_LOAD64(const uint8_t *p) {$/;" f namespace:doris::crc32c +LF src/http/http_parser.cpp 32;" d file: +LIKELY src/common/compiler_util.h 29;" d +LIKELY src/common/compiler_util.h 36;" d +LIMIT_EXCEEDED src/exec/exec_node.h 392;" d +LINKER_INITIALIZED src/gutil/macros.h /^enum LinkerInitialized { LINKER_INITIALIZED };$/;" e enum:base::LinkerInitialized +LIST_REMOTE_FILE_TIMEOUT src/olap/task/engine_clone_task.cpp /^const uint32_t LIST_REMOTE_FILE_TIMEOUT = 15;$/;" m namespace:doris file: +LOAD src/olap/delta_writer.h /^ LOAD = 1,$/;" e enum:doris::WriteType +LOAD_DELETE src/olap/delta_writer.h /^ LOAD_DELETE = 2,$/;" e enum:doris::WriteType +LOCATION src/http/http_headers.cpp /^const char* HttpHeaders::LOCATION = "Location";$/;" m class:doris::HttpHeaders file: +LOCATION src/http/http_headers.h /^ static const char* LOCATION;$/;" m class:doris::HttpHeaders +LOG src/common/logging.h 34;" d +LOGIC128 src/gutil/int128.h 163;" d +LOGIC128 src/gutil/int128.h 173;" d +LOGICASSIGN128 src/gutil/int128.h 175;" d +LOGICASSIGN128 src/gutil/int128.h 186;" d +LOG_MAX_ALLOCATION_BYTES src/runtime/bufferpool/suballocator.cc /^constexpr int Suballocator::LOG_MAX_ALLOCATION_BYTES;$/;" m class:doris::Suballocator file: +LOG_MAX_ALLOCATION_BYTES src/runtime/bufferpool/suballocator.h /^ static constexpr int LOG_MAX_ALLOCATION_BYTES = BufferPool::LOG_MAX_BUFFER_BYTES;$/;" m class:doris::Suballocator +LOG_MAX_BUFFER_BYTES src/runtime/bufferpool/buffer_pool.cc /^constexpr int BufferPool::LOG_MAX_BUFFER_BYTES;$/;" m class:doris::BufferPool file: +LOG_MAX_BUFFER_BYTES src/runtime/bufferpool/buffer_pool.h /^ static constexpr int LOG_MAX_BUFFER_BYTES = 48;$/;" m class:doris::BufferPool +LOG_MIN_ALLOCATION_BYTES src/runtime/bufferpool/suballocator.cc /^constexpr int Suballocator::LOG_MIN_ALLOCATION_BYTES;$/;" m class:doris::Suballocator file: +LOG_MIN_ALLOCATION_BYTES src/runtime/bufferpool/suballocator.h /^ static constexpr int LOG_MIN_ALLOCATION_BYTES = 12;$/;" m class:doris::Suballocator +LOW_56_BITS src/olap/olap_common.h 39;" d +LOW_MEMORY src/runtime/disk_io_mgr.cc /^static const int LOW_MEMORY = 64 * 1024 * 1024;$/;" m namespace:doris file: +LRUCache src/olap/lru_cache.cpp /^LRUCache::LRUCache() : _usage(0), _last_id(0), _lookup_count(0),$/;" f class:doris::LRUCache +LRUCache src/olap/lru_cache.h /^ class LRUCache {$/;" c class:doris::CachePriority +LRUHandle src/olap/lru_cache.h /^ typedef struct LRUHandle {$/;" s class:doris::CachePriority +LRUHandle src/olap/lru_cache.h /^ } LRUHandle;$/;" t class:doris::CachePriority typeref:struct:doris::CachePriority::LRUHandle +LRU_MAX_CASH_TASK_NUM src/runtime/export_task_mgr.cpp /^static size_t LRU_MAX_CASH_TASK_NUM = 1000;$/;" m namespace:doris file: +LSS_ASMINPUT_0 src/gutil/linux_syscall_support.h 1959;" d +LSS_ASMINPUT_0 src/gutil/linux_syscall_support.h 1960;" d +LSS_ASMINPUT_1 src/gutil/linux_syscall_support.h 1961;" d +LSS_ASMINPUT_1 src/gutil/linux_syscall_support.h 1962;" d +LSS_ASMINPUT_2 src/gutil/linux_syscall_support.h 1963;" d +LSS_ASMINPUT_2 src/gutil/linux_syscall_support.h 1964;" d +LSS_ASMINPUT_3 src/gutil/linux_syscall_support.h 1965;" d +LSS_ASMINPUT_3 src/gutil/linux_syscall_support.h 1966;" d +LSS_ASMINPUT_4 src/gutil/linux_syscall_support.h 1967;" d +LSS_ASMINPUT_4 src/gutil/linux_syscall_support.h 1968;" d +LSS_ASMINPUT_5 src/gutil/linux_syscall_support.h 1969;" d +LSS_ASMINPUT_5 src/gutil/linux_syscall_support.h 1970;" d +LSS_ASMINPUT_6 src/gutil/linux_syscall_support.h 1971;" d +LSS_ASMINPUT_6 src/gutil/linux_syscall_support.h 1972;" d +LSS_BODY src/gutil/linux_syscall_support.h 1059;" d +LSS_BODY src/gutil/linux_syscall_support.h 1060;" d +LSS_BODY src/gutil/linux_syscall_support.h 1313;" d +LSS_BODY src/gutil/linux_syscall_support.h 1314;" d +LSS_BODY src/gutil/linux_syscall_support.h 1530;" d +LSS_BODY src/gutil/linux_syscall_support.h 1531;" d +LSS_BODY src/gutil/linux_syscall_support.h 1718;" d +LSS_BODY src/gutil/linux_syscall_support.h 1719;" d +LSS_BODY src/gutil/linux_syscall_support.h 1973;" d +LSS_BODY src/gutil/linux_syscall_support.h 1974;" d +LSS_BODY src/gutil/linux_syscall_support.h 2202;" d +LSS_BODY src/gutil/linux_syscall_support.h 2203;" d +LSS_BODY src/gutil/linux_syscall_support.h 2334;" d +LSS_BODY src/gutil/linux_syscall_support.h 2335;" d +LSS_BODY_ARG0 src/gutil/linux_syscall_support.h 1347;" d +LSS_BODY_ARG0 src/gutil/linux_syscall_support.h 1354;" d +LSS_BODY_ARG1 src/gutil/linux_syscall_support.h 1348;" d +LSS_BODY_ARG1 src/gutil/linux_syscall_support.h 1355;" d +LSS_BODY_ARG2 src/gutil/linux_syscall_support.h 1349;" d +LSS_BODY_ARG2 src/gutil/linux_syscall_support.h 1357;" d +LSS_BODY_ARG3 src/gutil/linux_syscall_support.h 1350;" d +LSS_BODY_ARG3 src/gutil/linux_syscall_support.h 1359;" d +LSS_BODY_ARG4 src/gutil/linux_syscall_support.h 1351;" d +LSS_BODY_ARG4 src/gutil/linux_syscall_support.h 1361;" d +LSS_BODY_ARG5 src/gutil/linux_syscall_support.h 1352;" d +LSS_BODY_ARG5 src/gutil/linux_syscall_support.h 1363;" d +LSS_BODY_ARG6 src/gutil/linux_syscall_support.h 1353;" d +LSS_BODY_ARG6 src/gutil/linux_syscall_support.h 1365;" d +LSS_BODY_ASM0 src/gutil/linux_syscall_support.h 1317;" d +LSS_BODY_ASM0 src/gutil/linux_syscall_support.h 1324;" d +LSS_BODY_ASM1 src/gutil/linux_syscall_support.h 1318;" d +LSS_BODY_ASM1 src/gutil/linux_syscall_support.h 1325;" d +LSS_BODY_ASM2 src/gutil/linux_syscall_support.h 1319;" d +LSS_BODY_ASM2 src/gutil/linux_syscall_support.h 1326;" d +LSS_BODY_ASM3 src/gutil/linux_syscall_support.h 1320;" d +LSS_BODY_ASM3 src/gutil/linux_syscall_support.h 1327;" d +LSS_BODY_ASM4 src/gutil/linux_syscall_support.h 1321;" d +LSS_BODY_ASM4 src/gutil/linux_syscall_support.h 1328;" d +LSS_BODY_ASM5 src/gutil/linux_syscall_support.h 1322;" d +LSS_BODY_ASM5 src/gutil/linux_syscall_support.h 1329;" d +LSS_BODY_ASM6 src/gutil/linux_syscall_support.h 1323;" d +LSS_BODY_ASM6 src/gutil/linux_syscall_support.h 1330;" d +LSS_BODY_CLOBBER0 src/gutil/linux_syscall_support.h 1332;" d +LSS_BODY_CLOBBER0 src/gutil/linux_syscall_support.h 1339;" d +LSS_BODY_CLOBBER1 src/gutil/linux_syscall_support.h 1333;" d +LSS_BODY_CLOBBER1 src/gutil/linux_syscall_support.h 1340;" d +LSS_BODY_CLOBBER2 src/gutil/linux_syscall_support.h 1334;" d +LSS_BODY_CLOBBER2 src/gutil/linux_syscall_support.h 1341;" d +LSS_BODY_CLOBBER3 src/gutil/linux_syscall_support.h 1335;" d +LSS_BODY_CLOBBER3 src/gutil/linux_syscall_support.h 1342;" d +LSS_BODY_CLOBBER4 src/gutil/linux_syscall_support.h 1336;" d +LSS_BODY_CLOBBER4 src/gutil/linux_syscall_support.h 1343;" d +LSS_BODY_CLOBBER5 src/gutil/linux_syscall_support.h 1337;" d +LSS_BODY_CLOBBER5 src/gutil/linux_syscall_support.h 1344;" d +LSS_BODY_CLOBBER6 src/gutil/linux_syscall_support.h 1338;" d +LSS_BODY_CLOBBER6 src/gutil/linux_syscall_support.h 1345;" d +LSS_ENTRYPOINT src/gutil/linux_syscall_support.h 1282;" d +LSS_ENTRYPOINT src/gutil/linux_syscall_support.h 1283;" d +LSS_ERRNO src/gutil/linux_syscall_support.h 955;" d +LSS_ERRNO src/gutil/linux_syscall_support.h 960;" d +LSS_ERRNO src/gutil/linux_syscall_support.h 962;" d +LSS_INLINE src/gutil/linux_syscall_support.h 965;" d +LSS_INLINE src/gutil/linux_syscall_support.h 967;" d +LSS_INLINE src/gutil/linux_syscall_support.h 969;" d +LSS_LOADARGS_0 src/gutil/linux_syscall_support.h 1932;" d +LSS_LOADARGS_0 src/gutil/linux_syscall_support.h 1933;" d +LSS_LOADARGS_1 src/gutil/linux_syscall_support.h 1935;" d +LSS_LOADARGS_1 src/gutil/linux_syscall_support.h 1936;" d +LSS_LOADARGS_2 src/gutil/linux_syscall_support.h 1939;" d +LSS_LOADARGS_2 src/gutil/linux_syscall_support.h 1940;" d +LSS_LOADARGS_3 src/gutil/linux_syscall_support.h 1943;" d +LSS_LOADARGS_3 src/gutil/linux_syscall_support.h 1944;" d +LSS_LOADARGS_4 src/gutil/linux_syscall_support.h 1947;" d +LSS_LOADARGS_4 src/gutil/linux_syscall_support.h 1948;" d +LSS_LOADARGS_5 src/gutil/linux_syscall_support.h 1951;" d +LSS_LOADARGS_5 src/gutil/linux_syscall_support.h 1952;" d +LSS_LOADARGS_6 src/gutil/linux_syscall_support.h 1955;" d +LSS_LOADARGS_6 src/gutil/linux_syscall_support.h 1956;" d +LSS_LOAD_ARG src/gutil/linux_syscall_support.h 1527;" d +LSS_LOAD_ARG src/gutil/linux_syscall_support.h 1528;" d +LSS_NAME src/gutil/linux_syscall_support.h /^ LSS_INLINE void (*LSS_NAME(restore)(void))(void) {$/;" f +LSS_NAME src/gutil/linux_syscall_support.h /^ LSS_INLINE void (*LSS_NAME(restore_rt)(void))(void) {$/;" f +LSS_NAME src/gutil/linux_syscall_support.h 975;" d +LSS_NAME src/gutil/linux_syscall_support.h 977;" d +LSS_NAME src/gutil/linux_syscall_support.h 979;" d +LSS_NAME src/gutil/linux_syscall_support.h 981;" d +LSS_NAME src/gutil/linux_syscall_support.h 983;" d +LSS_NAME src/gutil/linux_syscall_support.h 985;" d +LSS_NAME src/gutil/linux_syscall_support.h 987;" d +LSS_NAME src/gutil/linux_syscall_support.h 989;" d +LSS_NAME src/gutil/linux_syscall_support.h 991;" d +LSS_NAME src/gutil/linux_syscall_support.h 993;" d +LSS_NAME src/gutil/linux_syscall_support.h 995;" d +LSS_NAME src/gutil/linux_syscall_support.h 997;" d +LSS_NAME src/gutil/linux_syscall_support.h 999;" d +LSS_REG src/gutil/linux_syscall_support.h 1517;" d +LSS_REG src/gutil/linux_syscall_support.h 1518;" d +LSS_REG src/gutil/linux_syscall_support.h 1704;" d +LSS_REG src/gutil/linux_syscall_support.h 1705;" d +LSS_REG src/gutil/linux_syscall_support.h 2200;" d +LSS_REG src/gutil/linux_syscall_support.h 2201;" d +LSS_REG src/gutil/linux_syscall_support.h 2332;" d +LSS_REG src/gutil/linux_syscall_support.h 2333;" d +LSS_RETURN src/gutil/linux_syscall_support.h 1002;" d +LSS_RETURN src/gutil/linux_syscall_support.h 1008;" d +LSS_RETURN src/gutil/linux_syscall_support.h 1020;" d +LSS_RETURN src/gutil/linux_syscall_support.h 1032;" d +LSS_RETURN src/gutil/linux_syscall_support.h 1302;" d +LSS_RETURN src/gutil/linux_syscall_support.h 1303;" d +LSS_SAVE_ARG src/gutil/linux_syscall_support.h 1525;" d +LSS_SAVE_ARG src/gutil/linux_syscall_support.h 1526;" d +LSS_SYSCALL_ARG src/gutil/linux_syscall_support.h 1291;" d +LSS_SYSCALL_ARG src/gutil/linux_syscall_support.h 1292;" d +LT src/olap/olap_common.h /^ LT = 2, \/\/ less than$/;" e enum:doris::RangeCondition +LZ4FRAME src/exec/decompressor.h /^ LZ4FRAME,$/;" e enum:doris::CompressType +LZOP src/exec/decompressor.h /^ LZOP$/;" e enum:doris::CompressType +LZOP_MAGIC src/exec/decompressor.h /^ const static uint8_t LZOP_MAGIC[9];$/;" m class:doris::LzopDecompressor +LZOP_MAGIC src/exec/lzo_decompressor.cpp /^const uint8_t LzopDecompressor::LZOP_MAGIC[9] =$/;" m class:doris::LzopDecompressor file: +LZOP_VERSION src/exec/decompressor.h /^ const static uint64_t LZOP_VERSION;$/;" m class:doris::LzopDecompressor +LZOP_VERSION src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::LZOP_VERSION = 0x1030;$/;" m class:doris::LzopDecompressor file: +LZO_MAX_BLOCK_SIZE src/exec/decompressor.h /^ const static uint32_t LZO_MAX_BLOCK_SIZE;$/;" m class:doris::LzopDecompressor +LZO_MAX_BLOCK_SIZE src/exec/lzo_decompressor.cpp /^const uint32_t LzopDecompressor::LZO_MAX_BLOCK_SIZE = (64*1024l*1024l);$/;" m class:doris::LzopDecompressor file: +LargeIntColumnReader src/olap/rowset/column_reader.cpp /^LargeIntColumnReader::LargeIntColumnReader(uint32_t column_id, uint32_t column_unique_id) :$/;" f class:doris::LargeIntColumnReader +LargeIntColumnReader src/olap/rowset/column_reader.h /^class LargeIntColumnReader : public ColumnReader {$/;" c namespace:doris +LargeIntColumnWriter src/olap/rowset/column_writer.cpp /^LargeIntColumnWriter::LargeIntColumnWriter(uint32_t column_id,$/;" f class:doris::LargeIntColumnWriter +LargeIntColumnWriter src/olap/rowset/column_writer.h /^class LargeIntColumnWriter : public ColumnWriter {$/;" c namespace:doris +LargeIntVal output/udf/include/udf.h /^ LargeIntVal() : val(0) { }$/;" f struct:doris_udf::LargeIntVal +LargeIntVal output/udf/include/udf.h /^ LargeIntVal(__int128 large_value) : val(large_value) { }$/;" f struct:doris_udf::LargeIntVal +LargeIntVal output/udf/include/udf.h /^struct LargeIntVal : public AnyVal {$/;" s namespace:doris_udf +LargeIntVal src/udf/udf.h /^ LargeIntVal() : val(0) { }$/;" f struct:doris_udf::LargeIntVal +LargeIntVal src/udf/udf.h /^ LargeIntVal(__int128 large_value) : val(large_value) { }$/;" f struct:doris_udf::LargeIntVal +LargeIntVal src/udf/udf.h /^struct LargeIntVal : public AnyVal {$/;" s namespace:doris_udf +LargeIntValue src/runtime/large_int_value.h /^class LargeIntValue {$/;" c namespace:doris +LargeIntValueTest test/runtime/large_int_value_test.cpp /^ LargeIntValueTest() {$/;" f class:doris::LargeIntValueTest +LargeIntValueTest test/runtime/large_int_value_test.cpp /^class LargeIntValueTest : public testing::Test {$/;" c namespace:doris file: +LargeIntWrapper src/exprs/scalar_fn_call.cpp /^typedef LargeIntVal (*LargeIntWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +Leave src/gutil/threading/thread_collision_warner.cc /^void ThreadCollisionWarner::Leave() {$/;" f class:base::ThreadCollisionWarner +Length src/gutil/strings/split.cc /^ int Length(StringPiece delimiter) {$/;" f struct:strings::delimiter::__anon29::AnyOfPolicy +Length src/gutil/strings/split.cc /^ int Length(StringPiece delimiter) {$/;" f struct:strings::delimiter::__anon29::LiteralPolicy +LessThan src/util/monotime.cpp /^bool MonoDelta::LessThan(const MonoDelta &rhs) const {$/;" f class:doris::MonoDelta +LikePredicate src/exprs/like_predicate.h /^class LikePredicate {$/;" c namespace:doris +LikePredicateFunction src/exprs/like_predicate.h /^ typedef doris_udf::BooleanVal (*LikePredicateFunction) ($/;" t class:doris::LikePredicate::doris_udf +LikePredicateState src/exprs/like_predicate.h /^ LikePredicateState() : escape_char('\\\\') {$/;" f struct:doris::LikePredicate::LikePredicateState +LikePredicateState src/exprs/like_predicate.h /^ struct LikePredicateState {$/;" s class:doris::LikePredicate +Limit src/gutil/strings/split.h /^inline LimitImpl Limit(Delimiter delim, int limit) {$/;" f namespace:strings::delimiter +Limit src/gutil/strings/split.h /^inline LimitImpl Limit(StringPiece s, int limit) {$/;" f namespace:strings::delimiter +Limit src/gutil/strings/split.h /^inline LimitImpl Limit(const char* s, int limit) {$/;" f namespace:strings::delimiter +Limit src/gutil/strings/split.h /^inline LimitImpl Limit(const string& s, int limit) {$/;" f namespace:strings::delimiter +LimitImpl src/gutil/strings/split.h /^ LimitImpl(Delimiter delimiter, int limit)$/;" f class:strings::delimiter::LimitImpl +LimitImpl src/gutil/strings/split.h /^class LimitImpl {$/;" c namespace:strings::delimiter +LineReader src/exec/line_reader.h /^class LineReader {$/;" c namespace:doris +LinkedSchemaChange src/olap/schema_change.h /^ explicit LinkedSchemaChange(const RowBlockChanger& row_block_changer)$/;" f class:doris::LinkedSchemaChange +LinkedSchemaChange src/olap/schema_change.h /^class LinkedSchemaChange : public SchemaChange {$/;" c namespace:doris +LinkerInitialized src/gutil/macros.h /^enum LinkerInitialized { LINKER_INITIALIZED };$/;" g namespace:base +ListIterator src/util/lru_cache.hpp /^ typedef typename std::list::iterator ListIterator;$/;" t class:doris::LruCache +Literal src/exprs/literal.cpp /^Literal::Literal(const TExprNode& node) : $/;" f class:doris::Literal +Literal src/exprs/literal.h /^class Literal : public Expr {$/;" c namespace:doris +Literal src/gutil/strings/split.cc /^Literal::Literal(StringPiece sp) : delimiter_(sp.ToString()) {$/;" f class:strings::delimiter::Literal +Literal src/gutil/strings/split.h /^class Literal {$/;" c namespace:strings::delimiter +LiteralPolicy src/gutil/strings/split.cc /^struct LiteralPolicy {$/;" s namespace:strings::delimiter::__anon29 file: +LittleEndian src/gutil/endian.h /^class LittleEndian {$/;" c +Load128 src/gutil/endian.h /^ static uint128 Load128(const void *p) {$/;" f class:BigEndian +Load128 src/gutil/endian.h /^ static uint128 Load128(const void *p) {$/;" f class:LittleEndian +Load128VariableLength src/gutil/endian.h /^ static uint128 Load128VariableLength(const void *p, int len) {$/;" f class:BigEndian +Load128VariableLength src/gutil/endian.h /^ static uint128 Load128VariableLength(const void *p, int len) {$/;" f class:LittleEndian +Load16 src/gutil/endian.h /^ static uint16 Load16(const void *p) {$/;" f class:BigEndian +Load16 src/gutil/endian.h /^ static uint16 Load16(const void *p) {$/;" f class:LittleEndian +Load32 src/gutil/endian.h /^ static uint32 Load32(const void *p) {$/;" f class:BigEndian +Load32 src/gutil/endian.h /^ static uint32 Load32(const void *p) {$/;" f class:LittleEndian +Load64 src/gutil/endian.h /^ static uint64 Load64(const void *p) {$/;" f class:BigEndian +Load64 src/gutil/endian.h /^ static uint64 Load64(const void *p) {$/;" f class:LittleEndian +Load64VariableLength src/gutil/endian.h /^ static uint64 Load64VariableLength(const void * const p, int len) {$/;" f class:BigEndian +Load64VariableLength src/gutil/endian.h /^ static uint64 Load64VariableLength(const void * const p, int len) {$/;" f class:LittleEndian +LoadChannel src/runtime/load_channel.cpp /^LoadChannel::LoadChannel(const UniqueId& load_id, int64_t mem_limit,$/;" f class:doris::LoadChannel +LoadChannel src/runtime/load_channel.h /^class LoadChannel {$/;" c namespace:doris +LoadChannelMgr src/runtime/load_channel_mgr.cpp /^LoadChannelMgr::LoadChannelMgr() : _is_stopped(false) {$/;" f class:doris::LoadChannelMgr +LoadChannelMgr src/runtime/load_channel_mgr.h /^class LoadChannelMgr {$/;" c namespace:doris +LoadChannelMgrTest test/runtime/load_channel_mgr_test.cpp /^ LoadChannelMgrTest() { }$/;" f class:doris::LoadChannelMgrTest +LoadChannelMgrTest test/runtime/load_channel_mgr_test.cpp /^class LoadChannelMgrTest : public testing::Test {$/;" c namespace:doris file: +LoadErrorHub src/util/load_error_hub.h /^ LoadErrorHub() {$/;" f class:doris::LoadErrorHub +LoadErrorHub src/util/load_error_hub.h /^class LoadErrorHub {$/;" c namespace:doris +LoadHandle src/http/action/mini_load.h /^struct LoadHandle {$/;" s namespace:doris +LoadHandleCmp src/http/action/mini_load.h /^struct LoadHandleCmp {$/;" s namespace:doris +LoadPathMgr src/runtime/load_path_mgr.cpp /^LoadPathMgr::LoadPathMgr(ExecEnv* exec_env) : _exec_env(exec_env),$/;" f class:doris::LoadPathMgr +LoadPathMgr src/runtime/load_path_mgr.h /^class LoadPathMgr {$/;" c namespace:doris +LoadStreamMgr src/runtime/stream_load/load_stream_mgr.h /^ LoadStreamMgr() { }$/;" f class:doris::LoadStreamMgr +LoadStreamMgr src/runtime/stream_load/load_stream_mgr.h /^class LoadStreamMgr {$/;" c namespace:doris +LoadUnsignedWord src/gutil/endian.h /^ static uword_t LoadUnsignedWord(const void *p) {$/;" f class:BigEndian +LoadUnsignedWord src/gutil/endian.h /^ static uword_t LoadUnsignedWord(const void *p) {$/;" f class:LittleEndian +LocalFileReader src/exec/local_file_reader.cpp /^LocalFileReader::LocalFileReader(const std::string& path, int64_t start_offset) $/;" f class:doris::LocalFileReader +LocalFileReader src/exec/local_file_reader.h /^class LocalFileReader : public FileReader {$/;" c namespace:doris +LocalFileWriter src/exec/local_file_writer.cpp /^LocalFileWriter::LocalFileWriter(const std::string& path, int64_t start_offset)$/;" f class:doris::LocalFileWriter +LocalFileWriter src/exec/local_file_writer.h /^class LocalFileWriter : public FileWriter {$/;" c namespace:doris +LockCounter src/util/metrics.h /^ LockCounter() :LockSimpleMetric(MetricType::COUNTER) { }$/;" f class:doris::LockCounter +LockCounter src/util/metrics.h /^class LockCounter : public LockSimpleMetric {$/;" c namespace:doris +LockGauge src/util/metrics.h /^ LockGauge() :LockSimpleMetric(MetricType::GAUGE) { }$/;" f class:doris::LockGauge +LockGauge src/util/metrics.h /^class LockGauge : public LockSimpleMetric {$/;" c namespace:doris +LockSimpleMetric src/util/metrics.h /^ LockSimpleMetric(MetricType type) :SimpleMetric(type), _value(T()) { }$/;" f class:doris::LockSimpleMetric +LockSimpleMetric src/util/metrics.h /^class LockSimpleMetric : public SimpleMetric {$/;" c namespace:doris +Log2Ceiling src/gutil/bits.cc /^int Bits::Log2Ceiling(uint32 n) {$/;" f class:Bits +Log2Ceiling64 src/gutil/bits.cc /^int Bits::Log2Ceiling64(uint64 n) {$/;" f class:Bits +Log2Ceiling64 src/util/bit_util.h /^ static inline int Log2Ceiling64(uint64_t n) {$/;" f class:doris::BitUtil +Log2CeilingNonZero64 src/util/bit_util.h /^ static inline int Log2CeilingNonZero64(uint64_t n) {$/;" f class:doris::BitUtil +Log2Floor src/gutil/bits.h /^inline int Bits::Log2Floor(uint32 n) {$/;" f class:Bits +Log2Floor64 src/gutil/bits.h /^inline int Bits::Log2Floor64(uint64 n) {$/;" f class:Bits +Log2Floor64 src/util/bit_util.h /^ static inline int Log2Floor64(uint64_t n) {$/;" f class:doris::BitUtil +Log2Floor64_Portable src/gutil/bits.h /^inline int Bits::Log2Floor64_Portable(uint64 n) {$/;" f class:Bits +Log2FloorNonZero src/gutil/bits.h /^inline int Bits::Log2FloorNonZero(uint32 n) {$/;" f class:Bits +Log2FloorNonZero64 src/gutil/bits.h /^inline int Bits::Log2FloorNonZero64(uint64 n) {$/;" f class:Bits +Log2FloorNonZero64 src/util/bit_util.h /^ static inline int Log2FloorNonZero64(uint64_t n) {$/;" f class:doris::BitUtil +Log2FloorNonZero64_Portable src/gutil/bits.h /^inline int Bits::Log2FloorNonZero64_Portable(uint64 n) {$/;" f class:Bits +Log2FloorNonZero_Portable src/gutil/bits.h /^inline int Bits::Log2FloorNonZero_Portable(uint32 n) {$/;" f class:Bits +Log2Floor_Portable src/gutil/bits.cc /^int Bits::Log2Floor_Portable(uint32 n) {$/;" f class:Bits +LogKuduMessage src/exec/kudu_util.cpp /^void LogKuduMessage(void* unused, kudu::client::KuduLogSeverity severity,$/;" f namespace:doris +LogUsage src/runtime/mem_tracker.cpp /^std::string MemTracker::LogUsage(const std::string& prefix, const std::list& trackers,$/;" f class:doris::MemTracker +LogUsage src/runtime/mem_tracker.cpp /^std::string MemTracker::LogUsage(const std::string& prefix, int64_t* logged_consumption) const {$/;" f class:doris::MemTracker +Lookup test/olap/lru_cache_test.cpp /^ int Lookup(int key) {$/;" f class:doris::CacheTest +LookupOrEmplace src/gutil/map-util.h /^LookupOrEmplace(Collection* const collection, Args&&... args) {$/;" f +LookupOrInsert src/gutil/map-util.h /^LookupOrInsert(Collection* const collection,$/;" f +LookupOrInsertNew src/gutil/map-util.h /^LookupOrInsertNew(Collection* const collection,$/;" f +LookupOrInsertNewSharedPtr src/gutil/map-util.h /^LookupOrInsertNewSharedPtr($/;" f +LruCache src/util/lru_cache.hpp /^ LruCache(size_t max_size) : _max_size(max_size) { }$/;" f class:doris::LruCache +LruCache src/util/lru_cache.hpp /^class LruCache {$/;" c namespace:doris +LruCacheTest test/util/lru_cache_util_test.cpp /^class LruCacheTest : public testing::Test {$/;" c namespace:doris file: +Lz4BlockCompression src/util/block_compression.cpp /^class Lz4BlockCompression : public BlockCompressionCodec {$/;" c namespace:doris file: +Lz4FrameDecompressor src/exec/decompressor.h /^ Lz4FrameDecompressor() : Decompressor(CompressType::LZ4FRAME) {}$/;" f class:doris::Lz4FrameDecompressor +Lz4FrameDecompressor src/exec/decompressor.h /^class Lz4FrameDecompressor : public Decompressor {$/;" c namespace:doris +Lz4fBlockCompression src/util/block_compression.cpp /^class Lz4fBlockCompression : public BlockCompressionCodec {$/;" c namespace:doris file: +LzoBinaryReader src/olap/push_handler.cpp /^LzoBinaryReader::LzoBinaryReader()$/;" f class:doris::LzoBinaryReader +LzoBinaryReader src/olap/push_handler.h /^class LzoBinaryReader: public IBinaryReader {$/;" c namespace:doris +LzoChecksum src/exec/decompressor.h /^ enum LzoChecksum {$/;" g class:doris::LzopDecompressor +LzopDecompressor src/exec/decompressor.h /^ LzopDecompressor() :$/;" f class:doris::LzopDecompressor +LzopDecompressor src/exec/decompressor.h /^class LzopDecompressor : public Decompressor {$/;" c namespace:doris +MACRO_CONCAT src/util/runtime_profile.h 43;" d +MAKE_ENUM_LIMITS src/gutil/casts.h 248;" d +MAKE_SNAPSHOT src/agent/task_worker_pool.h /^ MAKE_SNAPSHOT,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +MANGLE_PREFIX src/util/symbols_util.cpp /^const char* MANGLE_PREFIX = "_Z";$/;" m namespace:doris file: +MAP_ANONYMOUS src/gutil/port.h 217;" d +MATERIALIZE_TUPLE_TIMER src/exec/olap_scanner.cpp /^static const std::string MATERIALIZE_TUPLE_TIMER =$/;" m namespace:doris file: +MAX src/exprs/agg_fn.h /^ MAX,$/;" e enum:doris::AggFn::AggregationOp +MAX src/exprs/agg_fn_evaluator.h /^ MAX,$/;" e enum:doris::AggFnEvaluator::AggregationOp +MAXIMUM_BYTES src/olap/rowset/segment_v2/bloom_filter.h /^ static const uint32_t MAXIMUM_BYTES = 128 * 1024 * 1024;$/;" m class:doris::segment_v2::BloomFilter +MAXU32 src/util/minizip/ioapi.h 95;" d +MAX_ALLOCATION_BYTES src/runtime/bufferpool/suballocator.cc /^constexpr int64_t Suballocator::MAX_ALLOCATION_BYTES;$/;" m class:doris::Suballocator file: +MAX_ALLOCATION_BYTES src/runtime/bufferpool/suballocator.h /^ static constexpr int64_t MAX_ALLOCATION_BYTES = BufferPool::MAX_BUFFER_BYTES;$/;" m class:doris::Suballocator +MAX_BASE src/exprs/math_functions.h /^ static const int32_t MAX_BASE = 36;$/;" m class:doris::MathFunctions +MAX_BIGINT_WIDTH src/util/mysql_global.h 41;" d +MAX_BLOB_WIDTH src/util/mysql_global.h 43;" d +MAX_BUCKET_OCCUPANCY_FRACTION src/exec/hash_table.cpp /^const float HashTable::MAX_BUCKET_OCCUPANCY_FRACTION = 0.75f;$/;" m class:doris::HashTable file: +MAX_BUCKET_OCCUPANCY_FRACTION src/exec/hash_table.h /^ static const float MAX_BUCKET_OCCUPANCY_FRACTION;$/;" m class:doris::HashTable +MAX_BUFFER_BYTES src/runtime/bufferpool/buffer_pool.cc /^constexpr int64_t BufferPool::MAX_BUFFER_BYTES;$/;" m class:doris::BufferPool file: +MAX_BUFFER_BYTES src/runtime/bufferpool/buffer_pool.h /^ static constexpr int64_t MAX_BUFFER_BYTES = 1L << LOG_MAX_BUFFER_BYTES;$/;" m class:doris::BufferPool +MAX_BUFFER_SIZE test/runtime/disk_io_mgr_test.cpp /^const int MAX_BUFFER_SIZE = 1024;$/;" m namespace:doris file: +MAX_CHAR_INLINE_LENGTH src/runtime/types.h /^ static const int MAX_CHAR_INLINE_LENGTH = 128;$/;" m struct:doris::TypeDescriptor +MAX_CHAR_LENGTH src/runtime/types.h /^ static const int MAX_CHAR_LENGTH = 255;$/;" m struct:doris::TypeDescriptor +MAX_CHAR_WIDTH src/util/mysql_global.h 42;" d +MAX_CHUNK_SIZE src/runtime/mem_pool.cpp /^const int MemPool::MAX_CHUNK_SIZE;$/;" m class:doris::MemPool file: +MAX_CHUNK_SIZE src/runtime/mem_pool.h /^ static const int MAX_CHUNK_SIZE = 512 * 1024;$/;" m class:doris::MemPool +MAX_DATETIME_WIDTH src/util/mysql_global.h 46;" d +MAX_DECIMAL4_PRECISION src/runtime/types.h /^ static const int MAX_DECIMAL4_PRECISION = 9;$/;" m struct:doris::TypeDescriptor +MAX_DECIMAL8_PRECISION src/runtime/types.h /^ static const int MAX_DECIMAL8_PRECISION = 18;$/;" m struct:doris::TypeDescriptor +MAX_DECIMAL_STR_LENGTH src/util/mysql_global.h 54;" d +MAX_DECIMAL_VALUE src/runtime/decimalv2_value.h /^ static const int128_t MAX_DECIMAL_VALUE = $/;" m class:doris::DecimalV2Value +MAX_DECPT_FOR_F_FORMAT src/util/mysql_global.h 45;" d +MAX_DISK_IO_UTIL_PERCENT src/http/action/metrics_action.cpp /^ static const std::string MAX_DISK_IO_UTIL_PERCENT;$/;" m class:doris::SimpleCoreMetricsVisitor file: +MAX_DISK_IO_UTIL_PERCENT src/http/action/metrics_action.cpp /^const std::string SimpleCoreMetricsVisitor::MAX_DISK_IO_UTIL_PERCENT = "max_disk_io_util_percent";$/;" m class:doris::SimpleCoreMetricsVisitor file: +MAX_DOUBLE_STR_LENGTH src/util/mysql_global.h 51;" d +MAX_DTVALUE_STR_LEN src/runtime/datetime_value.h /^const int MAX_DTVALUE_STR_LEN = 27;$/;" m namespace:doris +MAX_ERROR_NUM src/runtime/runtime_state.cpp /^const int64_t MAX_ERROR_NUM = 50;$/;" m namespace:doris file: +MAX_EXPR_VALUES_ARRAY_SIZE src/exec/new_partitioned_hash_table.h /^ static const int MAX_EXPR_VALUES_ARRAY_SIZE = 256 << 10;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +MAX_FILL_FACTOR src/exec/new_partitioned_hash_table.cc /^constexpr double NewPartitionedHashTable::MAX_FILL_FACTOR;$/;" m class:NewPartitionedHashTable file: +MAX_FILL_FACTOR src/exec/new_partitioned_hash_table.h /^ static constexpr double MAX_FILL_FACTOR = 0.75;$/;" m class:doris::NewPartitionedHashTable +MAX_FILL_FACTOR src/exec/partitioned_hash_table.cc /^const double PartitionedHashTable::MAX_FILL_FACTOR = 0.75f;$/;" m class:doris::PartitionedHashTable file: +MAX_FILL_FACTOR src/exec/partitioned_hash_table.h /^ static const double MAX_FILL_FACTOR;$/;" m class:doris::PartitionedHashTable +MAX_FILTER_RATIO_KEY src/http/action/mini_load.cpp /^const std::string MAX_FILTER_RATIO_KEY = "max_filter_ratio";$/;" m namespace:doris file: +MAX_FLOAT_CHARS src/exprs/cast_functions.cpp /^const int MAX_FLOAT_CHARS = 24;$/;" m namespace:doris file: +MAX_FLOAT_STR_LENGTH src/util/mysql_global.h 49;" d +MAX_FORWARDS src/http/http_headers.cpp /^const char* HttpHeaders::MAX_FORWARDS = "Max-Forwards";$/;" m class:doris::HttpHeaders file: +MAX_FORWARDS src/http/http_headers.h /^ static const char* MAX_FORWARDS;$/;" m class:doris::HttpHeaders +MAX_FRAC_DIGITS_NUM src/olap/decimal12.h /^ static const int32_t MAX_FRAC_DIGITS_NUM = 9;$/;" m struct:doris::decimal12_t +MAX_FRAC_VALUE src/runtime/decimalv2_value.h /^ static const int32_t MAX_FRAC_VALUE = 999999999;$/;" m class:doris::DecimalV2Value +MAX_INT128 src/runtime/large_int_value.h /^const __int128 MAX_INT128 = ~((__int128) 0x01 << 127);$/;" m namespace:doris +MAX_INT64 src/runtime/decimalv2_value.h /^ static const int64_t MAX_INT64 = 9223372036854775807ll;$/;" m class:doris::DecimalV2Value +MAX_INTEL_MICRO_ARCHITECTURE src/gutil/cpu.h /^ MAX_INTEL_MICRO_ARCHITECTURE$/;" e enum:base::final::IntelMicroArchitecture +MAX_INT_DIGITS_NUM src/olap/decimal12.h /^ static const int32_t MAX_INT_DIGITS_NUM = 18;$/;" m struct:doris::decimal12_t +MAX_INT_VALUE src/runtime/decimalv2_value.h /^ static const int64_t MAX_INT_VALUE = 999999999999999999;$/;" m class:doris::DecimalV2Value +MAX_INT_WIDTH src/util/mysql_global.h 40;" d +MAX_LENGTH output/udf/include/udf.h /^ static const int MAX_LENGTH = (1 << 30);$/;" m struct:doris_udf::StringVal +MAX_LENGTH src/runtime/string_value.h /^ static const int MAX_LENGTH = (1 << 30);$/;" m struct:doris::StringValue +MAX_LENGTH src/udf/udf.h /^ static const int MAX_LENGTH = (1 << 30);$/;" m struct:doris_udf::StringVal +MAX_LITERAL_SIZE src/olap/rowset/run_length_byte_writer.cpp /^const int32_t RunLengthByteWriter::MAX_LITERAL_SIZE;$/;" m class:doris::RunLengthByteWriter file: +MAX_LITERAL_SIZE src/olap/rowset/run_length_byte_writer.h /^ static const int32_t MAX_LITERAL_SIZE = 128;$/;" m class:doris::RunLengthByteWriter +MAX_MEDIUMINT_WIDTH src/util/mysql_global.h 39;" d +MAX_MEM_POOL_SIZE src/runtime/row_batch.h /^ static const int MAX_MEM_POOL_SIZE = 32 * 1024 * 1024;$/;" m class:doris::RowBatch +MAX_MONOTONIC_SECONDS src/util/monotime.cpp 28;" d file: +MAX_NETWORK_RECEIVE_BYTES_RATE src/http/action/metrics_action.cpp /^ static const std::string MAX_NETWORK_RECEIVE_BYTES_RATE;$/;" m class:doris::SimpleCoreMetricsVisitor file: +MAX_NETWORK_RECEIVE_BYTES_RATE src/http/action/metrics_action.cpp /^const std::string SimpleCoreMetricsVisitor::MAX_NETWORK_RECEIVE_BYTES_RATE= "max_network_receive_bytes_rate";$/;" m class:doris::SimpleCoreMetricsVisitor file: +MAX_NETWORK_SEND_BYTES_RATE src/http/action/metrics_action.cpp /^ static const std::string MAX_NETWORK_SEND_BYTES_RATE;$/;" m class:doris::SimpleCoreMetricsVisitor file: +MAX_NETWORK_SEND_BYTES_RATE src/http/action/metrics_action.cpp /^const std::string SimpleCoreMetricsVisitor::MAX_NETWORK_SEND_BYTES_RATE= "max_network_send_bytes_rate";$/;" m class:doris::SimpleCoreMetricsVisitor file: +MAX_OP_IN_FIELD_NUM src/olap/olap_define.h /^static const uint32_t MAX_OP_IN_FIELD_NUM = 100;$/;" m namespace:doris +MAX_PARTITION_DEPTH src/exec/new_partitioned_aggregation_node.h /^ static const int MAX_PARTITION_DEPTH = 16;$/;" m class:doris::NewPartitionedAggregationNode +MAX_PARTITION_DEPTH src/exec/partitioned_aggregation_node.h /^ static const int MAX_PARTITION_DEPTH = 16;$/;" m class:doris::PartitionedAggregationNode +MAX_PATCH_LIST src/olap/rowset/run_length_integer_writer.h /^ static const uint16_t MAX_PATCH_LIST = uint16_t(MAX_SCOPE * 0.1);$/;" m class:doris::RunLengthIntegerWriter +MAX_PATH_LEN test/olap/delete_handler_test.cpp /^static const uint32_t MAX_PATH_LEN = 1024;$/;" m namespace:doris file: +MAX_PATH_LEN test/olap/delta_writer_test.cpp /^static const uint32_t MAX_PATH_LEN = 1024;$/;" m namespace:doris file: +MAX_PATH_LEN test/olap/rowset/alpha_rowset_test.cpp /^static const uint32_t MAX_PATH_LEN = 1024;$/;" m namespace:doris file: +MAX_PATH_LEN test/olap/rowset/rowset_converter_test.cpp /^static const uint32_t MAX_PATH_LEN = 1024;$/;" m namespace:doris file: +MAX_POSITION_SIZE src/olap/olap_define.h /^static const uint32_t MAX_POSITION_SIZE = 16;$/;" m namespace:doris +MAX_PRECISION src/runtime/types.h /^ static const int MAX_PRECISION = 38;$/;" m struct:doris::TypeDescriptor +MAX_QUEUE_CAPACITY src/runtime/disk_io_mgr_scan_range.cc /^const int MAX_QUEUE_CAPACITY = 128;$/;" m namespace:doris file: +MAX_RECEIVERS test/runtime/data_stream_test.cpp /^ static const int MAX_RECEIVERS = 16;$/;" m class:doris::DataStreamTest file: +MAX_REPEAT_SIZE src/olap/rowset/run_length_byte_writer.cpp /^const int32_t RunLengthByteWriter::MAX_REPEAT_SIZE;$/;" m class:doris::RunLengthByteWriter file: +MAX_REPEAT_SIZE src/olap/rowset/run_length_byte_writer.h /^ static const int32_t MAX_REPEAT_SIZE = 127 + MIN_REPEAT_SIZE;$/;" m class:doris::RunLengthByteWriter +MAX_RETRY src/olap/task/engine_batch_load_task.h /^const uint32_t MAX_RETRY = 3;$/;" m namespace:doris +MAX_RETRY_TIMES test/olap/delta_writer_test.cpp /^static const uint32_t MAX_RETRY_TIMES = 10;$/;" m namespace:doris file: +MAX_ROWSET_ID src/olap/olap_common.h /^static const int64_t MAX_ROWSET_ID = 1L << 56;$/;" m namespace:doris +MAX_SCALE src/runtime/types.h /^ static const int MAX_SCALE = MAX_PRECISION;$/;" m struct:doris::TypeDescriptor +MAX_SCAVENGE_ATTEMPTS src/runtime/bufferpool/buffer_allocator.h /^ static const int MAX_SCAVENGE_ATTEMPTS = 3;$/;" m struct:doris::BufferPool::BufferAllocator +MAX_SCOPE src/olap/rowset/run_length_integer_writer.h /^ static const uint16_t MAX_SCOPE = 512;$/;" m class:doris::RunLengthIntegerWriter +MAX_SEGMENT_SIZE src/olap/rowset/beta_rowset_writer.cpp /^const uint32_t MAX_SEGMENT_SIZE = static_cast($/;" m namespace:doris file: +MAX_SENDERS test/runtime/data_stream_test.cpp /^ static const int MAX_SENDERS = 16;$/;" m class:doris::DataStreamTest file: +MAX_SHARD_NUM src/olap/data_dir.h /^ static const uint32_t MAX_SHARD_NUM = 1024;$/;" m class:doris::DataDir +MAX_SHARD_NUM src/runtime/load_path_mgr.cpp /^static const uint32_t MAX_SHARD_NUM = 1024;$/;" m namespace:doris file: +MAX_SHORT_REPEAT_LENGTH src/olap/rowset/run_length_integer_writer.h /^ static const uint16_t MAX_SHORT_REPEAT_LENGTH = 10;$/;" m class:doris::RunLengthIntegerWriter +MAX_SMALLINT_WIDTH src/util/mysql_global.h 38;" d +MAX_STATISTIC_LENGTH src/olap/olap_define.h /^static const uint32_t MAX_STATISTIC_LENGTH = 34;$/;" m namespace:doris +MAX_TINYINT_WIDTH src/util/mysql_global.h 37;" d +MAX_TUPLE_POOL_SIZE src/exec/analytic_eval_node.cpp /^static const int MAX_TUPLE_POOL_SIZE = 8 * 1024 * 1024; \/\/ 8MB$/;" v file: +MAX_VARCHAR_LENGTH src/runtime/types.h /^ static const int MAX_VARCHAR_LENGTH = 65355;$/;" m struct:doris::TypeDescriptor +MAX_VINT_LEN src/exec/read_write_util.h /^ static const int MAX_VINT_LEN = 9;$/;" m class:doris::ReadWriteUtil +MAX_VLQ_BYTE_LEN src/util/bit_stream_utils.h /^ static const int MAX_VLQ_BYTE_LEN = 5;$/;" m class:doris::BitReader +MAX_WARNINGS src/udf/udf.cpp /^static const int MAX_WARNINGS = 1000;$/;" m namespace:doris_udf file: +MAX_ZINT_LEN src/exec/read_write_util.h /^ const static int MAX_ZINT_LEN = 5;$/;" m class:doris::ReadWriteUtil +MAX_ZLONG_LEN src/exec/read_write_util.h /^ const static int MAX_ZLONG_LEN = 10;$/;" m class:doris::ReadWriteUtil +MEDIUM_UC src/olap/options.cpp /^static std::string MEDIUM_UC = "MEDIUM";$/;" m namespace:doris file: +MEGABYTE src/util/debug_util.cpp 33;" d file: +MEGABYTE src/util/pretty_printer.h /^ static const int64_t MEGABYTE = KILOBYTE * 1024;$/;" m class:doris::PrettyPrinter +MEM_ALLOC_FAILED_ERROR_MSG_1 src/runtime/spill_sorter.cc /^const string MEM_ALLOC_FAILED_ERROR_MSG_1 = "Failed to allocate block for $0-length ";$/;" m namespace:doris file: +MEM_ALLOC_FAILED_ERROR_MSG_2 src/runtime/spill_sorter.cc /^const string MEM_ALLOC_FAILED_ERROR_MSG_2 = "-length "$/;" m namespace:doris file: +MEM_POOL_POISON src/runtime/mem_pool.cpp 32;" d file: +MERGE src/exec/olap_scan_node.h /^ MERGE = 4,$/;" e enum:doris::TransferStatus +META_COLUMN_FAMILY src/olap/olap_define.h /^static const std::string META_COLUMN_FAMILY = "meta";$/;" m namespace:doris +META_COLUMN_FAMILY_INDEX src/olap/olap_define.h /^ META_COLUMN_FAMILY_INDEX,$/;" e enum:doris::ColumnFamilyIndex +META_POSTFIX src/olap/olap_meta.cpp /^const std::string META_POSTFIX = "\/meta";$/;" m namespace:doris file: +META_TYPE src/http/action/meta_action.h /^enum META_TYPE {$/;" g namespace:doris +METHOD_NOT_ALLOWED src/http/http_status.h /^ METHOD_NOT_ALLOWED = 405,$/;" e enum:doris::HttpStatus +MICROSECOND src/runtime/datetime_value.h /^ MICROSECOND,$/;" e enum:doris::TimeUnit +MICROS_PER_MILLI src/util/time.h 28;" d +MICROS_PER_SEC src/util/time.h 27;" d +MILLION src/util/debug_util.cpp 41;" d file: +MILLION src/util/pretty_printer.h /^ static const int64_t MILLION = THOUSAND * 1000;$/;" m class:doris::PrettyPrinter +MILLIS_PER_SEC src/util/time.h 29;" d +MIN src/exprs/agg_fn.h /^ MIN,$/;" e enum:doris::AggFn::AggregationOp +MIN src/exprs/agg_fn_evaluator.h /^ MIN,$/;" e enum:doris::AggFnEvaluator::AggregationOp +MINIMUM_BYTES src/olap/rowset/segment_v2/bloom_filter.h /^ static const uint32_t MINIMUM_BYTES = 32;$/;" m class:doris::segment_v2::BloomFilter +MINI_PREFIX src/olap/olap_define.h /^static const std::string MINI_PREFIX = "\/mini_download";$/;" m namespace:doris +MINUTE src/runtime/datetime_value.h /^ MINUTE,$/;" e enum:doris::TimeUnit +MINUTE src/util/debug_util.cpp 37;" d file: +MINUTE src/util/pretty_printer.h /^ static const int64_t MINUTE = SECOND * 60;$/;" m class:doris::PrettyPrinter +MINUTE_MICROSECOND src/runtime/datetime_value.h /^ MINUTE_MICROSECOND,$/;" e enum:doris::TimeUnit +MINUTE_SECOND src/runtime/datetime_value.h /^ MINUTE_SECOND,$/;" e enum:doris::TimeUnit +MIN_ALLOCATION_BYTES src/runtime/bufferpool/suballocator.cc /^constexpr int64_t Suballocator::MIN_ALLOCATION_BYTES;$/;" m class:doris::Suballocator file: +MIN_ALLOCATION_BYTES src/runtime/bufferpool/suballocator.h /^ static constexpr int64_t MIN_ALLOCATION_BYTES = 1L << LOG_MIN_ALLOCATION_BYTES;$/;" m class:doris::Suballocator +MIN_BASE src/exprs/math_functions.h /^ static const int32_t MIN_BASE = 2;$/;" m class:doris::MathFunctions +MIN_BUFFER_SIZE test/runtime/disk_io_mgr_test.cpp /^const int MIN_BUFFER_SIZE = 512;$/;" m namespace:doris file: +MIN_FILTER_BLOCK_NUM src/olap/rowset/segment_reader.cpp /^static const uint32_t MIN_FILTER_BLOCK_NUM = 10;$/;" m namespace:doris file: +MIN_HEADER_SIZE src/exec/decompressor.h /^ const static uint32_t MIN_HEADER_SIZE;$/;" m class:doris::LzopDecompressor +MIN_HEADER_SIZE src/exec/lzo_decompressor.cpp /^const uint32_t LzopDecompressor::MIN_HEADER_SIZE = 34;$/;" m class:doris::LzopDecompressor file: +MIN_INT128 src/runtime/large_int_value.h /^const __int128 MIN_INT128 = ((__int128) 0x01 << 127);$/;" m namespace:doris +MIN_LZO_VERSION src/exec/decompressor.h /^ const static uint64_t MIN_LZO_VERSION;$/;" m class:doris::LzopDecompressor +MIN_LZO_VERSION src/exec/lzo_decompressor.cpp /^const uint64_t LzopDecompressor::MIN_LZO_VERSION = 0x0100;$/;" m class:doris::LzopDecompressor file: +MIN_QUEUE_CAPACITY src/runtime/disk_io_mgr_scan_range.cc /^const int MIN_QUEUE_CAPACITY = 2;$/;" m namespace:doris file: +MIN_REPEAT src/olap/rowset/run_length_integer_writer.h /^ static const uint16_t MIN_REPEAT = 3; \/\/ NOTE 不要修改这个值, 否则程序出错$/;" m class:doris::RunLengthIntegerWriter +MIN_REPEAT_SIZE src/olap/rowset/run_length_byte_writer.cpp /^const int32_t RunLengthByteWriter::MIN_REPEAT_SIZE;$/;" m class:doris::RunLengthByteWriter file: +MIN_REPEAT_SIZE src/olap/rowset/run_length_byte_writer.h /^ static const int32_t MIN_REPEAT_SIZE = 3;$/;" m class:doris::RunLengthByteWriter +MIPS_SYSCALL_CLOBBERS src/gutil/linux_syscall_support.h 1711;" d +MIPS_SYSCALL_CLOBBERS src/gutil/linux_syscall_support.h 1714;" d +MIX32 src/gutil/hash/legacy_hash.h /^static const uint32 MIX32 = 0x12b9b0a1UL; \/\/ pi; an arbitrary number$/;" v +MIX64 src/gutil/hash/legacy_hash.h /^static const uint64 MIX64 = GG_ULONGLONG(0x2b992ddfa23249d6); \/\/ more of pi$/;" v +MOD src/exprs/arithmetic_expr.h /^ MOD,$/;" e enum:doris::ArithmeticExpr::BinaryOpType +MODE_READ_AT_SNAPSHOT src/exec/kudu_scanner.cpp /^const string MODE_READ_AT_SNAPSHOT = "READ_AT_SNAPSHOT";$/;" m namespace:doris file: +MODULE_KEY src/http/monitor_action.cpp /^const std::string MODULE_KEY = "module";$/;" m namespace:doris file: +MONTH src/runtime/datetime_value.h /^ MONTH,$/;" e enum:doris::TimeUnit +MOVE src/agent/task_worker_pool.h /^ MOVE,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +MOVED_PERMANENTLY src/http/http_status.h /^ MOVED_PERMANENTLY = 301,$/;" e enum:doris::HttpStatus +MOVE_ONLY_TYPE_FOR_CPP_03 src/gutil/move.h 204;" d +MREMAP_FIXED src/gutil/linux_syscall_support.h 571;" d +MSG_NOSIGNAL src/gutil/port.h 238;" d +MUL src/exprs/arithmetic_expr.h /^ MUL,$/;" e enum:doris::ArithmeticExpr::BinaryOpType +MULTIPLE_CHOICES src/http/http_status.h /^ MULTIPLE_CHOICES = 300,$/;" e enum:doris::HttpStatus +MURMUR3_32_SEED src/util/hash_util.hpp /^ static const uint32_t MURMUR3_32_SEED = 104729;$/;" m class:doris::HashUtil +MURMUR_PRIME src/util/hash_util.hpp /^ static const uint64_t MURMUR_PRIME = 0xc6a4a7935bd1e995ULL;$/;" m class:doris::HashUtil +MURMUR_R src/util/hash_util.hpp /^ static const int MURMUR_R = 47;$/;" m class:doris::HashUtil +MURMUR_SEED src/util/hash_util.hpp /^ static const uint32_t MURMUR_SEED = 0xadc83b19ULL;$/;" m class:doris::HashUtil +MUST_CREATE src/env/env.h /^ MUST_CREATE,$/;" e enum:doris::Env::OpenMode +MUST_EXIST src/env/env.h /^ MUST_EXIST$/;" e enum:doris::Env::OpenMode +MUST_USE_RESULT src/gutil/port.h 464;" d +MY_ALIGN src/util/mysql_global.h 34;" d +Maintenance src/runtime/bufferpool/buffer_allocator.cc /^void BufferPool::BufferAllocator::Maintenance() {$/;" f class:doris::BufferPool::BufferAllocator +Maintenance src/runtime/bufferpool/buffer_allocator.cc /^void BufferPool::FreeBufferArena::Maintenance() {$/;" f class:doris::BufferPool::FreeBufferArena +Maintenance src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::Maintenance() {$/;" f class:doris::BufferPool +MakeBuffer test/util/arrow/arrow_row_batch_test.cpp /^void MakeBuffer(const std::string& data, std::shared_ptr* out) {$/;" f namespace:doris +MakeBuffer test/util/arrow/arrow_row_block_test.cpp /^void MakeBuffer(const std::string& data, std::shared_ptr* out) {$/;" f namespace:doris +MakeRandomSizeData test/runtime/buffered_block_mgr2_test.cpp /^ static int32_t* MakeRandomSizeData(BufferedBlockMgr2::Block* block) {$/;" f class:doris::BufferedBlockMgrTest +MakeScopedCleanup src/util/scoped_cleanup.h /^ScopedCleanup MakeScopedCleanup(F f) {$/;" f namespace:doris +MapUtilAssignNewDefaultInstance src/gutil/map-util.h /^void MapUtilAssignNewDefaultInstance(T** location) {$/;" f +MapUtilAssignNewInstance src/gutil/map-util.h /^void MapUtilAssignNewInstance(T** location, const Arg &arg) {$/;" f +Maskx src/gutil/utf/rune.c /^ Maskx = (1<& t_output_expr,$/;" f class:doris::MemoryScratchSink +MemoryScratchSink src/runtime/memory_scratch_sink.h /^class MemoryScratchSink : public DataSink {$/;" c namespace:doris +MemoryScratchSinkTest test/runtime/memory_scratch_sink_test.cpp /^ MemoryScratchSinkTest() {$/;" f class:doris::MemoryScratchSinkTest +MemoryScratchSinkTest test/runtime/memory_scratch_sink_test.cpp /^class MemoryScratchSinkTest : public testing::Test {$/;" c namespace:doris file: +Merge src/exprs/new_agg_fn_evaluator.cc /^void NewAggFnEvaluator::Merge(Tuple* src, Tuple* dst) {$/;" f class:NewAggFnEvaluator +MergeComparison src/exec/olap_scan_node.h /^ MergeComparison(CompareLargeFunc compute_fn, int offset) {$/;" f class:doris::OlapScanNode::MergeComparison +MergeComparison src/exec/olap_scan_node.h /^ class MergeComparison {$/;" c class:doris::OlapScanNode +MergeContextComparator src/olap/generic_iterators.cpp /^ struct MergeContextComparator {$/;" s class:doris::MergeIterator file: +MergeElement src/olap/schema_change.cpp /^ struct MergeElement {$/;" s class:doris::RowBlockMerger file: +MergeFn output/udf/include/uda_test_harness.h /^ typedef void (*MergeFn)(FunctionContext* context, const INTERMEDIATE& src,$/;" t class:doris_udf::UdaTestHarnessBase +MergeFn src/udf/uda_test_harness.h /^ typedef void (*MergeFn)(FunctionContext* context, const INTERMEDIATE& src,$/;" t class:doris_udf::UdaTestHarnessBase +MergeHeap src/olap/reader.cpp /^ typedef std::priority_queue, ChildCtxComparator> MergeHeap;$/;" t class:doris::CollectIterator file: +MergeIterator src/olap/generic_iterators.cpp /^ MergeIterator(std::vector iters)$/;" f class:doris::MergeIterator +MergeIterator src/olap/generic_iterators.cpp /^class MergeIterator : public RowwiseIterator {$/;" c namespace:doris file: +MergeIteratorContext src/olap/generic_iterators.cpp /^ MergeIteratorContext(RowwiseIterator* iter)$/;" f class:doris::MergeIteratorContext +MergeIteratorContext src/olap/generic_iterators.cpp /^class MergeIteratorContext {$/;" c namespace:doris file: +MergeJoinNode src/exec/merge_join_node.cpp /^MergeJoinNode::MergeJoinNode($/;" f class:doris::MergeJoinNode +MergeJoinNode src/exec/merge_join_node.h /^class MergeJoinNode : public ExecNode {$/;" c namespace:doris +MergeNode src/exec/merge_node.cpp /^MergeNode::MergeNode(ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::MergeNode +MergeNode src/exec/merge_node.h /^class MergeNode : public ExecNode {$/;" c namespace:doris +MergeSorter src/runtime/merge_sorter.cpp /^MergeSorter::MergeSorter(const TupleRowComparator& compare_less_than,$/;" f class:doris::MergeSorter +MergeSorter src/runtime/merge_sorter.h /^class MergeSorter {$/;" c namespace:doris +Merger src/olap/merger.h /^class Merger {$/;" c namespace:doris +MessageBodyFileSink src/runtime/message_body_sink.h /^ MessageBodyFileSink(const std::string& path) : _path(path) { }$/;" f class:doris::MessageBodyFileSink +MessageBodyFileSink src/runtime/message_body_sink.h /^class MessageBodyFileSink : public MessageBodySink {$/;" c namespace:doris +MessageBodySink src/runtime/message_body_sink.h /^class MessageBodySink {$/;" c namespace:doris +MessageBodySinkTest test/http/message_body_sink_test.cpp /^ MessageBodySinkTest() { }$/;" f class:doris::MessageBodySinkTest +MessageBodySinkTest test/http/message_body_sink_test.cpp /^class MessageBodySinkTest : public testing::Test {$/;" c namespace:doris file: +MetaAction src/http/action/meta_action.h /^ MetaAction(META_TYPE meta_type) : _meta_type(meta_type) {}$/;" f class:doris::MetaAction +MetaAction src/http/action/meta_action.h /^class MetaAction : public HttpHandler {$/;" c namespace:doris +MetaOpType src/olap/tablet_sync_service.h /^enum MetaOpType {$/;" g namespace:doris +Metric src/util/metrics.h /^ Metric(MetricType type) :_type(type), _registry(nullptr) { }$/;" f class:doris::Metric +Metric src/util/metrics.h /^class Metric {$/;" c namespace:doris +MetricCollector src/util/metrics.h /^class MetricCollector {$/;" c namespace:doris +MetricLabel src/util/metrics.h /^ MetricLabel() { }$/;" f struct:doris::MetricLabel +MetricLabel src/util/metrics.h /^ MetricLabel(const T& name_, const P& value_) :name(name_), value(value_) {$/;" f struct:doris::MetricLabel +MetricLabel src/util/metrics.h /^struct MetricLabel {$/;" s namespace:doris +MetricLabels src/util/metrics.h /^struct MetricLabels {$/;" s namespace:doris +MetricRegistry src/util/metrics.h /^ MetricRegistry(const std::string& name) : _name(name) { }$/;" f class:doris::MetricRegistry +MetricRegistry src/util/metrics.h /^class MetricRegistry {$/;" c namespace:doris +MetricType src/util/metrics.h /^enum class MetricType {$/;" c namespace:doris +MetricsAction src/http/action/metrics_action.h /^ MetricsAction(MetricRegistry* metrics) :_metrics(metrics) { }$/;" f class:doris::MetricsAction +MetricsAction src/http/action/metrics_action.h /^class MetricsAction : public HttpHandler {$/;" c namespace:doris +MetricsActionTest test/http/metrics_action_test.cpp /^ MetricsActionTest() { }$/;" f class:doris::MetricsActionTest +MetricsActionTest test/http/metrics_action_test.cpp /^class MetricsActionTest : public testing::Test {$/;" c namespace:doris file: +MetricsTest test/util/new_metrics_test.cpp /^ MetricsTest() { }$/;" f class:doris::MetricsTest +MetricsTest test/util/new_metrics_test.cpp /^class MetricsTest : public testing::Test {$/;" c namespace:doris file: +MetricsVisitor src/util/metrics.h /^class MetricsVisitor {$/;" c namespace:doris +Microsecond src/util/time.h /^ Microsecond,$/;" e enum:doris::TimePrecision +MicrosecondsInt64 src/gutil/walltime.h /^typedef int64 MicrosecondsInt64;$/;" t +Millisecond src/util/time.h /^ Millisecond,$/;" e enum:doris::TimePrecision +Min src/util/monotime.cpp /^MonoTime MonoTime::Min() {$/;" f class:doris::MonoTime +MinFinalize test/udf/uda_test.cpp /^StringVal MinFinalize(FunctionContext* context, const BufferVal& val) {$/;" f namespace:doris_udf +MinInit test/udf/uda_test.cpp /^void MinInit(FunctionContext* context, BufferVal* val) {$/;" f namespace:doris_udf +MinMerge test/udf/uda_test.cpp /^void MinMerge(FunctionContext* context, const BufferVal& src, BufferVal* dst) {$/;" f namespace:doris_udf +MinReservation src/exec/new_partitioned_aggregation_node.h /^ int64_t MinReservation() const {$/;" f class:doris::NewPartitionedAggregationNode +MinSerialize test/udf/uda_test.cpp /^const BufferVal MinSerialize(FunctionContext* context, const BufferVal& intermediate) {$/;" f namespace:doris_udf +MinState test/udf/uda_test.cpp /^struct MinState {$/;" s namespace:doris_udf file: +MinUpdate test/udf/uda_test.cpp /^void MinUpdate(FunctionContext* context, const StringVal& input, BufferVal* val) {$/;" f namespace:doris_udf +MiniLoadAction src/http/action/mini_load.cpp /^MiniLoadAction::MiniLoadAction(ExecEnv* exec_env) :$/;" f class:doris::MiniLoadAction +MiniLoadAction src/http/action/mini_load.h /^class MiniLoadAction : public HttpHandler {$/;" c namespace:doris +MiniLoadAsyncCtx src/http/action/mini_load.cpp /^ MiniLoadAsyncCtx(MiniLoadAction* handler_) : handler(handler_) { }$/;" f struct:doris::MiniLoadAsyncCtx +MiniLoadAsyncCtx src/http/action/mini_load.cpp /^struct MiniLoadAsyncCtx {$/;" s namespace:doris file: +MiniLoadCtx src/http/action/mini_load.cpp /^ MiniLoadCtx(bool is_streaming_) : is_streaming(is_streaming_) {} $/;" f struct:doris::MiniLoadCtx +MiniLoadCtx src/http/action/mini_load.cpp /^struct MiniLoadCtx {$/;" s namespace:doris file: +MinimumReservationUnavailable src/common/status.h /^ static Status MinimumReservationUnavailable(const Slice& msg, int16_t precise_code = 1, const Slice& msg2 = Slice()) {$/;" f class:doris::Status +MockAgentServerClient test/agent/mock_utils.h /^class MockAgentServerClient : public AgentServerClient {$/;" c namespace:doris +MockAgentUtils test/agent/mock_utils.h /^class MockAgentUtils : public AgentUtils {$/;" c namespace:doris +MockCommandExecutor test/olap/mock_command_executor.h /^class MockCommandExecutor : public StorageEngine {$/;" c namespace:doris +MockESServerTest test/exec/es_scan_reader_test.cpp /^ MockESServerTest() { }$/;" f class:doris::MockESServerTest +MockESServerTest test/exec/es_scan_reader_test.cpp /^class MockESServerTest : public testing::Test {$/;" c namespace:doris file: +MockMasterServerClient test/agent/mock_utils.h /^class MockMasterServerClient : public MasterServerClient {$/;" c namespace:doris +MockPusher test/agent/mock_pusher.h /^class MockPusher : public Pusher {$/;" c namespace:doris +MockTaskWorkerPool test/agent/mock_task_worker_pool.h /^class MockTaskWorkerPool : public TaskWorkerPool {$/;" c namespace:doris +ModExpr src/exprs/arithmetic_expr.h /^ ModExpr(const TExprNode& node) : ArithmeticExpr(node) { }$/;" f class:doris::ModExpr +ModExpr src/exprs/arithmetic_expr.h /^class ModExpr : public ArithmeticExpr {$/;" c namespace:doris +MonitorAction src/http/monitor_action.cpp /^MonitorAction::MonitorAction() {$/;" f class:doris::MonitorAction +MonitorAction src/http/monitor_action.h /^class MonitorAction : public HttpHandler {$/;" c namespace:doris +MonoDelta src/util/monotime.cpp /^MonoDelta::MonoDelta()$/;" f class:doris::MonoDelta +MonoDelta src/util/monotime.cpp /^MonoDelta::MonoDelta(int64_t delta)$/;" f class:doris::MonoDelta +MonoDelta src/util/monotime.h /^class MonoDelta {$/;" c namespace:doris +MonoTime src/util/monotime.cpp /^MonoTime::MonoTime()$/;" f class:doris::MonoTime +MonoTime src/util/monotime.cpp /^MonoTime::MonoTime(const struct timespec &ts) {$/;" f class:doris::MonoTime +MonoTime src/util/monotime.cpp /^MonoTime::MonoTime(int64_t nanos)$/;" f class:doris::MonoTime +MonoTime src/util/monotime.h /^class MonoTime {$/;" c namespace:doris +MonotonicMicros src/util/time.h /^inline int64_t MonotonicMicros() { \/\/ 63 bits ~= 5K years uptime$/;" f namespace:doris +MonotonicMillis src/util/time.h /^inline int64_t MonotonicMillis() {$/;" f namespace:doris +MonotonicNanos src/util/time.h /^inline int64_t MonotonicNanos() {$/;" f namespace:doris +MonotonicSeconds src/util/time.h /^inline int64_t MonotonicSeconds() {$/;" f namespace:doris +MonotonicStopWatch src/util/stopwatch.hpp /^ MonotonicStopWatch() {$/;" f class:doris::MonotonicStopWatch +MonotonicStopWatch src/util/stopwatch.hpp /^class MonotonicStopWatch {$/;" c namespace:doris +MoreThan src/util/monotime.cpp /^bool MonoDelta::MoreThan(const MonoDelta &rhs) const {$/;" f class:doris::MonoDelta +MoveHashPartitions src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::MoveHashPartitions(int64_t num_input_rows) {$/;" f class:doris::NewPartitionedAggregationNode +MoveToDirtyUnpinned src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::Client::MoveToDirtyUnpinned(Page* page) {$/;" f class:doris::BufferPool::Client +MulExpr src/exprs/arithmetic_expr.h /^ MulExpr(const TExprNode& node) : ArithmeticExpr(node) { }$/;" f class:doris::MulExpr +MulExpr src/exprs/arithmetic_expr.h /^class MulExpr : public ArithmeticExpr {$/;" c namespace:doris +MultiDistinctCountDateState src/exprs/aggregate_functions.cpp /^class MultiDistinctCountDateState {$/;" c namespace:doris file: +MultiDistinctDecimalState src/exprs/aggregate_functions.cpp /^class MultiDistinctDecimalState {$/;" c namespace:doris file: +MultiDistinctDecimalV2State src/exprs/aggregate_functions.cpp /^class MultiDistinctDecimalV2State {$/;" c namespace:doris file: +MultiDistinctNumericState src/exprs/aggregate_functions.cpp /^class MultiDistinctNumericState {$/;" c namespace:doris file: +MultiDistinctStringCountState src/exprs/aggregate_functions.cpp /^class MultiDistinctStringCountState {$/;" c namespace:doris file: +MultiNullableTupleStreamTest test/runtime/buffered_tuple_stream2_test.cpp /^class MultiNullableTupleStreamTest : public SimpleTupleStreamTest {$/;" c namespace:doris file: +MultiThreadTest test/util/blocking_queue_test.cpp /^ MultiThreadTest() : $/;" f class:doris::MultiThreadTest +MultiThreadTest test/util/blocking_queue_test.cpp /^class MultiThreadTest {$/;" c namespace:doris file: +MultiTupleStreamTest test/runtime/buffered_tuple_stream2_test.cpp /^class MultiTupleStreamTest : public SimpleTupleStreamTest {$/;" c namespace:doris file: +Mutex src/util/mutex.cpp /^Mutex::Mutex() {$/;" f class:doris::Mutex +Mutex src/util/mutex.h /^class Mutex {$/;" c namespace:doris +MutexLock src/common/logging.h 46;" d +MutexLock src/util/mutex.h /^ explicit MutexLock(Mutex* mutex, bool try_lock = false)$/;" f class:doris::MutexLock +MutexLock src/util/mutex.h /^class MutexLock {$/;" c namespace:doris +MySQLTableDescriptor src/runtime/descriptors.cpp /^MySQLTableDescriptor::MySQLTableDescriptor(const TTableDescriptor& tdesc)$/;" f class:doris::MySQLTableDescriptor +MySQLTableDescriptor src/runtime/descriptors.h /^class MySQLTableDescriptor : public TableDescriptor {$/;" c namespace:doris +MysqlConnInfo src/runtime/mysql_table_writer.h /^struct MysqlConnInfo {$/;" s namespace:doris +MysqlInfo src/util/mysql_load_error_hub.h /^ MysqlInfo(const TMysqlErrorHubInfo& t_info) :$/;" f struct:doris::MysqlLoadErrorHub::MysqlInfo +MysqlInfo src/util/mysql_load_error_hub.h /^ MysqlInfo(const std::string& h, int32_t p, const std::string& u,$/;" f struct:doris::MysqlLoadErrorHub::MysqlInfo +MysqlInfo src/util/mysql_load_error_hub.h /^ struct MysqlInfo {$/;" s class:doris::MysqlLoadErrorHub +MysqlLoadErrorHub src/util/mysql_load_error_hub.cpp /^MysqlLoadErrorHub::MysqlLoadErrorHub(const TMysqlErrorHubInfo& info) : _info(info) {$/;" f class:doris::MysqlLoadErrorHub +MysqlLoadErrorHub src/util/mysql_load_error_hub.h /^class MysqlLoadErrorHub : public LoadErrorHub {$/;" c namespace:doris +MysqlRowBuffer src/util/mysql_row_buffer.cpp /^MysqlRowBuffer::MysqlRowBuffer():$/;" f class:doris::MysqlRowBuffer +MysqlRowBuffer src/util/mysql_row_buffer.h /^class MysqlRowBuffer {$/;" c namespace:doris +MysqlScanNode src/exec/mysql_scan_node.cpp /^MysqlScanNode::MysqlScanNode(ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::MysqlScanNode +MysqlScanNode src/exec/mysql_scan_node.h /^class MysqlScanNode : public ScanNode {$/;" c namespace:doris +MysqlScanNodeTest test/exec/mysql_scan_node_test.cpp /^ MysqlScanNodeTest() : _runtim_state("test") {$/;" f class:doris::MysqlScanNodeTest +MysqlScanNodeTest test/exec/mysql_scan_node_test.cpp /^class MysqlScanNodeTest : public testing::Test {$/;" c namespace:doris file: +MysqlScanner src/exec/mysql_scanner.cpp /^MysqlScanner::MysqlScanner(const MysqlScannerParam& param)$/;" f class:doris::MysqlScanner +MysqlScanner src/exec/mysql_scanner.h /^class MysqlScanner {$/;" c namespace:doris +MysqlScannerParam src/exec/mysql_scanner.h /^ MysqlScannerParam(): client_flag(0) { }$/;" f struct:doris::MysqlScannerParam +MysqlScannerParam src/exec/mysql_scanner.h /^struct MysqlScannerParam {$/;" s namespace:doris +MysqlScannerTest test/exec/mysql_scanner_test.cpp /^ MysqlScannerTest() {$/;" f class:doris::MysqlScannerTest +MysqlScannerTest test/exec/mysql_scanner_test.cpp /^class MysqlScannerTest : public testing::Test {$/;" c namespace:doris file: +MysqlTableSink src/runtime/mysql_table_sink.cpp /^MysqlTableSink::MysqlTableSink(ObjectPool* pool, const RowDescriptor& row_desc,$/;" f class:doris::MysqlTableSink +MysqlTableSink src/runtime/mysql_table_sink.h /^class MysqlTableSink : public DataSink {$/;" c namespace:doris +MysqlTableWriter src/runtime/mysql_table_writer.cpp /^MysqlTableWriter::MysqlTableWriter(const std::vector& output_expr_ctxs) :$/;" f class:doris::MysqlTableWriter +MysqlTableWriter src/runtime/mysql_table_writer.h /^class MysqlTableWriter {$/;" c namespace:doris +MysqlTableWriterTest test/runtime/mysql_table_writer_test.cpp /^ MysqlTableWriterTest() :$/;" f class:doris::MysqlTableWriterTest +MysqlTableWriterTest test/runtime/mysql_table_writer_test.cpp /^class MysqlTableWriterTest : public testing::Test {$/;" c namespace:doris file: +NANOS_PER_MICRO src/util/time.h 26;" d +NANOS_PER_SEC src/util/time.h 25;" d +NAVIGATION_BAR_PREFIX src/http/web_page_handler.cpp /^static const std::string NAVIGATION_BAR_PREFIX =$/;" m namespace:doris file: +NAVIGATION_BAR_SUFFIX src/http/web_page_handler.cpp /^static const std::string NAVIGATION_BAR_SUFFIX =$/;" m namespace:doris file: +NDV src/exprs/agg_fn.h /^ NDV,$/;" e enum:doris::AggFn::AggregationOp +NDV src/exprs/agg_fn_evaluator.h /^ NDV,$/;" e enum:doris::AggFnEvaluator::AggregationOp +NEED_ALIGNED_LOADS src/gutil/port.h 1082;" d +NEGATIVE_INFINITY src/exec/olap_utils.h /^static const char* NEGATIVE_INFINITY = "-oo";$/;" m namespace:doris +NEVER_CACHE src/runtime/disk_io_mgr.h /^ const static int64_t NEVER_CACHE = -1;$/;" m class:doris::DiskIoMgr::ScanRange +NINE src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +NINETEEN src/olap/serialize.h /^ THIRTEEN, FOURTEEN, FIFTEEN, SIXTEEN, SEVENTEEN, EIGHTEEN, NINETEEN,$/;" e enum:doris::ser::FixedBitSize +NONE_ENCODING src/olap/rowset/run_length_integer_writer.h /^ NONE_ENCODING = 4$/;" e enum:doris::RunLengthIntegerWriter::EncodingType +NON_AUTHORITATIVE_INFORMATION src/http/http_status.h /^ NON_AUTHORITATIVE_INFORMATION = 203,$/;" e enum:doris::HttpStatus +NON_BLOCKING src/util/thrift_server.h /^ enum ServerType { THREAD_POOL = 0, THREADED, NON_BLOCKING };$/;" e enum:doris::ThriftServer::ServerType +NORMAL src/http/download_action.h /^ NORMAL = 1,$/;" e enum:doris::DownloadAction::DOWNLOAD_TYPE +NORMAL src/olap/lru_cache.h /^ NORMAL = 0,$/;" m class:doris::CachePriority +NOT_ACCEPTABLE src/http/http_status.h /^ NOT_ACCEPTABLE = 406,$/;" e enum:doris::HttpStatus +NOT_FETCH_DATA src/olap/tablet_sync_service.h 29;" d +NOT_FIXED_DEC src/runtime/decimal_value.h /^static const int32_t NOT_FIXED_DEC = 31;$/;" m namespace:doris +NOT_FOUND src/http/http_status.h /^ NOT_FOUND = 404,$/;" e enum:doris::HttpStatus +NOT_IMPLEMENTED src/http/http_status.h /^ NOT_IMPLEMENTED = 501,$/;" e enum:doris::HttpStatus +NOT_MODIFIED src/http/http_status.h /^ NOT_MODIFIED = 304,$/;" e enum:doris::HttpStatus +NOUNCRYPT src/util/minizip/unzip.c 72;" d file: +NO_CONTENT src/http/http_status.h /^ NO_CONTENT = 204,$/;" e enum:doris::HttpStatus +NO_FLAGS src/util/thread.h /^ NO_FLAGS = 0,$/;" e enum:doris::Thread::CreateFlags +NO_FLUSH_RESOURCES src/runtime/row_batch.h /^ NO_FLUSH_RESOURCES,$/;" m class:doris::RowBatch::FlushMode +NO_STACK_WATCHDOG src/util/thread.h /^ NO_STACK_WATCHDOG = 1$/;" e enum:doris::Thread::CreateFlags +NO_SYNC src/olap/fs/file_block_manager.cpp /^ NO_SYNC$/;" e enum:doris::fs::internal::FileWritableBlock::SyncMode file: +NULL_IF_COMPUTE_FUNCTION src/exprs/conditional_functions_ir.cpp 45;" d file: +NULL_IF_COMPUTE_FUNCTION_WRAPPER src/exprs/conditional_functions_ir.cpp 60;" d file: +NULL_VALUE src/exec/new_partitioned_hash_table.cc /^static int64_t NULL_VALUE[] = {$/;" v file: +NULL_VALUE src/exec/partitioned_hash_table.cc /^static int64_t NULL_VALUE[] = { HashUtil::FNV_SEED, HashUtil::FNV_SEED,$/;" m namespace:doris file: +NUM_BATCHES test/runtime/data_stream_test.cpp /^ static const int NUM_BATCHES = TOTAL_DATA_SIZE \/ BATCH_CAPACITY \/ PER_ROW_DATA;$/;" m class:doris::DataStreamTest file: +NUM_CACHE_LEVELS src/util/cpu_info.h /^ static const int NUM_CACHE_LEVELS = L3_CACHE + 1;$/;" m class:doris::CpuInfo +NUM_FREE_LISTS src/runtime/bufferpool/suballocator.h /^ static constexpr int NUM_FREE_LISTS =$/;" m class:doris::Suballocator +NUM_LISTS src/runtime/free_pool.hpp /^ static const int NUM_LISTS = 64;$/;" m class:doris::FreePool +NUM_OFFSET_BITS src/util/bitmap.h /^ static const int64_t NUM_OFFSET_BITS = 6;$/;" m class:doris::Bitmap +NUM_PARTITIONING_BITS src/exec/new_partitioned_aggregation_node.h /^ static const int NUM_PARTITIONING_BITS = 4;$/;" m class:doris::NewPartitionedAggregationNode +NUM_PARTITIONING_BITS src/exec/partitioned_aggregation_node.h /^ static const int NUM_PARTITIONING_BITS = 4;$/;" m class:doris::PartitionedAggregationNode +NUM_PASSES src/util/radix_sort.h /^ static constexpr size_t NUM_PASSES = (KEY_BITS + (Traits::PART_SIZE_BITS - 1)) \/ Traits::PART_SIZE_BITS;$/;" m struct:doris::RadixSort +NUM_PC_BITMAPS src/exprs/aggregate_functions.cpp /^const static int NUM_PC_BITMAPS = 64; \/\/ number of bitmaps$/;" m namespace:doris file: +NUM_SMALL_BLOCKS src/runtime/buffered_tuple_stream.cpp /^static const int NUM_SMALL_BLOCKS = sizeof(INITIAL_BLOCK_SIZES) \/ sizeof(int64_t);$/;" m namespace:doris file: +NUM_SMALL_BLOCKS src/runtime/buffered_tuple_stream2.cc /^static const int NUM_SMALL_BLOCKS = sizeof(INITIAL_BLOCK_SIZES) \/ sizeof(int64_t);$/;" m namespace:doris file: +NUM_SMALL_DATA_PAGES src/exec/partitioned_hash_table.cc /^static const int NUM_SMALL_DATA_PAGES = sizeof(INITIAL_DATA_PAGE_SIZES) \/ sizeof(int64_t);$/;" m namespace:doris file: +NUM_SPIN_CYCLES src/util/spinlock.h /^ static const int NUM_SPIN_CYCLES = 70;$/;" m class:doris::SpinLock +NUM_STRINGS test/runtime/buffered_tuple_stream2_test.cpp /^static const int NUM_STRINGS = sizeof(STRINGS) \/ sizeof(StringValue);$/;" m namespace:doris file: +NUM_THREADS test/util/thread_pool_test.cpp /^const int NUM_THREADS = 5;$/;" m namespace:doris file: +NVALGRIND src/gutil/valgrind.h 116;" d +NanosToTimeSpec src/util/monotime.cpp /^void MonoDelta::NanosToTimeSpec(int64_t nanos, struct timespec* ts) {$/;" f class:doris::MonoDelta +Nanosecond src/util/time.h /^ Nanosecond$/;" e enum:doris::TimePrecision +NeedReadReservation src/runtime/buffered_tuple_stream3.cc /^bool BufferedTupleStream3::NeedReadReservation() const {$/;" f class:BufferedTupleStream3 +NeedReadReservation src/runtime/buffered_tuple_stream3.cc /^bool BufferedTupleStream3::NeedReadReservation(bool stream_pinned) const {$/;" f class:BufferedTupleStream3 +NeedReadReservation src/runtime/buffered_tuple_stream3.cc /^bool BufferedTupleStream3::NeedReadReservation(bool stream_pinned, int64_t num_pages,$/;" f class:BufferedTupleStream3 +NeedWriteReservation src/runtime/buffered_tuple_stream3.cc /^bool BufferedTupleStream3::NeedWriteReservation() const {$/;" f class:BufferedTupleStream3 +NeedWriteReservation src/runtime/buffered_tuple_stream3.cc /^bool BufferedTupleStream3::NeedWriteReservation(bool stream_pinned) const {$/;" f class:BufferedTupleStream3 +NeedWriteReservation src/runtime/buffered_tuple_stream3.cc /^bool BufferedTupleStream3::NeedWriteReservation(bool stream_pinned, int64_t num_pages,$/;" f class:BufferedTupleStream3 +NetMetrics src/util/system_metrics.cpp /^struct NetMetrics {$/;" s namespace:doris file: +NetworkByteOrder src/gutil/endian.h /^typedef BigEndian NetworkByteOrder;$/;" t +NewAggFnEvaluator src/exprs/new_agg_fn_evaluator.cc /^NewAggFnEvaluator::NewAggFnEvaluator(const AggFn& agg_fn, MemPool* mem_pool, MemTracker* tracker, bool is_clone)$/;" f class:NewAggFnEvaluator +NewAggFnEvaluator src/exprs/new_agg_fn_evaluator.h /^class NewAggFnEvaluator {$/;" c namespace:doris +NewNode src/olap/skiplist.h /^SkipList::NewNode(const Key& key, int height) {$/;" f class:doris::SkipList +NewPartitionedAggregationNode src/exec/new_partitioned_aggregation_node.cc /^NewPartitionedAggregationNode::NewPartitionedAggregationNode($/;" f class:doris::NewPartitionedAggregationNode +NewPartitionedAggregationNode src/exec/new_partitioned_aggregation_node.h /^class NewPartitionedAggregationNode : public ExecNode {$/;" c namespace:doris +NewPartitionedHashTable src/exec/new_partitioned_hash_table.cc /^NewPartitionedHashTable::NewPartitionedHashTable(bool quadratic_probing, Suballocator* allocator,$/;" f class:NewPartitionedHashTable +NewPartitionedHashTable src/exec/new_partitioned_hash_table.h /^class NewPartitionedHashTable {$/;" c namespace:doris +NewPartitionedHashTableCtx src/exec/new_partitioned_hash_table.cc /^NewPartitionedHashTableCtx::NewPartitionedHashTableCtx(const std::vector& build_exprs,$/;" f class:NewPartitionedHashTableCtx +NewPartitionedHashTableCtx src/exec/new_partitioned_hash_table.h /^class NewPartitionedHashTableCtx {$/;" c namespace:doris +Next src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTable::Iterator::Next() {$/;" f class:doris::NewPartitionedHashTable::Iterator +Next src/olap/skiplist.h /^ Node* Next(int n) {$/;" f struct:doris::SkipList::Node +Next src/olap/skiplist.h /^inline void SkipList::Iterator::Next() {$/;" f class:doris::SkipList::Iterator +Next src/util/bitmap.h /^ size_t Next(bool* value) {$/;" f class:doris::BitmapIterator +Next src/util/bitmap.h /^ size_t Next(bool* value, size_t max_run) {$/;" f class:doris::BitmapIterator +Next src/util/random.h /^ uint32_t Next() {$/;" f class:doris::Random +NextCharUTF8 src/gutil/strings/util.cc /^struct NextCharUTF8 {$/;" s file: +NextDuplicate src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTable::Iterator::NextDuplicate() {$/;" f class:doris::NewPartitionedHashTable::Iterator +NextFilledBucket src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTable::NextFilledBucket(int64_t* bucket_idx, DuplicateNode** node) {$/;" f class:doris::NewPartitionedHashTable +NextPartition src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::NextPartition() {$/;" f class:doris::NewPartitionedAggregationNode +NextReadPage src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::NextReadPage() {$/;" f class:BufferedTupleStream3 +NextRow src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTableCtx::ExprValuesCache::NextRow() {$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +NextUnmatched src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTable::Iterator::NextUnmatched() {$/;" f class:doris::NewPartitionedHashTable::Iterator +NextWithLimit src/util/bitmap.h /^ size_t NextWithLimit(bool* value, size_t limit) {$/;" f class:doris::BitmapIterator +NoArg src/gutil/strings/substitute.cc /^const SubstituteArg SubstituteArg::NoArg;$/;" m class:strings::SubstituteArg file: +NoArg src/gutil/strings/substitute.h /^ static const SubstituteArg NoArg;$/;" m class:strings::internal::SubstituteArg +NoBarrier_AtomicExchange src/gutil/atomicops-internals-gcc.h /^inline Atomic32 NoBarrier_AtomicExchange(volatile Atomic32* ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicExchange src/gutil/atomicops-internals-gcc.h /^inline Atomic64 NoBarrier_AtomicExchange(volatile Atomic64* ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicExchange src/gutil/atomicops-internals-tsan.h /^inline Atomic32 NoBarrier_AtomicExchange(volatile Atomic32 *ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicExchange src/gutil/atomicops-internals-tsan.h /^inline Atomic64 NoBarrier_AtomicExchange(volatile Atomic64 *ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicExchange src/gutil/atomicops-internals-x86.h /^inline Atomic32 NoBarrier_AtomicExchange(volatile Atomic32* ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicExchange src/gutil/atomicops-internals-x86.h /^inline Atomic64 NoBarrier_AtomicExchange(volatile Atomic64* ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicExchange src/gutil/atomicops.h /^inline AtomicWord NoBarrier_AtomicExchange(volatile AtomicWord* ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicIncrement src/gutil/atomicops-internals-gcc.h /^inline Atomic32 NoBarrier_AtomicIncrement(volatile Atomic32* ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicIncrement src/gutil/atomicops-internals-gcc.h /^inline Atomic64 NoBarrier_AtomicIncrement(volatile Atomic64* ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicIncrement src/gutil/atomicops-internals-tsan.h /^inline Atomic32 NoBarrier_AtomicIncrement(volatile Atomic32 *ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicIncrement src/gutil/atomicops-internals-tsan.h /^inline Atomic64 NoBarrier_AtomicIncrement(volatile Atomic64 *ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicIncrement src/gutil/atomicops-internals-x86.h /^inline Atomic32 NoBarrier_AtomicIncrement(volatile Atomic32* ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicIncrement src/gutil/atomicops-internals-x86.h /^inline Atomic64 NoBarrier_AtomicIncrement(volatile Atomic64* ptr,$/;" f namespace:base::subtle +NoBarrier_AtomicIncrement src/gutil/atomicops.h /^inline AtomicWord NoBarrier_AtomicIncrement(volatile AtomicWord* ptr,$/;" f namespace:base::subtle +NoBarrier_CompareAndSwap src/gutil/atomicops-internals-gcc.h /^inline Atomic32 NoBarrier_CompareAndSwap(volatile Atomic32* ptr,$/;" f namespace:base::subtle +NoBarrier_CompareAndSwap src/gutil/atomicops-internals-gcc.h /^inline Atomic64 NoBarrier_CompareAndSwap(volatile Atomic64* ptr,$/;" f namespace:base::subtle +NoBarrier_CompareAndSwap src/gutil/atomicops-internals-tsan.h /^inline Atomic32 NoBarrier_CompareAndSwap(volatile Atomic32 *ptr,$/;" f namespace:base::subtle +NoBarrier_CompareAndSwap src/gutil/atomicops-internals-tsan.h /^inline Atomic64 NoBarrier_CompareAndSwap(volatile Atomic64 *ptr,$/;" f namespace:base::subtle +NoBarrier_CompareAndSwap src/gutil/atomicops-internals-x86.h /^inline Atomic32 NoBarrier_CompareAndSwap(volatile Atomic32* ptr,$/;" f namespace:base::subtle +NoBarrier_CompareAndSwap src/gutil/atomicops-internals-x86.h /^inline Atomic64 NoBarrier_CompareAndSwap(volatile Atomic64* ptr,$/;" f namespace:base::subtle +NoBarrier_CompareAndSwap src/gutil/atomicops.h /^inline AtomicWord NoBarrier_CompareAndSwap(volatile AtomicWord* ptr,$/;" f namespace:base::subtle +NoBarrier_Load src/gutil/atomicops-internals-gcc.h /^inline Atomic32 NoBarrier_Load(volatile const Atomic32* ptr) {$/;" f namespace:base::subtle +NoBarrier_Load src/gutil/atomicops-internals-gcc.h /^inline Atomic64 NoBarrier_Load(volatile const Atomic64* ptr) {$/;" f namespace:base::subtle +NoBarrier_Load src/gutil/atomicops-internals-tsan.h /^inline Atomic32 NoBarrier_Load(volatile const Atomic32 *ptr) {$/;" f namespace:base::subtle +NoBarrier_Load src/gutil/atomicops-internals-tsan.h /^inline Atomic64 NoBarrier_Load(volatile const Atomic64 *ptr) {$/;" f namespace:base::subtle +NoBarrier_Load src/gutil/atomicops-internals-x86.h /^inline Atomic32 NoBarrier_Load(volatile const Atomic32* ptr) {$/;" f namespace:base::subtle +NoBarrier_Load src/gutil/atomicops-internals-x86.h /^inline Atomic64 NoBarrier_Load(volatile const Atomic64* ptr) {$/;" f namespace:base::subtle +NoBarrier_Load src/gutil/atomicops.h /^inline AtomicWord NoBarrier_Load(volatile const AtomicWord *ptr) {$/;" f namespace:base::subtle +NoBarrier_Next src/olap/skiplist.h /^ Node* NoBarrier_Next(int n) {$/;" f struct:doris::SkipList::Node +NoBarrier_SetNext src/olap/skiplist.h /^ void NoBarrier_SetNext(int n, Node* x) {$/;" f struct:doris::SkipList::Node +NoBarrier_Store src/gutil/atomicops-internals-gcc.h /^inline void NoBarrier_Store(volatile Atomic32* ptr, Atomic32 value) {$/;" f namespace:base::subtle +NoBarrier_Store src/gutil/atomicops-internals-gcc.h /^inline void NoBarrier_Store(volatile Atomic64* ptr, Atomic64 value) {$/;" f namespace:base::subtle +NoBarrier_Store src/gutil/atomicops-internals-tsan.h /^inline void NoBarrier_Store(volatile Atomic32 *ptr, Atomic32 value) {$/;" f namespace:base::subtle +NoBarrier_Store src/gutil/atomicops-internals-tsan.h /^inline void NoBarrier_Store(volatile Atomic64 *ptr, Atomic64 value) {$/;" f namespace:base::subtle +NoBarrier_Store src/gutil/atomicops-internals-x86.h /^inline void NoBarrier_Store(volatile Atomic32* ptr, Atomic32 value) {$/;" f namespace:base::subtle +NoBarrier_Store src/gutil/atomicops-internals-x86.h /^inline void NoBarrier_Store(volatile Atomic64* ptr, Atomic64 value) {$/;" f namespace:base::subtle +NoBarrier_Store src/gutil/atomicops.h /^inline void NoBarrier_Store(volatile AtomicWord *ptr, AtomicWord value) {$/;" f namespace:base::subtle +NoFilter src/gutil/strings/split_internal.h /^struct NoFilter {$/;" s namespace:strings::internal +NoType src/gutil/template_util.h /^typedef big_ NoType;$/;" t namespace:base +Node src/exec/hash_table.h /^ Node():_next_idx(-1),$/;" f struct:doris::HashTable::Node +Node src/exec/hash_table.h /^ struct Node {$/;" s class:doris::HashTable +Node src/olap/skiplist.h /^ explicit Node(const Key& k) : key(k) { }$/;" f struct:doris::SkipList::Node +Node src/olap/skiplist.h /^struct SkipList::Node {$/;" s class:doris::SkipList +Node src/util/internal_queue.h /^ Node() : parent_queue(NULL), next_node(NULL), prev_node(NULL) {}$/;" f struct:doris::InternalQueueBase::Node +Node src/util/internal_queue.h /^ struct Node {$/;" s class:doris::InternalQueueBase +NodeChannel src/exec/tablet_sink.cpp /^NodeChannel::NodeChannel(OlapTableSink* parent, int64_t index_id,$/;" f class:doris::stream_load::NodeChannel +NodeChannel src/exec/tablet_sink.h /^class NodeChannel {$/;" c namespace:doris::stream_load +NodeInfo src/exec/tablet_info.h /^ NodeInfo(const TNodeInfo& tnode)$/;" f struct:doris::NodeInfo +NodeInfo src/exec/tablet_info.h /^struct NodeInfo {$/;" s namespace:doris +NotFound src/common/status.h /^ static Status NotFound(const Slice& msg,$/;" f class:doris::Status +NotPredicate src/exprs/compound_predicate.h /^ NotPredicate(const TExprNode& node) : CompoundPredicate(node) { }$/;" f class:doris::NotPredicate +NotPredicate src/exprs/compound_predicate.h /^class NotPredicate: public CompoundPredicate {$/;" c namespace:doris +NotSupported src/common/status.h /^ static Status NotSupported(const Slice& msg,$/;" f class:doris::Status +NotifiedCounter test/runtime/thread_resource_mgr_test.cpp /^ NotifiedCounter() : _counter(0) {$/;" f class:doris::NotifiedCounter +NotifiedCounter test/runtime/thread_resource_mgr_test.cpp /^class NotifiedCounter {$/;" c namespace:doris file: +Notify test/runtime/thread_resource_mgr_test.cpp /^ void Notify(ThreadResourceMgr::ResourcePool* consumer) {$/;" f class:doris::NotifiedCounter +Now src/gutil/cycleclock-inl.h /^inline int64 CycleClock::Now() {$/;" f class:CycleClock +Now src/util/monotime.cpp /^MonoTime MonoTime::Now() {$/;" f class:doris::MonoTime +NullBitmapBuilder src/olap/rowset/segment_v2/column_writer.cpp /^ NullBitmapBuilder() : _has_null(false), _bitmap_buf(512), _rle_encoder(&_bitmap_buf, 1) {$/;" f class:doris::segment_v2::NullBitmapBuilder +NullBitmapBuilder src/olap/rowset/segment_v2/column_writer.cpp /^ explicit NullBitmapBuilder(size_t reserve_bits)$/;" f class:doris::segment_v2::NullBitmapBuilder +NullBitmapBuilder src/olap/rowset/segment_v2/column_writer.cpp /^class NullBitmapBuilder {$/;" c namespace:doris::segment_v2 file: +NullIfExpr src/exprs/conditional_functions.h /^class NullIfExpr : public Expr {$/;" c namespace:doris +NullIndicatorBytesPerRow src/runtime/buffered_tuple_stream3.inline.h /^inline int BufferedTupleStream3::NullIndicatorBytesPerRow() const {$/;" f class:doris::BufferedTupleStream3 +NullIndicatorOffset src/runtime/descriptors.h /^ NullIndicatorOffset(int byte_offset, int bit_offset_)$/;" f struct:doris::NullIndicatorOffset +NullIndicatorOffset src/runtime/descriptors.h /^struct NullIndicatorOffset {$/;" s namespace:doris +NullLiteral src/exprs/null_literal.cpp /^NullLiteral::NullLiteral(const TExprNode& node) : $/;" f class:doris::NullLiteral +NullLiteral src/exprs/null_literal.h /^class NullLiteral : public Expr {$/;" c namespace:doris +NullLoadErrorHub src/util/null_load_error_hub.cpp /^NullLoadErrorHub::NullLoadErrorHub() {$/;" f class:doris::NullLoadErrorHub +NullLoadErrorHub src/util/null_load_error_hub.h /^class NullLoadErrorHub : public LoadErrorHub {$/;" c namespace:doris +NullPredicate src/olap/null_predicate.cpp /^NullPredicate::NullPredicate(uint32_t column_id, bool is_null)$/;" f class:doris::NullPredicate +NullPredicate src/olap/null_predicate.h /^class NullPredicate : public ColumnPredicate {$/;" c namespace:doris +NullValueReader src/olap/rowset/column_reader.h /^ NullValueReader(uint32_t column_id, uint32_t column_unique_id) :$/;" f class:doris::NullValueReader +NullValueReader src/olap/rowset/column_reader.h /^class NullValueReader : public ColumnReader {$/;" c namespace:doris +NumBufferSizes src/runtime/bufferpool/buffer_allocator.cc /^ int NumBufferSizes() const {$/;" f class:doris::BufferPool::FreeBufferArena file: +NumCPUs src/gutil/sysinfo.cc /^int NumCPUs(void) {$/;" f namespace:base +NumHashCollisions src/exec/new_partitioned_hash_table.h /^ int64_t NumHashCollisions() const { return num_hash_collisions_; }$/;" f class:doris::NewPartitionedHashTable +NumInsertsBeforeResize src/exec/new_partitioned_hash_table.inline.h /^inline int64_t NewPartitionedHashTable::NumInsertsBeforeResize() const {$/;" f class:doris::NewPartitionedHashTable +NumericHashHelper src/exprs/aggregate_functions.cpp /^ class NumericHashHelper {$/;" c class:doris::MultiDistinctNumericState file: +OF src/util/minizip/ioapi.h /^typedef ZPOS64_T (ZCALLBACK *tell64_file_func) OF((voidpf opaque, voidpf stream));$/;" t +OF src/util/minizip/ioapi.h /^typedef uLong (ZCALLBACK *read_file_func) OF((voidpf opaque, voidpf stream, void* buf, uLong size));$/;" t +OF src/util/minizip/ioapi.h /^typedef uLong (ZCALLBACK *write_file_func) OF((voidpf opaque, voidpf stream, const void* buf, uLong size));$/;" t +OF src/util/minizip/ioapi.h /^typedef voidpf (ZCALLBACK *open64_file_func) OF((voidpf opaque, const void* filename, int mode));$/;" t +OF src/util/minizip/ioapi.h /^typedef voidpf (ZCALLBACK *open_file_func) OF((voidpf opaque, const char* filename, int mode));$/;" t +OFFSETOF_MEMBER src/gutil/port.h 1182;" d +OFFSET_MASK src/runtime/buffered_tuple_stream.h /^ static const uint64_t OFFSET_MASK = 0xFFFFFF0000;$/;" m struct:doris::BufferedTupleStream::RowIdx +OFFSET_MASK src/runtime/buffered_tuple_stream2.h /^ static const uint64_t OFFSET_MASK = 0xFFFFFF0000;$/;" m struct:doris::BufferedTupleStream2::RowIdx +OFFSET_SHIFT src/runtime/buffered_tuple_stream.h /^ static const uint64_t OFFSET_SHIFT = 16;$/;" m struct:doris::BufferedTupleStream::RowIdx +OFFSET_SHIFT src/runtime/buffered_tuple_stream2.h /^ static const uint64_t OFFSET_SHIFT = 16;$/;" m struct:doris::BufferedTupleStream2::RowIdx +OK src/common/status.h /^ static Status OK() { return Status(); }$/;" f class:doris::Status +OK src/http/http_status.h /^ OK = 200,$/;" e enum:doris::HttpStatus +OLAPCompressionType src/olap/olap_common.h /^enum OLAPCompressionType {$/;" g namespace:doris +OLAPDataVersion src/olap/olap_define.h /^enum OLAPDataVersion {$/;" g namespace:doris +OLAPIndexFixedHeader src/olap/olap_index.h /^ OLAPIndexFixedHeader() : data_length(0), num_rows(0) {}$/;" f struct:doris::OLAPIndexFixedHeader +OLAPIndexFixedHeader src/olap/olap_index.h /^struct OLAPIndexFixedHeader {$/;" s namespace:doris +OLAPIndexOffset src/olap/olap_index.h /^ OLAPIndexOffset() : segment(0), offset(0) {}$/;" f struct:doris::OLAPIndexOffset +OLAPIndexOffset src/olap/olap_index.h /^ OLAPIndexOffset(const OLAPIndexOffset& off) : segment(off.segment), offset(off.offset) {}$/;" f struct:doris::OLAPIndexOffset +OLAPIndexOffset src/olap/olap_index.h /^ OLAPIndexOffset(const iterator_offset_t& seg, const iterator_offset_t& off) :$/;" f struct:doris::OLAPIndexOffset +OLAPIndexOffset src/olap/olap_index.h /^struct OLAPIndexOffset {$/;" s namespace:doris +OLAPStatus src/olap/olap_define.h /^enum OLAPStatus {$/;" g namespace:doris +OLAP_CACHE_NUMERIC_TO_BUF src/olap/lru_cache.h 34;" d +OLAP_CACHE_STRING_TO_BUF src/olap/lru_cache.h 22;" d +OLAP_COLUMN_FILE_SEGMENT_SIZE_SCALE src/olap/olap_define.h /^static const double OLAP_COLUMN_FILE_SEGMENT_SIZE_SCALE = 0.9;$/;" m namespace:doris +OLAP_COMMAND_SIGNATURE_LEN src/olap/olap_define.h /^static const uint32_t OLAP_COMMAND_SIGNATURE_LEN = 4;$/;" m namespace:doris +OLAP_COMPACTION_DEFAULT_CANDIDATE_SIZE src/olap/olap_define.h /^static constexpr uint32_t OLAP_COMPACTION_DEFAULT_CANDIDATE_SIZE = 10;$/;" m namespace:doris +OLAP_COMP_LZ4 src/olap/olap_common.h /^ OLAP_COMP_LZ4 = 3, \/\/ 用于储存的压缩算法,压缩率低,cpu开销低$/;" e enum:doris::OLAPCompressionType +OLAP_COMP_STORAGE src/olap/olap_common.h /^ OLAP_COMP_STORAGE = 2, \/\/ 用于硬盘数据的压缩算法,压缩率高,cpu开销大$/;" e enum:doris::OLAPCompressionType +OLAP_COMP_TRANSPORT src/olap/olap_common.h /^ OLAP_COMP_TRANSPORT = 1, \/\/ 用于网络传输的压缩算法,压缩率低,cpu开销低$/;" e enum:doris::OLAPCompressionType +OLAP_DATA_VERSION_APPLIED src/olap/olap_define.h /^static const int32_t OLAP_DATA_VERSION_APPLIED = DORIS_V1;$/;" m namespace:doris +OLAP_DEFAULT_COLUMN_DICT_KEY_SIZE_THRESHOLD src/olap/olap_define.h /^static const uint32_t OLAP_DEFAULT_COLUMN_DICT_KEY_SIZE_THRESHOLD = 80; \/\/ 30%$/;" m namespace:doris +OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE src/olap/olap_define.h /^static const uint32_t OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE = 10 * 1024;$/;" m namespace:doris +OLAP_DEFAULT_MAX_PACKED_ROW_BLOCK_SIZE src/olap/olap_define.h /^static const uint32_t OLAP_DEFAULT_MAX_PACKED_ROW_BLOCK_SIZE = 1024 * 1024 * 20;$/;" m namespace:doris +OLAP_DEFAULT_MAX_UNPACKED_ROW_BLOCK_SIZE src/olap/olap_define.h /^static const uint32_t OLAP_DEFAULT_MAX_UNPACKED_ROW_BLOCK_SIZE = 1024 * 1024 * 100;$/;" m namespace:doris +OLAP_ERR_ALTER_DELTA_DOES_NOT_EXISTS src/olap/olap_define.h /^ OLAP_ERR_ALTER_DELTA_DOES_NOT_EXISTS = -1601,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ALTER_MULTI_TABLE_ERR src/olap/olap_define.h /^ OLAP_ERR_ALTER_MULTI_TABLE_ERR = -1600,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ALTER_STATUS_ERR src/olap/olap_define.h /^ OLAP_ERR_ALTER_STATUS_ERR = -1602,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_COMPUTE_VERSION_HASH_ERROR src/olap/olap_define.h /^ OLAP_ERR_BE_COMPUTE_VERSION_HASH_ERROR = -803,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_ERROR_DELETE_ACTION src/olap/olap_define.h /^ OLAP_ERR_BE_ERROR_DELETE_ACTION = -811,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_INIT_OLAP_DATA src/olap/olap_define.h /^ OLAP_ERR_BE_INIT_OLAP_DATA = -806,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_INVALID_NEED_MERGED_VERSIONS src/olap/olap_define.h /^ OLAP_ERR_BE_INVALID_NEED_MERGED_VERSIONS = -810,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_MERGE_ERROR src/olap/olap_define.h /^ OLAP_ERR_BE_MERGE_ERROR = -802,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_NO_SUITABLE_VERSION src/olap/olap_define.h /^ OLAP_ERR_BE_NO_SUITABLE_VERSION = -808,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_REPLACE_VERSIONS_ERROR src/olap/olap_define.h /^ OLAP_ERR_BE_REPLACE_VERSIONS_ERROR = -801,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_SAVE_HEADER_ERROR src/olap/olap_define.h /^ OLAP_ERR_BE_SAVE_HEADER_ERROR = -805,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_SEGMENTS_OVERLAPPING src/olap/olap_define.h /^ OLAP_ERR_BE_SEGMENTS_OVERLAPPING = -812,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_TRY_BE_LOCK_ERROR src/olap/olap_define.h /^ OLAP_ERR_BE_TRY_BE_LOCK_ERROR = -809,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_TRY_OBTAIN_VERSION_LOCKS src/olap/olap_define.h /^ OLAP_ERR_BE_TRY_OBTAIN_VERSION_LOCKS = -807,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BE_VERSION_NOT_MATCH src/olap/olap_define.h /^ OLAP_ERR_BE_VERSION_NOT_MATCH = -800,$/;" e enum:doris::OLAPStatus +OLAP_ERR_BUFFER_OVERFLOW src/olap/olap_define.h /^ OLAP_ERR_BUFFER_OVERFLOW = -204,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CALL_SEQUENCE_ERROR src/olap/olap_define.h /^ OLAP_ERR_CALL_SEQUENCE_ERROR = -202,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CANNOT_CREATE_DIR src/olap/olap_define.h /^ OLAP_ERR_CANNOT_CREATE_DIR = -117,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CAPTURE_ROWSET_ERROR src/olap/olap_define.h /^ OLAP_ERR_CAPTURE_ROWSET_ERROR = -804,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CAPTURE_ROWSET_READER_ERROR src/olap/olap_define.h /^ OLAP_ERR_CAPTURE_ROWSET_READER_ERROR = -702,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CATCH_EXCEPTION src/olap/olap_define.h /^ OLAP_ERR_CATCH_EXCEPTION = -210,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CE_BUFFER_TOO_SMALL src/olap/olap_define.h /^ OLAP_ERR_CE_BUFFER_TOO_SMALL = -301,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CE_CMD_NOT_VALID src/olap/olap_define.h /^ OLAP_ERR_CE_CMD_NOT_VALID = -302,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CE_CMD_PARAMS_ERROR src/olap/olap_define.h /^ OLAP_ERR_CE_CMD_PARAMS_ERROR = -300,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CE_LOAD_TABLE_ERROR src/olap/olap_define.h /^ OLAP_ERR_CE_LOAD_TABLE_ERROR = -303,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CE_NOT_FINISHED src/olap/olap_define.h /^ OLAP_ERR_CE_NOT_FINISHED = -304,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CE_TABLET_ID_EXIST src/olap/olap_define.h /^ OLAP_ERR_CE_TABLET_ID_EXIST = -305,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CE_TRY_CE_LOCK_ERROR src/olap/olap_define.h /^ OLAP_ERR_CE_TRY_CE_LOCK_ERROR = -306,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CHECKSUM_ERROR src/olap/olap_define.h /^ OLAP_ERR_CHECKSUM_ERROR = -208,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CHECK_LINES_ERROR src/olap/olap_define.h /^ OLAP_ERR_CHECK_LINES_ERROR = -224,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COLUMN_DATA_LOAD_BLOCK src/olap/olap_define.h /^ OLAP_ERR_COLUMN_DATA_LOAD_BLOCK = -1700,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COLUMN_DATA_MAKE_FILE_HEADER src/olap/olap_define.h /^ OLAP_ERR_COLUMN_DATA_MAKE_FILE_HEADER = -1702,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COLUMN_DATA_PATCH_LIST_NUM src/olap/olap_define.h /^ OLAP_ERR_COLUMN_DATA_PATCH_LIST_NUM = -1704,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COLUMN_DATA_READ_VAR_INT src/olap/olap_define.h /^ OLAP_ERR_COLUMN_DATA_READ_VAR_INT = -1703,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COLUMN_DATA_RECORD_INDEX src/olap/olap_define.h /^ OLAP_ERR_COLUMN_DATA_RECORD_INDEX = -1701,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COLUMN_READ_STREAM src/olap/olap_define.h /^ OLAP_ERR_COLUMN_READ_STREAM = -1706,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COLUMN_SEEK_ERROR src/olap/olap_define.h /^ OLAP_ERR_COLUMN_SEEK_ERROR = -1719,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COLUMN_STREAM_EOF src/olap/olap_define.h /^ OLAP_ERR_COLUMN_STREAM_EOF = -1705,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COLUMN_STREAM_NOT_EXIST src/olap/olap_define.h /^ OLAP_ERR_COLUMN_STREAM_NOT_EXIST = -1716,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COLUMN_VALUE_NULL src/olap/olap_define.h /^ OLAP_ERR_COLUMN_VALUE_NULL = -1717,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COMPRESS_ERROR src/olap/olap_define.h /^ OLAP_ERR_COMPRESS_ERROR = -111,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CONFIG_ERROR src/olap/olap_define.h /^ OLAP_ERR_CONFIG_ERROR = -205,$/;" e enum:doris::OLAPStatus +OLAP_ERR_COPY_FILE_ERROR src/olap/olap_define.h /^ OLAP_ERR_COPY_FILE_ERROR = -121,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CREATE_FILE_ERROR src/olap/olap_define.h /^ OLAP_ERR_CREATE_FILE_ERROR = -103,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CUMULATIVE_ERROR_DELETE_ACTION src/olap/olap_define.h /^ OLAP_ERR_CUMULATIVE_ERROR_DELETE_ACTION = -2005,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CUMULATIVE_FAILED_ACQUIRE_DATA_SOURCE src/olap/olap_define.h /^ OLAP_ERR_CUMULATIVE_FAILED_ACQUIRE_DATA_SOURCE = -2003,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CUMULATIVE_INVALID_NEED_MERGED_VERSIONS src/olap/olap_define.h /^ OLAP_ERR_CUMULATIVE_INVALID_NEED_MERGED_VERSIONS = -2004,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CUMULATIVE_INVALID_PARAMETERS src/olap/olap_define.h /^ OLAP_ERR_CUMULATIVE_INVALID_PARAMETERS = -2002,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CUMULATIVE_MISS_VERSION src/olap/olap_define.h /^ OLAP_ERR_CUMULATIVE_MISS_VERSION = -2006,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSIONS src/olap/olap_define.h /^ OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSIONS = -2000,$/;" e enum:doris::OLAPStatus +OLAP_ERR_CUMULATIVE_REPEAT_INIT src/olap/olap_define.h /^ OLAP_ERR_CUMULATIVE_REPEAT_INIT = -2001,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DATA_EOF src/olap/olap_define.h /^ OLAP_ERR_DATA_EOF = -1102,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DATA_FILE_TYPE_ERROR src/olap/olap_define.h /^ OLAP_ERR_DATA_FILE_TYPE_ERROR = -1101,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DATA_ROW_BLOCK_ERROR src/olap/olap_define.h /^ OLAP_ERR_DATA_ROW_BLOCK_ERROR = -1100,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DECOMPRESS_ERROR src/olap/olap_define.h /^ OLAP_ERR_DECOMPRESS_ERROR = -112,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DELETE_INVALID_CONDITION src/olap/olap_define.h /^ OLAP_ERR_DELETE_INVALID_CONDITION = -1900,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DELETE_INVALID_PARAMETERS src/olap/olap_define.h /^ OLAP_ERR_DELETE_INVALID_PARAMETERS = -1903,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DELETE_INVALID_VERSION src/olap/olap_define.h /^ OLAP_ERR_DELETE_INVALID_VERSION = -1904,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DELETE_SAVE_HEADER_FAILED src/olap/olap_define.h /^ OLAP_ERR_DELETE_SAVE_HEADER_FAILED = -1902,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DELETE_UPDATE_HEADER_FAILED src/olap/olap_define.h /^ OLAP_ERR_DELETE_UPDATE_HEADER_FAILED = -1901,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DELETE_VERSION_ERROR src/olap/olap_define.h /^ OLAP_ERR_DELETE_VERSION_ERROR = -504,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DIR_NOT_EXIST src/olap/olap_define.h /^ OLAP_ERR_DIR_NOT_EXIST = -101,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DISK_FAILURE src/olap/olap_define.h /^ OLAP_ERR_DISK_FAILURE = -227,$/;" e enum:doris::OLAPStatus +OLAP_ERR_DISK_REACH_CAPACITY_LIMIT src/olap/olap_define.h /^ OLAP_ERR_DISK_REACH_CAPACITY_LIMIT = -232,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ENGINE_DROP_NOEXISTS_TABLE src/olap/olap_define.h /^ OLAP_ERR_ENGINE_DROP_NOEXISTS_TABLE = -501,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ENGINE_INSERT_EXISTS_TABLE src/olap/olap_define.h /^ OLAP_ERR_ENGINE_INSERT_EXISTS_TABLE = -500,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ENGINE_LOAD_INDEX_TABLE_ERROR src/olap/olap_define.h /^ OLAP_ERR_ENGINE_LOAD_INDEX_TABLE_ERROR = -502,$/;" e enum:doris::OLAPStatus +OLAP_ERR_EVAL_CONJUNCTS_ERROR src/olap/olap_define.h /^ OLAP_ERR_EVAL_CONJUNCTS_ERROR = -120,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FETCH_COMPRESSION_ERROR src/olap/olap_define.h /^ OLAP_ERR_FETCH_COMPRESSION_ERROR = -604,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FETCH_CONTEXT_NOT_EXIST src/olap/olap_define.h /^ OLAP_ERR_FETCH_CONTEXT_NOT_EXIST = -605,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FETCH_GET_READER_PARAMS_ERR src/olap/olap_define.h /^ OLAP_ERR_FETCH_GET_READER_PARAMS_ERR = -606,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FETCH_MEMORY_EXCEEDED src/olap/olap_define.h /^ OLAP_ERR_FETCH_MEMORY_EXCEEDED = -608,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FETCH_OTHER_ERROR src/olap/olap_define.h /^ OLAP_ERR_FETCH_OTHER_ERROR = -600,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FETCH_SAVE_SESSION_ERR src/olap/olap_define.h /^ OLAP_ERR_FETCH_SAVE_SESSION_ERR = -607,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FETCH_SCHEMA_ERROR src/olap/olap_define.h /^ OLAP_ERR_FETCH_SCHEMA_ERROR = -603,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FETCH_TABLE_NOT_EXIST src/olap/olap_define.h /^ OLAP_ERR_FETCH_TABLE_NOT_EXIST = -601,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FETCH_VERSION_ERROR src/olap/olap_define.h /^ OLAP_ERR_FETCH_VERSION_ERROR = -602,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FILE_ALREADY_EXIST src/olap/olap_define.h /^ OLAP_ERR_FILE_ALREADY_EXIST = -122,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FILE_DATA_ERROR src/olap/olap_define.h /^ OLAP_ERR_FILE_DATA_ERROR = -220,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FILE_FORMAT_ERROR src/olap/olap_define.h /^ OLAP_ERR_FILE_FORMAT_ERROR = -119,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FILE_NOT_EXIST src/olap/olap_define.h /^ OLAP_ERR_FILE_NOT_EXIST = -102,$/;" e enum:doris::OLAPStatus +OLAP_ERR_FUNC_NOT_IMPLEMENTED src/olap/olap_define.h /^ OLAP_ERR_FUNC_NOT_IMPLEMENTED = -201,$/;" e enum:doris::OLAPStatus +OLAP_ERR_GC_SCAN_PATH_ERROR src/olap/olap_define.h /^ OLAP_ERR_GC_SCAN_PATH_ERROR = -505,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_ADD_INCREMENTAL_VERSION src/olap/olap_define.h /^ OLAP_ERR_HEADER_ADD_INCREMENTAL_VERSION = -1403,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_ADD_PENDING_DELTA src/olap/olap_define.h /^ OLAP_ERR_HEADER_ADD_PENDING_DELTA = -1402,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_ADD_VERSION src/olap/olap_define.h /^ OLAP_ERR_HEADER_ADD_VERSION = -1400,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_DELETE src/olap/olap_define.h /^ OLAP_ERR_HEADER_DELETE = -1406,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_DELETE_VERSION src/olap/olap_define.h /^ OLAP_ERR_HEADER_DELETE_VERSION = -1401,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_FLAG_PUT src/olap/olap_define.h /^ OLAP_ERR_HEADER_FLAG_PUT = -1409,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_GET src/olap/olap_define.h /^ OLAP_ERR_HEADER_GET = -1407,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_HAS_PENDING_DATA src/olap/olap_define.h /^ OLAP_ERR_HEADER_HAS_PENDING_DATA = -1413,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_INIT_FAILED src/olap/olap_define.h /^ OLAP_ERR_HEADER_INIT_FAILED = -1411,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_INVALID_FLAG src/olap/olap_define.h /^ OLAP_ERR_HEADER_INVALID_FLAG = -1404,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_LOAD_INVALID_KEY src/olap/olap_define.h /^ OLAP_ERR_HEADER_LOAD_INVALID_KEY = -1408,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_LOAD_JSON_HEADER src/olap/olap_define.h /^ OLAP_ERR_HEADER_LOAD_JSON_HEADER = -1410,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_PB_PARSE_FAILED src/olap/olap_define.h /^ OLAP_ERR_HEADER_PB_PARSE_FAILED = -1412,$/;" e enum:doris::OLAPStatus +OLAP_ERR_HEADER_PUT src/olap/olap_define.h /^ OLAP_ERR_HEADER_PUT = -1405,$/;" e enum:doris::OLAPStatus +OLAP_ERR_INDEX_CHECKSUM_ERROR src/olap/olap_define.h /^ OLAP_ERR_INDEX_CHECKSUM_ERROR = -1002,$/;" e enum:doris::OLAPStatus +OLAP_ERR_INDEX_DELTA_PRUNING src/olap/olap_define.h /^ OLAP_ERR_INDEX_DELTA_PRUNING = -1003,$/;" e enum:doris::OLAPStatus +OLAP_ERR_INDEX_EOF src/olap/olap_define.h /^ OLAP_ERR_INDEX_EOF = -1001,$/;" e enum:doris::OLAPStatus +OLAP_ERR_INDEX_LOAD_ERROR src/olap/olap_define.h /^ OLAP_ERR_INDEX_LOAD_ERROR = -1000,$/;" e enum:doris::OLAPStatus +OLAP_ERR_INIT_FAILED src/olap/olap_define.h /^ OLAP_ERR_INIT_FAILED = -206,$/;" e enum:doris::OLAPStatus +OLAP_ERR_INPUT_PARAMETER_ERROR src/olap/olap_define.h /^ OLAP_ERR_INPUT_PARAMETER_ERROR = -203,$/;" e enum:doris::OLAPStatus +OLAP_ERR_INVALID_CLUSTER_INFO src/olap/olap_define.h /^ OLAP_ERR_INVALID_CLUSTER_INFO = -225,$/;" e enum:doris::OLAPStatus +OLAP_ERR_INVALID_ROOT_PATH src/olap/olap_define.h /^ OLAP_ERR_INVALID_ROOT_PATH = -222,$/;" e enum:doris::OLAPStatus +OLAP_ERR_INVALID_SCHEMA src/olap/olap_define.h /^ OLAP_ERR_INVALID_SCHEMA = -207,$/;" e enum:doris::OLAPStatus +OLAP_ERR_INVALID_SNAPSHOT_VERSION src/olap/olap_define.h /^ OLAP_ERR_INVALID_SNAPSHOT_VERSION = -234,$/;" e enum:doris::OLAPStatus +OLAP_ERR_IO_ERROR src/olap/olap_define.h /^ OLAP_ERR_IO_ERROR = -106,$/;" e enum:doris::OLAPStatus +OLAP_ERR_LZO_DISABLED src/olap/olap_define.h /^ OLAP_ERR_LZO_DISABLED = -231,$/;" e enum:doris::OLAPStatus +OLAP_ERR_MALLOC_ERROR src/olap/olap_define.h /^ OLAP_ERR_MALLOC_ERROR = -104,$/;" e enum:doris::OLAPStatus +OLAP_ERR_META_ALREADY_EXIST src/olap/olap_define.h /^ OLAP_ERR_META_ALREADY_EXIST = -3007,$/;" e enum:doris::OLAPStatus +OLAP_ERR_META_DELETE src/olap/olap_define.h /^ OLAP_ERR_META_DELETE = -3006,$/;" e enum:doris::OLAPStatus +OLAP_ERR_META_GET src/olap/olap_define.h /^ OLAP_ERR_META_GET = -3003,$/;" e enum:doris::OLAPStatus +OLAP_ERR_META_INVALID_ARGUMENT src/olap/olap_define.h /^ OLAP_ERR_META_INVALID_ARGUMENT = -3000,$/;" e enum:doris::OLAPStatus +OLAP_ERR_META_ITERATOR src/olap/olap_define.h /^ OLAP_ERR_META_ITERATOR = -3005,$/;" e enum:doris::OLAPStatus +OLAP_ERR_META_KEY_NOT_FOUND src/olap/olap_define.h /^ OLAP_ERR_META_KEY_NOT_FOUND = -3002,$/;" e enum:doris::OLAPStatus +OLAP_ERR_META_OPEN_DB src/olap/olap_define.h /^ OLAP_ERR_META_OPEN_DB = -3001,$/;" e enum:doris::OLAPStatus +OLAP_ERR_META_PUT src/olap/olap_define.h /^ OLAP_ERR_META_PUT = -3004,$/;" e enum:doris::OLAPStatus +OLAP_ERR_MMAP_ERROR src/olap/olap_define.h /^ OLAP_ERR_MMAP_ERROR = -114,$/;" e enum:doris::OLAPStatus +OLAP_ERR_MUTEX_ERROR src/olap/olap_define.h /^ OLAP_ERR_MUTEX_ERROR = -107,$/;" e enum:doris::OLAPStatus +OLAP_ERR_NETWORK_ERROR src/olap/olap_define.h /^ OLAP_ERR_NETWORK_ERROR = -109,$/;" e enum:doris::OLAPStatus +OLAP_ERR_NOT_INITED src/olap/olap_define.h /^ OLAP_ERR_NOT_INITED = -200,$/;" e enum:doris::OLAPStatus +OLAP_ERR_NO_AVAILABLE_ROOT_PATH src/olap/olap_define.h /^ OLAP_ERR_NO_AVAILABLE_ROOT_PATH = -223,$/;" e enum:doris::OLAPStatus +OLAP_ERR_OS_ERROR src/olap/olap_define.h /^ OLAP_ERR_OS_ERROR = -100,$/;" e enum:doris::OLAPStatus +OLAP_ERR_OTHER_ERROR src/olap/olap_define.h /^ OLAP_ERR_OTHER_ERROR = -1,$/;" e enum:doris::OLAPStatus +OLAP_ERR_OUT_OF_BOUND src/olap/olap_define.h /^ OLAP_ERR_OUT_OF_BOUND = -218,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PARSE_PROTOBUF_ERROR src/olap/olap_define.h /^ OLAP_ERR_PARSE_PROTOBUF_ERROR = -211,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PREVIOUS_SCHEMA_CHANGE_NOT_FINISHED src/olap/olap_define.h /^ OLAP_ERR_PREVIOUS_SCHEMA_CHANGE_NOT_FINISHED = -1603,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PTHREAD_ERROR src/olap/olap_define.h /^ OLAP_ERR_PTHREAD_ERROR = -108,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_ACQUIRE_DATASOURCE_ERROR src/olap/olap_define.h /^ OLAP_ERR_PUSH_ACQUIRE_DATASOURCE_ERROR = -905,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_BATCH_PROCESS_REMOVED src/olap/olap_define.h /^ OLAP_ERR_PUSH_BATCH_PROCESS_REMOVED = -912,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_BUILD_DELTA_ERROR src/olap/olap_define.h /^ OLAP_ERR_PUSH_BUILD_DELTA_ERROR = -907,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_CHECKSUM_ERROR src/olap/olap_define.h /^ OLAP_ERR_PUSH_CHECKSUM_ERROR = -904,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_COMMIT_ROWSET src/olap/olap_define.h /^ OLAP_ERR_PUSH_COMMIT_ROWSET = -913,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_CREAT_CUMULATIVE_ERROR src/olap/olap_define.h /^ OLAP_ERR_PUSH_CREAT_CUMULATIVE_ERROR = -906,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_DELTA_FILE_EOF src/olap/olap_define.h /^ OLAP_ERR_PUSH_DELTA_FILE_EOF = -901,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_INIT_ERROR src/olap/olap_define.h /^ OLAP_ERR_PUSH_INIT_ERROR = -900,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_INPUT_DATA_ERROR src/olap/olap_define.h /^ OLAP_ERR_PUSH_INPUT_DATA_ERROR = -910,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_ROWSET_NOT_FOUND src/olap/olap_define.h /^ OLAP_ERR_PUSH_ROWSET_NOT_FOUND = -914,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_SCHEMA_MISMATCH src/olap/olap_define.h /^ OLAP_ERR_PUSH_SCHEMA_MISMATCH = -903,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_TABLE_NOT_EXIST src/olap/olap_define.h /^ OLAP_ERR_PUSH_TABLE_NOT_EXIST = -909,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST src/olap/olap_define.h /^ OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST = -911,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_VERSION_ALREADY_EXIST src/olap/olap_define.h /^ OLAP_ERR_PUSH_VERSION_ALREADY_EXIST = -908,$/;" e enum:doris::OLAPStatus +OLAP_ERR_PUSH_VERSION_INCORRECT src/olap/olap_define.h /^ OLAP_ERR_PUSH_VERSION_INCORRECT = -902,$/;" e enum:doris::OLAPStatus +OLAP_ERR_QUERY_SPLIT_KEY_ERR src/olap/olap_define.h /^ OLAP_ERR_QUERY_SPLIT_KEY_ERR = -1605,$/;" e enum:doris::OLAPStatus +OLAP_ERR_READER_GET_ITERATOR_ERROR src/olap/olap_define.h /^ OLAP_ERR_READER_GET_ITERATOR_ERROR = -701,$/;" e enum:doris::OLAPStatus +OLAP_ERR_READER_IS_UNINITIALIZED src/olap/olap_define.h /^ OLAP_ERR_READER_IS_UNINITIALIZED = -700,$/;" e enum:doris::OLAPStatus +OLAP_ERR_READER_READING_ERROR src/olap/olap_define.h /^ OLAP_ERR_READER_READING_ERROR = -703,$/;" e enum:doris::OLAPStatus +OLAP_ERR_READ_UNENOUGH src/olap/olap_define.h /^ OLAP_ERR_READ_UNENOUGH = -116,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWBLOCK_DECOMPRESS_ERROR src/olap/olap_define.h /^ OLAP_ERR_ROWBLOCK_DECOMPRESS_ERROR = -1300,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWBLOCK_FIND_ROW_EXCEPTION src/olap/olap_define.h /^ OLAP_ERR_ROWBLOCK_FIND_ROW_EXCEPTION = -1301,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWBLOCK_READ_INFO_ERROR src/olap/olap_define.h /^ OLAP_ERR_ROWBLOCK_READ_INFO_ERROR = -1302,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_ALREADY_EXIST src/olap/olap_define.h /^ OLAP_ERR_ROWSET_ALREADY_EXIST = -3106,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_BUILDER_INIT src/olap/olap_define.h /^ OLAP_ERR_ROWSET_BUILDER_INIT = -3104,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_CREATE_READER src/olap/olap_define.h /^ OLAP_ERR_ROWSET_CREATE_READER = -3107,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_DELETE_FILE_FAILED src/olap/olap_define.h /^ OLAP_ERR_ROWSET_DELETE_FILE_FAILED = -3103,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_GENERATE_ID_FAILED src/olap/olap_define.h /^ OLAP_ERR_ROWSET_GENERATE_ID_FAILED = -3102,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_INVALID src/olap/olap_define.h /^ OLAP_ERR_ROWSET_INVALID = -3108,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_INVALID_STATE_TRANSITION src/olap/olap_define.h /^ OLAP_ERR_ROWSET_INVALID_STATE_TRANSITION = -3112$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_LOAD_FAILED src/olap/olap_define.h /^ OLAP_ERR_ROWSET_LOAD_FAILED = -3109,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_READER_INIT src/olap/olap_define.h /^ OLAP_ERR_ROWSET_READER_INIT = -3110,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_READ_FAILED src/olap/olap_define.h /^ OLAP_ERR_ROWSET_READ_FAILED = -3111,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_SAVE_FAILED src/olap/olap_define.h /^ OLAP_ERR_ROWSET_SAVE_FAILED = -3101,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_TYPE_NOT_FOUND src/olap/olap_define.h /^ OLAP_ERR_ROWSET_TYPE_NOT_FOUND = -3105,$/;" e enum:doris::OLAPStatus +OLAP_ERR_ROWSET_WRITER_INIT src/olap/olap_define.h /^ OLAP_ERR_ROWSET_WRITER_INIT = -3100,$/;" e enum:doris::OLAPStatus +OLAP_ERR_RWLOCK_ERROR src/olap/olap_define.h /^ OLAP_ERR_RWLOCK_ERROR = -115,$/;" e enum:doris::OLAPStatus +OLAP_ERR_SCHEMA_CHANGE_INFO_INVALID src/olap/olap_define.h /^ OLAP_ERR_SCHEMA_CHANGE_INFO_INVALID = -1604,$/;" e enum:doris::OLAPStatus +OLAP_ERR_SCHEMA_SCHEMA_FIELD_INVALID src/olap/olap_define.h /^ OLAP_ERR_SCHEMA_SCHEMA_FIELD_INVALID = -1501,$/;" e enum:doris::OLAPStatus +OLAP_ERR_SCHEMA_SCHEMA_INVALID src/olap/olap_define.h /^ OLAP_ERR_SCHEMA_SCHEMA_INVALID = -1500,$/;" e enum:doris::OLAPStatus +OLAP_ERR_SERIALIZE_PROTOBUF_ERROR src/olap/olap_define.h /^ OLAP_ERR_SERIALIZE_PROTOBUF_ERROR = -212,$/;" e enum:doris::OLAPStatus +OLAP_ERR_SIGNATURE_ERROR src/olap/olap_define.h /^ OLAP_ERR_SIGNATURE_ERROR = -209,$/;" e enum:doris::OLAPStatus +OLAP_ERR_STL_ERROR src/olap/olap_define.h /^ OLAP_ERR_STL_ERROR = -105,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TABLE_ALREADY_DELETED_ERROR src/olap/olap_define.h /^ OLAP_ERR_TABLE_ALREADY_DELETED_ERROR = -406,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TABLE_CREATE_FROM_HEADER_ERROR src/olap/olap_define.h /^ OLAP_ERR_TABLE_CREATE_FROM_HEADER_ERROR = -404,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TABLE_CREATE_META_ERROR src/olap/olap_define.h /^ OLAP_ERR_TABLE_CREATE_META_ERROR = -405,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TABLE_INDEX_FIND_ERROR src/olap/olap_define.h /^ OLAP_ERR_TABLE_INDEX_FIND_ERROR = -403,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TABLE_INDEX_VALIDATE_ERROR src/olap/olap_define.h /^ OLAP_ERR_TABLE_INDEX_VALIDATE_ERROR = -402,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TABLE_INSERT_DUPLICATION_ERROR src/olap/olap_define.h /^ OLAP_ERR_TABLE_INSERT_DUPLICATION_ERROR = -503,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TABLE_NOT_FOUND src/olap/olap_define.h /^ OLAP_ERR_TABLE_NOT_FOUND = -215,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TABLE_VERSION_DUPLICATE_ERROR src/olap/olap_define.h /^ OLAP_ERR_TABLE_VERSION_DUPLICATE_ERROR = -400,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TABLE_VERSION_INDEX_MISMATCH_ERROR src/olap/olap_define.h /^ OLAP_ERR_TABLE_VERSION_INDEX_MISMATCH_ERROR = -401,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TEST_FILE_ERROR src/olap/olap_define.h /^ OLAP_ERR_TEST_FILE_ERROR = -221,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TOO_FEW_FILE_DESCRITPROR src/olap/olap_define.h /^ OLAP_ERR_TOO_FEW_FILE_DESCRITPROR = -123,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TOO_MANY_TRANSACTIONS src/olap/olap_define.h /^ OLAP_ERR_TOO_MANY_TRANSACTIONS = -233,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TRANSACTION_ALREADY_COMMITTED src/olap/olap_define.h /^ OLAP_ERR_TRANSACTION_ALREADY_COMMITTED = -228,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TRANSACTION_ALREADY_VISIBLE src/olap/olap_define.h /^ OLAP_ERR_TRANSACTION_ALREADY_VISIBLE = -229,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TRANSACTION_NOT_EXIST src/olap/olap_define.h /^ OLAP_ERR_TRANSACTION_NOT_EXIST = -226,$/;" e enum:doris::OLAPStatus +OLAP_ERR_TRY_LOCK_FAILED src/olap/olap_define.h /^ OLAP_ERR_TRY_LOCK_FAILED = -216,$/;" e enum:doris::OLAPStatus +OLAP_ERR_UB_FUNC_ERROR src/olap/olap_define.h /^ OLAP_ERR_UB_FUNC_ERROR = -110,$/;" e enum:doris::OLAPStatus +OLAP_ERR_UB_NETWORK_ERROR src/olap/olap_define.h /^ OLAP_ERR_UB_NETWORK_ERROR = -118,$/;" e enum:doris::OLAPStatus +OLAP_ERR_UNDERFLOW src/olap/olap_define.h /^ OLAP_ERR_UNDERFLOW = -219,$/;" e enum:doris::OLAPStatus +OLAP_ERR_UNKNOWN_COMPRESSION_TYPE src/olap/olap_define.h /^ OLAP_ERR_UNKNOWN_COMPRESSION_TYPE = -113,$/;" e enum:doris::OLAPStatus +OLAP_ERR_VERSION_ALREADY_MERGED src/olap/olap_define.h /^ OLAP_ERR_VERSION_ALREADY_MERGED = -230,$/;" e enum:doris::OLAPStatus +OLAP_ERR_VERSION_NOT_EXIST src/olap/olap_define.h /^ OLAP_ERR_VERSION_NOT_EXIST = -214,$/;" e enum:doris::OLAPStatus +OLAP_ERR_WRITER_DATA_WRITE_ERROR src/olap/olap_define.h /^ OLAP_ERR_WRITER_DATA_WRITE_ERROR = -1201,$/;" e enum:doris::OLAPStatus +OLAP_ERR_WRITER_INDEX_WRITE_ERROR src/olap/olap_define.h /^ OLAP_ERR_WRITER_INDEX_WRITE_ERROR = -1200,$/;" e enum:doris::OLAPStatus +OLAP_ERR_WRITER_ROW_BLOCK_ERROR src/olap/olap_define.h /^ OLAP_ERR_WRITER_ROW_BLOCK_ERROR = -1202,$/;" e enum:doris::OLAPStatus +OLAP_ERR_WRITER_SEGMENT_NOT_FINALIZED src/olap/olap_define.h /^ OLAP_ERR_WRITER_SEGMENT_NOT_FINALIZED = -1203,$/;" e enum:doris::OLAPStatus +OLAP_ERR_WRITE_PROTOBUF_ERROR src/olap/olap_define.h /^ OLAP_ERR_WRITE_PROTOBUF_ERROR = -213,$/;" e enum:doris::OLAPStatus +OLAP_FIELD_AGGREGATION_BITMAP_UNION src/olap/olap_common.h /^ OLAP_FIELD_AGGREGATION_BITMAP_UNION = 7,$/;" e enum:doris::FieldAggregationMethod +OLAP_FIELD_AGGREGATION_HLL_UNION src/olap/olap_common.h /^ OLAP_FIELD_AGGREGATION_HLL_UNION = 5,$/;" e enum:doris::FieldAggregationMethod +OLAP_FIELD_AGGREGATION_MAX src/olap/olap_common.h /^ OLAP_FIELD_AGGREGATION_MAX = 3,$/;" e enum:doris::FieldAggregationMethod +OLAP_FIELD_AGGREGATION_MIN src/olap/olap_common.h /^ OLAP_FIELD_AGGREGATION_MIN = 2,$/;" e enum:doris::FieldAggregationMethod +OLAP_FIELD_AGGREGATION_NONE src/olap/olap_common.h /^ OLAP_FIELD_AGGREGATION_NONE = 0,$/;" e enum:doris::FieldAggregationMethod +OLAP_FIELD_AGGREGATION_REPLACE src/olap/olap_common.h /^ OLAP_FIELD_AGGREGATION_REPLACE = 4,$/;" e enum:doris::FieldAggregationMethod +OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL src/olap/olap_common.h /^ OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL = 8,$/;" e enum:doris::FieldAggregationMethod +OLAP_FIELD_AGGREGATION_SUM src/olap/olap_common.h /^ OLAP_FIELD_AGGREGATION_SUM = 1,$/;" e enum:doris::FieldAggregationMethod +OLAP_FIELD_AGGREGATION_UNKNOWN src/olap/olap_common.h /^ OLAP_FIELD_AGGREGATION_UNKNOWN = 6,$/;" e enum:doris::FieldAggregationMethod +OLAP_FIELD_TYPE_BIGINT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_BIGINT = 7, \/\/ MYSQL_TYPE_LONGLONG$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_BOOL src/olap/olap_common.h /^ OLAP_FIELD_TYPE_BOOL = 24,$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_CHAR src/olap/olap_common.h /^ OLAP_FIELD_TYPE_CHAR = 13, \/\/ MYSQL_TYPE_STRING$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_DATE src/olap/olap_common.h /^ OLAP_FIELD_TYPE_DATE = 14, \/\/ MySQL_TYPE_NEWDATE$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_DATETIME src/olap/olap_common.h /^ OLAP_FIELD_TYPE_DATETIME = 15, \/\/ MySQL_TYPE_DATETIME$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_DECIMAL src/olap/olap_common.h /^ OLAP_FIELD_TYPE_DECIMAL = 16, \/\/ DECIMAL, using different store format against MySQL$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_DISCRETE_DOUBLE src/olap/olap_common.h /^ OLAP_FIELD_TYPE_DISCRETE_DOUBLE = 12,$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_DOUBLE src/olap/olap_common.h /^ OLAP_FIELD_TYPE_DOUBLE = 11, \/\/ MYSQL_TYPE_DOUBLE$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_FLOAT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_FLOAT = 10, \/\/ MYSQL_TYPE_FLOAT$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_HLL src/olap/olap_common.h /^ OLAP_FIELD_TYPE_HLL = 23,$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_INT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_INT = 5, \/\/ MYSQL_TYPE_LONG$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_LARGEINT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_LARGEINT = 9,$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_LIST src/olap/olap_common.h /^ OLAP_FIELD_TYPE_LIST = 19, \/\/ LIST$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_MAP src/olap/olap_common.h /^ OLAP_FIELD_TYPE_MAP = 20, \/\/ Map$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_NONE src/olap/olap_common.h /^ OLAP_FIELD_TYPE_NONE = 22,$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_OBJECT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_OBJECT = 25$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_SMALLINT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_SMALLINT = 3, \/\/ MYSQL_TYPE_SHORT$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_STRUCT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_STRUCT = 18, \/\/ Struct$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_TINYINT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_TINYINT = 1, \/\/ MYSQL_TYPE_TINY$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_UNKNOWN src/olap/olap_common.h /^ OLAP_FIELD_TYPE_UNKNOWN = 21, \/\/ UNKNOW Type$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_UNSIGNED_BIGINT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_UNSIGNED_BIGINT = 8,$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_UNSIGNED_INT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_UNSIGNED_INT = 6,$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_UNSIGNED_SMALLINT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_UNSIGNED_SMALLINT = 4,$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_UNSIGNED_TINYINT src/olap/olap_common.h /^ OLAP_FIELD_TYPE_UNSIGNED_TINYINT = 2,$/;" e enum:doris::FieldType +OLAP_FIELD_TYPE_VARCHAR src/olap/olap_common.h /^ OLAP_FIELD_TYPE_VARCHAR = 17,$/;" e enum:doris::FieldType +OLAP_FIX_HEADER_MAGIC_NUMBER src/olap/olap_define.h /^static const uint64_t OLAP_FIX_HEADER_MAGIC_NUMBER = 0;$/;" m namespace:doris +OLAP_GOTO src/olap/olap_define.h 93;" d +OLAP_LIKELY src/olap/olap_define.h 391;" d +OLAP_LIKELY src/olap/olap_define.h 394;" d +OLAP_LOG_NOTICE_DIRECT_SOCK src/olap/utils.h 339;" d +OLAP_LOG_SETBASIC src/olap/utils.h 341;" d +OLAP_LOG_WARNING src/olap/utils.h 338;" d +OLAP_LOG_WARNING_SOCK src/olap/utils.h 340;" d +OLAP_LOG_WRITE src/olap/utils.h 320;" d +OLAP_LRU_CACHE_MAX_KEY_LENTH src/olap/olap_define.h /^static const size_t OLAP_LRU_CACHE_MAX_KEY_LENTH = OLAP_MAX_PATH_LEN * 2;$/;" m namespace:doris +OLAP_MAX_COLUMN_SEGMENT_FILE_SIZE src/olap/olap_define.h /^static const uint32_t OLAP_MAX_COLUMN_SEGMENT_FILE_SIZE = 268435456;$/;" m namespace:doris +OLAP_MAX_PATH_LEN src/olap/olap_define.h /^static const uint32_t OLAP_MAX_PATH_LEN = 512;$/;" m namespace:doris +OLAP_REQUEST_FAILED src/olap/olap_define.h /^ OLAP_REQUEST_FAILED = -2,$/;" e enum:doris::OLAPStatus +OLAP_STRING_MAX_BYTES src/olap/olap_define.h /^static const uint16_t OLAP_STRING_MAX_BYTES = sizeof(StringLengthType);$/;" m namespace:doris +OLAP_STRING_MAX_LENGTH src/olap/olap_define.h /^static const uint16_t OLAP_STRING_MAX_LENGTH = 65535;$/;" m namespace:doris +OLAP_SUCCESS src/olap/olap_define.h /^ OLAP_SUCCESS = 0,$/;" e enum:doris::OLAPStatus +OLAP_UNLIKELY src/olap/olap_define.h 392;" d +OLAP_UNLIKELY src/olap/olap_define.h 395;" d +OLAP_UNUSED_ARG src/olap/olap_define.h 425;" d +OLAP_V1 src/olap/olap_define.h /^ OLAP_V1 = 0,$/;" e enum:doris::OLAPDataVersion +OLAP_VLOG_WRITE src/olap/utils.h 327;" d +OLD_HEADER_PREFIX src/olap/tablet_meta_manager.h /^const std::string OLD_HEADER_PREFIX = "hdr_";$/;" m namespace:doris +ONE src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +ONE_ARG_MATH_FN src/exprs/math_functions.cpp 118;" d file: +ONE_BILLION src/runtime/decimalv2_value.h /^ static const uint32_t ONE_BILLION = 1000000000;$/;" m class:doris::DecimalV2Value +ONE_LEVEL output/udf/include/uda_test_harness.h /^ ONE_LEVEL = 2,$/;" e enum:doris_udf::UdaExecutionMode +ONE_LEVEL src/udf/uda_test_harness.h /^ ONE_LEVEL = 2,$/;" e enum:doris_udf::UdaExecutionMode +OPTIONS src/http/http_method.h /^ OPTIONS,$/;" e enum:doris::HttpMethod +OP_EQ src/olap/olap_cond.h /^ OP_EQ = 0, \/\/ equal$/;" e enum:doris::CondOp +OP_GE src/olap/olap_cond.h /^ OP_GE = 5, \/\/ greater or equal$/;" e enum:doris::CondOp +OP_GT src/olap/olap_cond.h /^ OP_GT = 4, \/\/ greater than$/;" e enum:doris::CondOp +OP_IN src/olap/olap_cond.h /^ OP_IN = 6, \/\/ IN$/;" e enum:doris::CondOp +OP_IS src/olap/olap_cond.h /^ OP_IS = 7, \/\/ is null or not null$/;" e enum:doris::CondOp +OP_LE src/olap/olap_cond.h /^ OP_LE = 3, \/\/ less or equal$/;" e enum:doris::CondOp +OP_LT src/olap/olap_cond.h /^ OP_LT = 2, \/\/ less than$/;" e enum:doris::CondOp +OP_NE src/olap/olap_cond.h /^ OP_NE = 1, \/\/ not equal$/;" e enum:doris::CondOp +OP_NULL src/olap/olap_cond.h /^ OP_NULL = 8 \/\/ invalid OP$/;" e enum:doris::CondOp +ORCFileStream src/exec/orc_scanner.cpp /^ ORCFileStream(FileReader* file, std::string filename) : _file(file), _filename(std::move(filename)) {$/;" f class:doris::ORCFileStream +ORCFileStream src/exec/orc_scanner.cpp /^class ORCFileStream : public orc::InputStream {$/;" c namespace:doris file: +ORCScanner src/exec/orc_scanner.cpp /^ORCScanner::ORCScanner(RuntimeState* state,$/;" f class:doris::ORCScanner +ORCScanner src/exec/orc_scanner.h /^class ORCScanner : public BaseScanner {$/;" c namespace:doris +ORC_SCANNER_H src/exec/orc_scanner.h 19;" d +ORIGIN src/http/http_headers.cpp /^const char* HttpHeaders::ORIGIN = "Origin";$/;" m class:doris::HttpHeaders file: +ORIGIN src/http/http_headers.h /^ static const char* ORIGIN;$/;" m class:doris::HttpHeaders +OS_ANDROID src/gutil/cpu.h 12;" d +OS_LINUX src/gutil/cpu.h 14;" d +OS_MACOSX src/gutil/cpu.h 10;" d +OS_POSIX src/gutil/cpu.h 38;" d +OS_WIN src/gutil/cpu.h 16;" d +OTHER src/exprs/agg_fn.h /^ OTHER, $/;" e enum:doris::AggFn::AggregationOp +OTHER src/exprs/agg_fn_evaluator.h /^ OTHER,$/;" e enum:doris::AggFnEvaluator::AggregationOp +OUTPUT_CHUNK src/exec/plain_text_line_reader.cpp 29;" d file: +OVERRIDE src/gutil/port.h 476;" d +OVERRIDE src/service/brpc.h 35;" d +O_BINARY src/gutil/port.h 189;" d +O_DIRECTORY src/gutil/linux_syscall_support.h 550;" d +O_DIRECTORY src/gutil/linux_syscall_support.h 552;" d +ObjectPool src/common/object_pool.h /^ ObjectPool(): _objects() {}$/;" f class:doris::ObjectPool +ObjectPool src/common/object_pool.h /^class ObjectPool {$/;" c namespace:doris +OlapMeta src/olap/olap_meta.cpp /^OlapMeta::OlapMeta(const std::string& root_path)$/;" f class:doris::OlapMeta +OlapMeta src/olap/olap_meta.h /^class OlapMeta {$/;" c namespace:doris +OlapMetaTest test/olap/olap_meta_test.cpp /^class OlapMetaTest : public testing::Test {$/;" c namespace:doris file: +OlapReaderStatistics src/olap/olap_common.h /^struct OlapReaderStatistics {$/;" s namespace:doris +OlapRewriteNode src/exec/olap_rewrite_node.cpp /^OlapRewriteNode::OlapRewriteNode(ObjectPool* pool,$/;" f class:doris::OlapRewriteNode +OlapRewriteNode src/exec/olap_rewrite_node.h /^class OlapRewriteNode : public ExecNode {$/;" c namespace:doris +OlapScanKeys src/exec/olap_common.h /^ OlapScanKeys() :$/;" f class:doris::OlapScanKeys +OlapScanKeys src/exec/olap_common.h /^class OlapScanKeys {$/;" c namespace:doris +OlapScanKeysTest test/exec/olap_common_test.cpp /^class OlapScanKeysTest : public ::testing::Test {$/;" c namespace:doris file: +OlapScanNode src/exec/olap_scan_node.cpp /^OlapScanNode::OlapScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs):$/;" f class:doris::OlapScanNode +OlapScanNode src/exec/olap_scan_node.h /^class OlapScanNode : public ScanNode {$/;" c namespace:doris +OlapScanNodeTest test/exec/olap_scan_node_test.cpp /^ OlapScanNodeTest() : _runtime_stat("test") { }$/;" f class:doris::OlapScanNodeTest +OlapScanNodeTest test/exec/olap_scan_node_test.cpp /^class OlapScanNodeTest : public ::testing::Test {$/;" c namespace:doris file: +OlapScanRange src/exec/olap_utils.h /^ OlapScanRange($/;" f struct:doris::OlapScanRange +OlapScanRange src/exec/olap_utils.h /^ OlapScanRange() : begin_include(true), end_include(true) {$/;" f struct:doris::OlapScanRange +OlapScanRange src/exec/olap_utils.h /^typedef struct OlapScanRange {$/;" s namespace:doris +OlapScanRange src/exec/olap_utils.h /^} OlapScanRange;$/;" t namespace:doris typeref:struct:doris::OlapScanRange +OlapScanner src/exec/olap_scanner.cpp /^OlapScanner::OlapScanner($/;" f class:doris::OlapScanner +OlapScanner src/exec/olap_scanner.h /^class OlapScanner {$/;" c namespace:doris +OlapSnapshotConverter src/olap/olap_snapshot_converter.h /^class OlapSnapshotConverter {$/;" c namespace:doris +OlapSnapshotConverterTest test/olap/olap_snapshot_converter_test.cpp /^class OlapSnapshotConverterTest : public testing::Test {$/;" c namespace:doris file: +OlapStopWatch src/olap/utils.h /^ OlapStopWatch() {$/;" f class:doris::OlapStopWatch +OlapStopWatch src/olap/utils.h /^class OlapStopWatch {$/;" c namespace:doris +OlapTableDescriptor src/runtime/descriptors.cpp /^OlapTableDescriptor::OlapTableDescriptor(const TTableDescriptor& tdesc)$/;" f class:doris::OlapTableDescriptor +OlapTableDescriptor src/runtime/descriptors.h /^class OlapTableDescriptor : public TableDescriptor {$/;" c namespace:doris +OlapTableIndexSchema src/exec/tablet_info.h /^struct OlapTableIndexSchema {$/;" s namespace:doris +OlapTableLocationParam src/exec/tablet_info.h /^ OlapTableLocationParam(const TOlapTableLocationParam& t_param) : _t_param(t_param) {$/;" f class:doris::OlapTableLocationParam +OlapTableLocationParam src/exec/tablet_info.h /^class OlapTableLocationParam {$/;" c namespace:doris +OlapTablePartKeyComparator src/exec/tablet_info.h /^ OlapTablePartKeyComparator(const std::vector& slot_descs) :$/;" f class:doris::OlapTablePartKeyComparator +OlapTablePartKeyComparator src/exec/tablet_info.h /^class OlapTablePartKeyComparator {$/;" c namespace:doris +OlapTablePartition src/exec/tablet_info.h /^struct OlapTablePartition {$/;" s namespace:doris +OlapTablePartitionParam src/exec/tablet_info.cpp /^OlapTablePartitionParam::OlapTablePartitionParam($/;" f class:doris::OlapTablePartitionParam +OlapTablePartitionParam src/exec/tablet_info.h /^class OlapTablePartitionParam {$/;" c namespace:doris +OlapTablePartitionParamTest test/exec/tablet_info_test.cpp /^ OlapTablePartitionParamTest() { }$/;" f class:doris::OlapTablePartitionParamTest +OlapTablePartitionParamTest test/exec/tablet_info_test.cpp /^class OlapTablePartitionParamTest : public testing::Test {$/;" c namespace:doris file: +OlapTableSchemaParam src/exec/tablet_info.h /^ OlapTableSchemaParam() { }$/;" f class:doris::OlapTableSchemaParam +OlapTableSchemaParam src/exec/tablet_info.h /^class OlapTableSchemaParam {$/;" c namespace:doris +OlapTableSink src/exec/tablet_sink.cpp /^OlapTableSink::OlapTableSink(ObjectPool* pool,$/;" f class:doris::stream_load::OlapTableSink +OlapTableSink src/exec/tablet_sink.h /^class OlapTableSink : public DataSink {$/;" c namespace:doris::stream_load +OlapTableSinkTest test/exec/tablet_sink_test.cpp /^ OlapTableSinkTest() { }$/;" f class:doris::stream_load::OlapTableSinkTest +OlapTableSinkTest test/exec/tablet_sink_test.cpp /^class OlapTableSinkTest : public testing::Test {$/;" c namespace:doris::stream_load file: +OlapTuple src/olap/tuple.h /^ OlapTuple() { }$/;" f class:doris::OlapTuple +OlapTuple src/olap/tuple.h /^ OlapTuple(const std::vector& values)$/;" f class:doris::OlapTuple +OlapTuple src/olap/tuple.h /^class OlapTuple {$/;" c namespace:doris +OneIn src/util/random.h /^ bool OneIn(int n) { return (Next() % n) == 0; }$/;" f class:doris::Random +OnlyWhitespace src/gutil/strings/util.cc /^bool OnlyWhitespace(const StringPiece& s) {$/;" f +Open src/exec/new_partitioned_hash_table.cc /^Status NewPartitionedHashTableCtx::Open(RuntimeState* state) {$/;" f class:NewPartitionedHashTableCtx +Open src/exprs/new_agg_fn_evaluator.cc /^Status NewAggFnEvaluator::Open($/;" f class:NewAggFnEvaluator +Open src/exprs/new_agg_fn_evaluator.cc /^Status NewAggFnEvaluator::Open(RuntimeState* state) {$/;" f class:NewAggFnEvaluator +Open src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::BufferHandle::Open(uint8_t* data, int64_t len, int home_core) {$/;" f class:doris::BufferPool::BufferHandle +Open src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::PageHandle::Open(Page* page, ClientHandle* client) {$/;" f class:doris::BufferPool::PageHandle +OpenMode src/env/env.h /^ enum OpenMode {$/;" g class:doris::Env +OpenedFileHandle src/util/file_cache.h /^ OpenedFileHandle() : _cache(nullptr), _handle(nullptr) { }$/;" f class:doris::OpenedFileHandle +OpenedFileHandle src/util/file_cache.h /^ explicit OpenedFileHandle(Cache* cache, Cache::Handle* handle)$/;" f class:doris::OpenedFileHandle +OpenedFileHandle src/util/file_cache.h /^class OpenedFileHandle {$/;" c namespace:doris +Operators src/exprs/operators.h /^class Operators {$/;" c namespace:doris +OptionsTest test/olap/options_test.cpp /^ OptionsTest() {}$/;" f class:doris::OptionsTest +OptionsTest test/olap/options_test.cpp /^class OptionsTest : public testing::Test {$/;" c namespace:doris file: +OrPredicate src/exprs/compound_predicate.h /^ OrPredicate(const TExprNode& node) : CompoundPredicate(node) { }$/;" f class:doris::OrPredicate +OrPredicate src/exprs/compound_predicate.h /^class OrPredicate: public CompoundPredicate {$/;" c namespace:doris +OrcScannerTest test/exec/orc_scanner_test.cpp /^ OrcScannerTest() : _runtime_state(TQueryGlobals()) {$/;" f class:doris::OrcScannerTest +OrcScannerTest test/exec/orc_scanner_test.cpp /^class OrcScannerTest : public testing::Test {$/;" c namespace:doris file: +OrdinalIndexReader src/olap/rowset/segment_v2/ordinal_page_index.h /^ explicit OrdinalIndexReader(const std::string& filename,$/;" f class:doris::segment_v2::OrdinalIndexReader +OrdinalIndexReader src/olap/rowset/segment_v2/ordinal_page_index.h /^class OrdinalIndexReader {$/;" c namespace:doris::segment_v2 +OrdinalIndexWriter src/olap/rowset/segment_v2/ordinal_page_index.h /^ OrdinalIndexWriter() : _page_builder(new IndexPageBuilder(0, true)) {}$/;" f class:doris::segment_v2::OrdinalIndexWriter +OrdinalIndexWriter src/olap/rowset/segment_v2/ordinal_page_index.h /^class OrdinalIndexWriter {$/;" c namespace:doris::segment_v2 +OrdinalPageIndexIterator src/olap/rowset/segment_v2/ordinal_page_index.h /^ OrdinalPageIndexIterator() : _index(nullptr), _cur_idx(-1) { }$/;" f class:doris::segment_v2::OrdinalPageIndexIterator +OrdinalPageIndexIterator src/olap/rowset/segment_v2/ordinal_page_index.h /^ OrdinalPageIndexIterator(OrdinalIndexReader* index) : _index(index), _cur_idx(0) { }$/;" f class:doris::segment_v2::OrdinalPageIndexIterator +OrdinalPageIndexIterator src/olap/rowset/segment_v2/ordinal_page_index.h /^ OrdinalPageIndexIterator(OrdinalIndexReader* index, int cur_idx) : _index(index), _cur_idx(cur_idx) { }$/;" f class:doris::segment_v2::OrdinalPageIndexIterator +OrdinalPageIndexIterator src/olap/rowset/segment_v2/ordinal_page_index.h /^class OrdinalPageIndexIterator {$/;" c namespace:doris::segment_v2 +OrdinalPageIndexTest test/olap/rowset/segment_v2/ordinal_page_index_test.cpp /^class OrdinalPageIndexTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +OrigFn src/gutil/valgrind.h /^ OrigFn;$/;" t typeref:struct:__anon10 +OrigFn src/gutil/valgrind.h /^ OrigFn;$/;" t typeref:struct:__anon11 +OrigFn src/gutil/valgrind.h /^ OrigFn;$/;" t typeref:struct:__anon12 +OrigFn src/gutil/valgrind.h /^ OrigFn;$/;" t typeref:struct:__anon13 +OrigFn src/gutil/valgrind.h /^ OrigFn;$/;" t typeref:struct:__anon14 +OrigFn src/gutil/valgrind.h /^ OrigFn;$/;" t typeref:struct:__anon9 +OsInfo src/util/os_info.h /^class OsInfo {$/;" c namespace:doris +OutStream src/olap/out_stream.cpp /^OutStream::OutStream(uint32_t buffer_size, Compressor compressor) : $/;" f class:doris::OutStream +OutStream src/olap/out_stream.h /^class OutStream {$/;" c namespace:doris +OutStreamFactory src/olap/out_stream.cpp /^OutStreamFactory::OutStreamFactory(CompressKind compress_kind, uint32_t stream_buffer_size) : $/;" f class:doris::OutStreamFactory +OutStreamFactory src/olap/out_stream.h /^class OutStreamFactory {$/;" c namespace:doris +OutputWithMarkupTagsStripped src/gutil/strings/strip.cc /^string OutputWithMarkupTagsStripped(const string& s) {$/;" f +OwnedSlice src/util/slice.h /^ OwnedSlice() : _slice((uint8_t*)nullptr, 0) {}$/;" f class:doris::OwnedSlice +OwnedSlice src/util/slice.h /^ OwnedSlice(OwnedSlice&& src) : _slice(src._slice) {$/;" f class:doris::OwnedSlice +OwnedSlice src/util/slice.h /^ OwnedSlice(uint8_t* _data, size_t size) :_slice(_data, size) {}$/;" f class:doris::OwnedSlice +OwnedSlice src/util/slice.h /^class OwnedSlice {$/;" c namespace:doris +Ownership src/gutil/basictypes.h /^enum Ownership {$/;" g +PACKAGE src/util/bfd_parser.h 22;" d +PACKAGE_VERSION src/util/bfd_parser.h 26;" d +PACKED src/gutil/port.h 319;" d +PAGE_FOOTER src/http/web_page_handler.cpp /^static const std::string PAGE_FOOTER = "<\/div><\/body><\/html>";$/;" m namespace:doris file: +PAGE_HEADER src/http/web_page_handler.cpp /^static const std::string PAGE_HEADER = $/;" m namespace:doris file: +PAGE_SIZE src/runtime/memory/system_allocator.cpp 30;" d file: +PAGG_DEFAULT_HASH_TABLE_SZ src/exec/new_partitioned_aggregation_node.h /^ static const int64_t PAGG_DEFAULT_HASH_TABLE_SZ = 1024;$/;" m class:doris::NewPartitionedAggregationNode +PARENT_WAITING_TID src/util/thread.h /^ PARENT_WAITING_TID = -2,$/;" e enum:doris::Thread::__anon38 +PARSE_AGAIN src/http/http_parser.h /^ PARSE_AGAIN, \/\/ continue call this function$/;" e enum:doris::HttpParser::ParseState +PARSE_DONE src/http/http_parser.h /^ PARSE_DONE, \/\/ trunk is over$/;" e enum:doris::HttpParser::ParseState +PARSE_ERROR src/http/http_parser.h /^ PARSE_ERROR, \/\/ parse failed.$/;" e enum:doris::HttpParser::ParseState +PARSE_FAILURE src/util/string_parser.hpp /^ PARSE_FAILURE,$/;" e enum:doris::StringParser::ParseResult +PARSE_OK src/http/http_parser.h /^ PARSE_OK, \/\/ reach trunk data, you can read data$/;" e enum:doris::HttpParser::ParseState +PARSE_OVERFLOW src/util/string_parser.hpp /^ PARSE_OVERFLOW,$/;" e enum:doris::StringParser::ParseResult +PARSE_SUCCESS src/util/string_parser.hpp /^ PARSE_SUCCESS = 0,$/;" e enum:doris::StringParser::ParseResult +PARSE_UNDERFLOW src/util/string_parser.hpp /^ PARSE_UNDERFLOW$/;" e enum:doris::StringParser::ParseResult +PARTIAL_CONTENT src/http/http_status.h /^ PARTIAL_CONTENT = 206,$/;" e enum:doris::HttpStatus +PARTITION src/exec/analytic_eval_node.h /^ PARTITION,$/;" e enum:doris::AnalyticEvalNode::AnalyticFnScope +PARTITION_FANOUT src/exec/new_partitioned_aggregation_node.h /^ static const int PARTITION_FANOUT = 16;$/;" m class:doris::NewPartitionedAggregationNode +PARTITION_FANOUT src/exec/partitioned_aggregation_node.h /^ static const int PARTITION_FANOUT = 16;$/;" m class:doris::PartitionedAggregationNode +PART_BITMASK src/util/radix_sort.h /^ static constexpr size_t PART_BITMASK = HISTOGRAM_SIZE - 1;$/;" m struct:doris::RadixSort +PART_SIZE_BITS src/util/radix_sort.h /^ static constexpr size_t PART_SIZE_BITS = 8; \/\/\/ With what pieces of the key, in bits, to do one pass - reshuffle of the array.$/;" m struct:doris::RadixSortFloatTraits +PART_SIZE_BITS src/util/radix_sort.h /^ static constexpr size_t PART_SIZE_BITS = 8;$/;" m struct:doris::RadixSortIntTraits +PART_SIZE_BITS src/util/radix_sort.h /^ static constexpr size_t PART_SIZE_BITS = 8;$/;" m struct:doris::RadixSortUIntTraits +PART_SIZE_BITS src/util/tdigest.h /^ static constexpr size_t PART_SIZE_BITS = 8;$/;" m struct:doris::TDigest::TDigestRadixSortTraits +PART_SIZE_BITS test/util/radix_sort_test.cpp /^ static constexpr size_t PART_SIZE_BITS = 8;$/;" m struct:doris::RadixSortTestTraits file: +PATCHED_BASE src/olap/rowset/run_length_integer_writer.h /^ PATCHED_BASE = 2,$/;" e enum:doris::RunLengthIntegerWriter::EncodingType +PATH src/http/action/reload_tablet_action.cpp /^const std::string PATH = "path";$/;" m namespace:doris file: +PATH src/util/url_parser.h /^ PATH,$/;" e enum:doris::UrlParser::UrlPart +PATH_MAX src/gutil/port.h 829;" d +PATH_SEPARATOR src/gutil/port.h /^const char PATH_SEPARATOR = '\/';$/;" v +PATH_SEPARATOR src/gutil/port.h /^const char PATH_SEPARATOR = '\\\\';$/;" v +PATH_SEPARATOR src/gutil/port.h 178;" d +PAYMENT_REQUIRED src/http/http_status.h /^ PAYMENT_REQUIRED = 402,$/;" e enum:doris::HttpStatus +PC_BITMAP_LENGTH src/exprs/aggregate_functions.cpp /^const static int PC_BITMAP_LENGTH = 32; \/\/ the length of each bit map$/;" m namespace:doris file: +PC_THETA src/exprs/aggregate_functions.cpp /^const static float PC_THETA = 0.77351f; \/\/ the magic number to compute the final result$/;" m namespace:doris file: +PENDING_DELTA_PREFIX src/olap/olap_define.h /^static const std::string PENDING_DELTA_PREFIX = "\/pending_delta";$/;" m namespace:doris +PENTIUM src/gutil/cpu.h /^ PENTIUM,$/;" e enum:base::final::IntelMicroArchitecture +PERF_COUNTER_BYTES_READ src/util/perf_counters.h /^ PERF_COUNTER_BYTES_READ,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_BYTES_WRITE src/util/perf_counters.h /^ PERF_COUNTER_BYTES_WRITE,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_DISK_READ src/util/perf_counters.h /^ PERF_COUNTER_DISK_READ,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_DISK_WRITE src/util/perf_counters.h /^ PERF_COUNTER_DISK_WRITE,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_HW_BRANCHES src/util/perf_counters.h /^ PERF_COUNTER_HW_BRANCHES,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_HW_BRANCH_MISSES src/util/perf_counters.h /^ PERF_COUNTER_HW_BRANCH_MISSES,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_HW_BUS_CYCLES src/util/perf_counters.h /^ PERF_COUNTER_HW_BUS_CYCLES,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_HW_CACHE_HIT src/util/perf_counters.h /^ PERF_COUNTER_HW_CACHE_HIT,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_HW_CACHE_MISSES src/util/perf_counters.h /^ PERF_COUNTER_HW_CACHE_MISSES,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_HW_CPU_CYCLES src/util/perf_counters.h /^ PERF_COUNTER_HW_CPU_CYCLES,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_HW_INSTRUCTIONS src/util/perf_counters.h /^ PERF_COUNTER_HW_INSTRUCTIONS,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_RESIDENT_SET_SIZE src/util/perf_counters.h /^ PERF_COUNTER_RESIDENT_SET_SIZE,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_SW_CONTEXT_SWITCHES src/util/perf_counters.h /^ PERF_COUNTER_SW_CONTEXT_SWITCHES,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_SW_CPU_CLOCK src/util/perf_counters.h /^ PERF_COUNTER_SW_CPU_CLOCK,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_SW_CPU_MIGRATIONS src/util/perf_counters.h /^ PERF_COUNTER_SW_CPU_MIGRATIONS,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_SW_PAGE_FAULTS src/util/perf_counters.h /^ PERF_COUNTER_SW_PAGE_FAULTS,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_VM_PEAK_USAGE src/util/perf_counters.h /^ PERF_COUNTER_VM_PEAK_USAGE,$/;" e enum:doris::PerfCounters::Counter +PERF_COUNTER_VM_USAGE src/util/perf_counters.h /^ PERF_COUNTER_VM_USAGE,$/;" e enum:doris::PerfCounters::Counter +PERF_IO_IDX src/util/perf_counters.cpp /^enum PERF_IO_IDX {$/;" g namespace:doris file: +PER_ROW_DATA test/runtime/data_stream_test.cpp /^ static const int PER_ROW_DATA = 8;$/;" m class:doris::DataStreamTest file: +PIN_FAILED_ERROR_MSG_1 src/runtime/spill_sorter.cc /^const string PIN_FAILED_ERROR_MSG_1 = "Failed to pin block for ";$/;" m namespace:doris file: +PIN_FAILED_ERROR_MSG_2 src/runtime/spill_sorter.cc /^const string PIN_FAILED_ERROR_MSG_2 = "-length data needed "$/;" m namespace:doris file: +PInternalServiceImpl src/service/internal_service.cpp /^PInternalServiceImpl::PInternalServiceImpl(ExecEnv* exec_env)$/;" f class:doris::PInternalServiceImpl +PInternalServiceImpl src/service/internal_service.h /^class PInternalServiceImpl : public T {$/;" c namespace:doris +PLAIN_PAGE_HEADER_SIZE src/olap/rowset/segment_v2/plain_page.h /^static const size_t PLAIN_PAGE_HEADER_SIZE = sizeof(uint32_t);$/;" m namespace:doris::segment_v2 +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 337;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 340;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 343;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 346;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 349;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 352;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 355;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 358;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 361;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 364;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 367;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 370;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 373;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 376;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 379;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 382;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 385;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 388;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 391;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 394;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 397;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 400;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 403;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 406;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 409;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 413;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 416;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 419;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 422;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 425;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 429;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 322;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 325;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 328;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 331;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 334;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 337;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 340;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 343;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 346;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 349;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 352;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 355;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 358;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 361;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 364;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 367;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 370;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 373;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 376;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 379;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 382;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 385;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 388;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 391;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 394;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 398;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 401;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 404;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 407;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 410;" d file: +PLATFORM_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 414;" d file: +PLATFORM_WINDOWS src/gutil/sysinfo.cc 32;" d file: +PLAT_amd64_linux src/gutil/valgrind.h 3918;" d +PLAT_amd64_linux src/gutil/valgrind.h 89;" d +PLAT_amd64_linux src/gutil/valgrind.h 98;" d +PLAT_ppc32_aix5 src/gutil/valgrind.h 106;" d +PLAT_ppc32_aix5 src/gutil/valgrind.h 3921;" d +PLAT_ppc32_aix5 src/gutil/valgrind.h 92;" d +PLAT_ppc32_linux src/gutil/valgrind.h 100;" d +PLAT_ppc32_linux src/gutil/valgrind.h 3919;" d +PLAT_ppc32_linux src/gutil/valgrind.h 90;" d +PLAT_ppc64_aix5 src/gutil/valgrind.h 104;" d +PLAT_ppc64_aix5 src/gutil/valgrind.h 3922;" d +PLAT_ppc64_aix5 src/gutil/valgrind.h 93;" d +PLAT_ppc64_linux src/gutil/valgrind.h 102;" d +PLAT_ppc64_linux src/gutil/valgrind.h 3920;" d +PLAT_ppc64_linux src/gutil/valgrind.h 91;" d +PLAT_x86_linux src/gutil/valgrind.h 3917;" d +PLAT_x86_linux src/gutil/valgrind.h 88;" d +PLAT_x86_linux src/gutil/valgrind.h 96;" d +PLUGIN_INSTALL_EARLY src/plugin/plugin.h 22;" d +PLUGIN_NOT_DYNAMIC_INSTALL src/plugin/plugin.h 20;" d +PLUGIN_NOT_DYNAMIC_UNINSTALL src/plugin/plugin.h 21;" d +POINTER_RETRY_ON_EINTR src/gutil/macros.h 281;" d +POPCNT src/util/cpu_info.h /^ static const int64_t POPCNT = (1 << 4);$/;" m class:doris::CpuInfo +POSITIVE_INFINITY src/exec/olap_utils.h /^static const char* POSITIVE_INFINITY = "+oo";$/;" m namespace:doris +POST src/http/http_method.h /^ POST,$/;" e enum:doris::HttpMethod +POS_PARAM_VALIDATE src/olap/rowset/segment_group.cpp 53;" d file: +PRAGMA src/http/http_headers.cpp /^const char* HttpHeaders::PRAGMA = "Pragma";$/;" m class:doris::HttpHeaders file: +PRAGMA src/http/http_headers.h /^ static const char* PRAGMA;$/;" m class:doris::HttpHeaders +PRECISION src/runtime/decimalv2_value.h /^ static const int32_t PRECISION = 27;$/;" m class:doris::DecimalV2Value +PRECISION src/util/debug_util.cpp 31;" d file: +PRECISION src/util/pretty_printer.h /^ static const int PRECISION = 2;$/;" m class:doris::PrettyPrinter +PRECONDITION_FAILED src/http/http_status.h /^ PRECONDITION_FAILED = 412,$/;" e enum:doris::HttpStatus +PREDICT_FALSE src/gutil/port.h 594;" d +PREDICT_FALSE src/gutil/port.h 597;" d +PREDICT_TRUE src/gutil/port.h 595;" d +PREDICT_TRUE src/gutil/port.h 598;" d +PREFER_READING src/util/mutex.h /^ PREFER_READING,$/;" m class:doris::RWMutex::Priority +PREFER_WRITING src/util/mutex.h /^ PREFER_WRITING,$/;" m class:doris::RWMutex::Priority +PREFETCH src/common/compiler_util.h 39;" d +PREFETCH_HINT_NTA src/gutil/port.h /^ PREFETCH_HINT_NTA = 0 \/\/ No temporal locality$/;" e enum:PrefetchHint +PREFETCH_HINT_T0 src/gutil/port.h /^ PREFETCH_HINT_T0 = 3, \/\/ More temporal locality$/;" e enum:PrefetchHint +PREFETCH_HINT_T1 src/gutil/port.h /^ PREFETCH_HINT_T1 = 2,$/;" e enum:PrefetchHint +PREFETCH_HINT_T2 src/gutil/port.h /^ PREFETCH_HINT_T2 = 1, \/\/ Less temporal locality$/;" e enum:PrefetchHint +PREFIX_LENGTH src/olap/olap_meta.cpp /^const size_t PREFIX_LENGTH = 4;$/;" m namespace:doris file: +PRETTY_PRINT_DEBUG_COUNTERS src/util/debug_counters.h 57;" d +PRETTY_PRINT_DEBUG_COUNTERS src/util/debug_counters.h 65;" d +PRETTY_PRINT_WIDTH src/util/perf_counters.cpp 38;" d file: +PRIME test/runtime/buffered_tuple_stream2_test.cpp /^static const uint32_t PRIME = 479001599;$/;" v file: +PRIMITIVE_VISIT src/util/arrow/row_batch.cpp 192;" d file: +PRIMITIVE_VISIT src/util/arrow/row_batch.cpp 204;" d file: +PRIMITIVE_VISIT src/util/arrow/row_batch.cpp 369;" d file: +PRIMITIVE_VISIT src/util/arrow/row_batch.cpp 381;" d file: +PRIMITIVE_VISIT src/util/arrow/row_block.cpp 163;" d file: +PRIMITIVE_VISIT src/util/arrow/row_block.cpp 175;" d file: +PRIMITIVE_VISIT src/util/arrow/row_block.cpp 248;" d file: +PRIMITIVE_VISIT src/util/arrow/row_block.cpp 260;" d file: +PRINTABLE_PTHREAD src/gutil/port.h 1175;" d +PRINTABLE_PTHREAD src/gutil/port.h 1177;" d +PRINTF_ATTRIBUTE src/gutil/port.h 311;" d +PRIORITY_CIDR_SEPARATOR src/service/backend_options.cpp /^static const std::string PRIORITY_CIDR_SEPARATOR = ";";$/;" m namespace:doris file: +PRIXS src/gutil/port.h 1169;" d +PRIdS src/gutil/port.h 1166;" d +PRIoS src/gutil/port.h 1170;" d +PRIuS src/gutil/port.h 1168;" d +PRIxS src/gutil/port.h 1167;" d +PROCESS_FD_NUM_USED src/http/action/metrics_action.cpp /^ static const std::string PROCESS_FD_NUM_USED;$/;" m class:doris::SimpleCoreMetricsVisitor file: +PROCESS_FD_NUM_USED src/http/action/metrics_action.cpp /^const std::string SimpleCoreMetricsVisitor::PROCESS_FD_NUM_USED = "process_fd_num_used";$/;" m class:doris::SimpleCoreMetricsVisitor file: +PROCESS_THREAD_NUM src/http/action/metrics_action.cpp /^ static const std::string PROCESS_THREAD_NUM;$/;" m class:doris::SimpleCoreMetricsVisitor file: +PROCESS_THREAD_NUM src/http/action/metrics_action.cpp /^const std::string SimpleCoreMetricsVisitor::PROCESS_THREAD_NUM = "process_thread_num";$/;" m class:doris::SimpleCoreMetricsVisitor file: +PROC_IO_CANCELLED_WRITE src/util/perf_counters.cpp /^ PROC_IO_CANCELLED_WRITE,$/;" e enum:doris::PERF_IO_IDX file: +PROC_IO_DISK_READ src/util/perf_counters.cpp /^ PROC_IO_DISK_READ,$/;" e enum:doris::PERF_IO_IDX file: +PROC_IO_DISK_WRITE src/util/perf_counters.cpp /^ PROC_IO_DISK_WRITE,$/;" e enum:doris::PERF_IO_IDX file: +PROC_IO_LAST_COUNTER src/util/perf_counters.cpp /^ PROC_IO_LAST_COUNTER,$/;" e enum:doris::PERF_IO_IDX file: +PROC_IO_READ src/util/perf_counters.cpp /^ PROC_IO_READ = 0,$/;" e enum:doris::PERF_IO_IDX file: +PROC_IO_SYS_RREAD src/util/perf_counters.cpp /^ PROC_IO_SYS_RREAD,$/;" e enum:doris::PERF_IO_IDX file: +PROC_IO_SYS_WRITE src/util/perf_counters.cpp /^ PROC_IO_SYS_WRITE,$/;" e enum:doris::PERF_IO_IDX file: +PROC_IO_WRITE src/util/perf_counters.cpp /^ PROC_IO_WRITE,$/;" e enum:doris::PERF_IO_IDX file: +PROC_SELF_IO src/util/perf_counters.h /^ PROC_SELF_IO,$/;" e enum:doris::PerfCounters::DataSource +PROC_SELF_STATUS src/util/perf_counters.h /^ PROC_SELF_STATUS,$/;" e enum:doris::PerfCounters::DataSource +PROPAGATE_POD_FROM_TEMPLATE_ARGUMENT src/gutil/type_traits.h 359;" d +PROTOCOL src/util/url_parser.h /^ PROTOCOL,$/;" e enum:doris::UrlParser::UrlPart +PROXY_AUTHENTICATE src/http/http_headers.cpp /^const char* HttpHeaders::PROXY_AUTHENTICATE = "Proxy-Authenticate";$/;" m class:doris::HttpHeaders file: +PROXY_AUTHENTICATE src/http/http_headers.h /^ static const char* PROXY_AUTHENTICATE;$/;" m class:doris::HttpHeaders +PROXY_AUTHENTICATION src/http/http_status.h /^ PROXY_AUTHENTICATION = 407,$/;" e enum:doris::HttpStatus +PROXY_AUTHORIZATION src/http/http_headers.cpp /^const char* HttpHeaders::PROXY_AUTHORIZATION = "Proxy-Authorization";$/;" m class:doris::HttpHeaders file: +PROXY_AUTHORIZATION src/http/http_headers.h /^ static const char* PROXY_AUTHORIZATION;$/;" m class:doris::HttpHeaders +PR_GET_DUMPABLE src/gutil/linux_syscall_support.h 556;" d +PR_SET_DUMPABLE src/gutil/linux_syscall_support.h 559;" d +PTHREAD_MUTEX_DESTROY_WITH_LOG src/util/mutex.cpp 33;" d file: +PTHREAD_MUTEX_INIT_WITH_LOG src/util/mutex.cpp 25;" d file: +PTHREAD_MUTEX_LOCK_WITH_LOG src/util/mutex.cpp 41;" d file: +PTHREAD_MUTEX_UNLOCK_WITH_LOG src/util/mutex.cpp 49;" d file: +PTHREAD_RWLOCK_DESTROY_WITH_LOG src/util/mutex.cpp 65;" d file: +PTHREAD_RWLOCK_INIT_WITH_LOG src/util/mutex.cpp 57;" d file: +PTHREAD_RWLOCK_RDLOCK_WITH_LOG src/util/mutex.cpp 73;" d file: +PTHREAD_RWLOCK_UNLOCK_WITH_LOG src/util/mutex.cpp 89;" d file: +PTHREAD_RWLOCK_WRLOCK_WITH_LOG src/util/mutex.cpp 81;" d file: +PUBLISH_VERSION src/agent/task_worker_pool.h /^ PUBLISH_VERSION,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +PUBLISH_VERSION_MAX_RETRY src/agent/task_worker_pool.cpp /^const uint32_t PUBLISH_VERSION_MAX_RETRY = 3;$/;" m namespace:doris file: +PUSH src/agent/task_worker_pool.h /^ PUSH,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +PUSH_FOR_DELETE src/olap/olap_common.h /^ PUSH_FOR_DELETE = 2,$/;" e enum:doris::PushType +PUSH_FOR_LOAD_DELETE src/olap/olap_common.h /^ PUSH_FOR_LOAD_DELETE = 3,$/;" e enum:doris::PushType +PUSH_MAX_RETRY src/olap/task/engine_batch_load_task.h /^const uint32_t PUSH_MAX_RETRY = 1;$/;" m namespace:doris +PUSH_MAX_RETRY test/agent/mock_task_worker_pool.h /^const uint32_t PUSH_MAX_RETRY = 1;$/;" m namespace:doris +PUSH_META src/olap/tablet_sync_service.h /^ PUSH_META,$/;" e enum:doris::MetaOpType +PUSH_NORMAL src/olap/olap_common.h /^ PUSH_NORMAL = 1,$/;" e enum:doris::PushType +PUSH_REQUEST_WRITE_BYTES_PER_SECOND src/http/action/metrics_action.cpp /^ static const std::string PUSH_REQUEST_WRITE_BYTES_PER_SECOND;$/;" m class:doris::SimpleCoreMetricsVisitor file: +PUSH_REQUEST_WRITE_BYTES_PER_SECOND src/http/action/metrics_action.cpp /^const std::string SimpleCoreMetricsVisitor::PUSH_REQUEST_WRITE_BYTES_PER_SECOND = "push_request_write_bytes_per_second";$/;" m class:doris::SimpleCoreMetricsVisitor file: +PUT src/http/http_method.h /^ PUT,$/;" e enum:doris::HttpMethod +PackedInt128 src/util/types.h /^ PackedInt128() : value(0) {$/;" f struct:doris::PackedInt128 +PackedInt128 src/util/types.h /^ PackedInt128(const __int128& value_) {$/;" f struct:doris::PackedInt128 +PackedInt128 src/util/types.h /^struct PackedInt128 {$/;" s namespace:doris +Page src/olap/rowset/segment_v2/column_writer.h /^ struct Page {$/;" s class:doris::segment_v2::ColumnWriter +Page src/runtime/buffered_tuple_stream3.h /^ Page() : num_rows(0), retrieved_buffer(true) {}$/;" f struct:doris::BufferedTupleStream3::Page +Page src/runtime/buffered_tuple_stream3.h /^ struct Page {$/;" s class:doris::BufferedTupleStream3 +Page src/runtime/bufferpool/buffer_pool_internal.h /^ Page(Client* client, int64_t len)$/;" f class:doris::BufferPool::Page +Page src/runtime/bufferpool/buffer_pool_internal.h /^class BufferPool::Page : public InternalList::Node {$/;" c class:doris::BufferPool +PageBuilder src/olap/rowset/segment_v2/page_builder.h /^ PageBuilder() { }$/;" f class:doris::segment_v2::PageBuilder +PageBuilder src/olap/rowset/segment_v2/page_builder.h /^class PageBuilder {$/;" c namespace:doris::segment_v2 +PageBuilderOptions src/olap/rowset/segment_v2/options.h /^struct PageBuilderOptions {$/;" s namespace:doris::segment_v2 +PageCacheHandle src/olap/page_cache.h /^ PageCacheHandle() { }$/;" f class:doris::PageCacheHandle +PageCacheHandle src/olap/page_cache.h /^ PageCacheHandle(Cache* cache, Cache::Handle* handle) : _cache(cache), _handle(handle) { }$/;" f class:doris::PageCacheHandle +PageCacheHandle src/olap/page_cache.h /^class PageCacheHandle {$/;" c namespace:doris +PageDecoder src/olap/rowset/segment_v2/page_decoder.h /^ PageDecoder() { }$/;" f class:doris::segment_v2::PageDecoder +PageDecoder src/olap/rowset/segment_v2/page_decoder.h /^class PageDecoder {$/;" c namespace:doris::segment_v2 +PageDecoderOptions src/olap/rowset/segment_v2/options.h /^struct PageDecoderOptions {$/;" s namespace:doris::segment_v2 +PageHandle src/olap/rowset/segment_v2/page_handle.h /^ PageHandle() : _is_data_owner(false) { }$/;" f class:doris::segment_v2::PageHandle +PageHandle src/olap/rowset/segment_v2/page_handle.h /^ PageHandle(PageCacheHandle cache_data)$/;" f class:doris::segment_v2::PageHandle +PageHandle src/olap/rowset/segment_v2/page_handle.h /^ PageHandle(const Slice& data) : _is_data_owner(true), _data(data) { }$/;" f class:doris::segment_v2::PageHandle +PageHandle src/olap/rowset/segment_v2/page_handle.h /^class PageHandle {$/;" c namespace:doris::segment_v2 +PageHandle src/runtime/bufferpool/buffer_pool.cc /^BufferPool::PageHandle::PageHandle() {$/;" f class:doris::BufferPool::PageHandle +PageHandle src/runtime/bufferpool/buffer_pool.cc /^BufferPool::PageHandle::PageHandle(PageHandle&& src) {$/;" f class:doris::BufferPool::PageHandle +PageHandle src/runtime/bufferpool/buffer_pool.h /^class BufferPool::PageHandle {$/;" c class:doris::BufferPool +PageHandlerCallback src/http/web_page_handler.h /^ PageHandlerCallback;$/;" t class:doris::WebPageHandler +PageHandlers src/http/web_page_handler.h /^ class PageHandlers {$/;" c class:doris::WebPageHandler +PageHandlersMap src/http/web_page_handler.h /^ typedef std::map PageHandlersMap;$/;" t class:doris::WebPageHandler +PageHead src/olap/rowset/segment_v2/column_writer.h /^ struct PageHead {$/;" s class:doris::segment_v2::ColumnWriter +PageIO src/olap/rowset/segment_v2/page_io.h /^class PageIO {$/;" c namespace:doris::segment_v2 +PageList src/runtime/bufferpool/buffer_pool_internal.h /^ PageList() : bytes_(0) {}$/;" f class:doris::BufferPool::PageList +PageList src/runtime/bufferpool/buffer_pool_internal.h /^class BufferPool::PageList {$/;" c class:doris::BufferPool +PagePointer src/olap/rowset/segment_v2/page_pointer.h /^ PagePointer() : offset(0), size(0) { }$/;" f struct:doris::segment_v2::PagePointer +PagePointer src/olap/rowset/segment_v2/page_pointer.h /^ PagePointer(const PagePointerPB& from) : offset(from.offset()), size(from.size()) { }$/;" f struct:doris::segment_v2::PagePointer +PagePointer src/olap/rowset/segment_v2/page_pointer.h /^ PagePointer(uint64_t offset_, uint32_t size_) : offset(offset_), size(size_) { }$/;" f struct:doris::segment_v2::PagePointer +PagePointer src/olap/rowset/segment_v2/page_pointer.h /^struct PagePointer {$/;" s namespace:doris::segment_v2 +PageReadOptions src/olap/rowset/segment_v2/page_io.h /^struct PageReadOptions {$/;" s namespace:doris::segment_v2 +ParquetFile src/exec/parquet_reader.cpp /^ParquetFile::ParquetFile(FileReader *file): _file(file) {$/;" f class:doris::ParquetFile +ParquetFile src/exec/parquet_reader.h /^class ParquetFile : public arrow::io::RandomAccessFile {$/;" c namespace:doris +ParquetReaderWrap src/exec/parquet_reader.cpp /^ParquetReaderWrap::ParquetReaderWrap(FileReader *file_reader, int32_t num_of_columns_from_file) :$/;" f class:doris::ParquetReaderWrap +ParquetReaderWrap src/exec/parquet_reader.h /^class ParquetReaderWrap {$/;" c namespace:doris +ParquetSannerTest test/exec/parquet_scanner_test.cpp /^ ParquetSannerTest() : _runtime_state(TQueryGlobals()) {$/;" f class:doris::ParquetSannerTest +ParquetSannerTest test/exec/parquet_scanner_test.cpp /^class ParquetSannerTest : public testing::Test {$/;" c namespace:doris file: +ParquetScanner src/exec/parquet_scanner.cpp /^ParquetScanner::ParquetScanner(RuntimeState* state,$/;" f class:doris::ParquetScanner +ParquetScanner src/exec/parquet_scanner.h /^class ParquetScanner : public BaseScanner {$/;" c namespace:doris +ParseCPUFlags src/util/cpu_info.cpp /^int64_t ParseCPUFlags(const string& values) {$/;" f namespace:doris +ParseDoubleRange src/gutil/strings/numbers.cc /^bool ParseDoubleRange(const char* text, int len, const char** end,$/;" f +ParseLeadingBoolValue src/gutil/strings/numbers.cc /^bool ParseLeadingBoolValue(const char *str, bool deflt) {$/;" f +ParseLeadingBoolValue src/gutil/strings/numbers.h /^inline bool ParseLeadingBoolValue(const string& str, bool deflt) {$/;" f +ParseLeadingDec32Value src/gutil/strings/numbers.cc /^int32 ParseLeadingDec32Value(const char *str, int32 deflt) {$/;" f +ParseLeadingDec32Value src/gutil/strings/numbers.h /^inline int32 ParseLeadingDec32Value(const string& str, int32 deflt) {$/;" f +ParseLeadingDec64Value src/gutil/strings/numbers.cc /^int64 ParseLeadingDec64Value(const char *str, int64 deflt) {$/;" f +ParseLeadingDec64Value src/gutil/strings/numbers.h /^inline int64 ParseLeadingDec64Value(const string& str, int64 deflt) {$/;" f +ParseLeadingDoubleValue src/gutil/strings/numbers.cc /^double ParseLeadingDoubleValue(const char *str, double deflt) {$/;" f +ParseLeadingDoubleValue src/gutil/strings/numbers.h /^inline double ParseLeadingDoubleValue(const string& str, double deflt) {$/;" f +ParseLeadingHex64Value src/gutil/strings/numbers.cc /^uint64 ParseLeadingHex64Value(const char *str, uint64 deflt) {$/;" f +ParseLeadingHex64Value src/gutil/strings/numbers.h /^inline uint64 ParseLeadingHex64Value(const string& str, uint64 deflt) {$/;" f +ParseLeadingInt32Value src/gutil/strings/numbers.cc /^int32 ParseLeadingInt32Value(const char *str, int32 deflt) {$/;" f +ParseLeadingInt32Value src/gutil/strings/numbers.h /^inline int32 ParseLeadingInt32Value(const string& str, int32 deflt) {$/;" f +ParseLeadingInt64Value src/gutil/strings/numbers.cc /^int64 ParseLeadingInt64Value(const char *str, int64 deflt) {$/;" f +ParseLeadingInt64Value src/gutil/strings/numbers.h /^inline int64 ParseLeadingInt64Value(const string& str, int64 deflt) {$/;" f +ParseLeadingUDec32Value src/gutil/strings/numbers.cc /^uint32 ParseLeadingUDec32Value(const char *str, uint32 deflt) {$/;" f +ParseLeadingUDec32Value src/gutil/strings/numbers.h /^inline uint32 ParseLeadingUDec32Value(const string& str, uint32 deflt) {$/;" f +ParseLeadingUDec64Value src/gutil/strings/numbers.cc /^uint64 ParseLeadingUDec64Value(const char *str, uint64 deflt) {$/;" f +ParseLeadingUDec64Value src/gutil/strings/numbers.h /^inline uint64 ParseLeadingUDec64Value(const string& str, uint64 deflt) {$/;" f +ParseLeadingUInt32Value src/gutil/strings/numbers.cc /^uint32 ParseLeadingUInt32Value(const char *str, uint32 deflt) {$/;" f +ParseLeadingUInt32Value src/gutil/strings/numbers.h /^inline uint32 ParseLeadingUInt32Value(const string& str, uint32 deflt) {$/;" f +ParseLeadingUInt64Value src/gutil/strings/numbers.cc /^uint64 ParseLeadingUInt64Value(const char *str, uint64 deflt) {$/;" f +ParseLeadingUInt64Value src/gutil/strings/numbers.h /^inline uint64 ParseLeadingUInt64Value(const string& str, uint64 deflt) {$/;" f +ParseMaxCpuIndex src/gutil/sysinfo.cc /^int ParseMaxCpuIndex(const char* str) {$/;" f namespace:base +ParseResult src/util/string_parser.hpp /^ enum ParseResult {$/;" g class:doris::StringParser +ParseState src/http/http_parser.h /^ enum ParseState {$/;" g class:doris::HttpParser +ParseUtil src/util/parse_util.h /^class ParseUtil {$/;" c namespace:doris +ParsedPage src/olap/rowset/segment_v2/parsed_page.h /^struct ParsedPage {$/;" s namespace:doris::segment_v2 +PartRange src/runtime/dpp_sink_internal.h /^ PartRange() {$/;" f class:doris::PartRange +PartRange src/runtime/dpp_sink_internal.h /^ PartRange(const PartRangeKey& start_key, const PartRangeKey& end_key,$/;" f class:doris::PartRange +PartRange src/runtime/dpp_sink_internal.h /^class PartRange {$/;" c namespace:doris +PartRangeKey src/runtime/dpp_sink_internal.h /^ PartRangeKey() {$/;" f class:doris::PartRangeKey +PartRangeKey src/runtime/dpp_sink_internal.h /^ PartRangeKey(int sign) : _sign(sign) { }$/;" f class:doris::PartRangeKey +PartRangeKey src/runtime/dpp_sink_internal.h /^class PartRangeKey {$/;" c namespace:doris +Partition src/exec/new_partitioned_aggregation_node.h /^ Partition(NewPartitionedAggregationNode* parent, int level, int idx)$/;" f struct:doris::NewPartitionedAggregationNode::Partition +Partition src/exec/new_partitioned_aggregation_node.h /^ struct Partition {$/;" s class:doris::NewPartitionedAggregationNode +Partition src/exec/partitioned_aggregation_node.h /^ Partition(PartitionedAggregationNode* parent, int level) :$/;" f struct:doris::PartitionedAggregationNode::Partition +Partition src/exec/partitioned_aggregation_node.h /^ struct Partition {$/;" s class:doris::PartitionedAggregationNode +PartitionInfo src/runtime/dpp_sink_internal.h /^ PartitionInfo() : _id(-1), _distributed_bucket(0) {$/;" f class:doris::PartitionInfo +PartitionInfo src/runtime/dpp_sink_internal.h /^class PartitionInfo {$/;" c namespace:doris +PartitionedAggregationNode src/exec/partitioned_aggregation_node.cc /^PartitionedAggregationNode::PartitionedAggregationNode($/;" f class:doris::PartitionedAggregationNode +PartitionedAggregationNode src/exec/partitioned_aggregation_node.h /^class PartitionedAggregationNode : public ExecNode {$/;" c namespace:doris +PartitionedHashTable src/exec/partitioned_hash_table.cc /^PartitionedHashTable::PartitionedHashTable(bool quadratic_probing, RuntimeState* state,$/;" f class:doris::PartitionedHashTable +PartitionedHashTable src/exec/partitioned_hash_table.h /^class PartitionedHashTable {$/;" c namespace:doris +PartitionedHashTableCtx src/exec/partitioned_hash_table.cc /^PartitionedHashTableCtx::PartitionedHashTableCtx($/;" f class:doris::PartitionedHashTableCtx +PartitionedHashTableCtx src/exec/partitioned_hash_table.h /^class PartitionedHashTableCtx {$/;" c namespace:doris +PartitionedHashTableTest test/exec/partitioned_hash_table_test.cpp /^ PartitionedHashTableTest() : _limit(-1), _mem_pool(&_limit) {}$/;" f class:doris::PartitionedHashTableTest +PartitionedHashTableTest test/exec/partitioned_hash_table_test.cpp /^class PartitionedHashTableTest : public testing::Test {$/;" c namespace:doris file: +PassAs src/gutil/gscoped_ptr.h /^ gscoped_ptr PassAs() {$/;" f class:gscoped_ptr +PatchedBaseHead src/olap/rowset/run_length_integer_writer.h /^ struct PatchedBaseHead {$/;" s class:doris::RunLengthIntegerWriter +PathBuilder src/util/path_builder.h /^class PathBuilder {$/;" c namespace:doris +PathTrie src/util/path_trie.hpp /^ PathTrie() :$/;" f class:doris::PathTrie +PathTrie src/util/path_trie.hpp /^class PathTrie {$/;" c namespace:doris +PathTrieTest test/util/path_trie_test.cpp /^class PathTrieTest : public testing::Test {$/;" c namespace:doris file: +PauseCPU src/gutil/atomicops-internals-tsan.h /^inline void PauseCPU() {$/;" f namespace:base::subtle +PauseCPU src/gutil/atomicops-internals-x86.h /^inline void PauseCPU() {$/;" f namespace:base::subtle +PerDiskState src/runtime/disk_io_mgr_internal.h /^ PerDiskState() {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +PerDiskState src/runtime/disk_io_mgr_internal.h /^ class PerDiskState {$/;" c class:doris::DiskIoMgr::RequestContext +PerNodeScanRanges src/runtime/plan_fragment_executor.h /^ typedef std::map > PerNodeScanRanges;$/;" t class:doris::PlanFragmentExecutor +PerSizeLists src/runtime/bufferpool/buffer_allocator.cc /^ PerSizeLists() : num_free_buffers(0), low_water_mark(0), num_clean_pages(0) {}$/;" f struct:doris::BufferPool::FreeBufferArena::PerSizeLists +PerSizeLists src/runtime/bufferpool/buffer_allocator.cc /^ struct PerSizeLists {$/;" s class:doris::BufferPool::FreeBufferArena file: +PercentileApproxState src/exprs/aggregate_functions.cpp /^ PercentileApproxState() : digest(new TDigest()) {}$/;" f struct:doris::PercentileApproxState +PercentileApproxState src/exprs/aggregate_functions.cpp /^ PercentileApproxState(double compression) : digest(new TDigest(compression)) {}$/;" f struct:doris::PercentileApproxState +PercentileApproxState src/exprs/aggregate_functions.cpp /^struct PercentileApproxState {$/;" s namespace:doris file: +PercentileApproxTest test/exprs/percentile_approx_test.cpp /^ PercentileApproxTest() {}$/;" f class:doris::PercentileApproxTest +PercentileApproxTest test/exprs/percentile_approx_test.cpp /^class PercentileApproxTest : public testing::Test {$/;" c namespace:doris file: +PerfCounters src/util/perf_counters.cpp /^PerfCounters::PerfCounters() : _group_fd(-1) {$/;" f class:doris::PerfCounters +PerfCounters src/util/perf_counters.h /^class PerfCounters {$/;" c namespace:doris +PeriodicCounterType src/util/runtime_profile.h /^ enum PeriodicCounterType {$/;" g class:doris::RuntimeProfile +PeriodicCounterUpdateState src/util/runtime_profile.cpp /^RuntimeProfile::PeriodicCounterUpdateState::PeriodicCounterUpdateState() : _done(false) {$/;" f class:doris::RuntimeProfile::PeriodicCounterUpdateState +PeriodicCounterUpdateState src/util/runtime_profile.h /^ struct PeriodicCounterUpdateState {$/;" s class:doris::RuntimeProfile +Pin src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::Pin(ClientHandle* client, PageHandle* handle) {$/;" f class:doris::BufferPool +PinBlocks test/runtime/buffered_block_mgr2_test.cpp /^ void PinBlocks(const vector& blocks) {$/;" f class:doris::BufferedBlockMgrTest +PinPage src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::PinPage(Page* page) {$/;" f class:BufferedTupleStream3 +PinPageIfNeeded src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::PinPageIfNeeded(Page* page, bool stream_pinned) {$/;" f class:BufferedTupleStream3 +PinStream src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::PinStream(bool* pinned) {$/;" f class:BufferedTupleStream3 +PlTaskRoot src/exec/pl_task_root.h /^class PlTaskRoot : public ExecNode {$/;" c namespace:doris +PlainPageBuilder src/olap/rowset/segment_v2/plain_page.h /^ PlainPageBuilder(const PageBuilderOptions& options) :$/;" f class:doris::segment_v2::PlainPageBuilder +PlainPageBuilder src/olap/rowset/segment_v2/plain_page.h /^class PlainPageBuilder : public PageBuilder {$/;" c namespace:doris::segment_v2 +PlainPageDecoder src/olap/rowset/segment_v2/plain_page.h /^ PlainPageDecoder(Slice data, const PageDecoderOptions& options) : _data(data),$/;" f class:doris::segment_v2::PlainPageDecoder +PlainPageDecoder src/olap/rowset/segment_v2/plain_page.h /^class PlainPageDecoder : public PageDecoder {$/;" c namespace:doris::segment_v2 +PlainPageTest test/olap/rowset/segment_v2/plain_page_test.cpp /^ PlainPageTest() {}$/;" f class:doris::segment_v2::PlainPageTest +PlainPageTest test/olap/rowset/segment_v2/plain_page_test.cpp /^class PlainPageTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +PlainTextLineReader src/exec/plain_text_line_reader.cpp /^PlainTextLineReader::PlainTextLineReader($/;" f class:doris::PlainTextLineReader +PlainTextLineReader src/exec/plain_text_line_reader.h /^class PlainTextLineReader : public LineReader {$/;" c namespace:doris +PlainTextLineReaderTest test/exec/plain_text_line_reader_bzip_test.cpp /^ PlainTextLineReaderTest() : _profile(&_obj_pool, "TestProfile") {$/;" f class:doris::PlainTextLineReaderTest +PlainTextLineReaderTest test/exec/plain_text_line_reader_bzip_test.cpp /^class PlainTextLineReaderTest : public testing::Test {$/;" c namespace:doris file: +PlainTextLineReaderTest test/exec/plain_text_line_reader_gzip_test.cpp /^ PlainTextLineReaderTest() : _profile(&_obj_pool, "TestProfile") {$/;" f class:doris::PlainTextLineReaderTest +PlainTextLineReaderTest test/exec/plain_text_line_reader_gzip_test.cpp /^class PlainTextLineReaderTest : public testing::Test {$/;" c namespace:doris file: +PlainTextLineReaderTest test/exec/plain_text_line_reader_lz4frame_test.cpp /^ PlainTextLineReaderTest() : _profile(&_obj_pool, "TestProfile") {$/;" f class:doris::PlainTextLineReaderTest +PlainTextLineReaderTest test/exec/plain_text_line_reader_lz4frame_test.cpp /^class PlainTextLineReaderTest : public testing::Test {$/;" c namespace:doris file: +PlainTextLineReaderTest test/exec/plain_text_line_reader_lzop_test.cpp /^ PlainTextLineReaderTest() : _profile(&_obj_pool, "TestProfile") {$/;" f class:doris::PlainTextLineReaderTest +PlainTextLineReaderTest test/exec/plain_text_line_reader_lzop_test.cpp /^class PlainTextLineReaderTest : public testing::Test {$/;" c namespace:doris file: +PlainTextLineReaderTest test/exec/plain_text_line_reader_uncompressed_test.cpp /^ PlainTextLineReaderTest() : _profile(&_obj_pool, "TestProfile") {$/;" f class:doris::PlainTextLineReaderTest +PlainTextLineReaderTest test/exec/plain_text_line_reader_uncompressed_test.cpp /^class PlainTextLineReaderTest : public testing::Test {$/;" c namespace:doris file: +PlanFragmentExecutor src/runtime/plan_fragment_executor.cpp /^PlanFragmentExecutor::PlanFragmentExecutor($/;" f class:doris::PlanFragmentExecutor +PlanFragmentExecutor src/runtime/plan_fragment_executor.h /^class PlanFragmentExecutor {$/;" c namespace:doris +PlanFragmentExecutor test/runtime/fragment_mgr_test.cpp /^PlanFragmentExecutor::PlanFragmentExecutor(ExecEnv* exec_env, $/;" f class:doris::PlanFragmentExecutor +PlanNodeId src/common/global_types.h /^typedef int PlanNodeId;$/;" t namespace:doris +Plugin src/plugin/plugin.h /^struct Plugin {$/;" s namespace:doris +PluginManager src/plugin/plugin_manager.h /^class PluginManager {$/;" c namespace:doris +PmuProfileAction src/http/action/pprof_actions.cpp /^ PmuProfileAction() {}$/;" f class:doris::PmuProfileAction +PmuProfileAction src/http/action/pprof_actions.cpp /^class PmuProfileAction : public HttpHandler {$/;" c namespace:doris file: +Poison src/runtime/bufferpool/buffer_pool.h /^ void Poison() { ASAN_POISON_MEMORY_REGION(data(), len()); }$/;" f class:doris::BufferPool::BufferHandle +PoolMemTrackerRegistry src/runtime/mem_tracker.h /^class PoolMemTrackerRegistry {$/;" c namespace:doris +PoolTrackersMap src/runtime/mem_tracker.h /^ typedef std::unordered_map> PoolTrackersMap;$/;" t class:doris::PoolMemTrackerRegistry +Pools src/runtime/thread_resource_mgr.h /^ typedef std::set Pools;$/;" t class:doris::ThreadResourceMgr +PopFreeBuffer src/runtime/bufferpool/buffer_allocator.cc /^bool BufferPool::FreeBufferArena::PopFreeBuffer($/;" f class:doris::BufferPool::FreeBufferArena +PopFreeBuffer src/runtime/bufferpool/free_list.h /^ bool PopFreeBuffer(BufferHandle* buffer) {$/;" f class:doris::FreeList +PopFreeListHead src/runtime/bufferpool/suballocator.cc /^unique_ptr Suballocator::PopFreeListHead(int list_idx) {$/;" f class:doris::Suballocator +Popcount src/util/bit_util.h /^ static inline int Popcount(uint64_t x) {$/;" f class:doris::BitUtil +PopcountNoHw src/util/bit_util.h /^ static inline int PopcountNoHw(uint64_t x) {$/;" f class:doris::BitUtil +PopcountSigned src/util/bit_util.h /^ static inline int PopcountSigned(T v) {$/;" f class:doris::BitUtil +PortableHashBase src/gutil/port.h /^struct PortableHashBase { };$/;" s +PortableSafeGmtime src/gutil/strings/util.cc /^static struct tm* PortableSafeGmtime(const time_t* timep, struct tm* result) {$/;" f file: +PositionEntryReader src/olap/stream_index_reader.cpp /^PositionEntryReader::PositionEntryReader() :$/;" f class:doris::PositionEntryReader +PositionEntryReader src/olap/stream_index_reader.h /^class PositionEntryReader {$/;" c namespace:doris +PositionEntryWriter src/olap/stream_index_writer.cpp /^PositionEntryWriter::PositionEntryWriter() : _positions_count(0), _statistics_size(0) {$/;" f class:doris::PositionEntryWriter +PositionEntryWriter src/olap/stream_index_writer.h /^class PositionEntryWriter {$/;" c namespace:doris +PositionProvider src/olap/stream_index_reader.h /^ PositionProvider() : _entry(NULL), _index(0) {$/;" f class:doris::PositionProvider +PositionProvider src/olap/stream_index_reader.h /^ explicit PositionProvider(const PositionEntryReader* entry)$/;" f class:doris::PositionProvider +PositionProvider src/olap/stream_index_reader.h /^class PositionProvider {$/;" c namespace:doris +PosixEnv src/env/env_posix.cpp /^class PosixEnv : public Env {$/;" c namespace:doris file: +PosixRandomAccessFile src/env/env_posix.cpp /^ PosixRandomAccessFile(std::string filename, int fd) : _filename(std::move(filename)), _fd(fd) {$/;" f class:doris::PosixRandomAccessFile +PosixRandomAccessFile src/env/env_posix.cpp /^class PosixRandomAccessFile : public RandomAccessFile {$/;" c namespace:doris file: +PosixRandomRWFile src/env/env_posix.cpp /^ PosixRandomRWFile(string fname, int fd, bool sync_on_close)$/;" f class:doris::PosixRandomRWFile +PosixRandomRWFile src/env/env_posix.cpp /^class PosixRandomRWFile : public RandomRWFile {$/;" c namespace:doris file: +PosixSequentialFile src/env/env_posix.cpp /^ PosixSequentialFile(string fname, FILE* f)$/;" f class:doris::PosixSequentialFile +PosixSequentialFile src/env/env_posix.cpp /^class PosixSequentialFile: public SequentialFile {$/;" c namespace:doris file: +PosixWritableFile src/env/env_posix.cpp /^ PosixWritableFile(std::string filename, int fd, uint64_t filesize, bool sync_on_close)$/;" f class:doris::PosixWritableFile +PosixWritableFile src/env/env_posix.cpp /^class PosixWritableFile : public WritableFile {$/;" c namespace:doris file: +PprofActions src/http/action/pprof_actions.h /^class PprofActions {$/;" c namespace:doris +PreAggregationNode src/exec/pre_aggregation_node.cpp /^PreAggregationNode::PreAggregationNode(ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::PreAggregationNode +PreAggregationNode src/exec/pre_aggregation_node.h /^class PreAggregationNode : public ExecNode {$/;" c namespace:doris +Predicate src/exprs/predicate.h /^ Predicate(const TExprNode& node) : Expr(node) {}$/;" f class:doris::Predicate +Predicate src/exprs/predicate.h /^class Predicate: public Expr {$/;" c namespace:doris +PrefetchBucket src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTable::Iterator::PrefetchBucket() {$/;" f class:doris::NewPartitionedHashTable::Iterator +PrefetchBucket src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTable::PrefetchBucket(uint32_t hash) {$/;" f class:doris::NewPartitionedHashTable +PrefetchHint src/gutil/port.h /^enum PrefetchHint {$/;" g +PrefixSuccessor src/gutil/strings/util.cc /^string PrefixSuccessor(const StringPiece& prefix) {$/;" f +PrepareBucketForInsert src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTable::PrepareBucketForInsert(int64_t bucket_idx, uint32_t hash) {$/;" f class:doris::NewPartitionedHashTable +PrepareForRead src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::PrepareForRead(bool delete_on_read, bool* got_reservation) {$/;" f class:BufferedTupleStream3 +PrepareForReadInternal src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::PrepareForReadInternal(bool delete_on_read) {$/;" f class:BufferedTupleStream3 +PrepareForReadWrite src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::PrepareForReadWrite($/;" f class:BufferedTupleStream3 +PrepareForWrite src/runtime/buffered_tuple_stream3.cc /^Status BufferedTupleStream3::PrepareForWrite(bool* got_reservation) {$/;" f class:BufferedTupleStream3 +PrepareToAllocateBuffer src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::Client::PrepareToAllocateBuffer(int64_t len) {$/;" f class:doris::BufferPool::Client +PrettyPrinter src/util/pretty_printer.h /^class PrettyPrinter {$/;" c namespace:doris +Prev src/olap/skiplist.h /^inline void SkipList::Iterator::Prev() {$/;" f class:doris::SkipList::Iterator +PrimitiveType src/runtime/primitive_type.h /^enum PrimitiveType {$/;" g namespace:doris +PrintStats src/exec/new_partitioned_hash_table.cc /^string NewPartitionedHashTable::PrintStats() const {$/;" f class:NewPartitionedHashTable +Priority src/util/mutex.h /^ enum class Priority {$/;" c class:doris::RWMutex +PriorityThreadPool src/util/priority_thread_pool.hpp /^ PriorityThreadPool(uint32_t num_threads, uint32_t queue_size) :$/;" f class:doris::PriorityThreadPool +PriorityThreadPool src/util/priority_thread_pool.hpp /^class PriorityThreadPool {$/;" c namespace:doris +Probe src/exec/new_partitioned_hash_table.inline.h /^inline int64_t NewPartitionedHashTable::Probe(Bucket* buckets, int64_t num_buckets,$/;" f class:doris::NewPartitionedHashTable +ProbeTest test/exec/partitioned_hash_table_test.cpp /^ void ProbeTest(PartitionedHashTable* table, PartitionedHashTableCtx* ht_ctx,$/;" f class:doris::PartitionedHashTableTest +ProbeTestData test/exec/hash_table_test.cpp /^ struct ProbeTestData {$/;" s class:doris::HashTableTest file: +ProbeTestData test/exec/partitioned_hash_table_test.cpp /^ struct ProbeTestData {$/;" s class:doris::PartitionedHashTableTest file: +ProcessBatch src/exec/new_partitioned_aggregation_node_ir.cc /^Status NewPartitionedAggregationNode::ProcessBatch(RowBatch* batch,$/;" f class:NewPartitionedAggregationNode +ProcessBatchFn src/exec/new_partitioned_aggregation_node.h /^ typedef Status (*ProcessBatchFn)($/;" t class:doris::NewPartitionedAggregationNode +ProcessBatchNoGrouping src/exec/new_partitioned_aggregation_node_ir.cc /^Status NewPartitionedAggregationNode::ProcessBatchNoGrouping(RowBatch* batch) {$/;" f class:NewPartitionedAggregationNode +ProcessBatchNoGroupingFn src/exec/new_partitioned_aggregation_node.h /^ typedef Status (*ProcessBatchNoGroupingFn)(NewPartitionedAggregationNode*, RowBatch*);$/;" t class:doris::NewPartitionedAggregationNode +ProcessBatchStreaming src/exec/new_partitioned_aggregation_node_ir.cc /^Status NewPartitionedAggregationNode::ProcessBatchStreaming(bool needs_serialize,$/;" f class:NewPartitionedAggregationNode +ProcessBatchStreamingFn src/exec/new_partitioned_aggregation_node.h /^ typedef Status (*ProcessBatchStreamingFn)(NewPartitionedAggregationNode*, bool,$/;" t class:doris::NewPartitionedAggregationNode +ProcessBuildBatchFn src/exec/hash_join_node.h /^ typedef void (*ProcessBuildBatchFn)(HashJoinNode*, RowBatch*);$/;" t class:doris::HashJoinNode +ProcessProbeBatchFn src/exec/hash_join_node.h /^ typedef int (*ProcessProbeBatchFn)(HashJoinNode*, RowBatch*, RowBatch*, int);$/;" t class:doris::HashJoinNode +ProcessRow src/exec/new_partitioned_aggregation_node_ir.cc /^Status NewPartitionedAggregationNode::ProcessRow(TupleRow* row,$/;" f class:NewPartitionedAggregationNode +ProcessRowBatchFn src/exec/aggregation_node.h /^ typedef void (*ProcessRowBatchFn)(AggregationNode*, RowBatch*);$/;" t class:doris::AggregationNode +ProcessRowBatchFn src/exec/partitioned_aggregation_node.h /^ typedef Status (*ProcessRowBatchFn)($/;" t class:doris::PartitionedAggregationNode +ProcessStream src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::ProcessStream(BufferedTupleStream3* input_stream) {$/;" f class:doris::NewPartitionedAggregationNode +ProducerThread test/util/internal_queue_test.cpp /^void ProducerThread($/;" f namespace:doris +ProfileAction src/http/action/pprof_actions.cpp /^ ProfileAction() {}$/;" f class:doris::ProfileAction +ProfileAction src/http/action/pprof_actions.cpp /^class ProfileAction : public HttpHandler {$/;" c namespace:doris file: +ProgressUpdater src/util/progress_updater.cpp /^ProgressUpdater::ProgressUpdater() :$/;" f class:doris::ProgressUpdater +ProgressUpdater src/util/progress_updater.cpp /^ProgressUpdater::ProgressUpdater(const std::string& label, int64_t total, int period) :$/;" f class:doris::ProgressUpdater +ProgressUpdater src/util/progress_updater.h /^class ProgressUpdater {$/;" c namespace:doris +PrometheusMetricsVisitor src/http/action/metrics_action.cpp /^class PrometheusMetricsVisitor : public MetricsVisitor {$/;" c namespace:doris file: +Properties src/common/configbase.h /^class Properties {$/;" c namespace:doris::config +PublishTimeout src/common/status.h /^ static Status PublishTimeout(const Slice& msg, int16_t precise_code = 1, const Slice& msg2 = Slice()) {$/;" f class:doris::Status +PushHandler src/olap/push_handler.h /^ PushHandler() {}$/;" f class:doris::PushHandler +PushHandler src/olap/push_handler.h /^class PushHandler {$/;" c namespace:doris +PushRowsetMetaTask src/olap/tablet_sync_service.h /^struct PushRowsetMetaTask {$/;" s namespace:doris +PushSpilledPartition src/exec/new_partitioned_aggregation_node.cc /^void NewPartitionedAggregationNode::PushSpilledPartition(Partition* partition) {$/;" f class:doris::NewPartitionedAggregationNode +PushTabletMetaTask src/olap/tablet_sync_service.h /^struct PushTabletMetaTask {$/;" s namespace:doris +PushType src/olap/olap_common.h /^enum PushType {$/;" g namespace:doris +Pusher src/agent/pusher.cpp /^Pusher::Pusher(OLAPEngine* engine, const TPushReq& push_req) :$/;" f class:doris::Pusher +Put src/util/rle_encoding.h /^inline void RleEncoder::Put(T value, size_t run_length) {$/;" f class:doris::RleEncoder +PutAligned src/util/bit_stream_utils.inline.h /^inline void BitWriter::PutAligned(T val, int num_bytes) {$/;" f class:doris::BitWriter +PutFixed32 test/olap/lru_cache_test.cpp /^void PutFixed32(std::string* dst, uint32_t value) {$/;" f namespace:doris +PutTwoDigits src/gutil/strings/util.cc /^static inline void PutTwoDigits(int i, char* p) {$/;" f file: +PutValue src/util/bit_stream_utils.inline.h /^inline void BitWriter::PutValue(uint64_t v, int num_bits) {$/;" f class:doris::BitWriter +PutVlqInt src/util/bit_stream_utils.inline.h /^inline void BitWriter::PutVlqInt(int32_t v) {$/;" f class:doris::BitWriter +QEncodingUnescape src/gutil/strings/escaping.cc /^int QEncodingUnescape(const char *source, int slen,$/;" f namespace:strings +QSorter src/runtime/qsorter.cpp /^QSorter::QSorter($/;" f class:doris::QSorter +QSorter src/runtime/qsorter.h /^class QSorter : public Sorter {$/;" c namespace:doris +QSorterTest test/runtime/qsorter_test.cpp /^ QSorterTest() {$/;" f class:doris::QSorterTest +QSorterTest test/runtime/qsorter_test.cpp /^class QSorterTest : public testing::Test {$/;" c namespace:doris file: +QUARTER src/runtime/datetime_value.h /^ QUARTER,$/;" e enum:doris::TimeUnit +QUERY src/util/url_parser.h /^ QUERY,$/;" e enum:doris::UrlParser::UrlPart +QUERY_SCAN_BYTES_PER_SECOND src/http/action/metrics_action.cpp /^ static const std::string QUERY_SCAN_BYTES_PER_SECOND;$/;" m class:doris::SimpleCoreMetricsVisitor file: +QUERY_SCAN_BYTES_PER_SECOND src/http/action/metrics_action.cpp /^const std::string SimpleCoreMetricsVisitor::QUERY_SCAN_BYTES_PER_SECOND = "query_scan_bytes_per_second";$/;" m class:doris::SimpleCoreMetricsVisitor file: +QUERY_SPLIT_KEY src/agent/task_worker_pool.h /^ QUERY_SPLIT_KEY,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +QUIESCED src/util/threadpool.h /^ QUIESCED,$/;" m class:doris::ThreadPoolToken::State +QUIESCING src/util/threadpool.h /^ QUIESCING,$/;" m class:doris::ThreadPoolToken::State +QueryBuilder src/exec/es/es_query_builder.h /^class QueryBuilder {$/;" c namespace:doris +QueryMaintenance src/exec/exec_node.cpp /^Status ExecNode::QueryMaintenance(RuntimeState* state, const std::string& msg) {$/;" f class:doris::ExecNode +QueryStatistics src/runtime/query_statistics.h /^ QueryStatistics() : scan_rows(0), scan_bytes(0) {$/;" f class:doris::QueryStatistics +QueryStatistics src/runtime/query_statistics.h /^class QueryStatistics {$/;" c namespace:doris +QueryStatisticsRecvr src/runtime/query_statistics.h /^class QueryStatisticsRecvr {$/;" c namespace:doris +QuotedPrintableUnescape src/gutil/strings/escaping.cc /^int QuotedPrintableUnescape(const char *source, int slen,$/;" f namespace:strings +RADIXSORT_H_ src/util/radix_sort.h 27;" d +RANGE src/exec/analytic_eval_node.h /^ RANGE,$/;" e enum:doris::AnalyticEvalNode::AnalyticFnScope +RANGE src/http/http_headers.cpp /^const char* HttpHeaders::RANGE = "Range";$/;" m class:doris::HttpHeaders file: +RANGE src/http/http_headers.h /^ static const char* RANGE;$/;" m class:doris::HttpHeaders +RATE_COUNTER src/util/runtime_profile.h /^ RATE_COUNTER = 0,$/;" e enum:doris::RuntimeProfile::PeriodicCounterType +READ src/runtime/disk_io_mgr.h /^ READ,$/;" e enum:doris::DiskIoMgr::RequestType::type +READER_ALTER_TABLE src/olap/olap_common.h /^ READER_ALTER_TABLE = 1,$/;" e enum:doris::ReaderType +READER_BASE_COMPACTION src/olap/olap_common.h /^ READER_BASE_COMPACTION = 2,$/;" e enum:doris::ReaderType +READER_CHECKSUM src/olap/olap_common.h /^ READER_CHECKSUM = 4,$/;" e enum:doris::ReaderType +READER_CUMULATIVE_COMPACTION src/olap/olap_common.h /^ READER_CUMULATIVE_COMPACTION = 3,$/;" e enum:doris::ReaderType +READER_QUERY src/olap/olap_common.h /^ READER_QUERY = 0,$/;" e enum:doris::ReaderType +READ_ROWBATCH src/exec/olap_scan_node.h /^ READ_ROWBATCH = 1,$/;" e enum:doris::TransferStatus +REALTIME_PUSH src/agent/task_worker_pool.h /^ REALTIME_PUSH,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +RECOVER_TABLET src/agent/task_worker_pool.h /^ RECOVER_TABLET,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +REF src/util/url_parser.h /^ REF,$/;" e enum:doris::UrlParser::UrlPart +REFERER src/http/http_headers.cpp /^const char* HttpHeaders::REFERER = "Referer";$/;" m class:doris::HttpHeaders file: +REFERER src/http/http_headers.h /^ static const char* REFERER;$/;" m class:doris::HttpHeaders +REGISTER_DISK_METRIC src/util/system_metrics.cpp 190;" d file: +REGISTER_DORIS_METRIC src/util/doris_metrics.cpp 154;" d file: +REGISTER_ENGINE_REQUEST_METRIC src/util/doris_metrics.cpp 180;" d file: +REGISTER_METIRC src/runtime/memory/chunk_allocator.cpp 112;" d file: +REGISTER_METIRC_WITH_NAME src/runtime/memory/chunk_allocator.cpp 106;" d file: +REGISTER_METIRC_WITH_PREFIX src/runtime/memory/chunk_allocator.cpp 109;" d file: +REGISTER_NETWORK_METRIC src/util/system_metrics.cpp 288;" d file: +RELEASE_SNAPSHOT src/agent/task_worker_pool.h /^ RELEASE_SNAPSHOT,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +REMOTE_DFS_DISK_OFFSET src/runtime/disk_io_mgr.h /^ REMOTE_DFS_DISK_OFFSET = 0,$/;" e enum:doris::DiskIoMgr::__anon3 +REMOTE_NUM_DISKS src/runtime/disk_io_mgr.h /^ REMOTE_NUM_DISKS$/;" e enum:doris::DiskIoMgr::__anon3 +REMOTE_S3_DISK_OFFSET src/runtime/disk_io_mgr.h /^ REMOTE_S3_DISK_OFFSET,$/;" e enum:doris::DiskIoMgr::__anon3 +REPORT_DISK_STATE src/agent/task_worker_pool.h /^ REPORT_DISK_STATE,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +REPORT_DISK_STATE_WORKER_COUNT src/agent/task_worker_pool.cpp /^const uint32_t REPORT_DISK_STATE_WORKER_COUNT = 1;$/;" m namespace:doris file: +REPORT_DISK_STATE_WORKER_COUNT test/agent/mock_task_worker_pool.h /^const uint32_t REPORT_DISK_STATE_WORKER_COUNT = 1;$/;" m namespace:doris +REPORT_OLAP_TABLE src/agent/task_worker_pool.h /^ REPORT_OLAP_TABLE,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +REPORT_OLAP_TABLE_WORKER_COUNT src/agent/task_worker_pool.cpp /^const uint32_t REPORT_OLAP_TABLE_WORKER_COUNT = 1;$/;" m namespace:doris file: +REPORT_OLAP_TABLE_WORKER_COUNT test/agent/mock_task_worker_pool.h /^const uint32_t REPORT_OLAP_TABLE_WORKER_COUNT = 1;$/;" m namespace:doris +REPORT_TASK src/agent/task_worker_pool.h /^ REPORT_TASK,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +REPORT_TASK_WORKER_COUNT src/agent/task_worker_pool.cpp /^const uint32_t REPORT_TASK_WORKER_COUNT = 1;$/;" m namespace:doris file: +REPORT_TASK_WORKER_COUNT test/agent/mock_task_worker_pool.h /^const uint32_t REPORT_TASK_WORKER_COUNT = 1;$/;" m namespace:doris +REQUESTED_RANGE_NOT_SATISFIED src/http/http_status.h /^ REQUESTED_RANGE_NOT_SATISFIED = 416,$/;" e enum:doris::HttpStatus +REQUEST_ENTITY_TOO_LARGE src/http/http_status.h /^ REQUEST_ENTITY_TOO_LARGE = 413,$/;" e enum:doris::HttpStatus +REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT src/runtime/mem_tracker.cpp /^const std::string REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT = "RequestPool=$0";$/;" m namespace:doris file: +REQUEST_PREFERENCE_PREFIX src/exec/es/es_scan_reader.cpp /^const std::string REQUEST_PREFERENCE_PREFIX = "&preference=_shards:";$/;" m namespace:doris file: +REQUEST_SCROLL_PATH src/exec/es/es_scan_reader.cpp /^const std::string REQUEST_SCROLL_PATH = "_scroll";$/;" m namespace:doris file: +REQUEST_SEARCH_FILTER_PATH src/exec/es/es_scan_reader.cpp /^const std::string REQUEST_SEARCH_FILTER_PATH = "filter_path=hits.hits._source,hits.total,_id,hits.hits._source.fields,hits.hits.fields";$/;" m namespace:doris file: +REQUEST_SEARCH_SCROLL_PATH src/exec/es/es_scan_reader.cpp /^const std::string REQUEST_SEARCH_SCROLL_PATH = "\/_search\/scroll";$/;" m namespace:doris file: +REQUEST_SEPARATOR src/exec/es/es_scan_reader.cpp /^const std::string REQUEST_SEPARATOR = "\/";$/;" m namespace:doris file: +REQUEST_TIMEOUT src/http/http_status.h /^ REQUEST_TIMEOUT = 408,$/;" e enum:doris::HttpStatus +REQUEST_URI_TOO_LONG src/http/http_status.h /^ REQUEST_URI_TOO_LONG = 414,$/;" e enum:doris::HttpStatus +REQUIRE_STACK_ALIGN_TRAMPOLINE src/gutil/port.h 446;" d +RESERVATION_MEM_FRACTION src/runtime/bufferpool/reservation_util.cc /^const double ReservationUtil::RESERVATION_MEM_FRACTION = 0.8;$/;" m class:doris::ReservationUtil file: +RESERVATION_MEM_FRACTION src/runtime/bufferpool/reservation_util.h /^ static const double RESERVATION_MEM_FRACTION;$/;" m class:doris::ReservationUtil +RESERVATION_MEM_MIN_REMAINING src/runtime/bufferpool/reservation_util.cc /^const int64_t ReservationUtil::RESERVATION_MEM_MIN_REMAINING = 75 * 1024 * 1024;$/;" m class:doris::ReservationUtil file: +RESERVATION_MEM_MIN_REMAINING src/runtime/bufferpool/reservation_util.h /^ static const int64_t RESERVATION_MEM_MIN_REMAINING;$/;" m class:doris::ReservationUtil +RESET_CONTENT src/http/http_status.h /^ RESET_CONTENT = 205,$/;" e enum:doris::HttpStatus +RESTART_POINT_INTERVAL src/olap/rowset/segment_v2/binary_prefix_page.h /^ static const uint8_t RESTART_POINT_INTERVAL = 16;$/;" m class:doris::segment_v2::BinaryPrefixPageBuilder +RES_BUF_SIZE test/exec/olap_scanner_test.cpp /^static const int RES_BUF_SIZE = 100 * 1024 * 1024;$/;" m namespace:doris file: +RETRY_AFTER src/http/http_headers.cpp /^const char* HttpHeaders::RETRY_AFTER = "Retry-After";$/;" m class:doris::HttpHeaders file: +RETRY_AFTER src/http/http_headers.h /^ static const char* RETRY_AFTER;$/;" m class:doris::HttpHeaders +RETRY_ON_EINTR src/gutil/macros.h 266;" d +RETURN_ERROR_IF_CAST_FORMAT_ERROR src/exec/es/es_scroll_parser.cpp 120;" d file: +RETURN_ERROR_IF_COL_IS_ARRAY src/exec/es/es_scroll_parser.cpp 82;" d file: +RETURN_ERROR_IF_COL_IS_NOT_STRING src/exec/es/es_scroll_parser.cpp 95;" d file: +RETURN_ERROR_IF_PARSING_FAILED src/exec/es/es_scroll_parser.cpp 108;" d file: +RETURN_IF_CANCELLED src/runtime/runtime_state.h 630;" d +RETURN_IF_ERROR src/common/status.h 251;" d +RETURN_IF_FALSE src/exec/read_write_util.h 28;" d +RETURN_IF_LIMIT_EXCEEDED src/exec/exec_node.h 403;" d +RETURN_IF_STATUS_ERROR src/common/status.h 259;" d +RETURN_NOT_OK src/olap/olap_define.h 399;" d +RETURN_NOT_OK_LOG src/olap/olap_define.h 408;" d +RETURN_TYPE_SIZE src/exprs/expr.h /^ RETURN_TYPE_SIZE, \/\/ int$/;" e enum:doris::Expr::ExprConstant +RETURN_WITH_WARN_IF_ERROR src/common/status.h 286;" d +REUQEST_SCROLL_FILTER_PATH src/exec/es/es_scan_reader.cpp /^const std::string REUQEST_SCROLL_FILTER_PATH = "filter_path=_scroll_id,hits.hits._source,hits.total,_id,hits.hits._source.fields,hits.hits.fields";$/;" m namespace:doris file: +REWIND_LITERAL src/util/rle_encoding.h /^ REWIND_LITERAL,$/;" e enum:doris::RleDecoder::RewindState +REWIND_RUN src/util/rle_encoding.h /^ REWIND_RUN,$/;" e enum:doris::RleDecoder::RewindState +RLE_PAGE_HEADER_SIZE src/olap/rowset/segment_v2/rle_page.h /^ RLE_PAGE_HEADER_SIZE = 4$/;" e enum:doris::segment_v2::__anon48 +RM build/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/agent/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/common/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/env/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/exec/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/exprs/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/gen_cpp/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/geo/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/gutil/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/http/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/olap/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/olap/fs/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/olap/rowset/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/runtime/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/service/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/testutil/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/tools/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/udf/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/udf_samples/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +RM build/src/util/Makefile /^RM = \/usr\/local\/bin\/cmake -E remove -f$/;" m +ROOT_COUNTER src/util/runtime_profile.cpp /^static const std::string ROOT_COUNTER = "";$/;" m namespace:doris file: +ROTL32 src/util/murmur_hash3.cpp 23;" d file: +ROTL32 src/util/murmur_hash3.cpp 44;" d file: +ROTL64 src/util/murmur_hash3.cpp 24;" d file: +ROTL64 src/util/murmur_hash3.cpp 45;" d file: +ROWS src/exec/analytic_eval_node.h /^ ROWS$/;" e enum:doris::AnalyticEvalNode::AnalyticFnScope +ROWSET_ID_PREFIX src/olap/olap_define.h /^const std::string ROWSET_ID_PREFIX = "s_";$/;" m namespace:doris +ROWSET_LOADED src/olap/rowset/rowset.h /^ ROWSET_LOADED,$/;" e enum:doris::RowsetState +ROWSET_PREFIX src/olap/rowset/rowset_meta_manager.cpp /^const std::string ROWSET_PREFIX = "rst_";$/;" m namespace:doris file: +ROWSET_UNLOADED src/olap/rowset/rowset.h /^ ROWSET_UNLOADED,$/;" e enum:doris::RowsetState +ROWSET_UNLOADING src/olap/rowset/rowset.h /^ ROWSET_UNLOADING$/;" e enum:doris::RowsetState +ROW_THROUGHPUT_COUNTER src/exec/exec_node.cpp /^const std::string ExecNode::ROW_THROUGHPUT_COUNTER = "RowsReturnedRate";$/;" m class:doris::ExecNode file: +ROW_THROUGHPUT_COUNTER src/exec/exec_node.h /^ static const std::string ROW_THROUGHPUT_COUNTER;$/;" m class:doris::ExecNode +RUNNING src/util/threadpool.h /^ RUNNING,$/;" m class:doris::ThreadPoolToken::State +RUNNING test/olap/skiplist_test.cpp /^ RUNNING,$/;" e enum:doris::TestState::ReaderState file: +RUNNING_ON_VALGRIND src/gutil/valgrind.h 3639;" d +RUN_COMPACTION src/http/action/compaction_action.h /^ RUN_COMPACTION = 2$/;" e enum:doris::CompactionActionType +RUN_LENGTH_BYTE_POSITIONS src/olap/rowset/segment_reader.h /^ static const int32_t RUN_LENGTH_BYTE_POSITIONS = BYTE_STREAM_POSITIONS + 1;$/;" m class:doris::SegmentReader +RUN_LENGTH_INT_POSITIONS src/olap/rowset/segment_reader.h /^ static const int32_t RUN_LENGTH_INT_POSITIONS = BYTE_STREAM_POSITIONS + 1;$/;" m class:doris::SegmentReader +RWMutex src/util/mutex.cpp /^RWMutex::RWMutex(Priority prio) {$/;" f class:doris::RWMutex +RWMutex src/util/mutex.h /^class RWMutex {$/;" c namespace:doris +RadixSort src/util/radix_sort.h /^struct RadixSort {$/;" s namespace:doris +RadixSortFloatTraits src/util/radix_sort.h /^struct RadixSortFloatTraits {$/;" s namespace:doris +RadixSortFloatTransform src/util/radix_sort.h /^struct RadixSortFloatTransform {$/;" s namespace:doris +RadixSortIdentityTransform src/util/radix_sort.h /^struct RadixSortIdentityTransform {$/;" s namespace:doris +RadixSortIntTraits src/util/radix_sort.h /^struct RadixSortIntTraits {$/;" s namespace:doris +RadixSortMallocAllocator src/util/radix_sort.h /^struct RadixSortMallocAllocator {$/;" s namespace:doris +RadixSortSignedTransform src/util/radix_sort.h /^struct RadixSortSignedTransform$/;" s namespace:doris +RadixSortTest test/util/radix_sort_test.cpp /^ RadixSortTest() {$/;" f class:doris::RadixSortTest +RadixSortTest test/util/radix_sort_test.cpp /^class RadixSortTest : public ::testing::Test {$/;" c namespace:doris file: +RadixSortTestTraits test/util/radix_sort_test.cpp /^struct RadixSortTestTraits {$/;" s namespace:doris file: +RadixSortUIntTraits src/util/radix_sort.h /^struct RadixSortUIntTraits {$/;" s namespace:doris +Random src/util/random.h /^ explicit Random(uint32_t s) : seed_(s & 0x7fffffffu) {$/;" f class:doris::Random +Random src/util/random.h /^class Random {$/;" c namespace:doris +RandomAccessFile src/env/env.h /^ RandomAccessFile() { }$/;" f class:doris::RandomAccessFile +RandomAccessFile src/env/env.h /^class RandomAccessFile {$/;" c namespace:doris +RandomAccessFileOptions src/env/env.h /^ RandomAccessFileOptions() { }$/;" f struct:doris::RandomAccessFileOptions +RandomAccessFileOptions src/env/env.h /^struct RandomAccessFileOptions {$/;" s namespace:doris +RandomHeight src/olap/skiplist.h /^int SkipList::RandomHeight() {$/;" f class:doris::SkipList +RandomRWFile src/env/env.h /^ RandomRWFile() {}$/;" f class:doris::RandomRWFile +RandomRWFile src/env/env.h /^class RandomRWFile {$/;" c namespace:doris +RandomRWFileOptions src/env/env.h /^struct RandomRWFileOptions {$/;" s namespace:doris +RandomString test/util/faststring_test.cpp /^void RandomString(void* dest, size_t n, doris::Random* rng) {$/;" f namespace:doris +RangeCondition src/olap/olap_common.h /^enum RangeCondition {$/;" g namespace:doris +RangeQueryBuilder src/exec/es/es_query_builder.cpp /^RangeQueryBuilder::RangeQueryBuilder(const ExtBinaryPredicate& range_predicate) : _field(range_predicate.col.name), _value(range_predicate.value.to_string()), _op(range_predicate.op) {$/;" f class:doris::RangeQueryBuilder +RangeQueryBuilder src/exec/es/es_query_builder.h /^class RangeQueryBuilder : public QueryBuilder {$/;" c namespace:doris +RankState src/exprs/aggregate_functions.cpp /^ RankState() : rank(1), count(0) { }$/;" f struct:doris::RankState +RankState src/exprs/aggregate_functions.cpp /^struct RankState {$/;" s namespace:doris file: +RateCounterInfo src/util/runtime_profile.h /^ struct RateCounterInfo {$/;" s class:doris::RuntimeProfile +RateCounterMap src/util/runtime_profile.h /^ typedef std::map RateCounterMap;$/;" t struct:doris::RuntimeProfile::PeriodicCounterUpdateState +RawValue src/runtime/raw_value.h /^class RawValue {$/;" c namespace:doris +RawValueTest test/runtime/raw_value_test.cpp /^class RawValueTest : public testing::Test {$/;" c namespace:doris file: +Read src/exec/parquet_reader.cpp /^arrow::Status ParquetFile::Read(int64_t nbytes, int64_t* bytes_read, void* buffer) {$/;" f class:doris::ParquetFile +Read src/exec/parquet_reader.cpp /^arrow::Status ParquetFile::Read(int64_t nbytes, std::shared_ptr* out) {$/;" f class:doris::ParquetFile +ReadAt src/exec/parquet_reader.cpp /^arrow::Status ParquetFile::ReadAt(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) {$/;" f class:doris::ParquetFile +ReadHeader src/util/rle_encoding.h /^inline bool RleDecoder::ReadHeader() {$/;" f class:doris::RleDecoder +ReadIntFromFile src/gutil/sysinfo.cc /^static bool ReadIntFromFile(const char *file, int *value) {$/;" f namespace:base +ReadLock src/util/mutex.h /^ explicit ReadLock(RWMutex* mutex, bool try_lock = false)$/;" f class:doris::ReadLock +ReadLock src/util/mutex.h /^class ReadLock {$/;" c namespace:doris +ReadMaxCPUIndex src/gutil/sysinfo.cc /^static int ReadMaxCPUIndex() {$/;" f namespace:base +ReadOnlyFileStream src/olap/file_stream.cpp /^ReadOnlyFileStream::ReadOnlyFileStream($/;" f class:doris::ReadOnlyFileStream +ReadOnlyFileStream src/olap/file_stream.h /^class ReadOnlyFileStream {$/;" c namespace:doris +ReadValues test/runtime/buffered_tuple_stream2_test.cpp /^ void ReadValues(BufferedTupleStream2* stream, RowDescriptor* desc, vector* results,$/;" f class:doris::SimpleTupleStreamTest +ReadWriteUtil src/exec/read_write_util.h /^class ReadWriteUtil {$/;" c namespace:doris +ReadableBlock src/olap/fs/block_manager.h /^class ReadableBlock : public Block {$/;" c namespace:doris::fs +Reader src/olap/reader.cpp /^Reader::Reader()$/;" f class:doris::Reader +Reader src/olap/reader.h /^class Reader {$/;" c namespace:doris +ReaderParams src/olap/reader.h /^ ReaderParams() :$/;" f struct:doris::ReaderParams +ReaderParams src/olap/reader.h /^struct ReaderParams {$/;" s namespace:doris +ReaderState test/olap/skiplist_test.cpp /^ enum ReaderState {$/;" g class:doris::TestState file: +ReaderType src/olap/olap_common.h /^enum ReaderType {$/;" g namespace:doris +ReceiverInfo test/runtime/data_stream_test.cpp /^ ReceiverInfo(TPartitionType::type stream_type, int num_senders, int receiver_num) :$/;" f struct:doris::DataStreamTest::ReceiverInfo +ReceiverInfo test/runtime/data_stream_test.cpp /^ struct ReceiverInfo {$/;" s class:doris::DataStreamTest file: +RecordBatchQueue src/runtime/record_batch_queue.h /^ RecordBatchQueue(u_int32_t max_elements) : _queue(max_elements) {}$/;" f class:doris::RecordBatchQueue +RecordBatchQueue src/runtime/record_batch_queue.h /^class RecordBatchQueue {$/;" c namespace:doris +RefCountClosure src/util/ref_count_closure.h /^ RefCountClosure() : _refs(0) { }$/;" f class:doris::RefCountClosure +RefCountClosure src/util/ref_count_closure.h /^class RefCountClosure : public google::protobuf::Closure {$/;" c namespace:doris +RefCountDec src/gutil/atomic_refcount.h /^inline bool RefCountDec(volatile Atomic32 *ptr) {$/;" f namespace:base +RefCountDec src/gutil/atomic_refcount.h /^inline bool RefCountDec(volatile AtomicWord *ptr) {$/;" f namespace:base +RefCountDec src/gutil/atomic_refcount.h /^inline bool RefCountDec(volatile base::subtle::Atomic64 *ptr) {$/;" f namespace:base +RefCountDecN src/gutil/atomic_refcount.h /^inline bool RefCountDecN(volatile Atomic32 *ptr, Atomic32 decrement) {$/;" f namespace:base +RefCountDecN src/gutil/atomic_refcount.h /^inline bool RefCountDecN(volatile AtomicWord *ptr, AtomicWord decrement) {$/;" f namespace:base +RefCountDecN src/gutil/atomic_refcount.h /^inline bool RefCountDecN(volatile base::subtle::Atomic64 *ptr,$/;" f namespace:base +RefCountInc src/gutil/atomic_refcount.h /^inline void RefCountInc(volatile Atomic32 *ptr) {$/;" f namespace:base +RefCountInc src/gutil/atomic_refcount.h /^inline void RefCountInc(volatile AtomicWord *ptr) {$/;" f namespace:base +RefCountInc src/gutil/atomic_refcount.h /^inline void RefCountInc(volatile base::subtle::Atomic64 *ptr) {$/;" f namespace:base +RefCountIncN src/gutil/atomic_refcount.h /^inline void RefCountIncN(volatile Atomic32 *ptr, Atomic32 increment) {$/;" f namespace:base +RefCountIncN src/gutil/atomic_refcount.h /^inline void RefCountIncN(volatile AtomicWord *ptr, AtomicWord increment) {$/;" f namespace:base +RefCountIncN src/gutil/atomic_refcount.h /^inline void RefCountIncN(volatile base::subtle::Atomic64 *ptr,$/;" f namespace:base +RefCountIsOne src/gutil/atomic_refcount.h /^inline bool RefCountIsOne(const volatile Atomic32 *ptr) {$/;" f namespace:base +RefCountIsOne src/gutil/atomic_refcount.h /^inline bool RefCountIsOne(const volatile AtomicWord *ptr) {$/;" f namespace:base +RefCountIsOne src/gutil/atomic_refcount.h /^inline bool RefCountIsOne(const volatile base::subtle::Atomic64 *ptr) {$/;" f namespace:base +RefCountIsZero src/gutil/atomic_refcount.h /^inline bool RefCountIsZero(const volatile Atomic32 *ptr) {$/;" f namespace:base +RefCountIsZero src/gutil/atomic_refcount.h /^inline bool RefCountIsZero(const volatile AtomicWord *ptr) {$/;" f namespace:base +RefCountIsZero src/gutil/atomic_refcount.h /^inline bool RefCountIsZero(const volatile base::subtle::Atomic64 *ptr) {$/;" f namespace:base +RefCounted src/gutil/ref_counted.h /^ RefCounted() {}$/;" f class:doris::RefCounted +RefCounted src/gutil/ref_counted.h /^class RefCounted : public subtle::RefCountedBase {$/;" c namespace:doris +RefCountedBase src/gutil/ref_counted.cc /^RefCountedBase::RefCountedBase()$/;" f class:doris::subtle::RefCountedBase +RefCountedBase src/gutil/ref_counted.h /^class RefCountedBase {$/;" c namespace:doris::subtle +RefCountedData src/gutil/ref_counted.h /^ RefCountedData() : data() {}$/;" f class:doris::RefCountedData +RefCountedData src/gutil/ref_counted.h /^ RefCountedData(const T& in_value) : data(in_value) {}$/;" f class:doris::RefCountedData +RefCountedData src/gutil/ref_counted.h /^class RefCountedData$/;" c namespace:doris +RefCountedThreadSafe src/gutil/ref_counted.h /^ RefCountedThreadSafe() {}$/;" f class:doris::RefCountedThreadSafe +RefCountedThreadSafe src/gutil/ref_counted.h /^class RefCountedThreadSafe : public subtle::RefCountedThreadSafeBase {$/;" c namespace:doris +RefCountedThreadSafeBase src/gutil/ref_counted.cc /^RefCountedThreadSafeBase::RefCountedThreadSafeBase() : ref_count_(0) {$/;" f class:doris::subtle::RefCountedThreadSafeBase +RefCountedThreadSafeBase src/gutil/ref_counted.h /^class RefCountedThreadSafeBase {$/;" c namespace:doris::subtle +RefreshConsumptionFromMetric src/runtime/mem_tracker.h /^ void RefreshConsumptionFromMetric() {$/;" f class:doris::MemTracker +Register src/common/configbase.h /^ Register(const char* ftype, const char* fname, void* fstorage, const char* fdefval) {$/;" f class:doris::config::Register +Register src/common/configbase.h /^class Register {$/;" c namespace:doris::config +RegisterClient src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::RegisterClient(const string& name, \/\/TmpFileMgr::FileGroup* file_group,$/;" f class:doris::BufferPool +Release src/gutil/ref_counted.cc /^bool RefCountedBase::Release() const {$/;" f class:doris::subtle::RefCountedBase +Release src/gutil/ref_counted.cc /^bool RefCountedThreadSafeBase::Release() const {$/;" f class:doris::subtle::RefCountedThreadSafeBase +Release src/gutil/ref_counted.h /^ void Release() const {$/;" f class:doris::RefCounted +Release src/gutil/ref_counted.h /^ void Release() const {$/;" f class:doris::RefCountedThreadSafe +ReleaseMemory src/runtime/bufferpool/buffer_allocator.cc /^void BufferPool::BufferAllocator::ReleaseMemory(int64_t bytes_to_free) {$/;" f class:doris::BufferPool::BufferAllocator +ReleaseMemory src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::ReleaseMemory(int64_t bytes_to_free) {$/;" f class:doris::BufferPool +ReleaseResources src/runtime/initial_reservations.cc /^void InitialReservations::ReleaseResources() {$/;" f class:doris::InitialReservations +ReleaseTo src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::ReleaseTo(int64_t bytes) {$/;" f class:doris::ReservationTracker +ReleaseToMemTracker src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::ReleaseToMemTracker(int64_t reservation_decrease) {$/;" f class:doris::ReservationTracker +Release_AtomicExchange src/gutil/atomicops-internals-gcc.h /^inline Atomic32 Release_AtomicExchange(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Release_AtomicExchange src/gutil/atomicops-internals-gcc.h /^inline Atomic64 Release_AtomicExchange(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Release_AtomicExchange src/gutil/atomicops-internals-tsan.h /^inline Atomic32 Release_AtomicExchange(volatile Atomic32 *ptr,$/;" f namespace:base::subtle +Release_AtomicExchange src/gutil/atomicops-internals-tsan.h /^inline Atomic64 Release_AtomicExchange(volatile Atomic64 *ptr,$/;" f namespace:base::subtle +Release_AtomicExchange src/gutil/atomicops-internals-x86.h /^inline Atomic32 Release_AtomicExchange(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Release_AtomicExchange src/gutil/atomicops-internals-x86.h /^inline Atomic64 Release_AtomicExchange(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Release_AtomicExchange src/gutil/atomicops.h /^inline AtomicWord Release_AtomicExchange(volatile AtomicWord* ptr,$/;" f namespace:base::subtle +Release_CompareAndSwap src/gutil/atomicops-internals-gcc.h /^inline Atomic32 Release_CompareAndSwap(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Release_CompareAndSwap src/gutil/atomicops-internals-gcc.h /^inline Atomic64 Release_CompareAndSwap(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Release_CompareAndSwap src/gutil/atomicops-internals-tsan.h /^inline Atomic32 Release_CompareAndSwap(volatile Atomic32 *ptr,$/;" f namespace:base::subtle +Release_CompareAndSwap src/gutil/atomicops-internals-tsan.h /^inline Atomic64 Release_CompareAndSwap(volatile Atomic64 *ptr,$/;" f namespace:base::subtle +Release_CompareAndSwap src/gutil/atomicops-internals-x86.h /^inline Atomic32 Release_CompareAndSwap(volatile Atomic32* ptr,$/;" f namespace:base::subtle +Release_CompareAndSwap src/gutil/atomicops-internals-x86.h /^inline Atomic64 Release_CompareAndSwap(volatile Atomic64* ptr,$/;" f namespace:base::subtle +Release_CompareAndSwap src/gutil/atomicops.h /^inline Atomic32 Release_CompareAndSwap(volatile Atomic32* ptr,$/;" f +Release_CompareAndSwap src/gutil/atomicops.h /^inline AtomicWord Release_CompareAndSwap(volatile AtomicWord* ptr,$/;" f +Release_CompareAndSwap src/gutil/atomicops.h /^inline AtomicWord Release_CompareAndSwap(volatile AtomicWord* ptr,$/;" f namespace:base::subtle +Release_CompareAndSwap src/gutil/atomicops.h /^inline base::subtle::Atomic64 Release_CompareAndSwap($/;" f +Release_Load src/gutil/atomicops-internals-gcc.h /^inline Atomic32 Release_Load(volatile const Atomic32* ptr) {$/;" f namespace:base::subtle +Release_Load src/gutil/atomicops-internals-gcc.h /^inline Atomic64 Release_Load(volatile const Atomic64* ptr) {$/;" f namespace:base::subtle +Release_Load src/gutil/atomicops-internals-tsan.h /^inline Atomic32 Release_Load(volatile const Atomic32 *ptr) {$/;" f namespace:base::subtle +Release_Load src/gutil/atomicops-internals-tsan.h /^inline Atomic64 Release_Load(volatile const Atomic64 *ptr) {$/;" f namespace:base::subtle +Release_Load src/gutil/atomicops-internals-x86.h /^inline Atomic32 Release_Load(volatile const Atomic32* ptr) {$/;" f namespace:base::subtle +Release_Load src/gutil/atomicops-internals-x86.h /^inline Atomic64 Release_Load(volatile const Atomic64* ptr) {$/;" f namespace:base::subtle +Release_Load src/gutil/atomicops.h /^inline Atomic32 Release_Load(volatile const Atomic32* ptr) {$/;" f +Release_Load src/gutil/atomicops.h /^inline AtomicWord Release_Load(volatile const AtomicWord* ptr) {$/;" f +Release_Load src/gutil/atomicops.h /^inline AtomicWord Release_Load(volatile const AtomicWord* ptr) {$/;" f namespace:base::subtle +Release_Load src/gutil/atomicops.h /^inline base::subtle::Atomic64 Release_Load($/;" f +Release_Store src/gutil/atomicops-internals-gcc.h /^inline void Release_Store(volatile Atomic32* ptr, Atomic32 value) {$/;" f namespace:base::subtle +Release_Store src/gutil/atomicops-internals-gcc.h /^inline void Release_Store(volatile Atomic64* ptr, Atomic64 value) {$/;" f namespace:base::subtle +Release_Store src/gutil/atomicops-internals-tsan.h /^inline void Release_Store(volatile Atomic32 *ptr, Atomic32 value) {$/;" f namespace:base::subtle +Release_Store src/gutil/atomicops-internals-tsan.h /^inline void Release_Store(volatile Atomic64 *ptr, Atomic64 value) {$/;" f namespace:base::subtle +Release_Store src/gutil/atomicops-internals-x86.h /^inline void Release_Store(volatile Atomic32* ptr, Atomic32 value) {$/;" f namespace:base::subtle +Release_Store src/gutil/atomicops-internals-x86.h /^inline void Release_Store(volatile Atomic64* ptr, Atomic64 value) {$/;" f namespace:base::subtle +Release_Store src/gutil/atomicops.h /^inline void Release_Store($/;" f +Release_Store src/gutil/atomicops.h /^inline void Release_Store(volatile Atomic32* ptr, Atomic32 value) {$/;" f +Release_Store src/gutil/atomicops.h /^inline void Release_Store(volatile AtomicWord* ptr, AtomicWord value) {$/;" f +Release_Store src/gutil/atomicops.h /^inline void Release_Store(volatile AtomicWord* ptr, AtomicWord value) {$/;" f namespace:base::subtle +ReloadTabletAction src/http/action/reload_tablet_action.cpp /^ReloadTabletAction::ReloadTabletAction(ExecEnv* exec_env) :$/;" f class:doris::ReloadTabletAction +ReloadTabletAction src/http/action/reload_tablet_action.h /^class ReloadTabletAction : public HttpHandler {$/;" c namespace:doris +Remove src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::Remove(const TupleRow* row, Tuple* dst) {$/;" f class:doris::NewAggFnEvaluator +Remove src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::Remove(const std::vector& evals,$/;" f class:doris::NewAggFnEvaluator +Remove src/gutil/strings/charset.h /^ void Remove(unsigned char c) { bits_[Word(c)] &= ~BitMask(c); }$/;" f class:strings::CharSet +RemoveCleanPage src/runtime/bufferpool/buffer_allocator.cc /^bool BufferPool::BufferAllocator::RemoveCleanPage($/;" f class:doris::BufferPool::BufferAllocator +RemoveCleanPage src/runtime/bufferpool/buffer_allocator.cc /^bool BufferPool::FreeBufferArena::RemoveCleanPage(bool claim_buffer, Page* page) {$/;" f class:doris::BufferPool::FreeBufferArena +RemoveExtraWhitespace src/gutil/strings/strip.cc /^void RemoveExtraWhitespace(string* s) {$/;" f +RemoveFromFreeList src/runtime/bufferpool/suballocator.cc /^unique_ptr Suballocator::RemoveFromFreeList(Suballocation* node) {$/;" f class:doris::Suballocator +RemoveNullsInString src/gutil/strings/strip.cc /^void RemoveNullsInString(string* s) {$/;" f +RemoveStrings src/gutil/strings/util.cc /^void RemoveStrings(vector* v, const vector& indices) {$/;" f +RemoverThread test/util/blocking_queue_test.cpp /^ void RemoverThread() {$/;" f class:doris::MultiThreadTest +RepartitionSpilledPartition src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::RepartitionSpilledPartition() {$/;" f class:doris::NewPartitionedAggregationNode +RepeatNode src/exec/repeat_node.cpp /^RepeatNode::RepeatNode(ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::RepeatNode +RepeatNode src/exec/repeat_node.h /^class RepeatNode : public ExecNode {$/;" c namespace:doris +RequestContext src/runtime/disk_io_mgr_internal.h /^class DiskIoMgr::RequestContext {$/;" c class:doris::DiskIoMgr +RequestContext src/runtime/disk_io_mgr_reader_context.cc /^DiskIoMgr::RequestContext::RequestContext(DiskIoMgr* parent, int num_disks) :$/;" f class:doris::DiskIoMgr::RequestContext +RequestContextCache src/runtime/disk_io_mgr.cc /^ RequestContextCache(DiskIoMgr* io_mgr) : _io_mgr(io_mgr) {}$/;" f class:doris::DiskIoMgr::RequestContextCache +RequestContextCache src/runtime/disk_io_mgr.cc /^class DiskIoMgr::RequestContextCache {$/;" c class:doris::DiskIoMgr file: +RequestRange src/runtime/disk_io_mgr.h /^ class RequestRange : public InternalQueue::Node {$/;" c class:doris::DiskIoMgr +RequestTrackersMap src/runtime/mem_tracker.h /^ typedef std::unordered_map > RequestTrackersMap;$/;" t class:doris::MemTracker +RequestType src/runtime/disk_io_mgr.h /^ struct RequestType {$/;" s class:doris::DiskIoMgr +ReservationTracker src/runtime/bufferpool/reservation_tracker.cc /^ReservationTracker::ReservationTracker() {}$/;" f class:doris::ReservationTracker +ReservationTracker src/runtime/bufferpool/reservation_tracker.h /^class ReservationTracker {$/;" c namespace:doris +ReservationTrackerCounters src/runtime/bufferpool/reservation_tracker_counters.h /^struct ReservationTrackerCounters {$/;" s namespace:doris +ReservationUtil src/runtime/bufferpool/reservation_util.h /^class ReservationUtil {$/;" c namespace:doris +Reserve src/util/rle_encoding.h /^inline void RleEncoder::Reserve(int num_bytes, uint8_t val) {$/;" f class:doris::RleEncoder +ReserveCapacity src/gutil/strings/split_internal.h /^ void ReserveCapacity(...) {}$/;" f class:strings::internal::Splitter +ReserveCapacity src/gutil/strings/split_internal.h /^ void ReserveCapacity(vector* v, size_t size) {$/;" f class:strings::internal::Splitter +Reset src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::PageHandle::Reset() {$/;" f class:doris::BufferPool::PageHandle +Reset src/runtime/bufferpool/buffer_pool.h /^inline void BufferPool::BufferHandle::Reset() {$/;" f class:doris::BufferPool::BufferHandle +Reset src/util/bitmap.h /^ void Reset() {$/;" f class:doris::BitmapIterator +Reset src/util/bitmap.h /^ void Reset(const uint8_t* map, size_t num_bits) {$/;" f class:doris::BitmapIterator +Reset src/util/bitmap.h /^ void Reset(int64_t num_bits) {$/;" f class:doris::Bitmap +ResetForRead src/exec/new_partitioned_hash_table.cc /^void NewPartitionedHashTableCtx::ExprValuesCache::ResetForRead() {$/;" f class:NewPartitionedHashTableCtx::ExprValuesCache +ResetIterators src/exec/new_partitioned_hash_table.cc /^void NewPartitionedHashTableCtx::ExprValuesCache::ResetIterators() {$/;" f class:NewPartitionedHashTableCtx::ExprValuesCache +ResetWritePage src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::ResetWritePage() {$/;" f class:BufferedTupleStream3 +ResizeBuckets src/exec/new_partitioned_hash_table.cc /^Status NewPartitionedHashTable::ResizeBuckets($/;" f class:NewPartitionedHashTable +ResizeTable test/exec/partitioned_hash_table_test.cpp /^ void ResizeTable($/;" f class:doris::PartitionedHashTableTest +ResourcePool src/runtime/thread_resource_mgr.cpp /^ThreadResourceMgr::ResourcePool::ResourcePool(ThreadResourceMgr* parent)$/;" f class:doris::ThreadResourceMgr::ResourcePool +ResourcePool src/runtime/thread_resource_mgr.h /^ class ResourcePool {$/;" c class:doris::ThreadResourceMgr +ResourceTls src/common/resource_tls.h /^class ResourceTls {$/;" c namespace:doris +ResourceTlsTest test/common/resource_tls_test.cpp /^class ResourceTlsTest : public testing::Test {$/;" c namespace:doris file: +RestClearScrollAction test/exec/es_scan_reader_test.cpp /^class RestClearScrollAction : public HttpHandler {$/;" c namespace:doris file: +RestMonitorIface src/http/rest_monitor_iface.h /^class RestMonitorIface {$/;" c namespace:doris +RestSearchAction test/exec/es_scan_reader_test.cpp /^class RestSearchAction : public HttpHandler {$/;" c namespace:doris file: +RestSearchScrollAction test/exec/es_scan_reader_test.cpp /^class RestSearchScrollAction : public HttpHandler {$/;" c namespace:doris file: +RestoreReservation src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::ClientHandle::RestoreReservation(SubReservation* src, int64_t bytes) {$/;" f class:doris::BufferPool::ClientHandle +RestoreTabletAction src/http/action/restore_tablet_action.cpp /^RestoreTabletAction::RestoreTabletAction(ExecEnv* exec_env) : _exec_env(exec_env) {$/;" f class:doris::RestoreTabletAction +RestoreTabletAction src/http/action/restore_tablet_action.h /^class RestoreTabletAction : public HttpHandler {$/;" c namespace:doris +ResultBufferMgr src/runtime/result_buffer_mgr.cpp /^ResultBufferMgr::ResultBufferMgr()$/;" f class:doris::ResultBufferMgr +ResultBufferMgr src/runtime/result_buffer_mgr.h /^class ResultBufferMgr {$/;" c namespace:doris +ResultBufferMgrTest test/runtime/result_buffer_mgr_test.cpp /^ ResultBufferMgrTest() {$/;" f class:doris::ResultBufferMgrTest +ResultBufferMgrTest test/runtime/result_buffer_mgr_test.cpp /^class ResultBufferMgrTest : public testing::Test {$/;" c namespace:doris file: +ResultComparator output/udf/include/uda_test_harness.h /^ typedef bool (*ResultComparator)(const RESULT& x, const RESULT& y);$/;" t class:doris_udf::UdaTestHarnessBase +ResultComparator src/udf/uda_test_harness.h /^ typedef bool (*ResultComparator)(const RESULT& x, const RESULT& y);$/;" t class:doris_udf::UdaTestHarnessBase +ResultQueue src/runtime/buffer_control_block.h /^ typedef std::list ResultQueue;$/;" t class:doris::BufferControlBlock +ResultQueueMgr src/runtime/result_queue_mgr.cpp /^ResultQueueMgr::ResultQueueMgr() {$/;" f class:doris::ResultQueueMgr +ResultQueueMgr src/runtime/result_queue_mgr.h /^class ResultQueueMgr {$/;" c namespace:doris +ResultQueueMgrTest test/runtime/result_queue_mgr_test.cpp /^ ResultQueueMgrTest() {$/;" f class:doris::ResultQueueMgrTest +ResultQueueMgrTest test/runtime/result_queue_mgr_test.cpp /^class ResultQueueMgrTest : public testing::Test {$/;" c namespace:doris file: +ResultSink src/runtime/result_sink.cpp /^ResultSink::ResultSink(const RowDescriptor& row_desc, const std::vector& t_output_expr,$/;" f class:doris::ResultSink +ResultSink src/runtime/result_sink.h /^class ResultSink : public DataSink {$/;" c namespace:doris +ResultSinkTest test/runtime/result_sink_test.cpp /^ ResultSinkTest() {$/;" f class:doris::ResultSinkTest +ResultSinkTest test/runtime/result_sink_test.cpp /^class ResultSinkTest : public testing::Test {$/;" c namespace:doris file: +ResultType output/udf/include/udf.h /^typedef AnyVal ResultType;$/;" t namespace:doris_udf +ResultType src/udf/udf.h /^typedef AnyVal ResultType;$/;" t namespace:doris_udf +ResultWriter src/runtime/result_writer.cpp /^ResultWriter::ResultWriter($/;" f class:doris::ResultWriter +ResultWriter src/runtime/result_writer.h /^class ResultWriter {$/;" c namespace:doris +ResultWriterTest test/runtime/result_writer_test.cpp /^ ResultWriterTest() {$/;" f class:doris::ResultWriterTest +ResultWriterTest test/runtime/result_writer_test.cpp /^class ResultWriterTest : public testing::Test {$/;" c namespace:doris file: +Return src/runtime/initial_reservations.cc /^void InitialReservations::Return(BufferPool::ClientHandle* src, int64_t bytes) {$/;" f class:doris::InitialReservations +ReverseBits32 src/gutil/bits.h /^inline uint32 Bits::ReverseBits32(uint32 n) {$/;" f class:Bits +ReverseBits64 src/gutil/bits.h /^inline uint64 Bits::ReverseBits64(uint64 n) {$/;" f class:Bits +ReverseBits8 src/gutil/bits.h /^inline uint8 Bits::ReverseBits8(unsigned char n) {$/;" f class:Bits +ReverseFindNth src/gutil/strings/util.cc /^int ReverseFindNth(StringPiece s, char c, int n) {$/;" f +ReverseMap src/gutil/map-util.h /^void ReverseMap(const Collection& collection,$/;" f +Rewind src/util/bit_stream_utils.inline.h /^inline void BitReader::Rewind(int num_bits) {$/;" f class:doris::BitReader +RewindOne src/util/rle_encoding.h /^inline void RleDecoder::RewindOne() {$/;" f class:doris::RleDecoder +RewindState src/util/rle_encoding.h /^ enum RewindState {$/;" g class:doris::RleDecoder +RleDecoder src/util/rle_encoding.h /^ RleDecoder() {}$/;" f class:doris::RleDecoder +RleDecoder src/util/rle_encoding.h /^ RleDecoder(const uint8_t* buffer, int buffer_len, int bit_width)$/;" f class:doris::RleDecoder +RleDecoder src/util/rle_encoding.h /^class RleDecoder {$/;" c namespace:doris +RleEncoder src/util/rle_encoding.h /^ explicit RleEncoder(faststring *buffer, int bit_width)$/;" f class:doris::RleEncoder +RleEncoder src/util/rle_encoding.h /^class RleEncoder {$/;" c namespace:doris +RlePageBuilder src/olap/rowset/segment_v2/rle_page.h /^ RlePageBuilder(const PageBuilderOptions& options) :$/;" f class:doris::segment_v2::RlePageBuilder +RlePageBuilder src/olap/rowset/segment_v2/rle_page.h /^class RlePageBuilder : public PageBuilder {$/;" c namespace:doris::segment_v2 +RlePageDecoder src/olap/rowset/segment_v2/rle_page.h /^ RlePageDecoder(Slice slice, const PageDecoderOptions& options) :$/;" f class:doris::segment_v2::RlePageDecoder +RlePageDecoder src/olap/rowset/segment_v2/rle_page.h /^class RlePageDecoder : public PageDecoder {$/;" c namespace:doris::segment_v2 +RlePageTest test/olap/rowset/segment_v2/rle_page_test.cpp /^class RlePageTest : public testing::Test {$/;" c namespace:doris file: +Roaring64Map src/util/bitmap_value.h /^ Roaring64Map(const Roaring& r) { emplaceOrInsert(0, r); }$/;" f class:doris::detail::Roaring64Map +Roaring64Map src/util/bitmap_value.h /^ Roaring64Map(roaring_bitmap_t* s) { emplaceOrInsert(0, s); }$/;" f class:doris::detail::Roaring64Map +Roaring64Map src/util/bitmap_value.h /^ Roaring64Map(size_t n, const uint32_t* data) { addMany(n, data); }$/;" f class:doris::detail::Roaring64Map +Roaring64Map src/util/bitmap_value.h /^ Roaring64Map(size_t n, const uint64_t* data) { addMany(n, data); }$/;" f class:doris::detail::Roaring64Map +Roaring64Map src/util/bitmap_value.h /^class Roaring64Map {$/;" c namespace:doris::detail +Roaring64MapSetBitForwardIterator src/util/bitmap_value.h /^ Roaring64MapSetBitForwardIterator(const Roaring64Map& parent, bool exhausted = false)$/;" f class:doris::detail::final +RollupSchema src/runtime/dpp_sink_internal.cpp /^RollupSchema::RollupSchema() {$/;" f class:doris::RollupSchema +RollupSchema src/runtime/dpp_sink_internal.h /^class RollupSchema {$/;" c namespace:doris +Rotate src/gutil/hash/city.cc /^static uint64 Rotate(uint64 val, int shift) {$/;" f namespace:util_hash +RotateByAtLeast1 src/gutil/hash/city.cc /^static uint64 RotateByAtLeast1(uint64 val, int shift) {$/;" f namespace:util_hash +RoundDown src/util/bit_util.h /^ constexpr static inline int64_t RoundDown(int64_t value, int64_t factor) {$/;" f class:doris::BitUtil +RoundUpNumBytes src/util/bit_util.h /^ constexpr static inline uint32_t RoundUpNumBytes(uint32_t bits) {$/;" f class:doris::BitUtil +RoundUpToPowerOf2 src/util/bit_util.h /^ static inline int64_t RoundUpToPowerOf2(int64_t value, int64_t factor) {$/;" f class:doris::BitUtil +RoundUpToPowerOfTwo src/util/bit_util.h /^ static inline int64_t RoundUpToPowerOfTwo(int64_t v) {$/;" f class:doris::BitUtil +RoutineLoadTaskExecutor src/runtime/routine_load/routine_load_task_executor.h /^ RoutineLoadTaskExecutor(ExecEnv* exec_env):$/;" f class:doris::RoutineLoadTaskExecutor +RoutineLoadTaskExecutor src/runtime/routine_load/routine_load_task_executor.h /^class RoutineLoadTaskExecutor {$/;" c namespace:doris +RoutineLoadTaskExecutorTest test/runtime/routine_load_task_executor_test.cpp /^ RoutineLoadTaskExecutorTest() { }$/;" f class:doris::RoutineLoadTaskExecutorTest +RoutineLoadTaskExecutorTest test/runtime/routine_load_task_executor_test.cpp /^class RoutineLoadTaskExecutorTest : public testing::Test {$/;" c namespace:doris file: +RowBatch src/runtime/row_batch.cpp /^RowBatch::RowBatch(const RowDescriptor& row_desc, const TRowBatch& input_batch, MemTracker* tracker) :$/;" f class:doris::RowBatch +RowBatch src/runtime/row_batch.cpp /^RowBatch::RowBatch(const RowDescriptor& row_desc, int capacity, MemTracker* mem_tracker) :$/;" f class:doris::RowBatch +RowBatch src/runtime/row_batch.cpp /^RowBatch::RowBatch(const RowDescriptor& row_desc,$/;" f class:doris::RowBatch +RowBatch src/runtime/row_batch.h /^class RowBatch : public RowBatchInterface {$/;" c namespace:doris +RowBatchInterface src/runtime/row_batch_interface.hpp /^class RowBatchInterface {$/;" c namespace:doris +RowBatchList src/exec/row_batch_list.h /^ RowBatchList() : _total_num_rows(0) { }$/;" f class:doris::RowBatchList +RowBatchList src/exec/row_batch_list.h /^class RowBatchList {$/;" c namespace:doris +RowBatchQueue src/exec/exec_node.cpp /^ExecNode::RowBatchQueue::RowBatchQueue(int max_batches) :$/;" f class:doris::ExecNode::RowBatchQueue +RowBatchQueue src/exec/exec_node.h /^ class RowBatchQueue : public BlockingQueue {$/;" c class:doris::ExecNode +RowBatchQueue src/runtime/data_stream_recvr.cc /^ typedef list > RowBatchQueue;$/;" t class:doris::DataStreamRecvr::SenderQueue file: +RowBlock src/olap/row_block.cpp /^RowBlock::RowBlock(const TabletSchema* schema) :$/;" f class:doris::RowBlock +RowBlock src/olap/row_block.h /^class RowBlock {$/;" c namespace:doris +RowBlockAllocator src/olap/schema_change.cpp /^RowBlockAllocator::RowBlockAllocator(const TabletSchema& tablet_schema,$/;" f class:doris::RowBlockAllocator +RowBlockAllocator src/olap/schema_change.h /^class RowBlockAllocator {$/;" c namespace:doris +RowBlockChanger src/olap/schema_change.cpp /^RowBlockChanger::RowBlockChanger(const TabletSchema& tablet_schema,$/;" f class:doris::RowBlockChanger +RowBlockChanger src/olap/schema_change.h /^class RowBlockChanger {$/;" c namespace:doris +RowBlockInfo src/olap/row_block.h /^ RowBlockInfo() : checksum(0), row_num(0) { }$/;" f struct:doris::RowBlockInfo +RowBlockInfo src/olap/row_block.h /^ RowBlockInfo(uint32_t value, uint32_t num) : checksum(value), row_num(num) { }$/;" f struct:doris::RowBlockInfo +RowBlockInfo src/olap/row_block.h /^struct RowBlockInfo {$/;" s namespace:doris +RowBlockMerger src/olap/schema_change.cpp /^RowBlockMerger::RowBlockMerger(TabletSharedPtr tablet) : _tablet(tablet) {}$/;" f class:doris::RowBlockMerger +RowBlockMerger src/olap/schema_change.cpp /^class RowBlockMerger {$/;" c namespace:doris file: +RowBlockPosition src/olap/olap_index.h /^ RowBlockPosition() : segment(0), block_size(0), data_offset(0), index_offset(0) {}$/;" f struct:doris::RowBlockPosition +RowBlockPosition src/olap/olap_index.h /^struct RowBlockPosition {$/;" s namespace:doris +RowBlockRow src/olap/row_block2.h /^ RowBlockRow(const RowBlockV2* block, size_t row_index) : _block(block), _row_index(row_index) { }$/;" f class:doris::RowBlockRow +RowBlockRow src/olap/row_block2.h /^class RowBlockRow {$/;" c namespace:doris +RowBlockSorter src/olap/schema_change.cpp /^RowBlockSorter::RowBlockSorter(RowBlockAllocator* row_block_allocator) :$/;" f class:doris::RowBlockSorter +RowBlockSorter src/olap/schema_change.cpp /^class RowBlockSorter {$/;" c namespace:doris file: +RowBlockV2 src/olap/row_block2.cpp /^RowBlockV2::RowBlockV2(const Schema& schema, uint16_t capacity)$/;" f class:doris::RowBlockV2 +RowBlockV2 src/olap/row_block2.h /^class RowBlockV2 {$/;" c namespace:doris +RowCursor src/olap/row_cursor.cpp /^RowCursor::RowCursor() :$/;" f class:doris::RowCursor +RowCursor src/olap/row_cursor.h /^class RowCursor {$/;" c namespace:doris +RowCursorCell src/olap/row_cursor_cell.h /^ RowCursorCell(const void* ptr) : _ptr((void*)ptr) { }$/;" f struct:doris::RowCursorCell +RowCursorCell src/olap/row_cursor_cell.h /^ RowCursorCell(void* ptr) : _ptr(ptr) { }$/;" f struct:doris::RowCursorCell +RowCursorCell src/olap/row_cursor_cell.h /^struct RowCursorCell {$/;" s namespace:doris +RowCursorComparator src/olap/memtable.cpp /^MemTable::RowCursorComparator::RowCursorComparator(const Schema* schema) : _schema(schema) {}$/;" f class:doris::MemTable::RowCursorComparator +RowCursorComparator src/olap/memtable.h /^ class RowCursorComparator {$/;" c class:doris::MemTable +RowDescriptor src/runtime/descriptors.cpp /^RowDescriptor::RowDescriptor($/;" f class:doris::RowDescriptor +RowDescriptor src/runtime/descriptors.cpp /^RowDescriptor::RowDescriptor(TupleDescriptor* tuple_desc, bool is_nullable) :$/;" f class:doris::RowDescriptor +RowDescriptor src/runtime/descriptors.h /^ RowDescriptor() {}$/;" f class:doris::RowDescriptor +RowDescriptor src/runtime/descriptors.h /^ RowDescriptor(const RowDescriptor& desc) :$/;" f class:doris::RowDescriptor +RowDescriptor src/runtime/descriptors.h /^class RowDescriptor {$/;" c namespace:doris +RowIdx src/runtime/buffered_tuple_stream.h /^ struct RowIdx {$/;" s class:doris::BufferedTupleStream +RowIdx src/runtime/buffered_tuple_stream2.h /^ struct RowIdx {$/;" s class:doris::BufferedTupleStream2 +RowNumType src/olap/push_handler.h /^ typedef uint32_t RowNumType;$/;" t class:doris::LzoBinaryReader +RowRange src/olap/rowset/segment_v2/row_ranges.h /^ RowRange() : _from(0), _to(0) { }$/;" f class:doris::segment_v2::RowRange +RowRange src/olap/rowset/segment_v2/row_ranges.h /^ RowRange(int64_t from, int64_t to) : _from(from), _to(to) { }$/;" f class:doris::segment_v2::RowRange +RowRange src/olap/rowset/segment_v2/row_ranges.h /^class RowRange {$/;" c namespace:doris::segment_v2 +RowRanges src/olap/rowset/segment_v2/row_ranges.h /^ RowRanges() : _count(0) { }$/;" f class:doris::segment_v2::RowRanges +RowRanges src/olap/rowset/segment_v2/row_ranges.h /^class RowRanges {$/;" c namespace:doris::segment_v2 +RowRangesTest test/olap/rowset/segment_v2/row_ranges_test.cpp /^class RowRangesTest : public testing::Test {$/;" c namespace:doris::segment_v2 file: +Rowset src/olap/rowset/rowset.cpp /^Rowset::Rowset(const TabletSchema *schema,$/;" f class:doris::Rowset +Rowset src/olap/rowset/rowset.h /^class Rowset : public std::enable_shared_from_this {$/;" c namespace:doris +RowsetConverter src/olap/rowset/rowset_converter.h /^ RowsetConverter(const TabletMetaSharedPtr& tablet_meta) : _tablet_meta(tablet_meta) { }$/;" f class:doris::RowsetConverter +RowsetConverter src/olap/rowset/rowset_converter.h /^class RowsetConverter {$/;" c namespace:doris +RowsetConverterTest test/olap/rowset/rowset_converter_test.cpp /^class RowsetConverterTest : public testing::Test {$/;" c namespace:doris file: +RowsetFactory src/olap/rowset/rowset_factory.h /^class RowsetFactory {$/;" c namespace:doris +RowsetGraph src/olap/rowset_graph.h /^class RowsetGraph {$/;" c namespace:doris +RowsetId src/olap/olap_common.h /^struct RowsetId {$/;" s namespace:doris +RowsetIdGenerator src/olap/rowset/rowset_id_generator.h /^ RowsetIdGenerator() {}$/;" f class:doris::RowsetIdGenerator +RowsetIdGenerator src/olap/rowset/rowset_id_generator.h /^class RowsetIdGenerator {$/;" c namespace:doris +RowsetMeta src/olap/rowset/rowset_meta.h /^class RowsetMeta {$/;" c namespace:doris +RowsetMetaManager src/olap/rowset/rowset_meta_manager.h /^class RowsetMetaManager {$/;" c namespace:doris +RowsetMetaManagerTest test/olap/rowset/rowset_meta_manager_test.cpp /^class RowsetMetaManagerTest : public testing::Test {$/;" c namespace:doris file: +RowsetMetaTest test/olap/rowset/rowset_meta_test.cpp /^class RowsetMetaTest : public testing::Test {$/;" c namespace:doris file: +RowsetReader src/olap/rowset/rowset_reader.h /^class RowsetReader {$/;" c namespace:doris +RowsetReaderContext src/olap/rowset/rowset_reader_context.h /^struct RowsetReaderContext {$/;" s namespace:doris +RowsetState src/olap/rowset/rowset.h /^enum RowsetState {$/;" g namespace:doris +RowsetStateMachine src/olap/rowset/rowset.h /^ RowsetStateMachine() : _rowset_state(ROWSET_UNLOADED) { }$/;" f class:doris::RowsetStateMachine +RowsetStateMachine src/olap/rowset/rowset.h /^class RowsetStateMachine {$/;" c namespace:doris +RowsetWriter src/olap/rowset/rowset_writer.h /^class RowsetWriter {$/;" c namespace:doris +RowsetWriterContext src/olap/rowset/rowset_writer_context.h /^ RowsetWriterContext() :$/;" f struct:doris::RowsetWriterContext +RowsetWriterContext src/olap/rowset/rowset_writer_context.h /^struct RowsetWriterContext {$/;" s namespace:doris +RowwiseIterator src/olap/iterators.h /^ RowwiseIterator() { }$/;" f class:doris::RowwiseIterator +RowwiseIterator src/olap/iterators.h /^class RowwiseIterator {$/;" c namespace:doris +Run src/runtime/merge_sorter.cpp /^MergeSorter::Run::Run(MergeSorter* parent, TupleDescriptor* sort_tuple_desc,$/;" f class:doris::MergeSorter::Run +Run src/runtime/merge_sorter.cpp /^class MergeSorter::Run {$/;" c class:doris::MergeSorter file: +Run src/runtime/spill_sorter.cc /^SpillSorter::Run::Run($/;" f class:doris::SpillSorter::Run +Run src/runtime/spill_sorter.cc /^class SpillSorter::Run {$/;" c class:doris::SpillSorter file: +Run test/util/blocking_queue_test.cpp /^ void Run() {$/;" f class:doris::MultiThreadTest +RunBatchSupplier src/runtime/sorted_run_merger.h /^ typedef boost::function RunBatchSupplier;$/;" t class:doris::SortedRunMerger +RunLengthByteReader src/olap/rowset/run_length_byte_reader.cpp /^RunLengthByteReader::RunLengthByteReader(ReadOnlyFileStream* input) : $/;" f class:doris::RunLengthByteReader +RunLengthByteReader src/olap/rowset/run_length_byte_reader.h /^class RunLengthByteReader {$/;" c namespace:doris +RunLengthByteWriter src/olap/rowset/run_length_byte_writer.cpp /^RunLengthByteWriter::RunLengthByteWriter(OutStream* output) : $/;" f class:doris::RunLengthByteWriter +RunLengthByteWriter src/olap/rowset/run_length_byte_writer.h /^class RunLengthByteWriter {$/;" c namespace:doris +RunLengthIntegerReader src/olap/rowset/run_length_integer_reader.cpp /^RunLengthIntegerReader::RunLengthIntegerReader(ReadOnlyFileStream* input, bool is_singed)$/;" f class:doris::RunLengthIntegerReader +RunLengthIntegerReader src/olap/rowset/run_length_integer_reader.h /^class RunLengthIntegerReader {$/;" c namespace:doris +RunLengthIntegerWriter src/olap/rowset/run_length_integer_writer.cpp /^RunLengthIntegerWriter::RunLengthIntegerWriter(OutStream* output, bool is_singed) : $/;" f class:doris::RunLengthIntegerWriter +RunLengthIntegerWriter src/olap/rowset/run_length_integer_writer.h /^class RunLengthIntegerWriter {$/;" c namespace:doris +RunTest test/util/decompress_test.cpp /^ void RunTest(THdfsCompression::type format) {$/;" f class:doris::DecompressorTest +Rune src/gutil/utf/utf.h /^typedef signed int Rune; \/* Code-point values in Unicode 4.0 are 21 bits wide.*\/$/;" t +Rune1 src/gutil/utf/rune.c /^ Rune1 = (1<<(Bit1+0*Bitx))-1, \/* 0000 0000 0111 1111 *\/$/;" e enum:__anon8 file: +Rune2 src/gutil/utf/rune.c /^ Rune2 = (1<<(Bit2+1*Bitx))-1, \/* 0000 0111 1111 1111 *\/$/;" e enum:__anon8 file: +Rune3 src/gutil/utf/rune.c /^ Rune3 = (1<<(Bit3+2*Bitx))-1, \/* 1111 1111 1111 1111 *\/$/;" e enum:__anon8 file: +Rune4 src/gutil/utf/rune.c /^ Rune4 = (1<<(Bit4+3*Bitx))-1,$/;" e enum:__anon8 file: +Runeerror src/gutil/utf/utf.h /^ Runeerror = 0xFFFD, \/* decoding error in UTF *\/$/;" e enum:__anon7 +Runemax src/gutil/utf/utf.h /^ Runemax = 0x10FFFF, \/* maximum rune value *\/$/;" e enum:__anon7 +Runeself src/gutil/utf/utf.h /^ Runeself = 0x80, \/* rune and UTF sequences are the same (<) *\/$/;" e enum:__anon7 +Runesync src/gutil/utf/utf.h /^ Runesync = 0x80, \/* cannot represent part of a UTF sequence (<) *\/$/;" e enum:__anon7 +Runnable src/util/threadpool.h /^class Runnable {$/;" c namespace:doris +RunningOnValgrind src/gutil/dynamic_annotations.c /^int RunningOnValgrind(void) {$/;" f +RuntimeError src/common/status.h /^ static Status RuntimeError(const Slice& msg,$/;" f class:doris::Status +RuntimeProfile src/util/runtime_profile.cpp /^RuntimeProfile::RuntimeProfile(ObjectPool* pool, const std::string& name, bool is_averaged_profile) :$/;" f class:doris::RuntimeProfile +RuntimeProfile src/util/runtime_profile.h /^class RuntimeProfile {$/;" c namespace:doris +RuntimeState src/runtime/runtime_state.cpp /^RuntimeState::RuntimeState($/;" f class:doris::RuntimeState +RuntimeState src/runtime/runtime_state.cpp /^RuntimeState::RuntimeState(const TQueryGlobals& query_globals)$/;" f class:doris::RuntimeState +RuntimeState src/runtime/runtime_state.h /^class RuntimeState {$/;" c namespace:doris +RuntimeState src/udf/udf.cpp /^class RuntimeState {$/;" c namespace:doris file: +SAFE_DELETE src/olap/olap_define.h 445;" d +SAFE_DELETE_ARRAY src/olap/olap_define.h 453;" d +SALT src/olap/rowset/segment_v2/block_split_bloom_filter.cpp /^const uint32_t BlockSplitBloomFilter::SALT[8] = {0x47b6137b, 0x44974d91, 0x8824ad5b, 0xa2b7289d,$/;" m class:doris::segment_v2::BlockSplitBloomFilter file: +SALT src/olap/rowset/segment_v2/block_split_bloom_filter.h /^ static const uint32_t SALT[BITS_SET_PER_BLOCK];$/;" m class:doris::segment_v2::BlockSplitBloomFilter +SAMPLING_COUNTER src/util/runtime_profile.h /^ SAMPLING_COUNTER,$/;" e enum:doris::RuntimeProfile::PeriodicCounterType +SA_RESTORER src/gutil/linux_syscall_support.h 574;" d +SCALE src/runtime/decimalv2_value.h /^ static const int32_t SCALE = 9;$/;" m class:doris::DecimalV2Value +SCANF_ATTRIBUTE src/gutil/port.h 313;" d +SCANNER_THREAD_TOTAL_WALLCLOCK_TIME src/exec/olap_scanner.cpp /^static const std::string SCANNER_THREAD_TOTAL_WALLCLOCK_TIME =$/;" m namespace:doris file: +SCHEMA_HASH src/http/action/checksum_action.cpp /^const std::string SCHEMA_HASH = "schema_hash";$/;" m namespace:doris file: +SCHEMA_HASH src/http/action/reload_tablet_action.cpp /^const std::string SCHEMA_HASH = "schema_hash";$/;" m namespace:doris file: +SCHEMA_HASH src/http/action/restore_tablet_action.cpp /^const std::string SCHEMA_HASH = "schema_hash";$/;" m namespace:doris file: +SCHEMA_HASH src/http/action/snapshot_action.cpp /^const std::string SCHEMA_HASH = "schema_hash";$/;" m namespace:doris file: +SCOPED_CLEANUP src/util/scoped_cleanup.h 34;" d +SCOPED_RAW_TIMER src/util/runtime_profile.h 52;" d +SCOPED_THREAD_COUNTER_MEASUREMENT src/util/runtime_profile.h 57;" d +SCOPED_TIMER src/util/runtime_profile.h 50;" d +SCRATCH_DIR test/runtime/buffered_block_mgr2_test.cpp /^const static string SCRATCH_DIR = "\/tmp\/doris-scratch";$/;" v file: +SCRATCH_SUFFIX test/runtime/buffered_block_mgr2_test.cpp /^const static string SCRATCH_SUFFIX = "\/doris-scratch";$/;" v file: +SECOND src/runtime/datetime_value.h /^ SECOND,$/;" e enum:doris::TimeUnit +SECOND src/util/debug_util.cpp 36;" d file: +SECOND src/util/pretty_printer.h /^ static const int64_t SECOND = 1000;$/;" m class:doris::PrettyPrinter +SECOND_KEY src/http/action/pprof_actions.cpp /^static const std::string SECOND_KEY = "seconds";$/;" m namespace:doris file: +SECOND_MICROSECOND src/runtime/datetime_value.h /^ SECOND_MICROSECOND,$/;" e enum:doris::TimeUnit +SEC_WEBSOCKET_ACCEPT src/http/http_headers.cpp /^const char* HttpHeaders::SEC_WEBSOCKET_ACCEPT = "Sec-WebSocket-Accept";$/;" m class:doris::HttpHeaders file: +SEC_WEBSOCKET_ACCEPT src/http/http_headers.h /^ static const char* SEC_WEBSOCKET_ACCEPT;$/;" m class:doris::HttpHeaders +SEC_WEBSOCKET_KEY src/http/http_headers.cpp /^const char* HttpHeaders::SEC_WEBSOCKET_KEY = "Sec-WebSocket-Key";$/;" m class:doris::HttpHeaders file: +SEC_WEBSOCKET_KEY src/http/http_headers.h /^ static const char* SEC_WEBSOCKET_KEY;$/;" m class:doris::HttpHeaders +SEC_WEBSOCKET_KEY1 src/http/http_headers.cpp /^const char* HttpHeaders::SEC_WEBSOCKET_KEY1 = "Sec-WebSocket-Key1";$/;" m class:doris::HttpHeaders file: +SEC_WEBSOCKET_KEY1 src/http/http_headers.h /^ static const char* SEC_WEBSOCKET_KEY1;$/;" m class:doris::HttpHeaders +SEC_WEBSOCKET_KEY2 src/http/http_headers.cpp /^const char* HttpHeaders::SEC_WEBSOCKET_KEY2 = "Sec-WebSocket-Key2";$/;" m class:doris::HttpHeaders file: +SEC_WEBSOCKET_KEY2 src/http/http_headers.h /^ static const char* SEC_WEBSOCKET_KEY2;$/;" m class:doris::HttpHeaders +SEC_WEBSOCKET_LOCATION src/http/http_headers.cpp /^const char* HttpHeaders::SEC_WEBSOCKET_LOCATION = "Sec-WebSocket-Location";$/;" m class:doris::HttpHeaders file: +SEC_WEBSOCKET_LOCATION src/http/http_headers.h /^ static const char* SEC_WEBSOCKET_LOCATION;$/;" m class:doris::HttpHeaders +SEC_WEBSOCKET_ORIGIN src/http/http_headers.cpp /^const char* HttpHeaders::SEC_WEBSOCKET_ORIGIN = "Sec-WebSocket-Origin";$/;" m class:doris::HttpHeaders file: +SEC_WEBSOCKET_ORIGIN src/http/http_headers.h /^ static const char* SEC_WEBSOCKET_ORIGIN;$/;" m class:doris::HttpHeaders +SEC_WEBSOCKET_PROTOCOL src/http/http_headers.cpp /^const char* HttpHeaders::SEC_WEBSOCKET_PROTOCOL = "Sec-WebSocket-Protocol";$/;" m class:doris::HttpHeaders file: +SEC_WEBSOCKET_PROTOCOL src/http/http_headers.h /^ static const char* SEC_WEBSOCKET_PROTOCOL;$/;" m class:doris::HttpHeaders +SEC_WEBSOCKET_VERSION src/http/http_headers.cpp /^const char* HttpHeaders::SEC_WEBSOCKET_VERSION = "Sec-WebSocket-Version";$/;" m class:doris::HttpHeaders file: +SEC_WEBSOCKET_VERSION src/http/http_headers.h /^ static const char* SEC_WEBSOCKET_VERSION;$/;" m class:doris::HttpHeaders +SEED_PRIMES src/exec/new_partitioned_hash_table.cc /^static uint32_t SEED_PRIMES[] = {$/;" v file: +SEED_PRIMES src/exec/partitioned_hash_table.cc /^static uint32_t SEED_PRIMES[] = {$/;" m namespace:doris file: +SEE_OTHER src/http/http_status.h /^ SEE_OTHER = 303,$/;" e enum:doris::HttpStatus +SEGMENT_GROUP_PARAM_VALIDATE src/olap/rowset/segment_group.cpp 44;" d file: +SERIAL src/util/threadpool.h /^ SERIAL,$/;" m class:doris::ThreadPool::ExecutionMode +SERVER src/http/http_headers.cpp /^const char* HttpHeaders::SERVER = "Server";$/;" m class:doris::HttpHeaders file: +SERVER src/http/http_headers.h /^ static const char* SERVER;$/;" m class:doris::HttpHeaders +SERVICE_UNAVAILABLE src/http/http_status.h /^ SERVICE_UNAVAILABLE = 503,$/;" e enum:doris::HttpStatus +SET_COOKIE src/http/http_headers.cpp /^const char* HttpHeaders::SET_COOKIE = "Set-Cookie";$/;" m class:doris::HttpHeaders file: +SET_COOKIE src/http/http_headers.h /^ static const char* SET_COOKIE;$/;" m class:doris::HttpHeaders +SET_COOKIE2 src/http/http_headers.cpp /^const char* HttpHeaders::SET_COOKIE2 = "Set-Cookie2";$/;" m class:doris::HttpHeaders file: +SET_COOKIE2 src/http/http_headers.h /^ static const char* SET_COOKIE2;$/;" m class:doris::HttpHeaders +SET_FIELD src/common/configbase.cpp 229;" d file: +SET_LOOKUP src/exprs/new_in_predicate.h /^ SET_LOOKUP,$/;" e enum:doris::InPredicate::Strategy +SEVEN src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +SEVENTEEN src/olap/serialize.h /^ THIRTEEN, FOURTEEN, FIFTEEN, SIXTEEN, SEVENTEEN, EIGHTEEN, NINETEEN,$/;" e enum:doris::ser::FixedBitSize +SHARD_PREFIX src/runtime/load_path_mgr.cpp /^static const std::string SHARD_PREFIX = "__shard_";$/;" m namespace:doris file: +SHELL build/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/agent/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/common/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/env/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/exec/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/exprs/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/gen_cpp/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/geo/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/gutil/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/http/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/olap/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/olap/fs/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/olap/rowset/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/runtime/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/service/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/testutil/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/tools/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/udf/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/udf_samples/Makefile /^SHELL = \/bin\/sh$/;" m +SHELL build/src/util/Makefile /^SHELL = \/bin\/sh$/;" m +SHORT_REPEAT src/olap/rowset/run_length_integer_writer.h /^ SHORT_REPEAT = 0,$/;" e enum:doris::RunLengthIntegerWriter::EncodingType +SHOW_INFO src/http/action/compaction_action.h /^ SHOW_INFO = 1,$/;" e enum:doris::CompactionActionType +SIGNED_RIGHT_SHIFT_IS src/util/thrift_util.cpp 40;" d file: +SIMDUTF8CHECK_H src/util/simdutf8check.h 19;" d +SIMULATE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 187;" d file: +SIMULATE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 202;" d file: +SIMULATE_ID build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 21;" d file: +SIMULATE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 181;" d file: +SIMULATE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 196;" d file: +SIMULATE_ID build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 21;" d file: +SIMULATE_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 194;" d file: +SIMULATE_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 209;" d file: +SIMULATE_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 37;" d file: +SIMULATE_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 188;" d file: +SIMULATE_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 203;" d file: +SIMULATE_VERSION_MAJOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 37;" d file: +SIMULATE_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 195;" d file: +SIMULATE_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 210;" d file: +SIMULATE_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 38;" d file: +SIMULATE_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 189;" d file: +SIMULATE_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 204;" d file: +SIMULATE_VERSION_MINOR build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 38;" d file: +SINGLE src/util/bitmap_value.h /^ SINGLE = 1, \/\/ single element$/;" e enum:doris::BitmapValue::BitmapDataType +SINGLE32 src/util/bitmap_value.h /^ SINGLE32 = 1,$/;" e enum:doris::BitmapTypeCode::type +SINGLE64 src/util/bitmap_value.h /^ SINGLE64 = 3,$/;" e enum:doris::BitmapTypeCode::type +SINGLE_NODE output/udf/include/uda_test_harness.h /^ SINGLE_NODE = 1,$/;" e enum:doris_udf::UdaExecutionMode +SINGLE_NODE src/udf/uda_test_harness.h /^ SINGLE_NODE = 1,$/;" e enum:doris_udf::UdaExecutionMode +SINGLE_THREADED_TID test/runtime/buffered_block_mgr2_test.cpp /^ static const int SINGLE_THREADED_TID = -1;$/;" m class:doris::BufferedBlockMgrTest file: +SIX src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +SIXTEEN src/olap/serialize.h /^ THIRTEEN, FOURTEEN, FIFTEEN, SIXTEEN, SEVENTEEN, EIGHTEEN, NINETEEN,$/;" e enum:doris::ser::FixedBitSize +SIXTYFOUR src/olap/serialize.h /^ TWENTYEIGHT, THIRTY, THIRTYTWO, FORTY, FORTYEIGHT, FIFTYSIX, SIXTYFOUR$/;" e enum:doris::ser::FixedBitSize +SIZECENTRALDIRITEM src/util/minizip/unzip.c 118;" d file: +SIZEOF_CHARP src/util/mysql_global.h 35;" d +SIZEOF_MEMBER src/gutil/port.h 1180;" d +SIZEZIPLOCALHEADER src/util/minizip/unzip.c 119;" d file: +SIZE_OF_TYPE src/olap/rowset/segment_v2/bitshuffle_page.h /^ SIZE_OF_TYPE = TypeTraits::size$/;" e enum:doris::segment_v2::BitShufflePageDecoder::__anon45 +SIZE_OF_TYPE src/olap/rowset/segment_v2/bitshuffle_page.h /^ SIZE_OF_TYPE = TypeTraits::size$/;" e enum:doris::segment_v2::BitshufflePageBuilder::__anon44 +SIZE_OF_TYPE src/olap/rowset/segment_v2/plain_page.h /^ SIZE_OF_TYPE = TypeTraits::size$/;" e enum:doris::segment_v2::PlainPageBuilder::__anon46 +SIZE_OF_TYPE src/olap/rowset/segment_v2/plain_page.h /^ SIZE_OF_TYPE = TypeTraits::size$/;" e enum:doris::segment_v2::PlainPageDecoder::__anon47 +SIZE_OF_TYPE src/olap/rowset/segment_v2/rle_page.h /^ SIZE_OF_TYPE = TypeTraits::size$/;" e enum:doris::segment_v2::RlePageBuilder::__anon49 +SIZE_OF_TYPE src/olap/rowset/segment_v2/rle_page.h /^ SIZE_OF_TYPE = TypeTraits::size$/;" e enum:doris::segment_v2::RlePageDecoder::__anon50 +SLICE_PARAM_VALIDATE src/olap/rowset/segment_group.cpp 61;" d file: +SMALLINT_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t SMALLINT_SIZE = sizeof(int16_t);$/;" m class:doris::AggFnEvaluator +SMALLINT_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t SMALLINT_SIZE = sizeof(int16_t);$/;" m class:doris::NewAggFnEvaluator +SMALL_PAGE_SIZE src/runtime/bufferpool/system_allocator.cc /^static int64_t SMALL_PAGE_SIZE = 4LL * 1024;$/;" m namespace:doris file: +SNAPSHOT_PREFIX src/olap/olap_define.h /^static const std::string SNAPSHOT_PREFIX = "\/snapshot";$/;" m namespace:doris +SPR_TB src/gutil/cycleclock-inl.h 83;" d +SPR_TBU src/gutil/cycleclock-inl.h 84;" d +SQLFilterOp src/exec/olap_utils.h /^enum SQLFilterOp {$/;" g namespace:doris +SRC_TUPLE_SLOT_ID_START test/exec/parquet_scanner_test.cpp 73;" d file: +SSD_UC src/olap/options.cpp /^static std::string SSD_UC = "SSD";$/;" m namespace:doris file: +SSE src/gutil/cpu.h /^ SSE,$/;" e enum:base::final::IntelMicroArchitecture +SSE2 src/gutil/cpu.h /^ SSE2,$/;" e enum:base::final::IntelMicroArchitecture +SSE3 src/gutil/cpu.h /^ SSE3,$/;" e enum:base::final::IntelMicroArchitecture +SSE41 src/gutil/cpu.h /^ SSE41,$/;" e enum:base::final::IntelMicroArchitecture +SSE42 src/gutil/cpu.h /^ SSE42,$/;" e enum:base::final::IntelMicroArchitecture +SSE4_1 src/util/cpu_info.h /^ static const int64_t SSE4_1 = (1 << 2);$/;" m class:doris::CpuInfo +SSE4_2 src/util/cpu_info.h /^ static const int64_t SSE4_2 = (1 << 3);$/;" m class:doris::CpuInfo +SSE_BITMASK src/util/sse_util.hpp /^static const int SSE_BITMASK[CHARS_PER_128_BIT_REGISTER] = {$/;" m namespace:doris::sse_util +SSSE3 src/gutil/cpu.h /^ SSSE3,$/;" e enum:base::final::IntelMicroArchitecture +SSSE3 src/util/cpu_info.h /^ static const int64_t SSSE3 = (1 << 1);$/;" m class:doris::CpuInfo +SStringPrintf src/gutil/stringprintf.cc /^const string& SStringPrintf(string* dst, const char* format, ...) {$/;" f +STARTING test/olap/skiplist_test.cpp /^ STARTING,$/;" e enum:doris::TestState::ReaderState file: +STATIC_ANALYSIS src/gutil/port.h 284;" d +STEP1 src/util/crc32c.cpp 346;" d file: +STEP1 src/util/crc32c.cpp 375;" d file: +STLAppendToString src/gutil/stl_util.h /^inline void STLAppendToString(string* str, const char* ptr, size_t n) {$/;" f +STLAssignToString src/gutil/stl_util.h /^inline void STLAssignToString(string* str, const char* ptr, size_t n) {$/;" f +STLAssignToVector src/gutil/stl_util.h /^inline void STLAssignToVector(vector* vec,$/;" f +STLAssignToVector src/gutil/stl_util.h /^inline void STLAssignToVector(vector* vec,$/;" f +STLAssignToVectorChar src/gutil/stl_util.h /^inline void STLAssignToVectorChar(vector* vec,$/;" f +STLClearHashIfBig src/gutil/stl_util.h /^template inline void STLClearHashIfBig(T *obj, size_t limit) {$/;" f +STLClearIfBig src/gutil/stl_util.h /^inline void STLClearIfBig(deque* obj, size_t limit = 1<<20) {$/;" f +STLClearIfBig src/gutil/stl_util.h /^template inline void STLClearIfBig(T* obj, size_t limit = 1<<20) {$/;" f +STLClearObject src/gutil/stl_util.h /^template void STLClearObject(T* obj) {$/;" f +STLClearObject src/gutil/stl_util.h /^void STLClearObject(deque* obj) {$/;" f +STLCountingAllocator src/gutil/stl_util.h /^ STLCountingAllocator() : bytes_used_(NULL) { }$/;" f class:STLCountingAllocator +STLCountingAllocator src/gutil/stl_util.h /^ STLCountingAllocator(const STLCountingAllocator& x)$/;" f class:STLCountingAllocator +STLCountingAllocator src/gutil/stl_util.h /^ STLCountingAllocator(int64* b) : bytes_used_(b) {} \/\/ TODO(user): explicit?$/;" f class:STLCountingAllocator +STLCountingAllocator src/gutil/stl_util.h /^class STLCountingAllocator : public Alloc {$/;" c +STLDeleteContainerPairFirstPointers src/gutil/stl_util.h /^void STLDeleteContainerPairFirstPointers(ForwardIterator begin,$/;" f +STLDeleteContainerPairPointers src/gutil/stl_util.h /^void STLDeleteContainerPairPointers(ForwardIterator begin,$/;" f +STLDeleteContainerPairSecondPointers src/gutil/stl_util.h /^void STLDeleteContainerPairSecondPointers(ForwardIterator begin,$/;" f +STLDeleteContainerPointers src/gutil/stl_util.h /^void STLDeleteContainerPointers(ForwardIterator begin,$/;" f +STLDeleteElements src/gutil/stl_util.h /^void STLDeleteElements(T *container) {$/;" f +STLDeleteValues src/gutil/stl_util.h /^void STLDeleteValues(T *v) {$/;" f +STLElementDeleter src/gutil/stl_util.h /^ STLElementDeleter(STLContainer *ptr) : container_ptr_(ptr) {}$/;" f class:STLElementDeleter +STLElementDeleter src/gutil/stl_util.h /^template class STLElementDeleter {$/;" c +STLEmptyBaseHandle src/gutil/stl_util.h /^ STLEmptyBaseHandle(const U &b, const Data &d)$/;" f struct:STLEmptyBaseHandle +STLEmptyBaseHandle src/gutil/stl_util.h /^struct STLEmptyBaseHandle : public Base {$/;" s +STLIncludes src/gutil/stl_util.h /^bool STLIncludes(const SortedSTLContainerA &a,$/;" f +STLReserveIfNeeded src/gutil/stl_util.h /^template void STLReserveIfNeeded(T* obj, int new_size) {$/;" f +STLSetDifference src/gutil/stl_util.h /^SortedSTLContainer STLSetDifference(const SortedSTLContainer &a,$/;" f +STLSetDifference src/gutil/stl_util.h /^void STLSetDifference(const SortedSTLContainerA &a,$/;" f +STLSetIntersection src/gutil/stl_util.h /^SortedSTLContainer STLSetIntersection(const SortedSTLContainer &a,$/;" f +STLSetIntersection src/gutil/stl_util.h /^void STLSetIntersection(const SortedSTLContainerA &a,$/;" f +STLSetSymmetricDifference src/gutil/stl_util.h /^SortedSTLContainer STLSetSymmetricDifference(const SortedSTLContainer &a,$/;" f +STLSetSymmetricDifference src/gutil/stl_util.h /^void STLSetSymmetricDifference(const SortedSTLContainerA &a,$/;" f +STLSetUnion src/gutil/stl_util.h /^SortedSTLContainer STLSetUnion(const SortedSTLContainer &a,$/;" f +STLSetUnion src/gutil/stl_util.h /^void STLSetUnion(const SortedSTLContainerA &a,$/;" f +STLSortAndRemoveDuplicates src/gutil/stl_util.h /^void STLSortAndRemoveDuplicates(T *v) {$/;" f +STLStringResizeUninitialized src/gutil/stl_util.h /^inline void STLStringResizeUninitialized(string* s, size_t new_size) {$/;" f +STLStringSupportsNontrashingResize src/gutil/stl_util.h /^inline bool STLStringSupportsNontrashingResize(const string& s) {$/;" f +STLValueDeleter src/gutil/stl_util.h /^ STLValueDeleter(STLContainer *ptr) : container_ptr_(ptr) {}$/;" f class:STLValueDeleter +STLValueDeleter src/gutil/stl_util.h /^template class STLValueDeleter {$/;" c +STORAGE_MEDIUM_MIGRATE src/agent/task_worker_pool.h /^ STORAGE_MEDIUM_MIGRATE,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +STRCHR_MODE src/util/sse_util.hpp /^static const int STRCHR_MODE = _SIDD_CMP_EQUAL_ANY | _SIDD_UBYTE_OPS;$/;" m namespace:doris::sse_util +STRCMPCASENOSENTIVEFUNCTION src/util/minizip/unzip.c 378;" d file: +STRCMP_MODE src/util/sse_util.hpp /^static const int STRCMP_MODE = _SIDD_CMP_EQUAL_EACH | _SIDD_UBYTE_OPS$/;" m namespace:doris::sse_util +STREAMING_HT_MIN_REDUCTION src/exec/new_partitioned_aggregation_node.cc /^static const StreamingHtMinReductionEntry STREAMING_HT_MIN_REDUCTION[] = {$/;" m namespace:doris file: +STREAMING_HT_MIN_REDUCTION_SIZE src/exec/new_partitioned_aggregation_node.cc /^static const int STREAMING_HT_MIN_REDUCTION_SIZE =$/;" m namespace:doris file: +STREAM_RETRY_ON_EINTR src/gutil/macros.h 273;" d +STREAM_SET src/gutil/port.h 1013;" d +STREAM_SET src/gutil/port.h 1016;" d +STREAM_SETF src/gutil/port.h 1014;" d +STREAM_SETF src/gutil/port.h 1017;" d +STRICT_MODE_KEY src/http/action/mini_load.cpp /^const std::string STRICT_MODE_KEY = "strict_mode";$/;" m namespace:doris file: +STRINGIFY build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 333;" d file: +STRINGIFY build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 318;" d file: +STRINGIFY_HELPER build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 332;" d file: +STRINGIFY_HELPER build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp 317;" d file: +STRINGS test/runtime/buffered_tuple_stream2_test.cpp /^static const StringValue STRINGS[] = {$/;" m namespace:doris file: +STRINGS_ASCII_CTYPE_H_ src/gutil/strings/ascii_ctype.h 25;" d +STRINGS_CHARSET_H_ src/gutil/strings/charset.h 4;" d +STRINGS_ESCAPING_H_ src/gutil/strings/escaping.h 22;" d +STRINGS_FASTMEM_H_ src/gutil/strings/fastmem.h 18;" d +STRINGS_JOIN_H_ src/gutil/strings/join.h 8;" d +STRINGS_MEMUTIL_H_ src/gutil/strings/memutil.h 54;" d +STRINGS_NUMBERS_H_ src/gutil/strings/numbers.h 7;" d +STRINGS_SPLIT_H_ src/gutil/strings/split.h 21;" d +STRINGS_SPLIT_INTERNAL_H_ src/gutil/strings/split_internal.h 18;" d +STRINGS_STRCAT_H_ src/gutil/strings/strcat.h 8;" d +STRINGS_STRINGPIECE_H_ src/gutil/strings/stringpiece.h 113;" d +STRINGS_STRIP_H_ src/gutil/strings/strip.h 8;" d +STRINGS_SUBSTITUTE_H_ src/gutil/strings/substitute.h 13;" d +STRINGS_UTIL_H_ src/gutil/strings/util.h 27;" d +STRING_BINARY_PRED_FN src/exprs/binary_predicate.cpp 360;" d file: +STRING_BINARY_PRED_FNS src/exprs/binary_predicate.cpp 375;" d file: +STRING_LENGTH_RECORD_LENGTH src/exprs/aggregate_functions.cpp /^ static const int STRING_LENGTH_RECORD_LENGTH = 4;$/;" m class:doris::MultiDistinctStringCountState file: +SUB src/exprs/arithmetic_expr.h /^ SUB,$/;" e enum:doris::ArithmeticExpr::BinaryOpType +SUB_LABEL_KEY src/http/action/mini_load.cpp /^const std::string SUB_LABEL_KEY = "sub_label";$/;" m namespace:doris file: +SUCCESS src/olap/compaction.h /^ SUCCESS = 1$/;" e enum:doris::Compaction::CompactionState +SUFFIXES build/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/agent/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/common/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/env/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/exec/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/exprs/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/gen_cpp/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/geo/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/gutil/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/http/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/olap/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/olap/fs/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/olap/rowset/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/runtime/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/service/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/testutil/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/tools/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/udf/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/udf_samples/Makefile /^SUFFIXES =$/;" m +SUFFIXES build/src/util/Makefile /^SUFFIXES =$/;" m +SUM src/exprs/agg_fn.h /^ SUM,$/;" e enum:doris::AggFn::AggregationOp +SUM src/exprs/agg_fn_evaluator.h /^ SUM,$/;" e enum:doris::AggFnEvaluator::AggregationOp +SUMMARY src/util/metrics.h /^ SUMMARY,$/;" m class:doris::MetricType +SUM_DISTINCT src/exprs/agg_fn.h /^ SUM_DISTINCT,$/;" e enum:doris::AggFn::AggregationOp +SUM_DISTINCT src/exprs/agg_fn_evaluator.h /^ SUM_DISTINCT,$/;" e enum:doris::AggFnEvaluator::AggregationOp +SWITCHING_PROTOCOLS src/http/http_status.h /^ SWITCHING_PROTOCOLS = 101,$/;" e enum:doris::HttpStatus +SYNC src/olap/fs/file_block_manager.cpp /^ SYNC,$/;" e enum:doris::fs::internal::FileWritableBlock::SyncMode file: +SYS_LINUX_SYSCALL_SUPPORT_H src/gutil/linux_syscall_support.h 131;" d +SYS_PERF_COUNTER src/util/perf_counters.h /^ SYS_PERF_COUNTER,$/;" e enum:doris::PerfCounters::DataSource +SafeSnprintf src/gutil/strings/util.cc /^int SafeSnprintf(char *str, size_t size, const char *format, ...) {$/;" f +SampleFn src/util/runtime_profile.h /^ typedef boost::function SampleFn;$/;" t class:doris::RuntimeProfile +SamplingCounterInfo src/util/runtime_profile.h /^ struct SamplingCounterInfo {$/;" s class:doris::RuntimeProfile +SamplingCounterMap src/util/runtime_profile.h /^ typedef std::map SamplingCounterMap;$/;" t struct:doris::RuntimeProfile::PeriodicCounterUpdateState +SaveReservation src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::ClientHandle::SaveReservation(SubReservation* dst, int64_t bytes) {$/;" f class:doris::BufferPool::ClientHandle +ScalarFnCall src/exprs/scalar_fn_call.cpp /^ScalarFnCall::ScalarFnCall(const TExprNode& node) : $/;" f class:doris::ScalarFnCall +ScalarFnCall src/exprs/scalar_fn_call.h /^class ScalarFnCall : public Expr {$/;" c namespace:doris +ScanContext src/runtime/external_scan_context_mgr.h /^ ScanContext(const TUniqueId& fragment_id, int64_t offset) : fragment_instance_id(fragment_id), offset(offset) {}$/;" f struct:doris::ScanContext +ScanContext src/runtime/external_scan_context_mgr.h /^ ScanContext(std::string id) : context_id(std::move(id)) {}$/;" f struct:doris::ScanContext +ScanContext src/runtime/external_scan_context_mgr.h /^struct ScanContext {$/;" s namespace:doris +ScanForFirstWord src/gutil/strings/util.cc /^const char* ScanForFirstWord(const char* the_string, const char** end_ptr) {$/;" f +ScanForFirstWord src/gutil/strings/util.h /^inline char* ScanForFirstWord(char* the_string, char** end_ptr) {$/;" f +ScanNode src/exec/scan_node.h /^ ScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)$/;" f class:doris::ScanNode +ScanNode src/exec/scan_node.h /^class ScanNode : public ExecNode {$/;" c namespace:doris +ScanRange src/runtime/disk_io_mgr.h /^ ScanRange() : ScanRange(-1) {}$/;" f class:doris::DiskIoMgr::ScanRange +ScanRange src/runtime/disk_io_mgr.h /^ class ScanRange : public RequestRange {$/;" c class:doris::DiskIoMgr +ScanRange src/runtime/disk_io_mgr_scan_range.cc /^DiskIoMgr::ScanRange::ScanRange(int capacity) : _ready_buffers_capacity(capacity) {$/;" f class:doris::DiskIoMgr::ScanRange +ScanTest test/exec/partitioned_hash_table_test.cpp /^ void ScanTest(bool quadratic, int initial_size, int rows_to_insert,$/;" f class:doris::PartitionedHashTableTest +ScannerCounter src/exec/base_scanner.h /^ ScannerCounter() :$/;" f struct:doris::ScannerCounter +ScannerCounter src/exec/base_scanner.h /^struct ScannerCounter {$/;" s namespace:doris +ScavengeBuffers src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::BufferAllocator::ScavengeBuffers($/;" f class:doris::BufferPool::BufferAllocator +Schema src/olap/schema.cpp /^Schema::Schema(const Schema& other) {$/;" f class:doris::Schema +Schema src/olap/schema.h /^ Schema(const TabletSchema& tablet_schema) {$/;" f class:doris::Schema +Schema src/olap/schema.h /^ Schema(const std::vector& columns, const std::vector& col_ids) {$/;" f class:doris::Schema +Schema src/olap/schema.h /^ Schema(const std::vector& columns, size_t num_key_columns) {$/;" f class:doris::Schema +Schema src/olap/schema.h /^ Schema(const std::vector& cols, size_t num_key_columns) {$/;" f class:doris::Schema +Schema src/olap/schema.h /^class Schema {$/;" c namespace:doris +SchemaAuthorScannerTest test/exec/schema_scanner/schema_authors_scanner_test.cpp /^ SchemaAuthorScannerTest() {$/;" f class:doris::SchemaAuthorScannerTest +SchemaAuthorScannerTest test/exec/schema_scanner/schema_authors_scanner_test.cpp /^class SchemaAuthorScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaChange src/olap/schema_change.h /^ SchemaChange() : _filtered_rows(0), _merged_rows(0) {}$/;" f class:doris::SchemaChange +SchemaChange src/olap/schema_change.h /^class SchemaChange {$/;" c namespace:doris +SchemaChangeDirectly src/olap/schema_change.cpp /^SchemaChangeDirectly::SchemaChangeDirectly($/;" f class:doris::SchemaChangeDirectly +SchemaChangeDirectly src/olap/schema_change.h /^class SchemaChangeDirectly : public SchemaChange {$/;" c namespace:doris +SchemaChangeHandler src/olap/schema_change.h /^ SchemaChangeHandler() {}$/;" f class:doris::SchemaChangeHandler +SchemaChangeHandler src/olap/schema_change.h /^class SchemaChangeHandler {$/;" c namespace:doris +SchemaChangeParams src/olap/schema_change.h /^ struct SchemaChangeParams {$/;" s class:doris::SchemaChangeHandler +SchemaChangeWithSorting src/olap/schema_change.cpp /^SchemaChangeWithSorting::SchemaChangeWithSorting(const RowBlockChanger& row_block_changer,$/;" f class:doris::SchemaChangeWithSorting +SchemaChangeWithSorting src/olap/schema_change.h /^class SchemaChangeWithSorting : public SchemaChange {$/;" c namespace:doris +SchemaCharsetsScanner src/exec/schema_scanner/schema_charsets_scanner.cpp /^SchemaCharsetsScanner::SchemaCharsetsScanner() : $/;" f class:doris::SchemaCharsetsScanner +SchemaCharsetsScanner src/exec/schema_scanner/schema_charsets_scanner.h /^class SchemaCharsetsScanner : public SchemaScanner {$/;" c namespace:doris +SchemaCharsetsScannerTest test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^ SchemaCharsetsScannerTest() {$/;" f class:doris::SchemaCharsetsScannerTest +SchemaCharsetsScannerTest test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^class SchemaCharsetsScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaCollationsScanner src/exec/schema_scanner/schema_collations_scanner.cpp /^SchemaCollationsScanner::SchemaCollationsScanner() : $/;" f class:doris::SchemaCollationsScanner +SchemaCollationsScanner src/exec/schema_scanner/schema_collations_scanner.h /^class SchemaCollationsScanner : public SchemaScanner {$/;" c namespace:doris +SchemaCollationsScannerTest test/exec/schema_scanner/schema_collations_scanner_test.cpp /^ SchemaCollationsScannerTest() {$/;" f class:doris::SchemaCollationsScannerTest +SchemaCollationsScannerTest test/exec/schema_scanner/schema_collations_scanner_test.cpp /^class SchemaCollationsScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaColumnsScanner src/exec/schema_scanner/schema_columns_scanner.cpp /^SchemaColumnsScanner::SchemaColumnsScanner() : $/;" f class:doris::SchemaColumnsScanner +SchemaColumnsScanner src/exec/schema_scanner/schema_columns_scanner.h /^class SchemaColumnsScanner : public SchemaScanner {$/;" c namespace:doris +SchemaColumnsScannerTest test/exec/schema_scanner/schema_columns_scanner_test.cpp /^ SchemaColumnsScannerTest() {$/;" f class:doris::SchemaColumnsScannerTest +SchemaColumnsScannerTest test/exec/schema_scanner/schema_columns_scanner_test.cpp /^class SchemaColumnsScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaCreateTableScannerTest test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^ SchemaCreateTableScannerTest() {$/;" f class:doris::SchemaCreateTableScannerTest +SchemaCreateTableScannerTest test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^class SchemaCreateTableScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaDummyScanner src/exec/schema_scanner/schema_dummy_scanner.cpp /^SchemaDummyScanner::SchemaDummyScanner() $/;" f class:doris::SchemaDummyScanner +SchemaDummyScanner src/exec/schema_scanner/schema_dummy_scanner.h /^class SchemaDummyScanner : public SchemaScanner {$/;" c namespace:doris +SchemaEnginesScannerTest test/exec/schema_scanner/schema_engines_scanner_test.cpp /^ SchemaEnginesScannerTest() {$/;" f class:doris::SchemaEnginesScannerTest +SchemaEnginesScannerTest test/exec/schema_scanner/schema_engines_scanner_test.cpp /^class SchemaEnginesScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaHash src/olap/olap_common.h /^typedef int32_t SchemaHash;$/;" t namespace:doris +SchemaHelper src/exec/schema_scanner/schema_helper.h /^class SchemaHelper {$/;" c namespace:doris +SchemaMapping src/olap/column_mapping.h /^typedef std::vector SchemaMapping;$/;" t namespace:doris +SchemaMapping src/olap/push_handler.h /^ typedef std::vector SchemaMapping;$/;" t class:doris::PushHandler +SchemaOpenTablesScannerTest test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^ SchemaOpenTablesScannerTest() {$/;" f class:doris::SchemaOpenTablesScannerTest +SchemaOpenTablesScannerTest test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^class SchemaOpenTablesScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaScanNode src/exec/schema_scan_node.cpp /^SchemaScanNode::SchemaScanNode(ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::SchemaScanNode +SchemaScanNode src/exec/schema_scan_node.h /^class SchemaScanNode : public ScanNode {$/;" c namespace:doris +SchemaScanNodeTest test/exec/schema_scan_node_test.cpp /^ SchemaScanNodeTest() : _runtim_state("test") {$/;" f class:doris::SchemaScanNodeTest +SchemaScanNodeTest test/exec/schema_scan_node_test.cpp /^class SchemaScanNodeTest : public testing::Test {$/;" c namespace:doris file: +SchemaScanner src/exec/schema_scanner.cpp /^SchemaScanner::SchemaScanner(ColumnDesc* columns, int column_num)$/;" f class:doris::SchemaScanner +SchemaScanner src/exec/schema_scanner.h /^class SchemaScanner {$/;" c namespace:doris +SchemaScannerParam src/exec/schema_scanner.h /^ SchemaScannerParam()$/;" f struct:doris::SchemaScannerParam +SchemaScannerParam src/exec/schema_scanner.h /^struct SchemaScannerParam {$/;" s namespace:doris +SchemaScannerTest test/exec/schema_scanner_test.cpp /^ SchemaScannerTest() {$/;" f class:doris::SchemaScannerTest +SchemaScannerTest test/exec/schema_scanner_test.cpp /^class SchemaScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaSchemataScanner src/exec/schema_scanner/schema_schemata_scanner.cpp /^SchemaSchemataScanner::SchemaSchemataScanner() : $/;" f class:doris::SchemaSchemataScanner +SchemaSchemataScanner src/exec/schema_scanner/schema_schemata_scanner.h /^class SchemaSchemataScanner : public SchemaScanner {$/;" c namespace:doris +SchemaSchemataScannerTest test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^ SchemaSchemataScannerTest() {$/;" f class:doris::SchemaSchemataScannerTest +SchemaSchemataScannerTest test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^class SchemaSchemataScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaTableDescriptor src/runtime/descriptors.cpp /^SchemaTableDescriptor::SchemaTableDescriptor(const TTableDescriptor& tdesc)$/;" f class:doris::SchemaTableDescriptor +SchemaTableDescriptor src/runtime/descriptors.h /^class SchemaTableDescriptor : public TableDescriptor {$/;" c namespace:doris +SchemaTableNamesScannerTest test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^ SchemaTableNamesScannerTest() {$/;" f class:doris::SchemaTableNamesScannerTest +SchemaTableNamesScannerTest test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^class SchemaTableNamesScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaTablesScanner src/exec/schema_scanner/schema_tables_scanner.cpp /^SchemaTablesScanner::SchemaTablesScanner()$/;" f class:doris::SchemaTablesScanner +SchemaTablesScanner src/exec/schema_scanner/schema_tables_scanner.h /^class SchemaTablesScanner : public SchemaScanner {$/;" c namespace:doris +SchemaTablesScannerTest test/exec/schema_scanner/schema_tables_scanner_test.cpp /^ SchemaTablesScannerTest() {$/;" f class:doris::SchemaTablesScannerTest +SchemaTablesScannerTest test/exec/schema_scanner/schema_tables_scanner_test.cpp /^class SchemaTablesScannerTest : public testing::Test {$/;" c namespace:doris file: +SchemaVariablesScanner src/exec/schema_scanner/schema_variables_scanner.cpp /^SchemaVariablesScanner::SchemaVariablesScanner(TVarType::type type) :$/;" f class:doris::SchemaVariablesScanner +SchemaVariablesScanner src/exec/schema_scanner/schema_variables_scanner.h /^class SchemaVariablesScanner : public SchemaScanner {$/;" c namespace:doris +SchemaVariablesScannerTest test/exec/schema_scanner/schema_variables_scanner_test.cpp /^ SchemaVariablesScannerTest() {$/;" f class:doris::SchemaVariablesScannerTest +SchemaVariablesScannerTest test/exec/schema_scanner/schema_variables_scanner_test.cpp /^class SchemaVariablesScannerTest : public testing::Test {$/;" c namespace:doris file: +ScopedCheck src/gutil/threading/thread_collision_warner.h /^ explicit ScopedCheck(ThreadCollisionWarner* warner)$/;" f class:base::ThreadCollisionWarner::ScopedCheck +ScopedCheck src/gutil/threading/thread_collision_warner.h /^ class BASE_EXPORT ScopedCheck {$/;" c class:base::ThreadCollisionWarner +ScopedCleanup src/util/scoped_cleanup.h /^ explicit ScopedCleanup(F f)$/;" f class:doris::ScopedCleanup +ScopedCleanup src/util/scoped_cleanup.h /^class ScopedCleanup {$/;" c namespace:doris +ScopedCounter src/util/runtime_profile.h /^ ScopedCounter(RuntimeProfile::Counter* counter, int64_t val) :$/;" f class:doris::ScopedCounter +ScopedCounter src/util/runtime_profile.h /^class ScopedCounter {$/;" c namespace:doris +ScopedFdCloser src/env/env_posix.cpp /^ explicit ScopedFdCloser(int fd) : fd_(fd) {}$/;" f class:doris::ScopedFdCloser +ScopedFdCloser src/env/env_posix.cpp /^class ScopedFdCloser {$/;" c namespace:doris file: +ScopedRawTimer src/util/runtime_profile.h /^ ScopedRawTimer(int64_t* counter) : _counter(counter) {$/;" f class:doris::ScopedRawTimer +ScopedRawTimer src/util/runtime_profile.h /^class ScopedRawTimer {$/;" c namespace:doris +ScopedRecursiveCheck src/gutil/threading/thread_collision_warner.h /^ explicit ScopedRecursiveCheck(ThreadCollisionWarner* warner)$/;" f class:base::ThreadCollisionWarner::ScopedRecursiveCheck +ScopedRecursiveCheck src/gutil/threading/thread_collision_warner.h /^ class BASE_EXPORT ScopedRecursiveCheck {$/;" c class:base::ThreadCollisionWarner +ScopedRefPtrEqualToFunctor src/gutil/ref_counted.h /^struct ScopedRefPtrEqualToFunctor {$/;" s +ScopedRefPtrHashFunctor src/gutil/ref_counted.h /^struct ScopedRefPtrHashFunctor {$/;" s +ScopedTimer src/util/runtime_profile.h /^ ScopedTimer(RuntimeProfile::Counter* counter) :$/;" f class:doris::ScopedTimer +ScopedTimer src/util/runtime_profile.h /^class ScopedTimer {$/;" c namespace:doris +ScrollParser src/exec/es/es_scroll_parser.cpp /^ScrollParser::ScrollParser() :$/;" f class:doris::ScrollParser +ScrollParser src/exec/es/es_scroll_parser.h /^class ScrollParser {$/;" c namespace:doris +Second src/util/time.h /^ Second,$/;" e enum:doris::TimePrecision +Seek src/exec/parquet_reader.cpp /^arrow::Status ParquetFile::Seek(int64_t position) {$/;" f class:doris::ParquetFile +Seek src/olap/skiplist.h /^inline void SkipList::Iterator::Seek(const Key& target) {$/;" f class:doris::SkipList::Iterator +SeekTo src/util/bitmap.h /^ void SeekTo(size_t bit) {$/;" f class:doris::BitmapIterator +SeekToBit src/util/bit_stream_utils.inline.h /^inline void BitReader::SeekToBit(uint stream_position) {$/;" f class:doris::BitReader +SeekToFirst src/olap/skiplist.h /^inline void SkipList::Iterator::SeekToFirst() {$/;" f class:doris::SkipList::Iterator +SeekToLast src/olap/skiplist.h /^inline void SkipList::Iterator::SeekToLast() {$/;" f class:doris::SkipList::Iterator +Segment src/olap/rowset/segment_v2/segment.cpp /^Segment::Segment($/;" f class:doris::segment_v2::Segment +Segment src/olap/rowset/segment_v2/segment.h /^class Segment : public std::enable_shared_from_this {$/;" c namespace:doris::segment_v2 +SegmentComparator src/olap/olap_index.h /^ SegmentComparator(const MemIndex* index, RowCursor* cursor) :$/;" f class:doris::SegmentComparator +SegmentComparator src/olap/olap_index.h /^class SegmentComparator {$/;" c namespace:doris +SegmentGroup src/olap/rowset/segment_group.cpp /^SegmentGroup::SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const TabletSchema* schema,$/;" f class:doris::SegmentGroup +SegmentGroup src/olap/rowset/segment_group.h /^class SegmentGroup {$/;" c namespace:doris +SegmentIterator src/olap/rowset/segment_v2/segment_iterator.cpp /^SegmentIterator::SegmentIterator(std::shared_ptr segment,$/;" f class:doris::segment_v2::SegmentIterator +SegmentIterator src/olap/rowset/segment_v2/segment_iterator.h /^class SegmentIterator : public RowwiseIterator {$/;" c namespace:doris::segment_v2 +SegmentMetaInfo src/olap/olap_index.h /^ SegmentMetaInfo() {$/;" f struct:doris::SegmentMetaInfo +SegmentMetaInfo src/olap/olap_index.h /^struct SegmentMetaInfo {$/;" s namespace:doris +SegmentReader src/olap/rowset/segment_reader.cpp /^SegmentReader::SegmentReader($/;" f class:doris::SegmentReader +SegmentReader src/olap/rowset/segment_reader.h /^class SegmentReader {$/;" c namespace:doris +SegmentReaderWriterTest test/olap/rowset/segment_v2/segment_test.cpp /^class SegmentReaderWriterTest : public ::testing::Test {$/;" c namespace:doris::segment_v2 file: +SegmentWriter src/olap/rowset/segment_v2/segment_writer.cpp /^SegmentWriter::SegmentWriter(fs::WritableBlock* wblock,$/;" f class:doris::segment_v2::SegmentWriter +SegmentWriter src/olap/rowset/segment_v2/segment_writer.h /^class SegmentWriter {$/;" c namespace:doris::segment_v2 +SegmentWriter src/olap/rowset/segment_writer.cpp /^SegmentWriter::SegmentWriter($/;" f class:doris::SegmentWriter +SegmentWriter src/olap/rowset/segment_writer.h /^class SegmentWriter {$/;" c namespace:doris +SegmentWriterOptions src/olap/rowset/segment_v2/segment_writer.h /^struct SegmentWriterOptions {$/;" s namespace:doris::segment_v2 +SelectBitshuffleFunctions src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp /^void SelectBitshuffleFunctions() {$/;" f namespace:doris::bitshuffle +SelectContainer src/gutil/strings/split_internal.h /^ struct SelectContainer {$/;" s class:strings::internal::Splitter +SelectContainer src/gutil/strings/split_internal.h /^ struct SelectContainer {$/;" s class:strings::internal::Splitter +SelectNode src/exec/select_node.cpp /^SelectNode::SelectNode($/;" f class:doris::SelectNode +SelectNode src/exec/select_node.h /^class SelectNode : public ExecNode {$/;" c namespace:doris +SelectionVector src/olap/selection_vector.h /^ explicit SelectionVector(size_t row_capacity)$/;" f class:doris::SelectionVector +SelectionVector src/olap/selection_vector.h /^class SelectionVector {$/;" c namespace:doris +SelectionVectorTest test/olap/selection_vector_test.cpp /^class SelectionVectorTest : public testing::Test {$/;" c namespace:doris file: +SelectionVectorView src/olap/selection_vector.h /^ explicit SelectionVectorView(SelectionVector* sel_vec)$/;" f class:doris::SelectionVectorView +SelectionVectorView src/olap/selection_vector.h /^class SelectionVectorView {$/;" c namespace:doris +Semaphore src/util/semaphore.hpp /^ explicit Semaphore(int count = 0) : _count(count) {$/;" f class:__anon35::Semaphore +Semaphore src/util/semaphore.hpp /^class Semaphore {$/;" c namespace:__anon35 +SenderInfo test/runtime/data_stream_test.cpp /^ SenderInfo(): thread_handle(NULL), num_bytes_sent(0) {}$/;" f struct:doris::DataStreamTest::SenderInfo +SenderInfo test/runtime/data_stream_test.cpp /^ struct SenderInfo {$/;" s class:doris::DataStreamTest file: +SenderQueue src/runtime/data_stream_recvr.cc /^DataStreamRecvr::SenderQueue::SenderQueue($/;" f class:doris::DataStreamRecvr::SenderQueue +SenderQueue src/runtime/data_stream_recvr.cc /^class DataStreamRecvr::SenderQueue {$/;" c class:doris::DataStreamRecvr file: +SequentialFile src/env/env.h /^ SequentialFile() { }$/;" f class:doris::SequentialFile +SequentialFile src/env/env.h /^class SequentialFile {$/;" c namespace:doris +Serialize src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::Serialize(Tuple* tuple) {$/;" f class:doris::NewAggFnEvaluator +Serialize src/exprs/new_agg_fn_evaluator.h /^inline void NewAggFnEvaluator::Serialize(const std::vector& evals,$/;" f class:doris::NewAggFnEvaluator +SerializeFn output/udf/include/uda_test_harness.h /^ typedef const INTERMEDIATE(*SerializeFn)(FunctionContext* context,$/;" t class:doris_udf::UdaTestHarnessBase +SerializeFn src/exprs/agg_fn_evaluator.cpp /^typedef StringVal(*SerializeFn)(FunctionContext*, const StringVal&);$/;" t namespace:doris file: +SerializeFn src/exprs/new_agg_fn_evaluator.cc /^typedef StringVal (*SerializeFn)(FunctionContext*, const StringVal&);$/;" t file: +SerializeFn src/udf/uda_test_harness.h /^ typedef const INTERMEDIATE(*SerializeFn)(FunctionContext* context,$/;" t class:doris_udf::UdaTestHarnessBase +SerializeOrFinalize src/exprs/new_agg_fn_evaluator.cc /^void NewAggFnEvaluator::SerializeOrFinalize(Tuple* src,$/;" f class:NewAggFnEvaluator +SerializeStreamForSpilling src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::Partition::SerializeStreamForSpilling() {$/;" f class:doris::NewPartitionedAggregationNode::Partition +SerializeTest test/olap/serialize_test.cpp /^ SerializeTest() { }$/;" f class:doris::ser::SerializeTest +SerializeTest test/olap/serialize_test.cpp /^class SerializeTest : public testing::Test {$/;" c namespace:doris::ser file: +ServerType src/util/thrift_server.h /^ enum ServerType { THREAD_POOL = 0, THREADED, NON_BLOCKING };$/;" g class:doris::ThriftServer +ServiceUnavailable src/common/status.h /^ static Status ServiceUnavailable(const Slice& msg,$/;" f class:doris::Status +SessionHandlerIf src/util/thrift_server.h /^ class SessionHandlerIf {$/;" c class:doris::ThriftServer +SessionKey src/util/thrift_server.h /^ typedef std::string SessionKey;$/;" t class:doris::ThriftServer +SessionKeySet src/util/thrift_server.h /^ typedef boost::unordered_map > SessionKeySet;$/;" t class:doris::ThriftServer +Set src/util/bitmap.h /^ void Set(int64_t bit_index, bool v) {$/;" f class:doris::Bitmap +SetAllBits src/util/bitmap.h /^ void SetAllBits(bool b) {$/;" f class:doris::Bitmap +SetAnyVal src/exprs/new_agg_fn_evaluator.cc /^static void SetAnyVal(const SlotDescriptor& desc, Tuple* tuple, AnyVal* dst) {$/;" f file: +SetAtEnd src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTable::Iterator::SetAtEnd() {$/;" f class:doris::NewPartitionedHashTable::Iterator +SetBit src/util/bit_util.h /^ constexpr static T SetBit(T v, int bitpos) {$/;" f class:doris::BitUtil +SetCurExprValuesHash src/exec/new_partitioned_hash_table.h /^ void ALWAYS_INLINE SetCurExprValuesHash(uint32_t hash) {$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +SetDebugDenyIncreaseReservation src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::ClientHandle::SetDebugDenyIncreaseReservation(double probability) {$/;" f class:doris::BufferPool::ClientHandle +SetDebugDenyIncreaseReservation src/runtime/bufferpool/reservation_tracker.h /^ void SetDebugDenyIncreaseReservation(double probability) {$/;" f class:doris::ReservationTracker +SetDstSlot src/exprs/new_agg_fn_evaluator.cc /^void NewAggFnEvaluator::SetDstSlot(const AnyVal* src, const SlotDescriptor& dst_slot_desc,$/;" f class:NewAggFnEvaluator +SetExecutorTest test/exec/set_executor_test.cpp /^ SetExecutorTest() :$/;" f class:doris::SetExecutorTest +SetExecutorTest test/exec/set_executor_test.cpp /^class SetExecutorTest : public testing::Test {$/;" c namespace:doris file: +SetLookupState src/exprs/new_in_predicate.h /^ struct SetLookupState {$/;" s class:doris::InPredicate +SetMatched src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTable::Iterator::SetMatched() {$/;" f class:doris::NewPartitionedHashTable::Iterator +SetNext src/olap/skiplist.h /^ void SetNext(int n, Node* x) {$/;" f struct:doris::SkipList::Node +SetRowNull src/exec/new_partitioned_hash_table.h /^ void ALWAYS_INLINE SetRowNull() { null_bitmap_.Set(CurIdx(), true); }$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +SetSamples src/util/streaming_sampler.h /^ void SetSamples(int period, const std::vector& samples) {$/;" f class:doris::StreamingSampler +SetTabletSchema test/olap/in_list_predicate_test.cpp /^ void SetTabletSchema(std::string name,$/;" f class:doris::TestInListPredicate +SetTabletSchema test/olap/null_predicate_test.cpp /^ void SetTabletSchema(std::string name,$/;" f class:doris::TestNullPredicate +SetTabletSchema test/olap/schema_change_test.cpp /^ void SetTabletSchema(const std::string& name,$/;" f class:doris::TestColumn +SetTabletSchemaWithOneColumn test/olap/column_reader_test.cpp /^ void SetTabletSchemaWithOneColumn(std::string name,$/;" f class:doris::TestColumn +SetTuple src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTable::Iterator::SetTuple(Tuple* tuple, uint32_t hash) {$/;" f class:doris::NewPartitionedHashTable::Iterator +SetTypeValueType src/olap/hll.h /^ typedef uint8_t SetTypeValueType;$/;" t class:doris::HllSetResolver +SetTypeValueType src/olap/hll.h /^ typedef uint8_t SetTypeValueType;$/;" t class:doris::HyperLogLog +SetUp test/exec/broker_reader_test.cpp /^ virtual void SetUp() {$/;" f class:doris::BrokerReaderTest +SetUp test/exec/broker_scan_node_test.cpp /^ virtual void SetUp() {$/;" f class:doris::BrokerScanNodeTest +SetUp test/exec/broker_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::BrokerScannerTest +SetUp test/exec/csv_scan_bench_test.cpp /^ virtual void SetUp() {$/;" f class:doris::CsvScanNodeBenchTest +SetUp test/exec/csv_scan_node_test.cpp /^ virtual void SetUp() {$/;" f class:doris::CsvScanNodeTest +SetUp test/exec/csv_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::CsvScannerTest +SetUp test/exec/es_http_scan_node_test.cpp /^ virtual void SetUp() {$/;" f class:doris::EsHttpScanNodeTest +SetUp test/exec/es_scan_node_test.cpp /^ virtual void SetUp() {$/;" f class:doris::EsScanNodeTest +SetUp test/exec/hash_table_test.cpp /^ virtual void SetUp() {$/;" f class:doris::HashTableTest +SetUp test/exec/mysql_scan_node_test.cpp /^ virtual void SetUp() {$/;" f class:doris::MysqlScanNodeTest +SetUp test/exec/mysql_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::MysqlScannerTest +SetUp test/exec/new_olap_scan_node_test.cpp /^ void SetUp() {$/;" f class:doris::TestOlapScanNode +SetUp test/exec/olap_common_test.cpp /^ virtual void SetUp() {$/;" f class:doris::ColumnValueRangeTest +SetUp test/exec/olap_common_test.cpp /^ virtual void SetUp() {$/;" f class:doris::OlapScanKeysTest +SetUp test/exec/olap_common_test.cpp /^ void SetUp() {$/;" f class:doris::DorisScanRangeTest +SetUp test/exec/olap_scan_node_test.cpp /^ virtual void SetUp() {$/;" f class:doris::OlapScanNodeTest +SetUp test/exec/orc_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::OrcScannerTest +SetUp test/exec/parquet_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::ParquetSannerTest +SetUp test/exec/partitioned_hash_table_test.cpp /^ virtual void SetUp() {$/;" f class:doris::PartitionedHashTableTest +SetUp test/exec/plain_text_line_reader_bzip_test.cpp /^ virtual void SetUp() {$/;" f class:doris::PlainTextLineReaderTest +SetUp test/exec/plain_text_line_reader_gzip_test.cpp /^ virtual void SetUp() {$/;" f class:doris::PlainTextLineReaderTest +SetUp test/exec/plain_text_line_reader_lz4frame_test.cpp /^ virtual void SetUp() {$/;" f class:doris::PlainTextLineReaderTest +SetUp test/exec/plain_text_line_reader_lzop_test.cpp /^ virtual void SetUp() {$/;" f class:doris::PlainTextLineReaderTest +SetUp test/exec/plain_text_line_reader_uncompressed_test.cpp /^ virtual void SetUp() {$/;" f class:doris::PlainTextLineReaderTest +SetUp test/exec/schema_scan_node_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaScanNodeTest +SetUp test/exec/schema_scanner/schema_authors_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaAuthorScannerTest +SetUp test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaCharsetsScannerTest +SetUp test/exec/schema_scanner/schema_collations_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaCollationsScannerTest +SetUp test/exec/schema_scanner/schema_columns_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaColumnsScannerTest +SetUp test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaCreateTableScannerTest +SetUp test/exec/schema_scanner/schema_engines_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaEnginesScannerTest +SetUp test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaOpenTablesScannerTest +SetUp test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaSchemataScannerTest +SetUp test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaTableNamesScannerTest +SetUp test/exec/schema_scanner/schema_tables_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaTablesScannerTest +SetUp test/exec/schema_scanner/schema_variables_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaVariablesScannerTest +SetUp test/exec/schema_scanner_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SchemaScannerTest +SetUp test/exec/set_executor_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SetExecutorTest +SetUp test/exprs/binary_predicate_test.cpp /^ virtual void SetUp() {$/;" f class:doris::BinaryOpTest +SetUp test/exprs/bitmap_function_test.cpp /^ void SetUp() {$/;" f class:doris::BitmapFunctionsTest +SetUp test/exprs/hll_function_test.cpp /^ void SetUp() {$/;" f class:doris::HllFunctionsTest +SetUp test/exprs/in_op_test.cpp /^ virtual void SetUp() {$/;" f class:doris::InOpTest +SetUp test/exprs/in_predicate_test.cpp /^ virtual void SetUp() {$/;" f class:doris::InPredicateTest +SetUp test/exprs/timestamp_functions_test.cpp /^ void SetUp() {$/;" f class:doris::TimestampFunctionsTest +SetUp test/olap/bit_field_test.cpp /^ void SetUp() {$/;" f class:doris::TestBitField +SetUp test/olap/bloom_filter_index_test.cpp /^ virtual void SetUp() {}$/;" f class:doris::TestBloomFilterIndex +SetUp test/olap/bloom_filter_test.cpp /^ virtual void SetUp() {}$/;" f class:doris::TestBloomFilter +SetUp test/olap/byte_buffer_test.cpp /^ virtual void SetUp() {$/;" f class:doris::TestByteBuffer +SetUp test/olap/column_reader_test.cpp /^ virtual void SetUp() {$/;" f class:doris::TestColumn +SetUp test/olap/delete_handler_test.cpp /^ void SetUp() {$/;" f class:doris::TestDeleteConditionHandler +SetUp test/olap/delete_handler_test.cpp /^ void SetUp() {$/;" f class:doris::TestDeleteConditionHandler2 +SetUp test/olap/delete_handler_test.cpp /^ void SetUp() {$/;" f class:doris::TestDeleteHandler +SetUp test/olap/delta_writer_test.cpp /^ void SetUp() {$/;" f class:doris::TestDeltaWriter +SetUp test/olap/file_helper_test.cpp /^ virtual void SetUp() {$/;" f class:doris::FileHandlerTest +SetUp test/olap/file_utils_test.cpp /^ virtual void SetUp() {$/;" f class:doris::FileUtilsTest +SetUp test/olap/lru_cache_test.cpp /^ void SetUp() {$/;" f class:doris::CacheTest +SetUp test/olap/memtable_flush_executor_test.cpp /^ void SetUp() {$/;" f class:doris::TestMemTableFlushExecutor +SetUp test/olap/olap_meta_test.cpp /^ virtual void SetUp() {$/;" f class:doris::OlapMetaTest +SetUp test/olap/olap_reader_test.cpp /^ void SetUp() {$/;" f class:doris::TestOLAPReaderColumn +SetUp test/olap/olap_reader_test.cpp /^ void SetUp() {$/;" f class:doris::TestOLAPReaderColumnDeleteCondition +SetUp test/olap/olap_reader_test.cpp /^ void SetUp() {$/;" f class:doris::TestOLAPReaderRow +SetUp test/olap/olap_snapshot_converter_test.cpp /^ virtual void SetUp() {$/;" f class:doris::OlapSnapshotConverterTest +SetUp test/olap/row_block_test.cpp /^ void SetUp() {$/;" f class:doris::TestRowBlock +SetUp test/olap/row_block_v2_test.cpp /^ void SetUp() {$/;" f class:doris::TestRowBlockV2 +SetUp test/olap/row_cursor_test.cpp /^ virtual void SetUp() {}$/;" f class:doris::TestRowCursor +SetUp test/olap/rowset/alpha_rowset_test.cpp /^ virtual void SetUp() {$/;" f class:doris::AlphaRowsetTest +SetUp test/olap/rowset/rowset_converter_test.cpp /^ virtual void SetUp() {$/;" f class:doris::RowsetConverterTest +SetUp test/olap/rowset/rowset_meta_manager_test.cpp /^ virtual void SetUp() {$/;" f class:doris::RowsetMetaManagerTest +SetUp test/olap/rowset/rowset_meta_test.cpp /^ virtual void SetUp() {$/;" f class:doris::RowsetMetaTest +SetUp test/olap/run_length_byte_test.cpp /^ virtual void SetUp() {$/;" f class:doris::TestRunLengthByte +SetUp test/olap/run_length_integer_test.cpp /^ virtual void SetUp() {$/;" f class:doris::TestRunLengthUnsignInteger +SetUp test/olap/run_length_integer_test.cpp /^virtual void SetUp() {$/;" f class:doris::TestRunLengthSignInteger +SetUp test/olap/schema_change_test.cpp /^ virtual void SetUp() {$/;" f class:doris::TestColumn +SetUp test/olap/stream_index_test.cpp /^ virtual void SetUp() { $/;" f class:doris::TestStreamIndex +SetUp test/olap/tablet_meta_manager_test.cpp /^ virtual void SetUp() {$/;" f class:doris::TabletMetaManagerTest +SetUp test/olap/tablet_mgr_test.cpp /^ virtual void SetUp() {$/;" f class:doris::TabletMgrTest +SetUp test/olap/txn_manager_test.cpp /^ virtual void SetUp() {$/;" f class:doris::TxnManagerTest +SetUp test/olap/vectorized_olap_reader_test.cpp /^ void SetUp() {$/;" f class:doris::TestVectorizedOLAPReader +SetUp test/runtime/buffer_control_block_test.cpp /^ virtual void SetUp() {$/;" f class:doris::BufferControlBlockTest +SetUp test/runtime/buffered_block_mgr2_test.cpp /^ virtual void SetUp() {$/;" f class:doris::BufferedBlockMgrTest +SetUp test/runtime/buffered_tuple_stream2_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SimpleTupleStreamTest +SetUp test/runtime/buffered_tuple_stream_test.cpp /^ virtual void SetUp() {$/;" f class:doris::BufferedTupleStreamTest +SetUp test/runtime/data_spliter_test.cpp /^ virtual void SetUp() {$/;" f class:doris::DataSplitTest +SetUp test/runtime/data_stream_test.cpp /^ virtual void SetUp() {$/;" f class:doris::DataStreamTest +SetUp test/runtime/datetime_value_test.cpp /^ virtual void SetUp() {$/;" f class:doris::DateTimeValueTest +SetUp test/runtime/decimal_value_test.cpp /^ virtual void SetUp() {$/;" f class:doris::DecimalValueTest +SetUp test/runtime/decimalv2_value_test.cpp /^ virtual void SetUp() {$/;" f class:doris::DecimalV2ValueTest +SetUp test/runtime/dpp_sink_internal_test.cpp /^ virtual void SetUp() {$/;" f class:doris::DppSinkInternalTest +SetUp test/runtime/dpp_sink_test.cpp /^ virtual void SetUp() {$/;" f class:doris::DppSinkTest +SetUp test/runtime/dpp_writer_test.cpp /^ virtual void SetUp() {$/;" f class:doris::DppWriterTest +SetUp test/runtime/external_scan_context_mgr_test.cpp /^ virtual void SetUp() {$/;" f class:doris::ExternalScanContextMgrTest +SetUp test/runtime/fragment_mgr_test.cpp /^ virtual void SetUp() {$/;" f class:doris::FragmentMgrTest +SetUp test/runtime/large_int_value_test.cpp /^ virtual void SetUp() {$/;" f class:doris::LargeIntValueTest +SetUp test/runtime/memory_scratch_sink_test.cpp /^ virtual void SetUp() {$/;" f class:doris::MemoryScratchSinkTest +SetUp test/runtime/mysql_table_writer_test.cpp /^ virtual void SetUp() {$/;" f class:doris::MysqlTableWriterTest +SetUp test/runtime/qsorter_test.cpp /^ virtual void SetUp() {$/;" f class:doris::QSorterTest +SetUp test/runtime/result_buffer_mgr_test.cpp /^ virtual void SetUp() {$/;" f class:doris::ResultBufferMgrTest +SetUp test/runtime/result_queue_mgr_test.cpp /^ virtual void SetUp() {$/;" f class:doris::ResultQueueMgrTest +SetUp test/runtime/result_sink_test.cpp /^ virtual void SetUp() {$/;" f class:doris::ResultSinkTest +SetUp test/runtime/result_writer_test.cpp /^ virtual void SetUp() {$/;" f class:doris::ResultWriterTest +SetUp test/runtime/sorter_test.cpp /^ virtual void SetUp() {$/;" f class:doris::SorterTest +SetUp test/runtime/tmp_file_mgr_test.cpp /^ virtual void SetUp() {$/;" f class:doris::TmpFileMgrTest +SetUp test/util/arrow/arrow_work_flow_test.cpp /^ virtual void SetUp() {$/;" f class:doris::ArrowWorkFlowTest +SetUp test/util/radix_sort_test.cpp /^ virtual void SetUp() {$/;" f class:doris::RadixSortTest +SetUp test/util/string_parser_test.cpp /^ virtual void SetUp() {$/;" f class:doris::StringParserTest +SetUp test/util/tdigest_test.cpp /^ virtual void SetUp() {$/;" f class:doris::TDigestTest +SetUp test/util/thread_test.cpp /^ virtual void SetUp() {}$/;" f class:doris::ThreadTest +SetUpTestCase test/agent/cgroups_mgr_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::CgroupsMgrTest +SetUpTestCase test/exec/broker_scan_node_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::BrokerScanNodeTest +SetUpTestCase test/exec/broker_scanner_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::BrokerScannerTest +SetUpTestCase test/exec/es_scan_reader_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::MockESServerTest +SetUpTestCase test/exec/orc_scanner_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::OrcScannerTest +SetUpTestCase test/exec/parquet_scanner_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::ParquetSannerTest +SetUpTestCase test/http/http_client_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::HttpClientTest +SetUpTestCase test/runtime/small_file_mgr_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::SmallFileMgrTest +SetUpTestCase test/runtime/user_function_cache_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::UserFunctionCacheTest +SetUpTestCase test/util/radix_sort_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::RadixSortTest +SetUpTestCase test/util/tdigest_test.cpp /^ static void SetUpTestCase() {$/;" f class:doris::TDigestTest +SetupTest test/exec/partitioned_hash_table_test.cpp /^ void SetupTest(bool quadratic, int64_t initial_num_buckets, bool too_big) {$/;" f class:doris::PartitionedHashTableTest +ShallowClone src/exprs/new_agg_fn_evaluator.cc /^void NewAggFnEvaluator::ShallowClone(ObjectPool* pool, MemPool* mem_pool,$/;" f class:NewAggFnEvaluator +ShardedLRUCache src/olap/lru_cache.cpp /^ShardedLRUCache::ShardedLRUCache(size_t capacity)$/;" f class:doris::ShardedLRUCache +ShardedLRUCache src/olap/lru_cache.h /^ class ShardedLRUCache : public Cache {$/;" c class:doris::CachePriority +ShellEscape src/gutil/strings/escaping.cc /^string ShellEscape(StringPiece src) {$/;" f namespace:strings +ShellEscapeCommandLine src/gutil/strings/escaping.h /^string ShellEscapeCommandLine(InputIterator begin, const InputIterator& end) {$/;" f namespace:strings +ShiftLeftZeroOnOverflow src/util/bit_util.h /^ static inline uint64_t ShiftLeftZeroOnOverflow(uint64_t v, int num_bits) {$/;" f class:doris::BitUtil +ShiftMix src/gutil/hash/city.cc /^static uint64 ShiftMix(uint64 val) {$/;" f namespace:util_hash +ShiftRightLogical src/util/bit_util.h /^ constexpr static T ShiftRightLogical(T v, int shift) {$/;" f class:doris::BitUtil +ShiftRightZeroOnOverflow src/util/bit_util.h /^ static inline uint64_t ShiftRightZeroOnOverflow(uint64_t v, int num_bits) {$/;" f class:doris::BitUtil +ShortKeyIndexBuilder src/olap/short_key_index.h /^ ShortKeyIndexBuilder(uint32_t segment_id, uint32_t num_rows_per_block) :$/;" f class:doris::ShortKeyIndexBuilder +ShortKeyIndexBuilder src/olap/short_key_index.h /^class ShortKeyIndexBuilder {$/;" c namespace:doris +ShortKeyIndexDecoder src/olap/short_key_index.h /^ ShortKeyIndexDecoder() : _parsed(false) {}$/;" f class:doris::ShortKeyIndexDecoder +ShortKeyIndexDecoder src/olap/short_key_index.h /^class ShortKeyIndexDecoder {$/;" c namespace:doris +ShortKeyIndexIterator src/olap/short_key_index.h /^ ShortKeyIndexIterator(const ShortKeyIndexDecoder* decoder, uint32_t ordinal = 0)$/;" f class:doris::ShortKeyIndexIterator +ShortKeyIndexIterator src/olap/short_key_index.h /^class ShortKeyIndexIterator {$/;" c namespace:doris +ShortKeyIndexTest test/olap/short_key_index_test.cpp /^ ShortKeyIndexTest() { }$/;" f class:doris::ShortKeyIndexTest +ShortKeyIndexTest test/olap/short_key_index_test.cpp /^class ShortKeyIndexTest : public testing::Test {$/;" c namespace:doris file: +ShortRepeatHead src/olap/rowset/run_length_integer_writer.h /^ struct ShortRepeatHead {$/;" s class:doris::RunLengthIntegerWriter +ShouldExpandPreaggHashTables src/exec/new_partitioned_aggregation_node.cc /^bool NewPartitionedAggregationNode::ShouldExpandPreaggHashTables() const {$/;" f class:doris::NewPartitionedAggregationNode +ShrinkToFitInternal src/util/faststring.cc /^void faststring::ShrinkToFitInternal() {$/;" f class:doris::faststring +SimpleAtoi src/gutil/strings/numbers.h /^bool MUST_USE_RESULT SimpleAtoi(const char* s, int_type* out) {$/;" f +SimpleAtoi src/gutil/strings/numbers.h /^bool MUST_USE_RESULT SimpleAtoi(const string& s, int_type* out) {$/;" f +SimpleCoreMetricsVisitor src/http/action/metrics_action.cpp /^class SimpleCoreMetricsVisitor : public MetricsVisitor {$/;" c namespace:doris file: +SimpleDtoa src/gutil/strings/numbers.cc /^string SimpleDtoa(double value) {$/;" f +SimpleFtoa src/gutil/strings/numbers.cc /^string SimpleFtoa(float value) {$/;" f +SimpleItoa src/gutil/strings/numbers.h /^inline string SimpleItoa(int32 i) {$/;" f +SimpleItoa src/gutil/strings/numbers.h /^inline string SimpleItoa(int64 i) {$/;" f +SimpleItoa src/gutil/strings/numbers.h /^inline string SimpleItoa(uint32 i) {$/;" f +SimpleItoa src/gutil/strings/numbers.h /^inline string SimpleItoa(uint64 i) {$/;" f +SimpleItoaWithCommas src/gutil/strings/numbers.cc /^string SimpleItoaWithCommas(int32 i) {$/;" f +SimpleItoaWithCommas src/gutil/strings/numbers.cc /^string SimpleItoaWithCommas(int64 i) {$/;" f +SimpleItoaWithCommas src/gutil/strings/numbers.cc /^string SimpleItoaWithCommas(uint32 i) {$/;" f +SimpleItoaWithCommas src/gutil/strings/numbers.cc /^string SimpleItoaWithCommas(uint64 i) {$/;" f +SimpleMetric src/util/metrics.h /^ SimpleMetric(MetricType type) :Metric(type) { }$/;" f class:doris::SimpleMetric +SimpleMetric src/util/metrics.h /^class SimpleMetric : public Metric {$/;" c namespace:doris +SimpleNullStreamTest test/runtime/buffered_tuple_stream2_test.cpp /^class SimpleNullStreamTest : public SimpleTupleStreamTest {$/;" c namespace:doris file: +SimpleTask test/util/threadpool_test.cpp /^ SimpleTask(int n, std::atomic* counter)$/;" f class:doris::SimpleTask +SimpleTask test/util/threadpool_test.cpp /^class SimpleTask : public Runnable {$/;" c namespace:doris file: +SimpleTupleStreamTest test/runtime/buffered_tuple_stream2_test.cpp /^ SimpleTupleStreamTest() : _tracker(-1) {}$/;" f class:doris::SimpleTupleStreamTest +SimpleTupleStreamTest test/runtime/buffered_tuple_stream2_test.cpp /^class SimpleTupleStreamTest : public testing::Test {$/;" c namespace:doris file: +Size src/runtime/bufferpool/free_list.h /^ int64_t Size() const { return free_list_.size(); }$/;" f class:doris::FreeList +SizeLocked src/util/blocking_queue.hpp /^ uint32_t SizeLocked(const std::unique_lock& lock) const {$/;" f class:doris::BlockingQueue +Skewed src/util/random.h /^ uint32_t Skewed(int max_log) {$/;" f class:doris::Random +Skip src/util/rle_encoding.h /^inline size_t RleDecoder::Skip(size_t to_skip) {$/;" f class:doris::RleDecoder +SkipEmpty src/gutil/strings/split.h /^struct SkipEmpty {$/;" s namespace:strings +SkipLeadingWhiteSpace src/gutil/strings/strip.h /^inline char* SkipLeadingWhiteSpace(char* str) {$/;" f +SkipLeadingWhiteSpace src/gutil/strings/strip.h /^inline const char* SkipLeadingWhiteSpace(const char* str) {$/;" f +SkipList src/olap/skiplist.h /^SkipList::SkipList(Comparator cmp, MemPool* mem_pool, bool can_dup) :$/;" f class:doris::SkipList +SkipList src/olap/skiplist.h /^class SkipList {$/;" c namespace:doris +SkipTest test/olap/skiplist_test.cpp /^class SkipTest : public testing::Test {};$/;" c namespace:doris file: +SkipWhitespace src/gutil/strings/split.h /^struct SkipWhitespace {$/;" s namespace:strings +SleepFor src/util/monotime.cpp /^void SleepFor(const MonoDelta& delta) {$/;" f namespace:doris +SleepForMilliseconds src/gutil/sysinfo.cc /^void SleepForMilliseconds(int64_t milliseconds) {$/;" f namespace:base +SleepForMs src/util/time.cpp /^void doris::SleepForMs(const int64_t duration_ms) {$/;" f class:doris +SleepForNanoseconds src/gutil/sysinfo.cc /^void SleepForNanoseconds(int64_t nanoseconds) {$/;" f namespace:base +Slice src/util/slice.cpp /^Slice::Slice(const faststring& s) : \/\/ NOLINT(runtime\/explicit)$/;" f class:doris::Slice +Slice src/util/slice.h /^ Slice() : data(const_cast("")), size(0) { }$/;" f struct:doris::Slice +Slice src/util/slice.h /^ Slice(const char* d, size_t n) :$/;" f struct:doris::Slice +Slice src/util/slice.h /^ Slice(const char* s) : \/\/ NOLINT(runtime\/explicit)$/;" f struct:doris::Slice +Slice src/util/slice.h /^ Slice(const std::string& s) : \/\/ NOLINT(runtime\/explicit)$/;" f struct:doris::Slice +Slice src/util/slice.h /^ Slice(const uint8_t* s, size_t n) :$/;" f struct:doris::Slice +Slice src/util/slice.h /^struct Slice {$/;" s namespace:doris +SliceMap src/util/slice.h /^struct SliceMap {$/;" s namespace:doris +SlotDescriptor src/runtime/descriptors.cpp /^SlotDescriptor::SlotDescriptor(const PSlotDescriptor& pdesc)$/;" f class:doris::SlotDescriptor +SlotDescriptor src/runtime/descriptors.cpp /^SlotDescriptor::SlotDescriptor(const TSlotDescriptor& tdesc)$/;" f class:doris::SlotDescriptor +SlotDescriptor src/runtime/descriptors.h /^class SlotDescriptor {$/;" c namespace:doris +SlotDescriptorMap src/runtime/descriptors.h /^ typedef std::tr1::unordered_map SlotDescriptorMap;$/;" t class:doris::DescriptorTbl +SlotId src/common/global_types.h /^typedef int SlotId;$/;" t namespace:doris +SlotRef src/exprs/slot_ref.cpp /^SlotRef::SlotRef(const SlotDescriptor* desc) :$/;" f class:doris::SlotRef +SlotRef src/exprs/slot_ref.cpp /^SlotRef::SlotRef(const SlotDescriptor* desc, const TypeDescriptor& type) :$/;" f class:doris::SlotRef +SlotRef src/exprs/slot_ref.cpp /^SlotRef::SlotRef(const TExprNode& node) :$/;" f class:doris::SlotRef +SlotRef src/exprs/slot_ref.cpp /^SlotRef::SlotRef(const TypeDescriptor& type, int offset) :$/;" f class:doris::SlotRef +SlotRef src/exprs/slot_ref.h /^class SlotRef : public Expr {$/;" c namespace:doris +SlowDestructorRunnable test/util/threadpool_test.cpp /^class SlowDestructorRunnable : public Runnable {$/;" c namespace:doris file: +SlowTask test/util/threadpool_test.cpp /^ explicit SlowTask(CountDownLatch* latch)$/;" f class:doris::SlowTask +SlowTask test/util/threadpool_test.cpp /^class SlowTask : public Runnable {$/;" c namespace:doris file: +Slow_CRC32 src/util/crc32c.cpp /^static inline void Slow_CRC32(uint64_t* l, uint8_t const **p) {$/;" f namespace:doris::crc32c +SlurpSmallTextFile src/gutil/sysinfo.cc /^static bool SlurpSmallTextFile(const char* file, char* buf, int buflen) {$/;" f namespace:base +SmallFileMgr src/runtime/small_file_mgr.cpp /^SmallFileMgr::SmallFileMgr($/;" f class:doris::SmallFileMgr +SmallFileMgr src/runtime/small_file_mgr.h /^class SmallFileMgr {$/;" c namespace:doris +SmallFileMgrTest test/runtime/small_file_mgr_test.cpp /^ SmallFileMgrTest() { }$/;" f class:doris::SmallFileMgrTest +SmallFileMgrTest test/runtime/small_file_mgr_test.cpp /^class SmallFileMgrTest: public testing::Test {$/;" c namespace:doris file: +SmallFileMgrTestHandler test/runtime/small_file_mgr_test.cpp /^class SmallFileMgrTestHandler : public HttpHandler {$/;" c namespace:doris file: +SmallIntVal output/udf/include/udf.h /^ SmallIntVal() : val(0) { }$/;" f struct:doris_udf::SmallIntVal +SmallIntVal output/udf/include/udf.h /^ SmallIntVal(int16_t val) : val(val) { }$/;" f struct:doris_udf::SmallIntVal +SmallIntVal output/udf/include/udf.h /^struct SmallIntVal : public AnyVal {$/;" s namespace:doris_udf +SmallIntVal src/udf/udf.h /^ SmallIntVal() : val(0) { }$/;" f struct:doris_udf::SmallIntVal +SmallIntVal src/udf/udf.h /^ SmallIntVal(int16_t val) : val(val) { }$/;" f struct:doris_udf::SmallIntVal +SmallIntVal src/udf/udf.h /^struct SmallIntVal : public AnyVal {$/;" s namespace:doris_udf +SmallIntWrapper src/exprs/scalar_fn_call.cpp /^typedef SmallIntVal (*SmallIntWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +SnappyBlockCompression src/util/block_compression.cpp /^class SnappyBlockCompression : public BlockCompressionCodec {$/;" c namespace:doris file: +SnappySlicesSource src/util/block_compression.cpp /^ SnappySlicesSource(const std::vector& slices) : _available(0), _cur_slice(0), _slice_off(0) {$/;" f class:doris::SnappySlicesSource +SnappySlicesSource src/util/block_compression.cpp /^class SnappySlicesSource : public snappy::Source {$/;" c namespace:doris file: +SnapshotAction src/http/action/snapshot_action.cpp /^SnapshotAction::SnapshotAction(ExecEnv* exec_env) :$/;" f class:doris::SnapshotAction +SnapshotAction src/http/action/snapshot_action.h /^class SnapshotAction : public HttpHandler {$/;" c namespace:doris +SnapshotLoader src/runtime/snapshot_loader.cpp /^SnapshotLoader::SnapshotLoader($/;" f class:doris::SnapshotLoader +SnapshotLoader src/runtime/snapshot_loader.h /^class SnapshotLoader {$/;" c namespace:doris +SnapshotLoaderTest test/runtime/snapshot_loader_test.cpp /^ SnapshotLoaderTest() {$/;" f class:doris::SnapshotLoaderTest +SnapshotLoaderTest test/runtime/snapshot_loader_test.cpp /^class SnapshotLoaderTest : public testing::Test {$/;" c namespace:doris file: +SnapshotManager src/olap/snapshot_manager.h /^ SnapshotManager()$/;" f class:doris::SnapshotManager +SnapshotManager src/olap/snapshot_manager.h /^class SnapshotManager {$/;" c namespace:doris +SortCompare src/runtime/bufferpool/free_list.h /^ inline static bool SortCompare(const BufferHandle& b1, const BufferHandle& b2) {$/;" f class:doris::FreeList +SortExecExprs src/exec/sort_exec_exprs.h /^class SortExecExprs {$/;" c namespace:doris +SortNode src/exec/sort_node.cpp /^SortNode::SortNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)$/;" f class:doris::SortNode +SortNode src/exec/sort_node.h /^class SortNode : public ExecNode {$/;" c namespace:doris +SortedRangesHaveIntersection src/gutil/stl_util.h /^bool SortedRangesHaveIntersection(InputIterator1 begin1, InputIterator1 end1,$/;" f +SortedRunMerger src/runtime/sorted_run_merger.cc /^SortedRunMerger::SortedRunMerger(const TupleRowComparator& compare_less_than,$/;" f class:doris::SortedRunMerger +SortedRunMerger src/runtime/sorted_run_merger.h /^class SortedRunMerger {$/;" c namespace:doris +Sorter src/runtime/sorter.h /^class Sorter {$/;" c namespace:doris +SorterTest test/runtime/sorter_test.cpp /^ SorterTest() {$/;" f class:doris::SorterTest +SorterTest test/runtime/sorter_test.cpp /^class SorterTest : public testing::Test {$/;" c namespace:doris file: +SparseIndexType src/olap/hll.h /^ typedef uint16_t SparseIndexType;$/;" t class:doris::HllSetResolver +SparseIndexType src/olap/hll.h /^ typedef uint16_t SparseIndexType;$/;" t class:doris::HyperLogLog +SparseLengthValueType src/olap/hll.h /^ typedef int32_t SparseLengthValueType;$/;" t class:doris::HllSetResolver +SparseLengthValueType src/olap/hll.h /^ typedef int32_t SparseLengthValueType;$/;" t class:doris::HyperLogLog +SparseValueType src/olap/hll.h /^ typedef uint8_t SparseValueType;$/;" t class:doris::HllSetResolver +SparseValueType src/olap/hll.h /^ typedef uint8_t SparseValueType;$/;" t class:doris::HyperLogLog +SpecificElement src/common/object_pool.h /^ SpecificElement(T* t): t(t) {}$/;" f struct:doris::ObjectPool::SpecificElement +SpecificElement src/common/object_pool.h /^ struct SpecificElement : GenericElement {$/;" s class:doris::ObjectPool +Spill src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::Partition::Spill(bool more_aggregate_rows) {$/;" f class:doris::NewPartitionedAggregationNode::Partition +SpillPartition src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::SpillPartition(bool more_aggregate_rows) {$/;" f class:doris::NewPartitionedAggregationNode +SpillSortNode src/exec/spill_sort_node.cc /^SpillSortNode::SpillSortNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) :$/;" f class:doris::SpillSortNode +SpillSortNode src/exec/spill_sort_node.h /^class SpillSortNode : public ExecNode {$/;" c namespace:doris +SpillSorter src/runtime/spill_sorter.cc /^SpillSorter::SpillSorter(const TupleRowComparator& compare_less_than,$/;" f class:doris::SpillSorter +SpillSorter src/runtime/spill_sorter.h /^class SpillSorter {$/;" c namespace:doris +SpinLock src/util/spinlock.h /^ SpinLock() : _locked(false) {$/;" f class:doris::SpinLock +SpinLock src/util/spinlock.h /^class SpinLock {$/;" c namespace:doris +SpinLockDelay src/gutil/spinlock_linux-inl.h /^void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop) {$/;" f namespace:base::internal +SpinLockDelay src/gutil/spinlock_posix-inl.h /^void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop) {$/;" f namespace:base::internal +SpinLockDelay src/gutil/spinlock_win32-inl.h /^void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop) {$/;" f namespace:base::internal +SpinLockWait src/gutil/spinlock_internal.cc /^int32 SpinLockWait(volatile Atomic32 *w, int n,$/;" f namespace:base::internal +SpinLockWaitTransition src/gutil/spinlock_internal.h /^struct SpinLockWaitTransition {$/;" s namespace:base::internal +SpinLockWake src/gutil/spinlock_linux-inl.h /^void SpinLockWake(volatile Atomic32 *w, bool all) {$/;" f namespace:base::internal +SpinLockWake src/gutil/spinlock_posix-inl.h /^void SpinLockWake(volatile Atomic32 *w, bool all) {$/;" f namespace:base::internal +SpinLockWake src/gutil/spinlock_win32-inl.h /^void SpinLockWake(volatile Atomic32 *w, bool all) {$/;" f namespace:base::internal +Split src/gutil/strings/split.h /^inline internal::Splitter Split($/;" f namespace:strings +Split src/gutil/strings/split.h /^inline internal::Splitter Split(StringPiece text, Delimiter d) {$/;" f namespace:strings +Split src/gutil/strings/split.h /^inline internal::Splitter Split($/;" f namespace:strings +Split src/gutil/strings/split.h /^inline internal::Splitter Split($/;" f namespace:strings +SplitCSVLine src/gutil/strings/split.cc /^void SplitCSVLine(char* line, vector* cols) {$/;" f +SplitCSVLineWithDelimiter src/gutil/strings/split.cc /^void SplitCSVLineWithDelimiter(char* line, char delimiter,$/;" f +SplitCSVLineWithDelimiterForStrings src/gutil/strings/split.cc /^void SplitCSVLineWithDelimiterForStrings(const string &line,$/;" f +SplitIterator src/gutil/strings/split_internal.h /^ SplitIterator(Delimiter d, Predicate p)$/;" f class:strings::internal::SplitIterator +SplitIterator src/gutil/strings/split_internal.h /^ SplitIterator(StringPiece text, Delimiter d)$/;" f class:strings::internal::SplitIterator +SplitIterator src/gutil/strings/split_internal.h /^ SplitIterator(StringPiece text, Delimiter d, Predicate p)$/;" f class:strings::internal::SplitIterator +SplitIterator src/gutil/strings/split_internal.h /^ explicit SplitIterator(Delimiter d)$/;" f class:strings::internal::SplitIterator +SplitIterator src/gutil/strings/split_internal.h /^class SplitIterator$/;" c namespace:strings::internal +SplitLeadingDec32Values src/gutil/strings/split.cc /^const char* SplitLeadingDec32Values(const char *str, vector *result) {$/;" f +SplitLeadingDec64Values src/gutil/strings/split.cc /^const char* SplitLeadingDec64Values(const char *str, vector *result) {$/;" f +SplitOneStringToken src/gutil/strings/split.cc /^string SplitOneStringToken(const char ** source, const char * delim) {$/;" f +SplitOneUInt32Token src/gutil/strings/split.h /^inline bool SplitOneUInt32Token(const char** source, const char* delim,$/;" f +SplitOneUInt64Token src/gutil/strings/split.h /^inline bool SplitOneUInt64Token(const char** source, const char* delim,$/;" f +SplitRange src/gutil/strings/split.cc /^bool SplitRange(const char* rangestr, int* from, int* to) {$/;" f +SplitStringAllowEmpty src/gutil/strings/split.cc /^void SplitStringAllowEmpty(const string& full, const char* delim,$/;" f +SplitStringAndParse src/gutil/strings/split.h /^bool SplitStringAndParse(StringPiece source, StringPiece delim,$/;" f +SplitStringAndParseToContainer src/gutil/strings/split.h /^bool SplitStringAndParseToContainer($/;" f +SplitStringAndParseToInserter src/gutil/strings/split.h /^bool SplitStringAndParseToInserter($/;" f namespace:strings::internal +SplitStringAndParseToList src/gutil/strings/split.h /^bool SplitStringAndParseToList($/;" f +SplitStringIntoKeyValuePairs src/gutil/strings/split.cc /^bool SplitStringIntoKeyValuePairs(const string& line,$/;" f +SplitStringIntoKeyValues src/gutil/strings/split.cc /^bool SplitStringIntoKeyValues(const string& line,$/;" f +SplitStringIntoNPiecesAllowEmpty src/gutil/strings/split.cc /^void SplitStringIntoNPiecesAllowEmpty(const string& full,$/;" f +SplitStringPieceToVector src/gutil/strings/split.cc /^void SplitStringPieceToVector(const StringPiece& full,$/;" f +SplitStringToHashmapUsing src/gutil/strings/split.cc /^void SplitStringToHashmapUsing(const string& full, const char* delim,$/;" f +SplitStringToHashsetUsing src/gutil/strings/split.cc /^void SplitStringToHashsetUsing(const string& full, const char* delim,$/;" f +SplitStringToIteratorAllowEmpty src/gutil/strings/split.cc /^void SplitStringToIteratorAllowEmpty(const StringType& full,$/;" f file: +SplitStringToIteratorUsing src/gutil/strings/split.cc /^void SplitStringToIteratorUsing(const StringType& full,$/;" f file: +SplitStringToLines src/gutil/strings/split.cc /^void SplitStringToLines(const char* full,$/;" f +SplitStringToMapUsing src/gutil/strings/split.cc /^void SplitStringToMapUsing(const string& full, const char* delim,$/;" f +SplitStringToSetUsing src/gutil/strings/split.cc /^void SplitStringToSetUsing(const string& full, const char* delim,$/;" f +SplitStringUsing src/gutil/strings/split.cc /^void SplitStringUsing(const string& full,$/;" f +SplitStringWithEscaping src/gutil/strings/split.cc /^void SplitStringWithEscaping(const string &full,$/;" f +SplitStringWithEscapingAllowEmpty src/gutil/strings/split.cc /^void SplitStringWithEscapingAllowEmpty(const string &full,$/;" f +SplitStringWithEscapingToHashset src/gutil/strings/split.cc /^void SplitStringWithEscapingToHashset(const string &full,$/;" f +SplitStringWithEscapingToIterator src/gutil/strings/split.cc /^void SplitStringWithEscapingToIterator(const string& src,$/;" f file: +SplitStringWithEscapingToSet src/gutil/strings/split.cc /^void SplitStringWithEscapingToSet(const string &full,$/;" f +SplitStructuredLine src/gutil/strings/split.cc /^bool SplitStructuredLine(StringPiece line,$/;" f +SplitStructuredLine src/gutil/strings/split.cc /^char* SplitStructuredLine(char* line,$/;" f +SplitStructuredLineInternal src/gutil/strings/split.cc /^bool SplitStructuredLineInternal(StringPiece line,$/;" f namespace:__anon31 +SplitStructuredLineInternal src/gutil/strings/split.cc /^char* SplitStructuredLineInternal(char* line,$/;" f namespace:__anon31 +SplitStructuredLineWithEscapes src/gutil/strings/split.cc /^bool SplitStructuredLineWithEscapes(StringPiece line,$/;" f +SplitStructuredLineWithEscapes src/gutil/strings/split.cc /^char* SplitStructuredLineWithEscapes(char* line,$/;" f +SplitToSize src/runtime/bufferpool/suballocator.cc /^Status Suballocator::SplitToSize(unique_ptr free_node,$/;" f class:doris::Suballocator +SplitToVector src/gutil/strings/split.cc /^void SplitToVector(char* full, const char* delim, vector* vec,$/;" f +SplitToVector src/gutil/strings/split.cc /^void SplitToVector(char* full, const char* delim, vector* vec,$/;" f +SplitUsing src/gutil/strings/split.cc /^vector* SplitUsing(char* full, const char* delim) {$/;" f +Splitter src/gutil/strings/split_internal.h /^ Splitter(StringPiece text, Delimiter d)$/;" f class:strings::internal::Splitter +Splitter src/gutil/strings/split_internal.h /^ Splitter(StringPiece text, Delimiter d, Predicate p)$/;" f class:strings::internal::Splitter +Splitter src/gutil/strings/split_internal.h /^class Splitter {$/;" c namespace:strings::internal +StConstructState src/geo/geo_functions.cpp /^ StConstructState() : is_null(false) { }$/;" f struct:doris::StConstructState +StConstructState src/geo/geo_functions.cpp /^struct StConstructState {$/;" s namespace:doris file: +StContainsState src/geo/geo_functions.cpp /^ StContainsState() : is_null(false), shapes{nullptr, nullptr} { }$/;" f struct:doris::StContainsState +StContainsState src/geo/geo_functions.cpp /^struct StContainsState {$/;" s namespace:doris file: +StartMoveToPinned src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::Client::StartMoveToPinned(ClientHandle* client, Page* page) {$/;" f class:doris::BufferPool::Client +StartSpilling src/runtime/runtime_state.cpp /^Status RuntimeState::StartSpilling(MemTracker* mem_tracker) {$/;" f class:doris::RuntimeState +StatUtil src/util/stat_util.hpp /^class StatUtil {$/;" c namespace:doris +State src/olap/fs/block_manager.h /^ enum State {$/;" g class:doris::fs::WritableBlock +State src/runtime/disk_io_mgr_internal.h /^ enum State {$/;" g class:doris::DiskIoMgr::RequestContext +State src/runtime/tablets_channel.h /^ enum State {$/;" g class:doris::TabletsChannel +State src/util/threadpool.h /^ enum class State {$/;" c class:doris::ThreadPoolToken +State test/olap/skiplist_test.cpp /^ State() {$/;" f struct:doris::ConcurrentTest::State +State test/olap/skiplist_test.cpp /^ struct State {$/;" s class:doris::ConcurrentTest file: +Statistics src/olap/merger.h /^ struct Statistics {$/;" s class:doris::Merger +Status src/common/status.cpp /^Status::Status(TStatusCode::type code, const Slice& msg, int16_t precise_code, const Slice& msg2)$/;" f class:doris::Status +Status src/common/status.cpp /^Status::Status(const PStatus& s) : _state(nullptr) {$/;" f class:doris::Status +Status src/common/status.cpp /^Status::Status(const TStatus& s) : _state(nullptr) {$/;" f class:doris::Status +Status src/common/status.h /^ Status(): _state(nullptr) {}$/;" f class:doris::Status +Status src/common/status.h /^ Status(const Status& s)$/;" f class:doris::Status +Status src/common/status.h /^class Status {$/;" c namespace:doris +StatusTest test/common/status_test.cpp /^class StatusTest : public testing::Test {$/;" c namespace:doris file: +StorageByteBuffer src/olap/byte_buffer.cpp /^StorageByteBuffer::StorageByteBuffer() : $/;" f class:doris::StorageByteBuffer +StorageByteBuffer src/olap/byte_buffer.h /^class StorageByteBuffer {$/;" c namespace:doris +StorageEngine src/olap/storage_engine.cpp /^StorageEngine::StorageEngine(const EngineOptions& options)$/;" f class:doris::StorageEngine +StorageEngine src/olap/storage_engine.h /^class StorageEngine {$/;" c namespace:doris +StoragePageCache src/olap/page_cache.cpp /^StoragePageCache::StoragePageCache(size_t capacity) : _cache(new_lru_cache(capacity)) {$/;" f class:doris::StoragePageCache +StoragePageCache src/olap/page_cache.h /^class StoragePageCache {$/;" c namespace:doris +StoragePageCacheTest test/olap/page_cache_test.cpp /^ StoragePageCacheTest() { }$/;" f class:doris::StoragePageCacheTest +StoragePageCacheTest test/olap/page_cache_test.cpp /^class StoragePageCacheTest : public testing::Test {$/;" c namespace:doris file: +StorageReadOptions src/olap/iterators.h /^class StorageReadOptions {$/;" c namespace:doris +Store128 src/gutil/endian.h /^ static void Store128(void *p, const uint128 v) {$/;" f class:BigEndian +Store128 src/gutil/endian.h /^ static void Store128(void *p, const uint128 v) {$/;" f class:LittleEndian +Store16 src/gutil/endian.h /^ static void Store16(void *p, uint16 v) {$/;" f class:BigEndian +Store16 src/gutil/endian.h /^ static void Store16(void *p, uint16 v) {$/;" f class:LittleEndian +Store32 src/gutil/endian.h /^ static void Store32(void *p, uint32 v) {$/;" f class:BigEndian +Store32 src/gutil/endian.h /^ static void Store32(void *p, uint32 v) {$/;" f class:LittleEndian +Store64 src/gutil/endian.h /^ static void Store64(void *p, uint64 v) {$/;" f class:BigEndian +Store64 src/gutil/endian.h /^ static void Store64(void *p, uint64 v) {$/;" f class:LittleEndian +StorePath src/olap/options.h /^ StorePath() : capacity_bytes(-1), storage_medium(TStorageMedium::HDD) {}$/;" f struct:doris::StorePath +StorePath src/olap/options.h /^ StorePath(const std::string& path_, int64_t capacity_bytes_)$/;" f struct:doris::StorePath +StorePath src/olap/options.h /^ StorePath(const std::string& path_, int64_t capacity_bytes_,$/;" f struct:doris::StorePath +StorePath src/olap/options.h /^struct StorePath {$/;" s namespace:doris +StoreUnsignedWord src/gutil/endian.h /^ static void StoreUnsignedWord(void *p, uword_t v) {$/;" f class:BigEndian +StoreUnsignedWord src/gutil/endian.h /^ static void StoreUnsignedWord(void *p, uword_t v) {$/;" f class:LittleEndian +StrAppend src/gutil/strings/strcat.cc /^void StrAppend(string *result, const AlphaNum &a) {$/;" f +StrAppend src/gutil/strings/strcat.cc /^void StrAppend(string *result, const AlphaNum &a, const AlphaNum &b) {$/;" f +StrAppend src/gutil/strings/strcat.cc /^void StrAppend(string *result,$/;" f +StrCat src/gutil/strings/strcat.cc /^string StrCat(const AlphaNum &a) {$/;" f +StrCat src/gutil/strings/strcat.cc /^string StrCat(const AlphaNum &a, const AlphaNum &b) {$/;" f +StrCat src/gutil/strings/strcat.cc /^string StrCat(const AlphaNum &a, const AlphaNum &b, const AlphaNum &c) {$/;" f +StrCat src/gutil/strings/strcat.cc /^string StrCat(const AlphaNum &a, const AlphaNum &b, const AlphaNum &c,$/;" f +StrCat src/gutil/strings/strcat.h /^inline string StrCat(const AlphaNum &a, const AlphaNum &b, const AlphaNum &c,$/;" f +StrCatNineOrMore src/gutil/strings/strcat.cc /^string StrCatNineOrMore(const AlphaNum *a, ...) {$/;" f namespace:strings::internal +Strategy src/exprs/new_in_predicate.h /^ enum Strategy {$/;" g class:doris::InPredicate +StreamHead src/olap/out_stream.h /^ StreamHead() : type(COMPRESSED), length(0), checksum(0) {}$/;" f struct:doris::StreamHead +StreamHead src/olap/out_stream.h /^struct StreamHead {$/;" s namespace:doris +StreamIndexHeader src/olap/stream_index_common.h /^ StreamIndexHeader() : $/;" f struct:doris::StreamIndexHeader +StreamIndexHeader src/olap/stream_index_common.h /^struct StreamIndexHeader {$/;" s namespace:doris +StreamIndexReader src/olap/stream_index_reader.cpp /^StreamIndexReader::StreamIndexReader() : $/;" f class:doris::StreamIndexReader +StreamIndexReader src/olap/stream_index_reader.h /^class StreamIndexReader {$/;" c namespace:doris +StreamIndexWriter src/olap/stream_index_writer.cpp /^StreamIndexWriter::StreamIndexWriter(FieldType field_type) : _field_type(field_type) {$/;" f class:doris::StreamIndexWriter +StreamIndexWriter src/olap/stream_index_writer.h /^class StreamIndexWriter {$/;" c namespace:doris +StreamLoadAction src/http/action/stream_load.cpp /^StreamLoadAction::StreamLoadAction(ExecEnv* exec_env) : _exec_env(exec_env) {$/;" f class:doris::StreamLoadAction +StreamLoadAction src/http/action/stream_load.h /^class StreamLoadAction : public HttpHandler {$/;" c namespace:doris +StreamLoadActionTest test/http/stream_load_test.cpp /^ StreamLoadActionTest() { }$/;" f class:doris::StreamLoadActionTest +StreamLoadActionTest test/http/stream_load_test.cpp /^class StreamLoadActionTest : public testing::Test {$/;" c namespace:doris file: +StreamLoadContext src/runtime/stream_load/stream_load_context.h /^ StreamLoadContext(ExecEnv* exec_env) :$/;" f class:doris::StreamLoadContext +StreamLoadContext src/runtime/stream_load/stream_load_context.h /^class StreamLoadContext {$/;" c namespace:doris +StreamLoadExecutor src/runtime/stream_load/stream_load_executor.h /^ StreamLoadExecutor(ExecEnv* exec_env) : _exec_env(exec_env) { }$/;" f class:doris::StreamLoadExecutor +StreamLoadExecutor src/runtime/stream_load/stream_load_executor.h /^class StreamLoadExecutor {$/;" c namespace:doris +StreamLoadPipe src/runtime/stream_load/stream_load_pipe.h /^ StreamLoadPipe(size_t max_buffered_bytes = 1024 * 1024,$/;" f class:doris::StreamLoadPipe +StreamLoadPipe src/runtime/stream_load/stream_load_pipe.h /^class StreamLoadPipe : public MessageBodySink, public FileReader {$/;" c namespace:doris +StreamLoadPipeTest test/runtime/stream_load_pipe_test.cpp /^ StreamLoadPipeTest() { }$/;" f class:doris::StreamLoadPipeTest +StreamLoadPipeTest test/runtime/stream_load_pipe_test.cpp /^class StreamLoadPipeTest : public testing::Test {$/;" c namespace:doris file: +StreamMap src/runtime/data_stream_mgr.h /^ boost::shared_ptr > StreamMap;$/;" t class:doris::DataStreamMgr +StreamName src/olap/stream_name.cpp /^StreamName::StreamName(uint32_t unique_column_id, StreamInfoMessage::Kind kind) : $/;" f class:doris::StreamName +StreamName src/olap/stream_name.h /^class StreamName {$/;" c namespace:doris +StreamType src/olap/out_stream.h /^ enum StreamType {$/;" g struct:doris::StreamHead +StreamingHtMinReductionEntry src/exec/new_partitioned_aggregation_node.cc /^struct StreamingHtMinReductionEntry {$/;" s namespace:doris file: +StreamingSampler src/util/streaming_sampler.h /^ StreamingSampler(int initial_period = 500)$/;" f class:doris::StreamingSampler +StreamingSampler src/util/streaming_sampler.h /^ StreamingSampler(int period, const std::vector& initial_samples)$/;" f class:doris::StreamingSampler +StreamingSampler src/util/streaming_sampler.h /^class StreamingSampler {$/;" c namespace:doris +StrictAutoDigitLessThan src/gutil/strings/numbers.cc /^bool StrictAutoDigitLessThan(const char* a, int alen,$/;" f +StringAppendF src/gutil/stringprintf.cc /^void StringAppendF(string* dst, const char* format, ...) {$/;" f +StringAppendV src/gutil/stringprintf.cc /^void StringAppendV(string* dst, const char* format, va_list ap) {$/;" f +StringBuffer src/runtime/string_buffer.hpp /^ StringBuffer(MemPool* pool) :$/;" f class:doris::StringBuffer +StringBuffer src/runtime/string_buffer.hpp /^ StringBuffer(MemPool* pool, StringValue* str) :$/;" f class:doris::StringBuffer +StringBuffer src/runtime/string_buffer.hpp /^class StringBuffer {$/;" c namespace:doris +StringCaseEqual src/util/string_util.h /^struct StringCaseEqual {$/;" s namespace:doris +StringCaseHasher src/util/string_util.h /^struct StringCaseHasher {$/;" s namespace:doris +StringCaseLess src/util/string_util.h /^struct StringCaseLess {$/;" s namespace:doris +StringColumnDictionaryReader src/olap/rowset/column_reader.cpp /^StringColumnDictionaryReader::StringColumnDictionaryReader($/;" f class:doris::StringColumnDictionaryReader +StringColumnDictionaryReader src/olap/rowset/column_reader.h /^class StringColumnDictionaryReader {$/;" c namespace:doris +StringColumnDirectReader src/olap/rowset/column_reader.cpp /^StringColumnDirectReader::StringColumnDirectReader($/;" f class:doris::StringColumnDirectReader +StringColumnDirectReader src/olap/rowset/column_reader.h /^class StringColumnDirectReader {$/;" c namespace:doris +StringConcatHeader src/exprs/aggregate_functions.cpp /^typedef int StringConcatHeader;$/;" t namespace:doris file: +StringDict src/olap/rowset/column_writer.h /^ typedef std::map StringDict;$/;" t class:doris::VarStringColumnWriter +StringFunctions src/exprs/string_functions.h /^class StringFunctions {$/;" c namespace:doris +StringFunctionsTest test/exprs/string_functions_test.cpp /^ StringFunctionsTest() {$/;" f class:doris::StringFunctionsTest +StringFunctionsTest test/exprs/string_functions_test.cpp /^class StringFunctionsTest : public testing::Test {$/;" c namespace:doris file: +StringParseTraits src/util/string_parser.hpp /^ class StringParseTraits {$/;" c class:doris::StringParser +StringParser src/util/string_parser.hpp /^class StringParser {$/;" c namespace:doris +StringParserTest test/util/string_parser_test.cpp /^ StringParserTest(){}$/;" f class:doris::StringParserTest +StringParserTest test/util/string_parser_test.cpp /^class StringParserTest : public testing::Test {$/;" c namespace:doris file: +StringPiece src/gutil/strings/stringpiece.cc /^StringPiece::StringPiece(StringPiece x, int pos)$/;" f class:StringPiece +StringPiece src/gutil/strings/stringpiece.cc /^StringPiece::StringPiece(StringPiece x, int pos, int len)$/;" f class:StringPiece +StringPiece src/gutil/strings/stringpiece.h /^ StringPiece() : ptr_(NULL), length_(0) {}$/;" f class:StringPiece +StringPiece src/gutil/strings/stringpiece.h /^ StringPiece(const char* offset, int len) : ptr_(offset), length_(len) {$/;" f class:StringPiece +StringPiece src/gutil/strings/stringpiece.h /^ StringPiece(const char* str) \/\/ NOLINT(runtime\/explicit)$/;" f class:StringPiece +StringPiece src/gutil/strings/stringpiece.h /^ StringPiece(const std::string& str) \/\/ NOLINT(runtime\/explicit)$/;" f class:StringPiece +StringPiece src/gutil/strings/stringpiece.h /^DECLARE_POD(StringPiece); \/\/ So vector becomes really fast$/;" v +StringPiece src/gutil/strings/stringpiece.h /^class StringPiece {$/;" c +StringPieceTo src/gutil/strings/split_internal.h /^struct StringPieceTo {$/;" s namespace:strings::internal +StringPieceTo src/gutil/strings/split_internal.h /^struct StringPieceTo {$/;" s namespace:strings::internal +StringPieceTo src/gutil/strings/split_internal.h /^struct StringPieceTo {$/;" s namespace:strings::internal +StringPrintf src/gutil/stringprintf.cc /^string StringPrintf(const char* format, ...) {$/;" f +StringPrintfVector src/gutil/stringprintf.cc /^string StringPrintfVector(const char* format, const vector& v) {$/;" f +StringRef src/exec/csv_scan_node.cpp /^ StringRef(char const* const begin, int const size) :$/;" f class:doris::StringRef +StringRef src/exec/csv_scan_node.cpp /^class StringRef {$/;" c namespace:doris file: +StringReplace src/gutil/strings/util.cc /^string StringReplace(const StringPiece& s, const StringPiece& oldsub,$/;" f +StringReplace src/gutil/strings/util.cc /^void StringReplace(const StringPiece& s, const StringPiece& oldsub,$/;" f +StringSearch src/runtime/string_search.hpp /^ StringSearch() : _pattern(NULL), _mask(0) {}$/;" f class:doris::StringSearch +StringSearch src/runtime/string_search.hpp /^ StringSearch(const StringValue* pattern) : _pattern(pattern), _mask(0), _skip(0) {$/;" f class:doris::StringSearch +StringSearch src/runtime/string_search.hpp /^class StringSearch {$/;" c namespace:doris +StringUtilTest test/util/string_util_test.cpp /^ StringUtilTest() { }$/;" f class:doris::StringUtilTest +StringUtilTest test/util/string_util_test.cpp /^class StringUtilTest : public testing::Test {$/;" c namespace:doris file: +StringVal output/udf/include/udf.h /^ StringVal() : len(0), ptr(NULL) { }$/;" f struct:doris_udf::StringVal +StringVal output/udf/include/udf.h /^ StringVal(const char* ptr) : len(strlen(ptr)), ptr((uint8_t*)ptr) {}$/;" f struct:doris_udf::StringVal +StringVal output/udf/include/udf.h /^ StringVal(uint8_t* ptr, int len) : len(len), ptr(ptr) {}$/;" f struct:doris_udf::StringVal +StringVal output/udf/include/udf.h /^struct StringVal : public AnyVal {$/;" s namespace:doris_udf +StringVal src/udf/udf.cpp /^StringVal::StringVal(FunctionContext* context, int len) : $/;" f class:doris_udf::StringVal +StringVal src/udf/udf.h /^ StringVal() : len(0), ptr(NULL) { }$/;" f struct:doris_udf::StringVal +StringVal src/udf/udf.h /^ StringVal(const char* ptr) : len(strlen(ptr)), ptr((uint8_t*)ptr) {}$/;" f struct:doris_udf::StringVal +StringVal src/udf/udf.h /^ StringVal(uint8_t* ptr, int len) : len(len), ptr(ptr) {}$/;" f struct:doris_udf::StringVal +StringVal src/udf/udf.h /^struct StringVal : public AnyVal {$/;" s namespace:doris_udf +StringValue src/runtime/string_value.h /^ StringValue(): ptr(NULL), len(0) {}$/;" f struct:doris::StringValue +StringValue src/runtime/string_value.h /^ StringValue(char* ptr, int len): ptr(ptr), len(len) {}$/;" f struct:doris::StringValue +StringValue src/runtime/string_value.h /^ explicit StringValue(const std::string& s) : $/;" f struct:doris::StringValue +StringValue src/runtime/string_value.h /^struct StringValue {$/;" s namespace:doris +StringValueSet src/exprs/hybird_set.h /^ StringValueSet() {$/;" f class:doris::StringValueSet +StringValueSet src/exprs/hybird_set.h /^class StringValueSet : public HybirdSetBase {$/;" c namespace:doris +StringWrapper src/exprs/scalar_fn_call.cpp /^typedef StringVal (*StringWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +StripBrackets src/gutil/strings/strip.cc /^void StripBrackets(char left, char right, string* s) {$/;" f +StripCurlyBraces src/gutil/strings/strip.cc /^void StripCurlyBraces(string* s) {$/;" f +StripDupCharacters src/gutil/strings/strip.cc /^int StripDupCharacters(string* s, char dup_char, int start_pos) {$/;" f +StripLeadingWhiteSpace src/gutil/strings/strip.cc /^void StripLeadingWhiteSpace(string* str) {$/;" f +StripLeadingWhiteSpace src/gutil/strings/strip.h /^inline char* StripLeadingWhiteSpace(char* line) {$/;" f +StripLeadingWhiteSpace src/gutil/strings/strip.h /^inline const char* StripLeadingWhiteSpace(const char* line) {$/;" f +StripMarkupTags src/gutil/strings/strip.cc /^void StripMarkupTags(string* s) {$/;" f +StripPrefixString src/gutil/strings/strip.cc /^string StripPrefixString(StringPiece str, const StringPiece& prefix) {$/;" f +StripString src/gutil/strings/strip.cc /^void StripString(char* str, StringPiece remove, char replacewith) {$/;" f +StripString src/gutil/strings/strip.cc /^void StripString(char* str, int len, StringPiece remove, char replacewith) {$/;" f +StripString src/gutil/strings/strip.cc /^void StripString(string* s, StringPiece remove, char replacewith) {$/;" f +StripString src/gutil/strings/strip.h /^inline void StripString(char* str, char remove, char replacewith) {$/;" f +StripSuffixString src/gutil/strings/strip.cc /^string StripSuffixString(StringPiece str, const StringPiece& suffix) {$/;" f +StripTrailingNewline src/gutil/strings/strip.cc /^bool StripTrailingNewline(string* s) {$/;" f +StripTrailingWhitespace src/gutil/strings/strip.cc /^void StripTrailingWhitespace(string* const s) {$/;" f +StripWhiteSpace src/gutil/strings/strip.cc /^void StripWhiteSpace(const char** str, int* len) {$/;" f +StripWhiteSpace src/gutil/strings/strip.cc /^void StripWhiteSpace(string* str) {$/;" f +StripWhiteSpace src/gutil/strings/strip.h /^inline void StripWhiteSpace(StringPiece* str) {$/;" f +StripWhiteSpace src/gutil/strings/strip.h /^inline void StripWhiteSpace(char** str, int* len) {$/;" f +StripWhiteSpaceInCollection src/gutil/strings/strip.h /^inline void StripWhiteSpaceInCollection(Collection* collection) {$/;" f namespace:strings +SubExpr src/exprs/arithmetic_expr.h /^ SubExpr(const TExprNode& node) : ArithmeticExpr(node) { }$/;" f class:doris::SubExpr +SubExpr src/exprs/arithmetic_expr.h /^class SubExpr : public ArithmeticExpr {$/;" c namespace:doris +SubReservation src/runtime/bufferpool/buffer_pool.cc /^BufferPool::SubReservation::SubReservation(ClientHandle* client) {$/;" f class:doris::BufferPool::SubReservation +SubReservation src/runtime/bufferpool/buffer_pool.h /^class BufferPool::SubReservation {$/;" c class:doris::BufferPool +Suballocation src/runtime/bufferpool/suballocator.h /^ Suballocation()$/;" f class:doris::Suballocation +Suballocation src/runtime/bufferpool/suballocator.h /^class Suballocation {$/;" c namespace:doris +Suballocator src/runtime/bufferpool/suballocator.cc /^Suballocator::Suballocator($/;" f class:doris::Suballocator +Suballocator src/runtime/bufferpool/suballocator.h /^class Suballocator {$/;" c namespace:doris +Substitute src/gutil/strings/substitute.h /^inline string Substitute($/;" f namespace:strings +SubstituteAndAppend src/gutil/strings/substitute.cc /^void SubstituteAndAppend($/;" f namespace:strings +SubstituteArg src/gutil/strings/substitute.cc /^SubstituteArg::SubstituteArg(const void* value) {$/;" f class:strings::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg() : text_(NULL), size_(-1) {}$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(bool value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(char value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(const StringPiece& value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(const char* value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(const string& value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(double value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(float value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(int value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(long long value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(long value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(short value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(unsigned int value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(unsigned long long value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(unsigned long value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^ inline SubstituteArg(unsigned short value) \/\/ NOLINT(runtime\/explicit)$/;" f class:strings::internal::SubstituteArg +SubstituteArg src/gutil/strings/substitute.h /^class SubstituteArg {$/;" c namespace:strings::internal +SubstituteToBuffer src/gutil/strings/substitute.cc /^char* SubstituteToBuffer(StringPiece format,$/;" f namespace:strings::internal +SubstitutedSize src/gutil/strings/substitute.cc /^int SubstitutedSize(StringPiece format,$/;" f namespace:strings::internal +SuggestedDelayNS src/gutil/spinlock_internal.cc /^static int SuggestedDelayNS(int loop) {$/;" f namespace:base::internal +SumOverArenas src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::BufferAllocator::SumOverArenas($/;" f class:doris::BufferPool::BufferAllocator +SumOverSizes src/runtime/bufferpool/buffer_allocator.cc /^int64_t BufferPool::FreeBufferArena::SumOverSizes($/;" f class:doris::BufferPool::FreeBufferArena +SupportsRemove src/exprs/agg_fn.h /^ bool SupportsRemove() const { return remove_fn_ != nullptr; }$/;" f class:doris::AggFn +SupportsSerialize src/exprs/agg_fn.h /^ bool SupportsSerialize() const { return serialize_fn_ != nullptr; }$/;" f class:doris::AggFn +SymbolAction src/http/action/pprof_actions.cpp /^ SymbolAction(BfdParser* parser) : _parser(parser) {}$/;" f class:doris::SymbolAction +SymbolAction src/http/action/pprof_actions.cpp /^class SymbolAction : public HttpHandler {$/;" c namespace:doris file: +SymbolsUtil src/util/symbols_util.h /^class SymbolsUtil {$/;" c namespace:doris +SyncMode src/olap/fs/file_block_manager.cpp /^ enum SyncMode {$/;" g class:doris::fs::internal::FileWritableBlock file: +SystemAllocator src/runtime/bufferpool/system_allocator.cc /^SystemAllocator::SystemAllocator(int64_t min_buffer_len)$/;" f class:doris::SystemAllocator +SystemAllocator src/runtime/bufferpool/system_allocator.h /^class SystemAllocator {$/;" c namespace:doris +SystemAllocator src/runtime/memory/system_allocator.h /^class SystemAllocator {$/;" c namespace:doris +SystemMetrics src/util/system_metrics.cpp /^SystemMetrics::SystemMetrics() {$/;" f class:doris::SystemMetrics +SystemMetrics src/util/system_metrics.h /^class SystemMetrics {$/;" c namespace:doris +SystemMetricsTest test/util/system_metrics_test.cpp /^ SystemMetricsTest() { }$/;" f class:doris::SystemMetricsTest +SystemMetricsTest test/util/system_metrics_test.cpp /^class SystemMetricsTest : public testing::Test {$/;" c namespace:doris file: +T1 src/gutil/utf/rune.c /^ T1 = ((1<<(Bit1+1))-1) ^ 0xFF, \/* 0000 0000 *\/$/;" e enum:__anon8 file: +T2 src/gutil/utf/rune.c /^ T2 = ((1<<(Bit2+1))-1) ^ 0xFF, \/* 1100 0000 *\/$/;" e enum:__anon8 file: +T3 src/gutil/utf/rune.c /^ T3 = ((1<<(Bit3+1))-1) ^ 0xFF, \/* 1110 0000 *\/$/;" e enum:__anon8 file: +T4 src/gutil/utf/rune.c /^ T4 = ((1<<(Bit4+1))-1) ^ 0xFF, \/* 1111 0000 *\/$/;" e enum:__anon8 file: +T5 src/gutil/utf/rune.c /^ T5 = ((1<<(Bit5+1))-1) ^ 0xFF, \/* 1111 1000 *\/$/;" e enum:__anon8 file: +T8_0 src/olap/utils.cpp /^static const unsigned int T8_0[256] = {$/;" m namespace:doris file: +T8_1 src/olap/utils.cpp /^static const unsigned int T8_1[256]= {$/;" m namespace:doris file: +T8_2 src/olap/utils.cpp /^static const unsigned int T8_2[256] = {$/;" m namespace:doris file: +T8_3 src/olap/utils.cpp /^static const unsigned int T8_3[256] = {$/;" m namespace:doris file: +T8_4 src/olap/utils.cpp /^static const unsigned int T8_4[256] = {$/;" m namespace:doris file: +T8_5 src/olap/utils.cpp /^static const unsigned int T8_5[256] = {$/;" m namespace:doris file: +T8_6 src/olap/utils.cpp /^static const unsigned int T8_6[256] = {$/;" m namespace:doris file: +T8_7 src/olap/utils.cpp /^static const unsigned int T8_7[256] = {$/;" m namespace:doris file: +TABLET_CONVERT_FINISHED src/olap/olap_define.h /^static const std::string TABLET_CONVERT_FINISHED = "tablet_convert_finished";$/;" m namespace:doris +TABLET_ID src/http/action/checksum_action.cpp /^const std::string TABLET_ID = "tablet_id";$/;" m namespace:doris file: +TABLET_ID src/http/action/reload_tablet_action.cpp /^const std::string TABLET_ID = "tablet_id";$/;" m namespace:doris file: +TABLET_ID src/http/action/restore_tablet_action.cpp /^const std::string TABLET_ID = "tablet_id";$/;" m namespace:doris file: +TABLET_ID src/http/action/snapshot_action.cpp /^const std::string TABLET_ID = "tablet_id";$/;" m namespace:doris file: +TABLET_ID_KEY src/olap/olap_define.h /^const std::string TABLET_ID_KEY = "tablet_id";$/;" m namespace:doris +TABLET_ID_PREFIX src/olap/olap_define.h /^const std::string TABLET_ID_PREFIX = "t_";$/;" m namespace:doris +TABLET_NOTREADY src/olap/tablet_meta.h /^ TABLET_NOTREADY,$/;" e enum:doris::TabletState +TABLET_RUNNING src/olap/tablet_meta.h /^ TABLET_RUNNING,$/;" e enum:doris::TabletState +TABLET_SCHEMA_HASH_KEY src/olap/olap_define.h /^const std::string TABLET_SCHEMA_HASH_KEY = "schema_hash";$/;" m namespace:doris +TABLET_SHUTDOWN src/olap/tablet_meta.h /^ TABLET_SHUTDOWN$/;" e enum:doris::TabletState +TABLET_STOPPED src/olap/tablet_meta.h /^ TABLET_STOPPED,$/;" e enum:doris::TabletState +TABLET_TOMBSTONED src/olap/tablet_meta.h /^ TABLET_TOMBSTONED,$/;" e enum:doris::TabletState +TABLET_VERSION src/http/action/checksum_action.cpp /^const std::string TABLET_VERSION = "version";$/;" m namespace:doris file: +TABLE_KEY src/http/action/mini_load.cpp /^const std::string TABLE_KEY = "table";$/;" m namespace:doris file: +TAKE_OWNERSHIP src/gutil/basictypes.h /^ TAKE_OWNERSHIP$/;" e enum:Ownership +TASK_FINISH_MAX_RETRY src/agent/task_worker_pool.cpp /^const uint32_t TASK_FINISH_MAX_RETRY = 3;$/;" m namespace:doris file: +TASK_FINISH_MAX_RETRY test/agent/mock_task_worker_pool.h /^const uint32_t TASK_FINISH_MAX_RETRY = 3;$/;" m namespace:doris +TDIGEST2_TDIGEST_H_ src/util/tdigest.h 44;" d +TDescriptorTableBuilder src/runtime/descriptor_helper.h /^class TDescriptorTableBuilder {$/;" c namespace:doris +TDigest src/util/tdigest.h /^ TDigest() : TDigest(10000) {}$/;" f class:doris::TDigest +TDigest src/util/tdigest.h /^ TDigest(TDigest&& o)$/;" f class:doris::TDigest +TDigest src/util/tdigest.h /^ TDigest(Value compression, Index bufferSize) : TDigest(compression, bufferSize, 0) {}$/;" f class:doris::TDigest +TDigest src/util/tdigest.h /^ TDigest(Value compression, Index unmergedSize, Index mergedSize)$/;" f class:doris::TDigest +TDigest src/util/tdigest.h /^ TDigest(std::vector&& processed, std::vector&& unprocessed, Value compression,$/;" f class:doris::TDigest +TDigest src/util/tdigest.h /^ explicit TDigest(Value compression) : TDigest(compression, 0) {}$/;" f class:doris::TDigest +TDigest src/util/tdigest.h /^class TDigest {$/;" c namespace:doris +TDigestComparator src/util/tdigest.h /^ TDigestComparator() {}$/;" f class:doris::TDigest::TDigestComparator +TDigestComparator src/util/tdigest.h /^ class TDigestComparator {$/;" c class:doris::TDigest +TDigestRadixSortTraits src/util/tdigest.h /^ struct TDigestRadixSortTraits$/;" s class:doris::TDigest +TDigestTest test/util/tdigest_test.cpp /^ TDigestTest() {$/;" f class:doris::TDigestTest +TDigestTest test/util/tdigest_test.cpp /^class TDigestTest : public ::testing::Test {$/;" c namespace:doris file: +TE src/http/http_headers.cpp /^const char* HttpHeaders::TE = "TE";$/;" m class:doris::HttpHeaders file: +TE src/http/http_headers.h /^ static const char* TE;$/;" m class:doris::HttpHeaders +TEMPORARY_REDIRECT src/http/http_status.h /^ TEMPORARY_REDIRECT = 307,$/;" e enum:doris::HttpStatus +TEN src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +TEST src/gutil/sysinfo-test.cc /^TEST(SysInfoTest, ReadMaxCpuIndexTest) {$/;" f namespace:doris +TEST test/agent/agent_server_test.cpp /^TEST(MakeSnapshotTest, TestMakeSnapshot) {$/;" f namespace:doris +TEST test/agent/agent_server_test.cpp /^TEST(ReleaseSnapshotTest, TestReleaseSnapshot) {$/;" f namespace:doris +TEST test/agent/agent_server_test.cpp /^TEST(SubmitTasksTest, TestSubmitTasks){$/;" f namespace:doris +TEST test/agent/heartbeat_server_test.cpp /^TEST(HeartbeatTest, TestHeartbeat){$/;" f namespace:doris +TEST test/agent/utils_test.cpp /^TEST(AgentUtilsTest, Test) {$/;" f namespace:doris +TEST test/exec/olap_scanner_test.cpp /^TEST(OlapIdlUtilTest, normalcase) {$/;" f namespace:doris +TEST test/olap/decimal12_test.cpp /^TEST(FieldInfoTest, Add) {$/;" f namespace:doris +TEST test/olap/generic_iterators_test.cpp /^TEST(GenericIteratorsTest, AutoIncrement) {$/;" f namespace:doris +TEST test/olap/generic_iterators_test.cpp /^TEST(GenericIteratorsTest, Merge) {$/;" f namespace:doris +TEST test/olap/generic_iterators_test.cpp /^TEST(GenericIteratorsTest, Union) {$/;" f namespace:doris +TEST test/olap/page_cache_test.cpp /^TEST(StoragePageCacheTest, normal) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, CompressInStream) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, CompressMassOutStream) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, CompressOutStream) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, CompressOutStream2) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, CompressOutStream3) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, CompressOutStream4) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, SeekCompress) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, SeekUncompress) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, SkipCompress) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, SkipUncompress) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, UncompressInStream) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, UncompressOutStream) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, UncompressOutStream2) {$/;" f namespace:doris +TEST test/olap/run_length_byte_test.cpp /^TEST(TestStream, UncompressOutStream3) {$/;" f namespace:doris +TEST test/olap/storage_types_test.cpp /^TEST(TypesTest, copy_and_equal) {$/;" f namespace:doris +TEST test/runtime/free_list_test.cpp /^TEST(FreeListTest, Basic) {$/;" f namespace:doris +TEST test/runtime/mem_limit_test.cpp /^TEST(MemTestTest, SingleTrackerNoLimit) {$/;" f namespace:doris +TEST test/runtime/mem_limit_test.cpp /^TEST(MemTestTest, SingleTrackerWithLimit) {$/;" f namespace:doris +TEST test/runtime/mem_limit_test.cpp /^TEST(MemTestTest, TrackerHierarchy) {$/;" f namespace:doris +TEST test/runtime/mem_limit_test.cpp /^TEST(MemTestTest, TrackerHierarchyTryConsume) {$/;" f namespace:doris +TEST test/runtime/mem_pool_test.cpp /^TEST(MemPoolTest, Basic) {$/;" f namespace:doris +TEST test/runtime/mem_pool_test.cpp /^TEST(MemPoolTest, Keep) {$/;" f namespace:doris +TEST test/runtime/mem_pool_test.cpp /^TEST(MemPoolTest, MaxAllocation) {$/;" f namespace:doris +TEST test/runtime/memory/chunk_allocator_test.cpp /^TEST(ChunkAllocatorTest, Normal) {$/;" f namespace:doris +TEST test/runtime/memory/system_allocator_test.cpp /^TEST(SystemAllocatorTest, TestNormal) {$/;" f namespace:doris +TEST test/runtime/string_buffer_test.cpp /^TEST(StringBufferTest, Basic) {$/;" f namespace:doris +TEST test/runtime/string_value_test.cpp /^TEST(StringValueTest, TestCompare) {$/;" f namespace:doris +TEST test/runtime/thread_resource_mgr_test.cpp /^TEST(ThreadResourceMgr, BasicTest) {$/;" f namespace:doris +TEST test/udf/uda_test.cpp /^TEST(CountMultiArgTest, Basic) {$/;" f namespace:doris_udf +TEST test/udf/uda_test.cpp /^TEST(CountTest, Basic) {$/;" f namespace:doris_udf +TEST test/udf/uda_test.cpp /^TEST(CountTest, FuzzyEquals) {$/;" f namespace:doris_udf +TEST test/udf/uda_test.cpp /^TEST(MinTest, Basic) {$/;" f namespace:doris_udf +TEST test/udf/udf_test.cpp /^TEST(UdfTest, TestFunctionContext) {$/;" f namespace:doris_udf +TEST test/udf/udf_test.cpp /^TEST(UdfTest, TestTimestampVal) {$/;" f namespace:doris_udf +TEST test/udf/udf_test.cpp /^TEST(UdfTest, TestValidate) {$/;" f namespace:doris_udf +TEST test/udf/udf_test.cpp /^TEST(UdfTest, TestVarArgs) {$/;" f namespace:doris_udf +TEST test/util/bit_stream_utils_test.cpp /^TEST(TestBitStreamUtil, TestBool) {$/;" f namespace:doris +TEST test/util/bit_stream_utils_test.cpp /^TEST(TestBitStreamUtil, TestMixed) {$/;" f namespace:doris +TEST test/util/bit_stream_utils_test.cpp /^TEST(TestBitStreamUtil, TestSeekToBit) {$/;" f namespace:doris +TEST test/util/bit_stream_utils_test.cpp /^TEST(TestBitStreamUtil, TestUint64) {$/;" f namespace:doris +TEST test/util/bit_stream_utils_test.cpp /^TEST(TestBitStreamUtil, TestValues) {$/;" f namespace:doris +TEST test/util/bit_util_test.cpp /^TEST(BitUtil, Ceil) {$/;" f namespace:doris +TEST test/util/bit_util_test.cpp /^TEST(BitUtil, Popcount) {$/;" f namespace:doris +TEST test/util/bitmap_value_test.cpp /^TEST(BitmapValueTest, Roaring64Map) {$/;" f namespace:doris +TEST test/util/bitmap_value_test.cpp /^TEST(BitmapValueTest, bitmap_intersect) {$/;" f namespace:doris +TEST test/util/bitmap_value_test.cpp /^TEST(BitmapValueTest, bitmap_serde) {$/;" f namespace:doris +TEST test/util/bitmap_value_test.cpp /^TEST(BitmapValueTest, bitmap_to_string) {$/;" f namespace:doris +TEST test/util/bitmap_value_test.cpp /^TEST(BitmapValueTest, bitmap_union) {$/;" f namespace:doris +TEST test/util/blocking_queue_test.cpp /^TEST(BlockingQueueTest, TestBasic) {$/;" f namespace:doris +TEST test/util/blocking_queue_test.cpp /^TEST(BlockingQueueTest, TestGetFromShutdownQueue) {$/;" f namespace:doris +TEST test/util/blocking_queue_test.cpp /^TEST(BlockingQueueTest, TestMultipleThreads) {$/;" f namespace:doris +TEST test/util/cidr_test.cpp /^TEST(CIDR, contains) {$/;" f namespace:doris +TEST test/util/cidr_test.cpp /^TEST(CIDR, normal) {$/;" f namespace:doris +TEST test/util/cidr_test.cpp /^TEST(CIDR, wrong_format) {$/;" f namespace:doris +TEST test/util/countdown_latch_test.cpp /^TEST(TestCountDownLatch, TestLatch) {$/;" f namespace:doris +TEST test/util/countdown_latch_test.cpp /^TEST(TestCountDownLatch, TestResetToZero) {$/;" f namespace:doris +TEST test/util/crc32c_test.cpp /^TEST(CRC, Extend) {$/;" f namespace:doris::crc32c +TEST test/util/crc32c_test.cpp /^TEST(CRC, StandardResults) {$/;" f namespace:doris::crc32c +TEST test/util/crc32c_test.cpp /^TEST(CRC, Values) {$/;" f namespace:doris::crc32c +TEST test/util/filesystem_util_test.cpp /^TEST(FileSystemUtil, CreateDirectory) {$/;" f namespace:doris +TEST test/util/filesystem_util_test.cpp /^TEST(FileSystemUtil, rlimit) {$/;" f namespace:doris +TEST test/util/filesystem_util_test.cpp /^TEST(FilesystemUtil, contain_path) {$/;" f namespace:doris +TEST test/util/internal_queue_test.cpp /^TEST(InternalQueue, TestBasic) {$/;" f namespace:doris +TEST test/util/internal_queue_test.cpp /^TEST(InternalQueue, TestClear) {$/;" f namespace:doris +TEST test/util/internal_queue_test.cpp /^TEST(InternalQueue, TestMultiProducerMultiConsumer) {$/;" f namespace:doris +TEST test/util/internal_queue_test.cpp /^TEST(InternalQueue, TestRemove) {$/;" f namespace:doris +TEST test/util/internal_queue_test.cpp /^TEST(InternalQueue, TestSingleProducerSingleConsumer) {$/;" f namespace:doris +TEST test/util/monotime_test.cpp /^TEST(TestMonoTime, TestComparison) {$/;" f namespace:doris +TEST test/util/monotime_test.cpp /^TEST(TestMonoTime, TestDeltaConversions) {$/;" f namespace:doris +TEST test/util/monotime_test.cpp /^TEST(TestMonoTime, TestDeltas) {$/;" f namespace:doris +TEST test/util/monotime_test.cpp /^TEST(TestMonoTime, TestMonotonicity) {$/;" f namespace:doris +TEST test/util/monotime_test.cpp /^TEST(TestMonoTime, TestOperators) {$/;" f namespace:doris +TEST test/util/monotime_test.cpp /^TEST(TestMonoTime, TestSleepFor) {$/;" f namespace:doris +TEST test/util/monotime_test.cpp /^TEST(TestMonoTime, TestTimeSpec) {$/;" f namespace:doris +TEST test/util/monotime_test.cpp /^TEST(TestMonoTime, TestTimeVal) {$/;" f namespace:doris +TEST test/util/monotime_test.cpp /^TEST(TestMonoTimePerf, TestMonoTimePerf) {$/;" f namespace:doris +TEST test/util/parse_util_test.cpp /^TEST(TestParseMemSpec, Bad) {$/;" f namespace:doris +TEST test/util/parse_util_test.cpp /^TEST(TestParseMemSpec, Normal) {$/;" f namespace:doris +TEST test/util/path_util_test.cpp /^TEST(TestPathUtil, BaseNameTest) {$/;" f namespace:doris +TEST test/util/path_util_test.cpp /^TEST(TestPathUtil, DirNameTest) {$/;" f namespace:doris +TEST test/util/path_util_test.cpp /^TEST(TestPathUtil, JoinPathSegments) {$/;" f namespace:doris +TEST test/util/path_util_test.cpp /^TEST(TestPathUtil, SplitPathTest) {$/;" f namespace:doris +TEST test/util/path_util_test.cpp /^TEST(TestPathUtil, file_extension_test) {$/;" f namespace:doris +TEST test/util/perf_counters_test.cpp /^TEST(CpuInfoTest, Basic) {$/;" f namespace:impala +TEST test/util/perf_counters_test.cpp /^TEST(DiskInfoTest, Basic) {$/;" f namespace:impala +TEST test/util/perf_counters_test.cpp /^TEST(PerfCounterTest, Basic) {$/;" f namespace:impala +TEST test/util/rle_encoding_test.cpp /^TEST(Rle, SpecificSequences) {$/;" f namespace:doris +TEST test/util/rle_encoding_test.cpp /^TEST(Rle, TestValues) {$/;" f namespace:doris +TEST test/util/runtime_profile_test.cpp /^TEST(CountersTest, Basic) {$/;" f namespace:impala +TEST test/util/runtime_profile_test.cpp /^TEST(CountersTest, BucketCounters) {$/;" f namespace:impala +TEST test/util/runtime_profile_test.cpp /^TEST(CountersTest, DerivedCounters) {$/;" f namespace:impala +TEST test/util/runtime_profile_test.cpp /^TEST(CountersTest, InfoStringTest) {$/;" f namespace:impala +TEST test/util/runtime_profile_test.cpp /^TEST(CountersTest, MergeAndUpdate) {$/;" f namespace:impala +TEST test/util/runtime_profile_test.cpp /^TEST(CountersTest, RateCounters) {$/;" f namespace:impala +TEST test/util/scoped_cleanup_test.cpp /^TEST(ScopedCleanup, TestCancelCleanup) {$/;" f namespace:doris +TEST test/util/scoped_cleanup_test.cpp /^TEST(ScopedCleanup, TestCleanup) {$/;" f namespace:doris +TEST test/util/scoped_cleanup_test.cpp /^TEST(ScopedCleanup, TestCleanupMacro) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToBool, Basic) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToFloat, Basic) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToFloat, BruteForce) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToFloat, InvalidLeadingTrailing) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToInt, Basic) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToInt, Int8_Exhaustive) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToInt, InvalidLeadingTrailing) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToInt, Limit) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToInt, Overflow) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToIntWithBase, Basic) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToIntWithBase, Int8_Exhaustive) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToIntWithBase, Limit) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToIntWithBase, NonNumericCharacters) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToIntWithBase, Overflow) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToUnsignedInt, Basic) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToUnsignedInt, Limit) {$/;" f namespace:doris +TEST test/util/string_parser_test.cpp /^TEST(StringToUnsignedInt, Overflow) {$/;" f namespace:doris +TEST test/util/thread_pool_test.cpp /^TEST(ThreadPoolTest, BasicTest) {$/;" f namespace:doris +TEST test/util/url_coding_test.cpp /^TEST(Base64Test, Basic) {$/;" f namespace:doris +TEST test/util/url_coding_test.cpp /^TEST(HtmlEscapingTest, Basic) {$/;" f namespace:doris +TEST test/util/url_coding_test.cpp /^TEST(UrlCodingTest, Basic) {$/;" f namespace:doris +TEST test/util/url_coding_test.cpp /^TEST(UrlCodingTest, BlankString) {$/;" f namespace:doris +TEST test/util/url_coding_test.cpp /^TEST(UrlCodingTest, HiveExceptions) {$/;" f namespace:doris +TEST test/util/url_coding_test.cpp /^TEST(UrlCodingTest, PathSeparators) {$/;" f namespace:doris +TEST test/util/zip_util_test.cpp /^TEST(ZipUtilTest, basic) {$/;" f namespace:doris +TEST test/util/zip_util_test.cpp /^TEST(ZipUtilTest, dir) {$/;" f namespace:doris +TEST test/util/zip_util_test.cpp /^TEST(ZipUtilTest, notzip) {$/;" f namespace:doris +TEST test/util/zip_util_test.cpp /^TEST(ZipUtilTest, targetAlready) {$/;" f namespace:doris +TEST_DIR test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^static const string TEST_DIR = ".\/ut_dir\/column_reader_writer_test";$/;" m namespace:doris::segment_v2 file: +TEST_EQUAL_PREDICATE test/olap/comparison_predicate_test.cpp 135;" d file: +TEST_F test/agent/cgroups_mgr_test.cpp /^TEST_F(CgroupsMgrTest, TestAssignThreadToCgroups) {$/;" f namespace:doris +TEST_F test/agent/cgroups_mgr_test.cpp /^TEST_F(CgroupsMgrTest, TestInitCgroups) {$/;" f namespace:doris +TEST_F test/agent/cgroups_mgr_test.cpp /^TEST_F(CgroupsMgrTest, TestIsDirectory) {$/;" f namespace:doris +TEST_F test/agent/cgroups_mgr_test.cpp /^TEST_F(CgroupsMgrTest, TestIsFileExist) {$/;" f namespace:doris +TEST_F test/agent/cgroups_mgr_test.cpp /^TEST_F(CgroupsMgrTest, TestModifyUserCgroups) {$/;" f namespace:doris +TEST_F test/agent/cgroups_mgr_test.cpp /^TEST_F(CgroupsMgrTest, TestRelocateTasks) {$/;" f namespace:doris +TEST_F test/agent/cgroups_mgr_test.cpp /^TEST_F(CgroupsMgrTest, TestUpdateLocalCgroups) {$/;" f namespace:doris +TEST_F test/common/resource_tls_test.cpp /^TEST_F(ResourceTlsTest, EmptyTest) {$/;" f namespace:doris +TEST_F test/common/resource_tls_test.cpp /^TEST_F(ResourceTlsTest, NormalTest) {$/;" f namespace:doris +TEST_F test/common/status_test.cpp /^TEST_F(StatusTest, Error) {$/;" f namespace:doris +TEST_F test/common/status_test.cpp /^TEST_F(StatusTest, OK) {$/;" f namespace:doris +TEST_F test/env/env_posix_test.cpp /^TEST_F(EnvPosixTest, iterate_dir) {$/;" f namespace:doris +TEST_F test/env/env_posix_test.cpp /^TEST_F(EnvPosixTest, random_access) {$/;" f namespace:doris +TEST_F test/env/env_posix_test.cpp /^TEST_F(EnvPosixTest, random_rw) {$/;" f namespace:doris +TEST_F test/exec/broker_reader_test.cpp /^TEST_F(BrokerReaderTest, normal) {$/;" f namespace:doris +TEST_F test/exec/broker_scan_node_test.cpp /^TEST_F(BrokerScanNodeTest, normal) {$/;" f namespace:doris +TEST_F test/exec/broker_scanner_test.cpp /^TEST_F(BrokerScannerTest, normal) {$/;" f namespace:doris +TEST_F test/exec/broker_scanner_test.cpp /^TEST_F(BrokerScannerTest, normal2) {$/;" f namespace:doris +TEST_F test/exec/broker_scanner_test.cpp /^TEST_F(BrokerScannerTest, normal3) {$/;" f namespace:doris +TEST_F test/exec/broker_scanner_test.cpp /^TEST_F(BrokerScannerTest, normal4) {$/;" f namespace:doris +TEST_F test/exec/broker_scanner_test.cpp /^TEST_F(BrokerScannerTest, normal5) {$/;" f namespace:doris +TEST_F test/exec/broker_scanner_test.cpp /^TEST_F(BrokerScannerTest, normal6) {$/;" f namespace:doris +TEST_F test/exec/broker_scanner_test.cpp /^TEST_F(BrokerScannerTest, normal7) {$/;" f namespace:doris +TEST_F test/exec/broker_scanner_test.cpp /^TEST_F(BrokerScannerTest, normal8) {$/;" f namespace:doris +TEST_F test/exec/broker_scanner_test.cpp /^TEST_F(BrokerScannerTest, normal9) {$/;" f namespace:doris +TEST_F test/exec/csv_scan_bench_test.cpp /^TEST_F(CsvScanNodeBenchTest, NormalUse) {$/;" f namespace:doris +TEST_F test/exec/csv_scan_node_test.cpp /^TEST_F(CsvScanNodeTest, NormalUse) {$/;" f namespace:doris +TEST_F test/exec/csv_scan_node_test.cpp /^TEST_F(CsvScanNodeTest, continuousDelim) {$/;" f namespace:doris +TEST_F test/exec/csv_scan_node_test.cpp /^TEST_F(CsvScanNodeTest, fill_fix_len_stringi_test) {$/;" f namespace:doris +TEST_F test/exec/csv_scan_node_test.cpp /^TEST_F(CsvScanNodeTest, wrong_decimal_format_test) {$/;" f namespace:doris +TEST_F test/exec/csv_scan_node_test.cpp /^TEST_F(CsvScanNodeTest, wrong_fix_len_string_format_test) {$/;" f namespace:doris +TEST_F test/exec/csv_scanner_test.cpp /^TEST_F(CsvScannerTest, no_exist_files) {$/;" f namespace:doris +TEST_F test/exec/csv_scanner_test.cpp /^TEST_F(CsvScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/es_http_scan_node_test.cpp /^TEST_F(EsHttpScanNodeTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/es_predicate_test.cpp /^TEST_F(EsPredicateTest, normal) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, bool_query) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, compound_bool_query) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, es_query) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, exists_query) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, like_query) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, match_all_query) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, range_query) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, term_query) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, terms_in_query) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, validate_esquery) {$/;" f namespace:doris +TEST_F test/exec/es_query_builder_test.cpp /^TEST_F(BooleanQueryBuilderTest, validate_partial) {$/;" f namespace:doris +TEST_F test/exec/es_scan_node_test.cpp /^TEST_F(EsScanNodeTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/es_scan_reader_test.cpp /^TEST_F(MockESServerTest, workflow) {$/;" f namespace:doris +TEST_F test/exec/hash_table_test.cpp /^TEST_F(HashTableTest, BasicTest) {$/;" f namespace:doris +TEST_F test/exec/hash_table_test.cpp /^TEST_F(HashTableTest, GrowTableTest) {$/;" f namespace:doris +TEST_F test/exec/hash_table_test.cpp /^TEST_F(HashTableTest, GrowTableTest2) {$/;" f namespace:doris +TEST_F test/exec/hash_table_test.cpp /^TEST_F(HashTableTest, ScanTest) {$/;" f namespace:doris +TEST_F test/exec/hash_table_test.cpp /^TEST_F(HashTableTest, SetupTest) {$/;" f namespace:doris +TEST_F test/exec/mysql_scan_node_test.cpp /^TEST_F(MysqlScanNodeTest, Prepare_fail_1) {$/;" f namespace:doris +TEST_F test/exec/mysql_scan_node_test.cpp /^TEST_F(MysqlScanNodeTest, Prepare_fail_2) {$/;" f namespace:doris +TEST_F test/exec/mysql_scan_node_test.cpp /^TEST_F(MysqlScanNodeTest, invalid_input) {$/;" f namespace:doris +TEST_F test/exec/mysql_scan_node_test.cpp /^TEST_F(MysqlScanNodeTest, no_init) {$/;" f namespace:doris +TEST_F test/exec/mysql_scan_node_test.cpp /^TEST_F(MysqlScanNodeTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/mysql_scan_node_test.cpp /^TEST_F(MysqlScanNodeTest, open_fail_1) {$/;" f namespace:doris +TEST_F test/exec/mysql_scan_node_test.cpp /^TEST_F(MysqlScanNodeTest, open_fail_2) {$/;" f namespace:doris +TEST_F test/exec/mysql_scan_node_test.cpp /^TEST_F(MysqlScanNodeTest, open_fail_3) {$/;" f namespace:doris +TEST_F test/exec/mysql_scanner_test.cpp /^TEST_F(MysqlScannerTest, no_init) {$/;" f namespace:doris +TEST_F test/exec/mysql_scanner_test.cpp /^TEST_F(MysqlScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/mysql_scanner_test.cpp /^TEST_F(MysqlScannerTest, open_failed) {$/;" f namespace:doris +TEST_F test/exec/mysql_scanner_test.cpp /^TEST_F(MysqlScannerTest, query_failed) {$/;" f namespace:doris +TEST_F test/exec/new_olap_scan_node_test.cpp /^TEST_F(TestOlapScanNode, MultiColumnMultiVersionTest) {$/;" f namespace:doris +TEST_F test/exec/new_olap_scan_node_test.cpp /^TEST_F(TestOlapScanNode, MultiColumnSingleVersionTest) {$/;" f namespace:doris +TEST_F test/exec/new_olap_scan_node_test.cpp /^TEST_F(TestOlapScanNode, SimpleTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(ColumnValueRangeTest, ExceptionCase) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(ColumnValueRangeTest, FixedAddRangeTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(ColumnValueRangeTest, FixedAndRangeIntersectionTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(ColumnValueRangeTest, FixedValueIntersectionTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(ColumnValueRangeTest, NormalCase) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(ColumnValueRangeTest, RangeAddRangeTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(ColumnValueRangeTest, RangeIntersectionTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(DorisScanRangeTest, FixedIntersectionTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(DorisScanRangeTest, RangeIntersectionTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(DorisScanRangeTest, ToOlapFilterTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(OlapScanKeysTest, EachtypeTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(OlapScanKeysTest, ExtendFixedAndRangeTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(OlapScanKeysTest, ExtendFixedTest) {$/;" f namespace:doris +TEST_F test/exec/olap_common_test.cpp /^TEST_F(OlapScanKeysTest, ExtendRangeTest) {$/;" f namespace:doris +TEST_F test/exec/olap_scan_node_test.cpp /^TEST_F(OlapScanNodeTest, NormalUse) {$/;" f namespace:doris +TEST_F test/exec/olap_scan_node_test.cpp /^TEST_F(OlapScanNodeTest, PushDownBinaryEqualPredicate) {$/;" f namespace:doris +TEST_F test/exec/olap_scan_node_test.cpp /^TEST_F(OlapScanNodeTest, PushDownBinaryPredicate) {$/;" f namespace:doris +TEST_F test/exec/olap_scan_node_test.cpp /^TEST_F(OlapScanNodeTest, PushDownInPredicateCase) {$/;" f namespace:doris +TEST_F test/exec/orc_scanner_test.cpp /^TEST_F(OrcScannerTest, normal) {$/;" f namespace:doris +TEST_F test/exec/orc_scanner_test.cpp /^TEST_F(OrcScannerTest, normal2) {$/;" f namespace:doris +TEST_F test/exec/orc_scanner_test.cpp /^TEST_F(OrcScannerTest, normal3) {$/;" f namespace:doris +TEST_F test/exec/parquet_scanner_test.cpp /^TEST_F(ParquetSannerTest, normal) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, HashEmpty) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, LinearBasicTest) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, LinearGrowTableTest) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, LinearInsertFullTest) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, LinearScanTest) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, LinearSetupTest) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, QuadraticBasicTest) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, QuadraticGrowTableTest) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, QuadraticInsertFullTest) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, QuadraticScanTest) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, QuadraticSetupTest) {$/;" f namespace:doris +TEST_F test/exec/partitioned_hash_table_test.cpp /^TEST_F(PartitionedHashTableTest, VeryLowMemTest) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_bzip_test.cpp /^TEST_F(PlainTextLineReaderTest, bzip2_normal_use) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_bzip_test.cpp /^TEST_F(PlainTextLineReaderTest, bzip2_test_limit) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_bzip_test.cpp /^TEST_F(PlainTextLineReaderTest, bzip2_test_limit2) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_bzip_test.cpp /^TEST_F(PlainTextLineReaderTest, bzip2_test_limit3) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_bzip_test.cpp /^TEST_F(PlainTextLineReaderTest, bzip2_test_limit4) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_bzip_test.cpp /^TEST_F(PlainTextLineReaderTest, bzip2_test_limit5) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_gzip_test.cpp /^TEST_F(PlainTextLineReaderTest, gzip_normal_use) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_gzip_test.cpp /^TEST_F(PlainTextLineReaderTest, gzip_test_limit) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_gzip_test.cpp /^TEST_F(PlainTextLineReaderTest, gzip_test_limit2) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_gzip_test.cpp /^TEST_F(PlainTextLineReaderTest, gzip_test_limit3) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_gzip_test.cpp /^TEST_F(PlainTextLineReaderTest, gzip_test_limit4) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_gzip_test.cpp /^TEST_F(PlainTextLineReaderTest, gzip_test_limit5) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_gzip_test.cpp /^TEST_F(PlainTextLineReaderTest, uncompressed_no_newline) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lz4frame_test.cpp /^TEST_F(PlainTextLineReaderTest, lz4_normal_use) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lz4frame_test.cpp /^TEST_F(PlainTextLineReaderTest, lz4_test_limit) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lz4frame_test.cpp /^TEST_F(PlainTextLineReaderTest, lz4_test_limit2) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lz4frame_test.cpp /^TEST_F(PlainTextLineReaderTest, lz4_test_limit3) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lz4frame_test.cpp /^TEST_F(PlainTextLineReaderTest, lz4_test_limit4) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lz4frame_test.cpp /^TEST_F(PlainTextLineReaderTest, lz4_test_limit5) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lzop_test.cpp /^TEST_F(PlainTextLineReaderTest, lzop_normal_use) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lzop_test.cpp /^TEST_F(PlainTextLineReaderTest, lzop_test_larger) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lzop_test.cpp /^TEST_F(PlainTextLineReaderTest, lzop_test_limit) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lzop_test.cpp /^TEST_F(PlainTextLineReaderTest, lzop_test_limit2) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lzop_test.cpp /^TEST_F(PlainTextLineReaderTest, lzop_test_limit3) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lzop_test.cpp /^TEST_F(PlainTextLineReaderTest, lzop_test_limit4) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_lzop_test.cpp /^TEST_F(PlainTextLineReaderTest, lzop_test_limit5) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_uncompressed_test.cpp /^TEST_F(PlainTextLineReaderTest, uncompressed_no_newline) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_uncompressed_test.cpp /^TEST_F(PlainTextLineReaderTest, uncompressed_normal_use) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_uncompressed_test.cpp /^TEST_F(PlainTextLineReaderTest, uncompressed_test_empty) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_uncompressed_test.cpp /^TEST_F(PlainTextLineReaderTest, uncompressed_test_limit) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_uncompressed_test.cpp /^TEST_F(PlainTextLineReaderTest, uncompressed_test_limit2) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_uncompressed_test.cpp /^TEST_F(PlainTextLineReaderTest, uncompressed_test_limit3) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_uncompressed_test.cpp /^TEST_F(PlainTextLineReaderTest, uncompressed_test_limit4) {$/;" f namespace:doris +TEST_F test/exec/plain_text_line_reader_uncompressed_test.cpp /^TEST_F(PlainTextLineReaderTest, uncompressed_test_limit5) {$/;" f namespace:doris +TEST_F test/exec/schema_scan_node_test.cpp /^TEST_F(SchemaScanNodeTest, Prepare_fail_1) {$/;" f namespace:doris +TEST_F test/exec/schema_scan_node_test.cpp /^TEST_F(SchemaScanNodeTest, Prepare_fail_2) {$/;" f namespace:doris +TEST_F test/exec/schema_scan_node_test.cpp /^TEST_F(SchemaScanNodeTest, dummy) {$/;" f namespace:doris +TEST_F test/exec/schema_scan_node_test.cpp /^TEST_F(SchemaScanNodeTest, get_dest_desc_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scan_node_test.cpp /^TEST_F(SchemaScanNodeTest, invalid_param) {$/;" f namespace:doris +TEST_F test/exec/schema_scan_node_test.cpp /^TEST_F(SchemaScanNodeTest, no_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scan_node_test.cpp /^TEST_F(SchemaScanNodeTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_authors_scanner_test.cpp /^TEST_F(SchemaAuthorScannerTest, invalid_param) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_authors_scanner_test.cpp /^TEST_F(SchemaAuthorScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_authors_scanner_test.cpp /^TEST_F(SchemaAuthorScannerTest, use_with_no_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^TEST_F(SchemaCharsetsScannerTest, invalid_param) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^TEST_F(SchemaCharsetsScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^TEST_F(SchemaCharsetsScannerTest, use_with_no_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_collations_scanner_test.cpp /^TEST_F(SchemaCollationsScannerTest, invalid_param) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_collations_scanner_test.cpp /^TEST_F(SchemaCollationsScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_collations_scanner_test.cpp /^TEST_F(SchemaCollationsScannerTest, use_with_no_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_columns_scanner_test.cpp /^TEST_F(SchemaColumnsScannerTest, desc_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_columns_scanner_test.cpp /^TEST_F(SchemaColumnsScannerTest, input_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_columns_scanner_test.cpp /^TEST_F(SchemaColumnsScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_columns_scanner_test.cpp /^TEST_F(SchemaColumnsScannerTest, one_column) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_columns_scanner_test.cpp /^TEST_F(SchemaColumnsScannerTest, op_before_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_columns_scanner_test.cpp /^TEST_F(SchemaColumnsScannerTest, start_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_columns_scanner_test.cpp /^TEST_F(SchemaColumnsScannerTest, table_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^TEST_F(SchemaCreateTableScannerTest, desc_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^TEST_F(SchemaCreateTableScannerTest, input_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^TEST_F(SchemaCreateTableScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^TEST_F(SchemaCreateTableScannerTest, one_column) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^TEST_F(SchemaCreateTableScannerTest, op_before_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^TEST_F(SchemaCreateTableScannerTest, start_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^TEST_F(SchemaCreateTableScannerTest, table_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_engines_scanner_test.cpp /^TEST_F(SchemaEnginesScannerTest, invalid_param) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_engines_scanner_test.cpp /^TEST_F(SchemaEnginesScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_engines_scanner_test.cpp /^TEST_F(SchemaEnginesScannerTest, use_with_no_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^TEST_F(SchemaOpenTablesScannerTest, desc_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^TEST_F(SchemaOpenTablesScannerTest, input_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^TEST_F(SchemaOpenTablesScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^TEST_F(SchemaOpenTablesScannerTest, one_column) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^TEST_F(SchemaOpenTablesScannerTest, op_before_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^TEST_F(SchemaOpenTablesScannerTest, start_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^TEST_F(SchemaOpenTablesScannerTest, table_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^TEST_F(SchemaSchemataScannerTest, input_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^TEST_F(SchemaSchemataScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^TEST_F(SchemaSchemataScannerTest, one_column) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^TEST_F(SchemaSchemataScannerTest, op_before_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^TEST_F(SchemaSchemataScannerTest, start_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^TEST_F(SchemaTableNamesScannerTest, input_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^TEST_F(SchemaTableNamesScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^TEST_F(SchemaTableNamesScannerTest, one_column) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^TEST_F(SchemaTableNamesScannerTest, op_before_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^TEST_F(SchemaTableNamesScannerTest, start_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^TEST_F(SchemaTableNamesScannerTest, table_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_tables_scanner_test.cpp /^TEST_F(SchemaTablesScannerTest, desc_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_tables_scanner_test.cpp /^TEST_F(SchemaTablesScannerTest, input_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_tables_scanner_test.cpp /^TEST_F(SchemaTablesScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_tables_scanner_test.cpp /^TEST_F(SchemaTablesScannerTest, one_column) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_tables_scanner_test.cpp /^TEST_F(SchemaTablesScannerTest, op_before_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_tables_scanner_test.cpp /^TEST_F(SchemaTablesScannerTest, start_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_tables_scanner_test.cpp /^TEST_F(SchemaTablesScannerTest, table_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_variables_scanner_test.cpp /^TEST_F(SchemaVariablesScannerTest, invalid_param) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_variables_scanner_test.cpp /^TEST_F(SchemaVariablesScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner/schema_variables_scanner_test.cpp /^TEST_F(SchemaVariablesScannerTest, use_with_no_init) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner_test.cpp /^TEST_F(SchemaScannerTest, input_fail) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner_test.cpp /^TEST_F(SchemaScannerTest, invalid_param) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner_test.cpp /^TEST_F(SchemaScannerTest, no_init_use) {$/;" f namespace:doris +TEST_F test/exec/schema_scanner_test.cpp /^TEST_F(SchemaScannerTest, normal_use) {$/;" f namespace:doris +TEST_F test/exec/set_executor_test.cpp /^TEST_F(SetExecutorTest, failed_case) {$/;" f namespace:doris +TEST_F test/exec/set_executor_test.cpp /^TEST_F(SetExecutorTest, normal_case) {$/;" f namespace:doris +TEST_F test/exec/tablet_info_test.cpp /^TEST_F(OlapTablePartitionParamTest, NodesInfo) {$/;" f namespace:doris +TEST_F test/exec/tablet_info_test.cpp /^TEST_F(OlapTablePartitionParamTest, bad_index) {$/;" f namespace:doris +TEST_F test/exec/tablet_info_test.cpp /^TEST_F(OlapTablePartitionParamTest, normal) {$/;" f namespace:doris +TEST_F test/exec/tablet_info_test.cpp /^TEST_F(OlapTablePartitionParamTest, tableLoacation) {$/;" f namespace:doris +TEST_F test/exec/tablet_info_test.cpp /^TEST_F(OlapTablePartitionParamTest, to_protobuf) {$/;" f namespace:doris +TEST_F test/exec/tablet_info_test.cpp /^TEST_F(OlapTablePartitionParamTest, unknown_distributed_col) {$/;" f namespace:doris +TEST_F test/exec/tablet_info_test.cpp /^TEST_F(OlapTablePartitionParamTest, unknown_index_column) {$/;" f namespace:doris +TEST_F test/exec/tablet_info_test.cpp /^TEST_F(OlapTablePartitionParamTest, unknown_partition_column) {$/;" f namespace:doris +TEST_F test/exec/tablet_info_test.cpp /^TEST_F(OlapTablePartitionParamTest, unpartitioned) {$/;" f namespace:doris +TEST_F test/exec/tablet_sink_test.cpp /^TEST_F(OlapTableSinkTest, add_batch_failed) {$/;" f namespace:doris::stream_load +TEST_F test/exec/tablet_sink_test.cpp /^TEST_F(OlapTableSinkTest, convert) {$/;" f namespace:doris::stream_load +TEST_F test/exec/tablet_sink_test.cpp /^TEST_F(OlapTableSinkTest, decimal) {$/;" f namespace:doris::stream_load +TEST_F test/exec/tablet_sink_test.cpp /^TEST_F(OlapTableSinkTest, init_fail1) {$/;" f namespace:doris::stream_load +TEST_F test/exec/tablet_sink_test.cpp /^TEST_F(OlapTableSinkTest, init_fail3) {$/;" f namespace:doris::stream_load +TEST_F test/exec/tablet_sink_test.cpp /^TEST_F(OlapTableSinkTest, init_fail4) {$/;" f namespace:doris::stream_load +TEST_F test/exec/tablet_sink_test.cpp /^TEST_F(OlapTableSinkTest, normal) {$/;" f namespace:doris::stream_load +TEST_F test/exprs/binary_predicate_test.cpp /^TEST_F(BinaryOpTest, NormalTest) {$/;" f namespace:doris +TEST_F test/exprs/binary_predicate_test.cpp /^TEST_F(BinaryOpTest, PrepareTest) {$/;" f namespace:doris +TEST_F test/exprs/binary_predicate_test.cpp /^TEST_F(BinaryOpTest, SimplePerformanceTest) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, bitmap_count) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, bitmap_empty) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, bitmap_from_string) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, bitmap_get_value) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, bitmap_union) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, bitmap_union_int) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, test_bitmap_intersect) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, to_bitmap) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, to_bitmap_invalid_argument) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, to_bitmap_null) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest, to_bitmap_out_of_range) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest,bitmap_and) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest,bitmap_contains) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest,bitmap_has_any) {$/;" f namespace:doris +TEST_F test/exprs/bitmap_function_test.cpp /^TEST_F(BitmapFunctionsTest,bitmap_or) {$/;" f namespace:doris +TEST_F test/exprs/hll_function_test.cpp /^TEST_F(HllFunctionsTest, hll_hash) {$/;" f namespace:doris +TEST_F test/exprs/hll_function_test.cpp /^TEST_F(HllFunctionsTest, hll_hash_null) {$/;" f namespace:doris +TEST_F test/exprs/hll_function_test.cpp /^TEST_F(HllFunctionsTest, hll_merge) {$/;" f namespace:doris +TEST_F test/exprs/hll_function_test.cpp /^TEST_F(HllFunctionsTest, hll_update) {$/;" f namespace:doris +TEST_F test/exprs/hybird_set_test.cpp /^TEST_F(HybirdSetTest, bigint) {$/;" f namespace:doris +TEST_F test/exprs/hybird_set_test.cpp /^TEST_F(HybirdSetTest, bool) {$/;" f namespace:doris +TEST_F test/exprs/hybird_set_test.cpp /^TEST_F(HybirdSetTest, double) {$/;" f namespace:doris +TEST_F test/exprs/hybird_set_test.cpp /^TEST_F(HybirdSetTest, float) {$/;" f namespace:doris +TEST_F test/exprs/hybird_set_test.cpp /^TEST_F(HybirdSetTest, int) {$/;" f namespace:doris +TEST_F test/exprs/hybird_set_test.cpp /^TEST_F(HybirdSetTest, smallint) {$/;" f namespace:doris +TEST_F test/exprs/hybird_set_test.cpp /^TEST_F(HybirdSetTest, string) {$/;" f namespace:doris +TEST_F test/exprs/hybird_set_test.cpp /^TEST_F(HybirdSetTest, timestamp) {$/;" f namespace:doris +TEST_F test/exprs/hybird_set_test.cpp /^TEST_F(HybirdSetTest, tinyint) {$/;" f namespace:doris +TEST_F test/exprs/in_op_test.cpp /^TEST_F(InOpTest, NormalTest) {$/;" f namespace:doris +TEST_F test/exprs/in_op_test.cpp /^TEST_F(InOpTest, PrepareTest) {$/;" f namespace:doris +TEST_F test/exprs/in_op_test.cpp /^TEST_F(InOpTest, SimplePerformanceTest) {$/;" f namespace:doris +TEST_F test/exprs/in_predicate_test.cpp /^TEST_F(InPredicateTest, 100_const) {$/;" f namespace:doris +TEST_F test/exprs/in_predicate_test.cpp /^TEST_F(InPredicateTest, diff_type) {$/;" f namespace:doris +TEST_F test/exprs/in_predicate_test.cpp /^TEST_F(InPredicateTest, no_child) {$/;" f namespace:doris +TEST_F test/exprs/in_predicate_test.cpp /^TEST_F(InPredicateTest, push_100_const) {$/;" f namespace:doris +TEST_F test/exprs/json_function_test.cpp /^TEST_F(JsonFunctionTest, double)$/;" f namespace:doris +TEST_F test/exprs/json_function_test.cpp /^TEST_F(JsonFunctionTest, int)$/;" f namespace:doris +TEST_F test/exprs/json_function_test.cpp /^TEST_F(JsonFunctionTest, special_char)$/;" f namespace:doris +TEST_F test/exprs/json_function_test.cpp /^TEST_F(JsonFunctionTest, string)$/;" f namespace:doris +TEST_F test/exprs/percentile_approx_test.cpp /^TEST_F(PercentileApproxTest, testNoMerge) {$/;" f namespace:doris +TEST_F test/exprs/percentile_approx_test.cpp /^TEST_F(PercentileApproxTest, testNullVale) {$/;" f namespace:doris +TEST_F test/exprs/percentile_approx_test.cpp /^TEST_F(PercentileApproxTest, testSample) {$/;" f namespace:doris +TEST_F test/exprs/percentile_approx_test.cpp /^TEST_F(PercentileApproxTest, testSerialize) {$/;" f namespace:doris +TEST_F test/exprs/string_functions_test.cpp /^TEST_F(StringFunctionsTest, ends_with) {$/;" f namespace:doris +TEST_F test/exprs/string_functions_test.cpp /^TEST_F(StringFunctionsTest, money_format_bigint) {$/;" f namespace:doris +TEST_F test/exprs/string_functions_test.cpp /^TEST_F(StringFunctionsTest, money_format_decimal) {$/;" f namespace:doris +TEST_F test/exprs/string_functions_test.cpp /^TEST_F(StringFunctionsTest, money_format_decimal_v2) {$/;" f namespace:doris +TEST_F test/exprs/string_functions_test.cpp /^TEST_F(StringFunctionsTest, money_format_double) {$/;" f namespace:doris +TEST_F test/exprs/string_functions_test.cpp /^TEST_F(StringFunctionsTest, money_format_large_int) {$/;" f namespace:doris +TEST_F test/exprs/string_functions_test.cpp /^TEST_F(StringFunctionsTest, null_or_empty) {$/;" f namespace:doris +TEST_F test/exprs/string_functions_test.cpp /^TEST_F(StringFunctionsTest, split_part) {$/;" f namespace:doris +TEST_F test/exprs/string_functions_test.cpp /^TEST_F(StringFunctionsTest, starts_with) {$/;" f namespace:doris +TEST_F test/exprs/timestamp_functions_test.cpp /^TEST_F(TimestampFunctionsTest, convert_tz_test) {$/;" f namespace:doris +TEST_F test/exprs/timestamp_functions_test.cpp /^TEST_F(TimestampFunctionsTest, curtime) {$/;" f namespace:doris +TEST_F test/exprs/timestamp_functions_test.cpp /^TEST_F(TimestampFunctionsTest, day_of_week_test) {$/;" f namespace:doris +TEST_F test/exprs/timestamp_functions_test.cpp /^TEST_F(TimestampFunctionsTest, from_unix) {$/;" f namespace:doris +TEST_F test/exprs/timestamp_functions_test.cpp /^TEST_F(TimestampFunctionsTest, now) {$/;" f namespace:doris +TEST_F test/exprs/timestamp_functions_test.cpp /^TEST_F(TimestampFunctionsTest, time_diff_test) {$/;" f namespace:doris +TEST_F test/exprs/timestamp_functions_test.cpp /^TEST_F(TimestampFunctionsTest, timestampdiff_test) {$/;" f namespace:doris +TEST_F test/exprs/timestamp_functions_test.cpp /^TEST_F(TimestampFunctionsTest, to_unix) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, as_wkt) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, st_circle) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, st_contains) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, st_contains_cached) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, st_dist_sphere) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, st_from_wkt) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, st_line) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, st_point) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, st_poly_line_fail) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, st_polygon) {$/;" f namespace:doris +TEST_F test/geo/geo_functions_test.cpp /^TEST_F(GeoFunctionsTest, st_x_y) {$/;" f namespace:doris +TEST_F test/geo/geo_types_test.cpp /^TEST_F(GeoTypesTest, circle) {$/;" f namespace:doris +TEST_F test/geo/geo_types_test.cpp /^TEST_F(GeoTypesTest, linestring) {$/;" f namespace:doris +TEST_F test/geo/geo_types_test.cpp /^TEST_F(GeoTypesTest, point_invalid) {$/;" f namespace:doris +TEST_F test/geo/geo_types_test.cpp /^TEST_F(GeoTypesTest, point_normal) {$/;" f namespace:doris +TEST_F test/geo/geo_types_test.cpp /^TEST_F(GeoTypesTest, polygon_contains) {$/;" f namespace:doris +TEST_F test/geo/geo_types_test.cpp /^TEST_F(GeoTypesTest, polygon_hole_contains) {$/;" f namespace:doris +TEST_F test/geo/geo_types_test.cpp /^TEST_F(GeoTypesTest, polygon_parse_fail) {$/;" f namespace:doris +TEST_F test/geo/wkt_parse_test.cpp /^TEST_F(WktParseTest, invalid_wkt) {$/;" f namespace:doris +TEST_F test/geo/wkt_parse_test.cpp /^TEST_F(WktParseTest, normal) {$/;" f namespace:doris +TEST_F test/http/http_client_test.cpp /^TEST_F(HttpClientTest, download) {$/;" f namespace:doris +TEST_F test/http/http_client_test.cpp /^TEST_F(HttpClientTest, get_failed) {$/;" f namespace:doris +TEST_F test/http/http_client_test.cpp /^TEST_F(HttpClientTest, get_normal) {$/;" f namespace:doris +TEST_F test/http/http_client_test.cpp /^TEST_F(HttpClientTest, post_failed) {$/;" f namespace:doris +TEST_F test/http/http_client_test.cpp /^TEST_F(HttpClientTest, post_normal) {$/;" f namespace:doris +TEST_F test/http/http_utils_test.cpp /^TEST_F(HttpUtilsTest, parse_basic_auth) {$/;" f namespace:doris +TEST_F test/http/message_body_sink_test.cpp /^TEST_F(MessageBodySinkTest, file_sink) {$/;" f namespace:doris +TEST_F test/http/metrics_action_test.cpp /^TEST_F(MetricsActionTest, prometheus_no_name) {$/;" f namespace:doris +TEST_F test/http/metrics_action_test.cpp /^TEST_F(MetricsActionTest, prometheus_no_prefix) {$/;" f namespace:doris +TEST_F test/http/metrics_action_test.cpp /^TEST_F(MetricsActionTest, prometheus_output) {$/;" f namespace:doris +TEST_F test/http/stream_load_test.cpp /^TEST_F(StreamLoadActionTest, begin_fail) {$/;" f namespace:doris +TEST_F test/http/stream_load_test.cpp /^TEST_F(StreamLoadActionTest, commit_fail) {$/;" f namespace:doris +TEST_F test/http/stream_load_test.cpp /^TEST_F(StreamLoadActionTest, no_auth) {$/;" f namespace:doris +TEST_F test/http/stream_load_test.cpp /^TEST_F(StreamLoadActionTest, normal) {$/;" f namespace:doris +TEST_F test/http/stream_load_test.cpp /^TEST_F(StreamLoadActionTest, plan_fail) {$/;" f namespace:doris +TEST_F test/http/stream_load_test.cpp /^TEST_F(StreamLoadActionTest, put_fail) {$/;" f namespace:doris +TEST_F test/olap/aggregate_func_test.cpp /^TEST_F(AggregateFuncTest, max) {$/;" f namespace:doris +TEST_F test/olap/aggregate_func_test.cpp /^TEST_F(AggregateFuncTest, min) {$/;" f namespace:doris +TEST_F test/olap/aggregate_func_test.cpp /^TEST_F(AggregateFuncTest, replace) {$/;" f namespace:doris +TEST_F test/olap/aggregate_func_test.cpp /^TEST_F(AggregateFuncTest, sum) {$/;" f namespace:doris +TEST_F test/olap/bit_field_test.cpp /^TEST_F(TestBitField, ReadWriteMultiBits) {$/;" f namespace:doris +TEST_F test/olap/bit_field_test.cpp /^TEST_F(TestBitField, ReadWriteOneBit) {$/;" f namespace:doris +TEST_F test/olap/bit_field_test.cpp /^TEST_F(TestBitField, Seek) {$/;" f namespace:doris +TEST_F test/olap/bit_field_test.cpp /^TEST_F(TestBitField, Skip) {$/;" f namespace:doris +TEST_F test/olap/bloom_filter_index_test.cpp /^TEST_F(TestBloomFilterIndex, abnormal_read) {$/;" f namespace:doris +TEST_F test/olap/bloom_filter_index_test.cpp /^TEST_F(TestBloomFilterIndex, abnormal_write) {$/;" f namespace:doris +TEST_F test/olap/bloom_filter_index_test.cpp /^TEST_F(TestBloomFilterIndex, normal_read_and_write) {$/;" f namespace:doris +TEST_F test/olap/bloom_filter_test.cpp /^TEST_F(TestBloomFilter, add_and_test_bytes) {$/;" f namespace:doris +TEST_F test/olap/bloom_filter_test.cpp /^TEST_F(TestBloomFilter, bloom_filter_info) {$/;" f namespace:doris +TEST_F test/olap/bloom_filter_test.cpp /^TEST_F(TestBloomFilter, init_bloom_filter) {$/;" f namespace:doris +TEST_F test/olap/byte_buffer_test.cpp /^TEST_F(TestByteBuffer, TestMmap) {$/;" f namespace:doris +TEST_F test/olap/byte_buffer_test.cpp /^TEST_F(TestByteBuffer, TestReadWrite) {$/;" f namespace:doris +TEST_F test/olap/byte_buffer_test.cpp /^TEST_F(TestByteBuffer, TestRef) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SeekDecimalColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SeekDirectVarcharColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SeekDirectVarcharColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SeekFloatColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SeekShortColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SeekShortColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SeekTinyColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SeekTinyColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SkipDecimalColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SkipDirectVarcharColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SkipFloatColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SkipLargeIntColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SkipShortColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SkipShortColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SkipTinyColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, SkipTinyColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, TinyColumnIndex) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDateColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDateColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDatetimeColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDatetimeColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDecimalColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDecimalColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDirectVarcharColumnWith65533) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDirectVarcharColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDirectVarcharColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDoubleColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedDoubleColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedFloatColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedFloatColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedIntColumnMassWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedIntColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedIntColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedLargeIntColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedLargeIntColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedLongColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedLongColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedShortColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedShortColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedStringColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedStringColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedStringColumnWithoutoutPresent2) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedTinyColumnWithPresent) {$/;" f namespace:doris +TEST_F test/olap/column_reader_test.cpp /^TEST_F(TestColumn, VectorizedTinyColumnWithoutPresent) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestEqualPredicate, DATETIME_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestEqualPredicate, DATE_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestEqualPredicate, DECIMAL_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestEqualPredicate, DOUBLE_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestEqualPredicate, FLOAT_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestEqualPredicate, STRING_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestLessPredicate, DATETIME_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestLessPredicate, DATE_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestLessPredicate, DECIMAL_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestLessPredicate, DOUBLE_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestLessPredicate, FLOAT_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/comparison_predicate_test.cpp /^TEST_F(TestLessPredicate, STRING_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/delete_handler_test.cpp /^TEST_F(TestDeleteConditionHandler, StoreCondInvalidParameters) {$/;" f namespace:doris +TEST_F test/olap/delete_handler_test.cpp /^TEST_F(TestDeleteConditionHandler, StoreCondNonexistentColumn) {$/;" f namespace:doris +TEST_F test/olap/delete_handler_test.cpp /^TEST_F(TestDeleteConditionHandler, StoreCondSucceed) {$/;" f namespace:doris +TEST_F test/olap/delete_handler_test.cpp /^TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {$/;" f namespace:doris +TEST_F test/olap/delete_handler_test.cpp /^TEST_F(TestDeleteConditionHandler2, ValidConditionValue) {$/;" f namespace:doris +TEST_F test/olap/delete_handler_test.cpp /^TEST_F(TestDeleteHandler, FilterDataConditions) {$/;" f namespace:doris +TEST_F test/olap/delete_handler_test.cpp /^TEST_F(TestDeleteHandler, FilterDataSubconditions) {$/;" f namespace:doris +TEST_F test/olap/delete_handler_test.cpp /^TEST_F(TestDeleteHandler, FilterDataVersion) {$/;" f namespace:doris +TEST_F test/olap/delete_handler_test.cpp /^TEST_F(TestDeleteHandler, InitSuccess) {$/;" f namespace:doris +TEST_F test/olap/delta_writer_test.cpp /^TEST_F(TestDeltaWriter, open) {$/;" f namespace:doris +TEST_F test/olap/delta_writer_test.cpp /^TEST_F(TestDeltaWriter, write) {$/;" f namespace:doris +TEST_F test/olap/file_helper_test.cpp /^TEST_F(FileHandlerTest, TestWrite) {$/;" f namespace:doris +TEST_F test/olap/file_utils_test.cpp /^TEST_F(FileUtilsTest, TestCopyFile) {$/;" f namespace:doris +TEST_F test/olap/file_utils_test.cpp /^TEST_F(FileUtilsTest, TestCreateDir) {$/;" f namespace:doris +TEST_F test/olap/file_utils_test.cpp /^TEST_F(FileUtilsTest, TestListDirsFiles) {$/;" f namespace:doris +TEST_F test/olap/file_utils_test.cpp /^TEST_F(FileUtilsTest, TestRemove) {$/;" f namespace:doris +TEST_F test/olap/fs/file_block_manager_test.cpp /^TEST_F(FileBlockManagerTest, NormalTest) {$/;" f namespace:doris +TEST_F test/olap/hll_test.cpp /^TEST_F(TestHll, InvalidPtr) {$/;" f namespace:doris +TEST_F test/olap/hll_test.cpp /^TEST_F(TestHll, Normal) {$/;" f namespace:doris +TEST_F test/olap/in_list_predicate_test.cpp /^TEST_F(TestInListPredicate, CHAR_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/in_list_predicate_test.cpp /^TEST_F(TestInListPredicate, DATETIME_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/in_list_predicate_test.cpp /^TEST_F(TestInListPredicate, DATE_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/in_list_predicate_test.cpp /^TEST_F(TestInListPredicate, DECIMAL_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/in_list_predicate_test.cpp /^TEST_F(TestInListPredicate, DOUBLE_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/in_list_predicate_test.cpp /^TEST_F(TestInListPredicate, FLOAT_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/in_list_predicate_test.cpp /^TEST_F(TestInListPredicate, VARCHAR_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/key_coder_test.cpp /^TEST_F(KeyCoderTest, test_char) {$/;" f namespace:doris +TEST_F test/olap/key_coder_test.cpp /^TEST_F(KeyCoderTest, test_date) {$/;" f namespace:doris +TEST_F test/olap/key_coder_test.cpp /^TEST_F(KeyCoderTest, test_decimal) {$/;" f namespace:doris +TEST_F test/olap/key_coder_test.cpp /^TEST_F(KeyCoderTest, test_int) {$/;" f namespace:doris +TEST_F test/olap/key_coder_test.cpp /^TEST_F(KeyCoderTest, test_varchar) {$/;" f namespace:doris +TEST_F test/olap/lru_cache_test.cpp /^TEST_F(CacheTest, EntriesArePinned) {$/;" f namespace:doris +TEST_F test/olap/lru_cache_test.cpp /^TEST_F(CacheTest, Erase) {$/;" f namespace:doris +TEST_F test/olap/lru_cache_test.cpp /^TEST_F(CacheTest, EvictionPolicy) {$/;" f namespace:doris +TEST_F test/olap/lru_cache_test.cpp /^TEST_F(CacheTest, EvictionPolicyWithDurable) {$/;" f namespace:doris +TEST_F test/olap/lru_cache_test.cpp /^TEST_F(CacheTest, HeavyEntries) {$/;" f namespace:doris +TEST_F test/olap/lru_cache_test.cpp /^TEST_F(CacheTest, HitAndMiss) {$/;" f namespace:doris +TEST_F test/olap/lru_cache_test.cpp /^TEST_F(CacheTest, NewId) {$/;" f namespace:doris +TEST_F test/olap/lru_cache_test.cpp /^TEST_F(CacheTest, Usage) {$/;" f namespace:doris +TEST_F test/olap/memtable_flush_executor_test.cpp /^TEST_F(TestMemTableFlushExecutor, create_flush_handler) {$/;" f namespace:doris +TEST_F test/olap/null_predicate_test.cpp /^TEST_F(TestNullPredicate, DATETIME_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/null_predicate_test.cpp /^TEST_F(TestNullPredicate, DATE_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/null_predicate_test.cpp /^TEST_F(TestNullPredicate, DECIMAL_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/null_predicate_test.cpp /^TEST_F(TestNullPredicate, DOUBLE_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/null_predicate_test.cpp /^TEST_F(TestNullPredicate, FLOAT_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/null_predicate_test.cpp /^TEST_F(TestNullPredicate, STRING_COLUMN) {$/;" f namespace:doris +TEST_F test/olap/olap_meta_test.cpp /^TEST_F(OlapMetaTest, TestGetRootPath) {$/;" f namespace:doris +TEST_F test/olap/olap_meta_test.cpp /^TEST_F(OlapMetaTest, TestIterate) {$/;" f namespace:doris +TEST_F test/olap/olap_meta_test.cpp /^TEST_F(OlapMetaTest, TestPutAndGet) {$/;" f namespace:doris +TEST_F test/olap/olap_meta_test.cpp /^TEST_F(OlapMetaTest, TestRemove) {$/;" f namespace:doris +TEST_F test/olap/olap_reader_test.cpp /^TEST_F(TestOLAPReaderColumn, next_tuple) {$/;" f namespace:doris +TEST_F test/olap/olap_reader_test.cpp /^TEST_F(TestOLAPReaderColumn, next_tuple_with_aggregation) {$/;" f namespace:doris +TEST_F test/olap/olap_reader_test.cpp /^TEST_F(TestOLAPReaderColumn, next_tuple_without_aggregation) {$/;" f namespace:doris +TEST_F test/olap/olap_reader_test.cpp /^TEST_F(TestOLAPReaderColumnDeleteCondition, next_tuple) {$/;" f namespace:doris +TEST_F test/olap/olap_reader_test.cpp /^TEST_F(TestOLAPReaderRow, next_tuple_with_aggregation) {$/;" f namespace:doris +TEST_F test/olap/olap_reader_test.cpp /^TEST_F(TestOLAPReaderRow, next_tuple_with_key_range) {$/;" f namespace:doris +TEST_F test/olap/olap_reader_test.cpp /^TEST_F(TestOLAPReaderRow, next_tuple_with_where_condition) {$/;" f namespace:doris +TEST_F test/olap/olap_reader_test.cpp /^TEST_F(TestOLAPReaderRow, next_tuple_without_aggregation) {$/;" f namespace:doris +TEST_F test/olap/olap_snapshot_converter_test.cpp /^TEST_F(OlapSnapshotConverterTest, ToNewAndToOldSnapshot) {$/;" f namespace:doris +TEST_F test/olap/options_test.cpp /^TEST_F(OptionsTest, parse_root_path) {$/;" f namespace:doris +TEST_F test/olap/row_block_test.cpp /^TEST_F(TestRowBlock, clear) {$/;" f namespace:doris +TEST_F test/olap/row_block_test.cpp /^TEST_F(TestRowBlock, compress_failed) {$/;" f namespace:doris +TEST_F test/olap/row_block_test.cpp /^TEST_F(TestRowBlock, decompress_failed) {$/;" f namespace:doris +TEST_F test/olap/row_block_test.cpp /^TEST_F(TestRowBlock, init) {$/;" f namespace:doris +TEST_F test/olap/row_block_test.cpp /^TEST_F(TestRowBlock, pos_limit) {$/;" f namespace:doris +TEST_F test/olap/row_block_test.cpp /^TEST_F(TestRowBlock, write_and_read) {$/;" f namespace:doris +TEST_F test/olap/row_block_test.cpp /^TEST_F(TestRowBlock, write_and_read_without_nullbyte) {$/;" f namespace:doris +TEST_F test/olap/row_block_v2_test.cpp /^TEST_F(TestRowBlockV2, test_convert) {$/;" f namespace:doris +TEST_F test/olap/row_cursor_test.cpp /^TEST_F(TestRowCursor, AggregateWithNull) {$/;" f namespace:doris +TEST_F test/olap/row_cursor_test.cpp /^TEST_F(TestRowCursor, AggregateWithoutNull) {$/;" f namespace:doris +TEST_F test/olap/row_cursor_test.cpp /^TEST_F(TestRowCursor, EqualAndCompare) {$/;" f namespace:doris +TEST_F test/olap/row_cursor_test.cpp /^TEST_F(TestRowCursor, FullKeyCmp) {$/;" f namespace:doris +TEST_F test/olap/row_cursor_test.cpp /^TEST_F(TestRowCursor, IndexCmp) {$/;" f namespace:doris +TEST_F test/olap/row_cursor_test.cpp /^TEST_F(TestRowCursor, InitRowCursor) {$/;" f namespace:doris +TEST_F test/olap/row_cursor_test.cpp /^TEST_F(TestRowCursor, InitRowCursorWithColIds) {$/;" f namespace:doris +TEST_F test/olap/row_cursor_test.cpp /^TEST_F(TestRowCursor, InitRowCursorWithColumnCount) {$/;" f namespace:doris +TEST_F test/olap/row_cursor_test.cpp /^TEST_F(TestRowCursor, InitRowCursorWithScanKey) {$/;" f namespace:doris +TEST_F test/olap/rowset/alpha_rowset_test.cpp /^TEST_F(AlphaRowsetTest, TestAlphaRowsetReader) {$/;" f namespace:doris +TEST_F test/olap/rowset/alpha_rowset_test.cpp /^TEST_F(AlphaRowsetTest, TestAlphaRowsetWriter) {$/;" f namespace:doris +TEST_F test/olap/rowset/alpha_rowset_test.cpp /^TEST_F(AlphaRowsetTest, TestRowCursorWithOrdinal) {$/;" f namespace:doris +TEST_F test/olap/rowset/beta_rowset_test.cpp /^TEST_F(BetaRowsetTest, BasicFunctionTest) {$/;" f namespace:doris +TEST_F test/olap/rowset/rowset_converter_test.cpp /^TEST_F(RowsetConverterTest, TestConvertAlphaRowsetToBeta) {$/;" f namespace:doris +TEST_F test/olap/rowset/rowset_converter_test.cpp /^TEST_F(RowsetConverterTest, TestConvertBetaRowsetToAlpha) {$/;" f namespace:doris +TEST_F test/olap/rowset/rowset_meta_manager_test.cpp /^TEST_F(RowsetMetaManagerTest, TestLoad) {$/;" f namespace:doris +TEST_F test/olap/rowset/rowset_meta_manager_test.cpp /^TEST_F(RowsetMetaManagerTest, TestSaveAndGetAndRemove) {$/;" f namespace:doris +TEST_F test/olap/rowset/rowset_meta_test.cpp /^TEST_F(RowsetMetaTest, TestAlphaRowsetMeta) {$/;" f namespace:doris +TEST_F test/olap/rowset/rowset_meta_test.cpp /^TEST_F(RowsetMetaTest, TestAlphaRowsetMetaAdd) {$/;" f namespace:doris +TEST_F test/olap/rowset/rowset_meta_test.cpp /^TEST_F(RowsetMetaTest, TestAlphaRowsetMetaClear) {$/;" f namespace:doris +TEST_F test/olap/rowset/rowset_meta_test.cpp /^TEST_F(RowsetMetaTest, TestInit) {$/;" f namespace:doris +TEST_F test/olap/rowset/rowset_meta_test.cpp /^TEST_F(RowsetMetaTest, TestInitWithInvalidData) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/binary_dict_page_test.cpp /^TEST_F(BinaryDictPageTest, TestBySmallDataSize) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/binary_dict_page_test.cpp /^TEST_F(BinaryDictPageTest, TestEncodingRatio) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/binary_plain_page_test.cpp /^TEST_F(BinaryPlainPageTest, TestBinaryPlainPageBuilderSeekByValueSmallPage) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/binary_prefix_page_test.cpp /^TEST_F(BinaryPrefixPageTest, TestEncodeAndDecode) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/binary_prefix_page_test.cpp /^TEST_F(BinaryPrefixPageTest, TestEncodeAndDecode2) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bitmap_index_test.cpp /^TEST_F(BitmapIndexTest, test_invert) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bitmap_index_test.cpp /^TEST_F(BitmapIndexTest, test_invert_2) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bitmap_index_test.cpp /^TEST_F(BitmapIndexTest, test_multi_pages) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bitmap_index_test.cpp /^TEST_F(BitmapIndexTest, test_null) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleDecimal12BlockEncoderSeekValue) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleDoubleBlockEncoderEqual) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleDoubleBlockEncoderRandom) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleDoubleBlockEncoderSeekValue) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleDoubleBlockEncoderSequence) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleFloatBlockEncoderRandom) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleFloatBlockEncoderSeekValue) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleInt32BlockEncoderEqual) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleInt32BlockEncoderMaxNumberEqual) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleInt32BlockEncoderMaxNumberSequence) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleInt32BlockEncoderRandom) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleInt32BlockEncoderSequence) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^TEST_F(BitShufflePageTest, TestBitShuffleInt64BlockEncoderRandom) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/block_bloom_filter_test.cpp /^TEST_F(BlockBloomFilterTest, Normal) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/block_bloom_filter_test.cpp /^TEST_F(BlockBloomFilterTest, SP) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/block_bloom_filter_test.cpp /^TEST_F(BlockBloomFilterTest, slice) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^TEST_F(BloomFilterIndexReaderWriterTest, test_bigint) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^TEST_F(BloomFilterIndexReaderWriterTest, test_char) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^TEST_F(BloomFilterIndexReaderWriterTest, test_date) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^TEST_F(BloomFilterIndexReaderWriterTest, test_datetime) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^TEST_F(BloomFilterIndexReaderWriterTest, test_decimal) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^TEST_F(BloomFilterIndexReaderWriterTest, test_int) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^TEST_F(BloomFilterIndexReaderWriterTest, test_largeint) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^TEST_F(BloomFilterIndexReaderWriterTest, test_varchar_type) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_page_test.cpp /^TEST_F(BloomFilterPageTest, TestBigIntFieldBloomFilterPage) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_page_test.cpp /^TEST_F(BloomFilterPageTest, TestCharFieldBloomFilterPage) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_page_test.cpp /^TEST_F(BloomFilterPageTest, TestIntFieldBloomFilterPage) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/bloom_filter_page_test.cpp /^TEST_F(BloomFilterPageTest, TestVarcharFieldBloomFilterPage) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^TEST_F(ColumnReaderWriterTest, test_default_value) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^TEST_F(ColumnReaderWriterTest, test_nullable) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^TEST_F(ColumnReaderWriterTest, test_types) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/encoding_info_test.cpp /^TEST_F(EncodingInfoTest, no_encoding) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/encoding_info_test.cpp /^TEST_F(EncodingInfoTest, normal) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestFindBitsOfInt) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestFirstLastValue) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestInt128BlockEncoderMinMax) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestInt128BlockEncoderSequence) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestInt24BlockEncoderMinMax) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestInt24BlockEncoderSequence) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestInt32BlockEncoderEqual) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestInt32BlockEncoderRandom) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestInt32BlockEncoderSequence) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestInt32NormalBlockEncoderSize) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestInt32SequenceBlockEncoderSize) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^TEST_F(FrameOfReferencePageTest, TestInt64BlockEncoderSequence) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/ordinal_page_index_test.cpp /^TEST_F(OrdinalPageIndexTest, normal) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/ordinal_page_index_test.cpp /^TEST_F(OrdinalPageIndexTest, one_data_page) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestBoolPlainPageSeekValue) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestDoublePageEncoderEqual) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestDoublePageEncoderRandom) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestDoublePageEncoderSequence) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestInt32PageEncoderSequence) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestInt32PlainPageRandom) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestInt32PlainPageSeekValue) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestInt64PlainPageRandom) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestInt64PlainPageSeekValue) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestPlainFloatBlockEncoderRandom) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/plain_page_test.cpp /^TEST_F(PlainPageTest, TestPlainInt32PageEncoderEqual) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/rle_page_test.cpp /^TEST_F(RlePageTest, TestRleBoolBlockEncoderRandom) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/rle_page_test.cpp /^TEST_F(RlePageTest, TestRleBoolBlockEncoderSize) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/rle_page_test.cpp /^TEST_F(RlePageTest, TestRleInt32BlockEncoderEqual) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/rle_page_test.cpp /^TEST_F(RlePageTest, TestRleInt32BlockEncoderRandom) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/rle_page_test.cpp /^TEST_F(RlePageTest, TestRleInt32BlockEncoderSequence) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/rle_page_test.cpp /^TEST_F(RlePageTest, TestRleInt32BlockEncoderSize) {$/;" f namespace:doris +TEST_F test/olap/rowset/segment_v2/row_ranges_test.cpp /^TEST_F(RowRangesTest, TestRange) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/row_ranges_test.cpp /^TEST_F(RowRangesTest, TestRangesToRoaring) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/row_ranges_test.cpp /^TEST_F(RowRangesTest, TestRowRanges) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/segment_test.cpp /^TEST_F(SegmentReaderWriterTest, LazyMaterialization) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/segment_test.cpp /^TEST_F(SegmentReaderWriterTest, TestBitmapPredicate) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/segment_test.cpp /^TEST_F(SegmentReaderWriterTest, TestBloomFilterIndexUniqueModel) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/segment_test.cpp /^TEST_F(SegmentReaderWriterTest, TestDefaultValueColumn) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/segment_test.cpp /^TEST_F(SegmentReaderWriterTest, TestIndex) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/segment_test.cpp /^TEST_F(SegmentReaderWriterTest, TestStringDict) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/segment_test.cpp /^TEST_F(SegmentReaderWriterTest, estimate_segment_size) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/segment_test.cpp /^TEST_F(SegmentReaderWriterTest, normal) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/zone_map_index_test.cpp /^TEST_F(ColumnZoneMapTest, NormalTestCharPage) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/zone_map_index_test.cpp /^TEST_F(ColumnZoneMapTest, NormalTestIntPage) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/segment_v2/zone_map_index_test.cpp /^TEST_F(ColumnZoneMapTest, NormalTestVarcharPage) {$/;" f namespace:doris::segment_v2 +TEST_F test/olap/rowset/unique_rowset_id_generator_test.cpp /^TEST_F(UniqueRowsetIdGeneratorTest, GenerateIdTest) {$/;" f namespace:doris +TEST_F test/olap/rowset/unique_rowset_id_generator_test.cpp /^TEST_F(UniqueRowsetIdGeneratorTest, RowsetIdFormatTest) {$/;" f namespace:doris +TEST_F test/olap/run_length_byte_test.cpp /^TEST_F(TestRunLengthByte, ReadWriteMultiBytes) {$/;" f namespace:doris +TEST_F test/olap/run_length_byte_test.cpp /^TEST_F(TestRunLengthByte, ReadWriteOneByte) {$/;" f namespace:doris +TEST_F test/olap/run_length_byte_test.cpp /^TEST_F(TestRunLengthByte, ReadWriteSameBytes) {$/;" f namespace:doris +TEST_F test/olap/run_length_byte_test.cpp /^TEST_F(TestRunLengthByte, Seek) {$/;" f namespace:doris +TEST_F test/olap/run_length_byte_test.cpp /^TEST_F(TestRunLengthByte, Skip) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, DirectEncoding) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, DirectEncodingForDeltaOverflows1) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, DirectEncodingForDeltaOverflows2) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, PatchedBaseEncoding1) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, PatchedBaseEncoding2) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, PatchedBaseEncoding3) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, PatchedBaseEncoding4) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, PatchedBaseEncoding5) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, PatchedBaseEncoding6) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, ReadWriteMultiInteger) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, ReadWriteOneInteger) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, ReadWriteOneInteger2) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, ShortRepeatEncoding) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, seek) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthSignInteger, skip) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, DirectEncoding) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, DirectEncoding2) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, PatchedBaseEncoding1) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, PatchedBaseEncoding2) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, ReadWriteMassInteger) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, ReadWriteMultiInteger) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, ReadWriteOneInteger) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, ShortRepeatEncoding) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, ShortRepeatEncoding2) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, ShortRepeatEncoding3) { $/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, seek) {$/;" f namespace:doris +TEST_F test/olap/run_length_integer_test.cpp /^TEST_F(TestRunLengthUnsignInteger, skip) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertBigIntToVarchar) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertDateToDatetime) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertDatetimeToDate) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertDecimalToVarchar) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertDoubleToVarchar) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertFloatToDouble) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertFloatToVarchar) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertIntToDate) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertIntToVarchar) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertLargeIntToVarchar) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertSmallIntToVarchar) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertTinyIntToVarchar) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertVarcharToBigInt) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertVarcharToDate) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertVarcharToDouble) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertVarcharToFloat) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertVarcharToInt) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertVarcharToLargeInt) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertVarcharToSmallInt) {$/;" f namespace:doris +TEST_F test/olap/schema_change_test.cpp /^TEST_F(TestColumn, ConvertVarcharToTinyInt) {$/;" f namespace:doris +TEST_F test/olap/selection_vector_test.cpp /^TEST_F(SelectionVectorTest, Normal) {$/;" f namespace:doris +TEST_F test/olap/serialize_test.cpp /^TEST_F(SerializeTest, decode_bit_width) {$/;" f namespace:doris::ser +TEST_F test/olap/serialize_test.cpp /^TEST_F(SerializeTest, encode_bit_width) {$/;" f namespace:doris::ser +TEST_F test/olap/serialize_test.cpp /^TEST_F(SerializeTest, find_closet_num_bits) {$/;" f namespace:doris::ser +TEST_F test/olap/serialize_test.cpp /^TEST_F(SerializeTest, get_closet_fixed_bits) {$/;" f namespace:doris::ser +TEST_F test/olap/serialize_test.cpp /^TEST_F(SerializeTest, new_percentile_bits) {$/;" f namespace:doris::ser +TEST_F test/olap/serialize_test.cpp /^TEST_F(SerializeTest, percentile_bits) {$/;" f namespace:doris::ser +TEST_F test/olap/short_key_index_test.cpp /^TEST_F(ShortKeyIndexTest, buider) {$/;" f namespace:doris +TEST_F test/olap/short_key_index_test.cpp /^TEST_F(ShortKeyIndexTest, enocde) {$/;" f namespace:doris +TEST_F test/olap/skiplist_test.cpp /^TEST_F(SkipTest, Concurrent1) { run_concurrent(1); }$/;" f namespace:doris +TEST_F test/olap/skiplist_test.cpp /^TEST_F(SkipTest, Concurrent2) { run_concurrent(2); }$/;" f namespace:doris +TEST_F test/olap/skiplist_test.cpp /^TEST_F(SkipTest, Concurrent3) { run_concurrent(3); }$/;" f namespace:doris +TEST_F test/olap/skiplist_test.cpp /^TEST_F(SkipTest, Concurrent4) { run_concurrent(4); }$/;" f namespace:doris +TEST_F test/olap/skiplist_test.cpp /^TEST_F(SkipTest, Concurrent5) { run_concurrent(5); }$/;" f namespace:doris +TEST_F test/olap/skiplist_test.cpp /^TEST_F(SkipTest, ConcurrentWithoutThreads) {$/;" f namespace:doris +TEST_F test/olap/skiplist_test.cpp /^TEST_F(SkipTest, Empty) {$/;" f namespace:doris +TEST_F test/olap/skiplist_test.cpp /^TEST_F(SkipTest, InsertAndLookup) {$/;" f namespace:doris +TEST_F test/olap/skiplist_test.cpp /^TEST_F(SkipTest, InsertWithHintNoneDupModel) {$/;" f namespace:doris +TEST_F test/olap/stream_index_test.cpp /^TEST_F(TestStreamIndex, index_write) {$/;" f namespace:doris +TEST_F test/olap/stream_index_test.cpp /^TEST_F(TestStreamIndex, remove_written_position) {$/;" f namespace:doris +TEST_F test/olap/stream_index_test.cpp /^TEST_F(TestStreamIndex, statistic) {$/;" f namespace:doris +TEST_F test/olap/stream_index_test.cpp /^TEST_F(TestStreamIndex, test_statistic) {$/;" f namespace:doris +TEST_F test/olap/tablet_meta_manager_test.cpp /^TEST_F(TabletMetaManagerTest, TestLoad) {$/;" f namespace:doris +TEST_F test/olap/tablet_meta_manager_test.cpp /^TEST_F(TabletMetaManagerTest, TestSaveAndGetAndRemove) {$/;" f namespace:doris +TEST_F test/olap/tablet_mgr_test.cpp /^TEST_F(TabletMgrTest, CreateTablet) {$/;" f namespace:doris +TEST_F test/olap/tablet_mgr_test.cpp /^TEST_F(TabletMgrTest, DropTablet) {$/;" f namespace:doris +TEST_F test/olap/tablet_mgr_test.cpp /^TEST_F(TabletMgrTest, GetRowsetId) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, CommitTxnTwiceWithDiffRowsetId) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, CommitTxnTwiceWithSameRowsetId) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, CommitTxnWithNoPrepare) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, CommitTxnWithPrepare) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, DeleteCommittedTxn) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, DeletePreparedTxn) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, PrepareNewTxn) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, PrepareNewTxnTwice) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, PublishNotExistedTxn) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, PublishVersionSuccessful) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, RollbackCommittedTxn) {$/;" f namespace:doris +TEST_F test/olap/txn_manager_test.cpp /^TEST_F(TxnManagerTest, RollbackNotCommittedTxn) {$/;" f namespace:doris +TEST_F test/olap/vectorized_olap_reader_test.cpp /^TEST_F(TestVectorizedOLAPReader, binary_predicate) {$/;" f namespace:doris +TEST_F test/olap/vectorized_olap_reader_test.cpp /^TEST_F(TestVectorizedOLAPReader, column_test) {$/;" f namespace:doris +TEST_F test/olap/vectorized_olap_reader_test.cpp /^TEST_F(TestVectorizedOLAPReader, empty_result) {$/;" f namespace:doris +TEST_F test/olap/vectorized_olap_reader_test.cpp /^TEST_F(TestVectorizedOLAPReader, in_predicate) {$/;" f namespace:doris +TEST_F test/runtime/buffer_control_block_test.cpp /^TEST_F(BufferControlBlockTest, add_one_get_one) {$/;" f namespace:doris +TEST_F test/runtime/buffer_control_block_test.cpp /^TEST_F(BufferControlBlockTest, add_then_cancel) {$/;" f namespace:doris +TEST_F test/runtime/buffer_control_block_test.cpp /^TEST_F(BufferControlBlockTest, get_add_after_cancel) {$/;" f namespace:doris +TEST_F test/runtime/buffer_control_block_test.cpp /^TEST_F(BufferControlBlockTest, get_one_after_close) {$/;" f namespace:doris +TEST_F test/runtime/buffer_control_block_test.cpp /^TEST_F(BufferControlBlockTest, get_then_add) {$/;" f namespace:doris +TEST_F test/runtime/buffer_control_block_test.cpp /^TEST_F(BufferControlBlockTest, get_then_cancel) {$/;" f namespace:doris +TEST_F test/runtime/buffer_control_block_test.cpp /^TEST_F(BufferControlBlockTest, get_then_close) {$/;" f namespace:doris +TEST_F test/runtime/buffer_control_block_test.cpp /^TEST_F(BufferControlBlockTest, init_normal) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, AllocationErrorHandling) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, ClientOversubscription) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, Close) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, CreateDestroyMulti) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, DISABLED_Multi8Random_plain) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, DISABLED_WriteErrorBlacklist) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, DeleteSingleBlocks) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, Deletion) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, Eviction) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, GetNewBlockSmallBlocks) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, Multi2Random_plain) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, Multi4Random_plain) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, MultipleClients) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, MultipleClientsExtraBuffers) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, NoDirsAllocationError) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, Pin) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, SingleRandom_plain) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, TmpFileAllocateError) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, WriteError) {$/;" f namespace:doris +TEST_F test/runtime/buffered_block_mgr2_test.cpp /^TEST_F(BufferedBlockMgrTest, get_new_block) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream2_test.cpp /^TEST_F(MultiNullableTupleStreamTest, MultiNullableTupleManyBufferSpill) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream2_test.cpp /^TEST_F(MultiNullableTupleStreamTest, MultiNullableTupleOneBufferSpill) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream2_test.cpp /^TEST_F(MultiTupleStreamTest, MultiTupleManyBufferSpill) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream2_test.cpp /^TEST_F(MultiTupleStreamTest, MultiTupleOneBufferSpill) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream2_test.cpp /^TEST_F(SimpleNullStreamTest, Basic) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream2_test.cpp /^TEST_F(SimpleTupleStreamTest, Basic) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream2_test.cpp /^TEST_F(SimpleTupleStreamTest, ManyBufferSpill) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream2_test.cpp /^TEST_F(SimpleTupleStreamTest, OneBufferSpill) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream2_test.cpp /^TEST_F(SimpleTupleStreamTest, SmallBuffers) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream2_test.cpp /^TEST_F(SimpleTupleStreamTest, UnpinPin) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream_test.cpp /^TEST_F(BufferedTupleStreamTest, addRow_bufferStream) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream_test.cpp /^TEST_F(BufferedTupleStreamTest, getNext_bufferStream) {$/;" f namespace:doris +TEST_F test/runtime/buffered_tuple_stream_test.cpp /^TEST_F(BufferedTupleStreamTest, init_bufferStream) {$/;" f namespace:doris +TEST_F test/runtime/data_spliter_test.cpp /^TEST_F(DataSplitTest, NoData) {$/;" f namespace:doris +TEST_F test/runtime/data_stream_test.cpp /^TEST_F(DataStreamTest, BasicTest) {$/;" f namespace:doris +TEST_F test/runtime/data_stream_test.cpp /^TEST_F(DataStreamTest, Cancel) {$/;" f namespace:doris +TEST_F test/runtime/data_stream_test.cpp /^TEST_F(DataStreamTest, UnknownSenderLargeResult) {$/;" f namespace:doris +TEST_F test/runtime/data_stream_test.cpp /^TEST_F(DataStreamTest, UnknownSenderSmallResult) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, acc) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, add_interval) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, calc_daynr) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, calc_form_daynr) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, check_date) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, check_range) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, construct_int64) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, day_of_year) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, equal) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, format_str) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, from_date_format_str) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, from_date_format_str_invalid) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, from_int_value) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, from_int_value_invalid) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, from_time_int) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, from_time_int_invalid) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, from_time_str) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, from_time_str_invalid) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, from_unixtime) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, local_time) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, min_max) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, operatro_minus) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, packed_time) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, random_convert) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, struct_size) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, to_int64) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, to_string) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, unix_timestamp) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, week) {$/;" f namespace:doris +TEST_F test/runtime/datetime_value_test.cpp /^TEST_F(DateTimeValueTest, weekday) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, add) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, compound_add) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, div) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, double_to_decimal) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, float_to_decimal) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, int_to_decimal) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, mul) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, negative_zero) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, round_minus) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, round_ops) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, round_to_int) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, string_to_decimal) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, sub) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, to_int_frac_value) {$/;" f namespace:doris +TEST_F test/runtime/decimal_value_test.cpp /^TEST_F(DecimalValueTest, unary_minus_operator) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, add) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, compound_add) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, div) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, double_to_decimal) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, float_to_decimal) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, int_to_decimal) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, mul) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, negative_zero) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, round_minus) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, round_ops) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, round_to_int) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, string_to_decimal) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, sub) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, to_int_frac_value) {$/;" f namespace:doris +TEST_F test/runtime/decimalv2_value_test.cpp /^TEST_F(DecimalV2ValueTest, unary_minus_operator) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, AddScanRangeTest) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, Buffers) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, InvalidWrite) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, MemTrackers) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, MultipleReader) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, MultipleReaderWriter) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, PartialRead) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, SingleReader) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, SingleReaderCancel) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, SingleWriter) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, SingleWriterCancel) {$/;" f namespace:doris +TEST_F test/runtime/disk_io_mgr_test.cpp /^TEST_F(DiskIoMgrTest, SyncReadTest) {$/;" f namespace:doris +TEST_F test/runtime/dpp_sink_internal_test.cpp /^TEST_F(DppSinkInternalTest, PartitionInfoNormal) {$/;" f namespace:doris +TEST_F test/runtime/dpp_sink_internal_test.cpp /^TEST_F(DppSinkInternalTest, RollupNormal) {$/;" f namespace:doris +TEST_F test/runtime/dpp_sink_internal_test.cpp /^TEST_F(DppSinkInternalTest, ValueSizeNotEq) {$/;" f namespace:doris +TEST_F test/runtime/dpp_sink_internal_test.cpp /^TEST_F(DppSinkInternalTest, ZeroBucket) {$/;" f namespace:doris +TEST_F test/runtime/dpp_sink_test.cpp /^TEST_F(DppSinkTest, NoData) {$/;" f namespace:doris +TEST_F test/runtime/dpp_sink_test.cpp /^TEST_F(DppSinkTest, WithData) {$/;" f namespace:doris +TEST_F test/runtime/dpp_writer_test.cpp /^TEST_F(DppWriterTest, normalCase) {$/;" f namespace:doris +TEST_F test/runtime/etl_job_mgr_test.cpp /^TEST_F(EtlJobMgrTest, CancelJob) {$/;" f namespace:doris +TEST_F test/runtime/etl_job_mgr_test.cpp /^TEST_F(EtlJobMgrTest, DuplicateCase) {$/;" f namespace:doris +TEST_F test/runtime/etl_job_mgr_test.cpp /^TEST_F(EtlJobMgrTest, FinishUnknowJob) {$/;" f namespace:doris +TEST_F test/runtime/etl_job_mgr_test.cpp /^TEST_F(EtlJobMgrTest, NormalCase) {$/;" f namespace:doris +TEST_F test/runtime/etl_job_mgr_test.cpp /^TEST_F(EtlJobMgrTest, RunAfterFail) {$/;" f namespace:doris +TEST_F test/runtime/etl_job_mgr_test.cpp /^TEST_F(EtlJobMgrTest, RunAfterSuccess) {$/;" f namespace:doris +TEST_F test/runtime/export_task_mgr_test.cpp /^TEST_F(ExportTaskMgrTest, CancelJob) {$/;" f namespace:doris +TEST_F test/runtime/export_task_mgr_test.cpp /^TEST_F(ExportTaskMgrTest, DuplicateCase) {$/;" f namespace:doris +TEST_F test/runtime/export_task_mgr_test.cpp /^TEST_F(ExportTaskMgrTest, FinishUnknowJob) {$/;" f namespace:doris +TEST_F test/runtime/export_task_mgr_test.cpp /^TEST_F(ExportTaskMgrTest, NormalCase) {$/;" f namespace:doris +TEST_F test/runtime/export_task_mgr_test.cpp /^TEST_F(ExportTaskMgrTest, RunAfterFail) {$/;" f namespace:doris +TEST_F test/runtime/export_task_mgr_test.cpp /^TEST_F(ExportTaskMgrTest, RunAfterSuccess) {$/;" f namespace:doris +TEST_F test/runtime/external_scan_context_mgr_test.cpp /^TEST_F(ExternalScanContextMgrTest, clear_context) {$/;" f namespace:doris +TEST_F test/runtime/external_scan_context_mgr_test.cpp /^TEST_F(ExternalScanContextMgrTest, create_normal) {$/;" f namespace:doris +TEST_F test/runtime/external_scan_context_mgr_test.cpp /^TEST_F(ExternalScanContextMgrTest, get_abnormal) {$/;" f namespace:doris +TEST_F test/runtime/external_scan_context_mgr_test.cpp /^TEST_F(ExternalScanContextMgrTest, get_normal) {$/;" f namespace:doris +TEST_F test/runtime/fragment_mgr_test.cpp /^TEST_F(FragmentMgrTest, AddNormal) {$/;" f namespace:doris +TEST_F test/runtime/fragment_mgr_test.cpp /^TEST_F(FragmentMgrTest, CancelNormal) {$/;" f namespace:doris +TEST_F test/runtime/fragment_mgr_test.cpp /^TEST_F(FragmentMgrTest, CancelWithoutAdd) {$/;" f namespace:doris +TEST_F test/runtime/fragment_mgr_test.cpp /^TEST_F(FragmentMgrTest, Normal) {$/;" f namespace:doris +TEST_F test/runtime/fragment_mgr_test.cpp /^TEST_F(FragmentMgrTest, PrepareFailed) {$/;" f namespace:doris +TEST_F test/runtime/heartbeat_flags_test.cpp /^TEST_F(HeartbeatFlagsTest, normal) {$/;" f namespace:doris +TEST_F test/runtime/kafka_consumer_pipe_test.cpp /^TEST_F(KafkaConsumerPipeTest, append_read) {$/;" f namespace:doris +TEST_F test/runtime/large_int_value_test.cpp /^TEST_F(LargeIntValueTest, largeint_to_string) {$/;" f namespace:doris +TEST_F test/runtime/large_int_value_test.cpp /^TEST_F(LargeIntValueTest, string_to_largeint) {$/;" f namespace:doris +TEST_F test/runtime/load_channel_mgr_test.cpp /^TEST_F(LoadChannelMgrTest, add_failed) {$/;" f namespace:doris +TEST_F test/runtime/load_channel_mgr_test.cpp /^TEST_F(LoadChannelMgrTest, cancel) {$/;" f namespace:doris +TEST_F test/runtime/load_channel_mgr_test.cpp /^TEST_F(LoadChannelMgrTest, check_builder) {$/;" f namespace:doris +TEST_F test/runtime/load_channel_mgr_test.cpp /^TEST_F(LoadChannelMgrTest, close_failed) {$/;" f namespace:doris +TEST_F test/runtime/load_channel_mgr_test.cpp /^TEST_F(LoadChannelMgrTest, duplicate_packet) {$/;" f namespace:doris +TEST_F test/runtime/load_channel_mgr_test.cpp /^TEST_F(LoadChannelMgrTest, normal) {$/;" f namespace:doris +TEST_F test/runtime/load_channel_mgr_test.cpp /^TEST_F(LoadChannelMgrTest, open_failed) {$/;" f namespace:doris +TEST_F test/runtime/load_channel_mgr_test.cpp /^TEST_F(LoadChannelMgrTest, unknown_tablet) {$/;" f namespace:doris +TEST_F test/runtime/memory_scratch_sink_test.cpp /^TEST_F(MemoryScratchSinkTest, work_flow_normal) {$/;" f namespace:doris +TEST_F test/runtime/mysql_table_writer_test.cpp /^TEST_F(MysqlTableWriterTest, NormalTest) {$/;" f namespace:doris +TEST_F test/runtime/qsorter_test.cpp /^TEST_F(QSorterTest, normalCase) {$/;" f namespace:doris +TEST_F test/runtime/raw_value_test.cpp /^TEST_F(RawValueTest, Compare) {$/;" f namespace:doris +TEST_F test/runtime/result_buffer_mgr_test.cpp /^TEST_F(ResultBufferMgrTest, cancel_no_block) {$/;" f namespace:doris +TEST_F test/runtime/result_buffer_mgr_test.cpp /^TEST_F(ResultBufferMgrTest, create_normal) {$/;" f namespace:doris +TEST_F test/runtime/result_buffer_mgr_test.cpp /^TEST_F(ResultBufferMgrTest, create_same_buffer) {$/;" f namespace:doris +TEST_F test/runtime/result_buffer_mgr_test.cpp /^TEST_F(ResultBufferMgrTest, fetch_data_no_block) {$/;" f namespace:doris +TEST_F test/runtime/result_buffer_mgr_test.cpp /^TEST_F(ResultBufferMgrTest, fetch_data_normal) {$/;" f namespace:doris +TEST_F test/runtime/result_buffer_mgr_test.cpp /^TEST_F(ResultBufferMgrTest, normal_cancel) {$/;" f namespace:doris +TEST_F test/runtime/result_queue_mgr_test.cpp /^TEST_F(ResultQueueMgrTest, cancel_no_block) {$/;" f namespace:doris +TEST_F test/runtime/result_queue_mgr_test.cpp /^TEST_F(ResultQueueMgrTest, create_normal) {$/;" f namespace:doris +TEST_F test/runtime/result_queue_mgr_test.cpp /^TEST_F(ResultQueueMgrTest, create_same_queue) {$/;" f namespace:doris +TEST_F test/runtime/result_queue_mgr_test.cpp /^TEST_F(ResultQueueMgrTest, fetch_result_end) {$/;" f namespace:doris +TEST_F test/runtime/result_queue_mgr_test.cpp /^TEST_F(ResultQueueMgrTest, fetch_result_normal) {$/;" f namespace:doris +TEST_F test/runtime/result_queue_mgr_test.cpp /^TEST_F(ResultQueueMgrTest, normal_cancel) {$/;" f namespace:doris +TEST_F test/runtime/result_sink_test.cpp /^TEST_F(ResultSinkTest, init_normal) {$/;" f namespace:doris +TEST_F test/runtime/result_writer_test.cpp /^TEST_F(ResultWriterTest, AppendRowBatch_empty) {$/;" f namespace:doris +TEST_F test/runtime/result_writer_test.cpp /^TEST_F(ResultWriterTest, AppendRowBatch_normal) {$/;" f namespace:doris +TEST_F test/runtime/result_writer_test.cpp /^TEST_F(ResultWriterTest, init_failed) {$/;" f namespace:doris +TEST_F test/runtime/result_writer_test.cpp /^TEST_F(ResultWriterTest, init_normal) {$/;" f namespace:doris +TEST_F test/runtime/routine_load_task_executor_test.cpp /^TEST_F(RoutineLoadTaskExecutorTest, exec_task) {$/;" f namespace:doris +TEST_F test/runtime/small_file_mgr_test.cpp /^TEST_F(SmallFileMgrTest, test_get_file) {$/;" f namespace:doris +TEST_F test/runtime/snapshot_loader_test.cpp /^TEST_F(SnapshotLoaderTest, NormalCase) {$/;" f namespace:doris +TEST_F test/runtime/sorter_test.cpp /^TEST_F(SorterTest, init_sort_exec_exprs) {$/;" f namespace:doris +TEST_F test/runtime/sorter_test.cpp /^TEST_F(SorterTest, prepare_sort_exec_exprs) {$/;" f namespace:doris +TEST_F test/runtime/sorter_test.cpp /^TEST_F(SorterTest, sorter_run_asc) {$/;" f namespace:doris +TEST_F test/runtime/sorter_test.cpp /^TEST_F(SorterTest, sorter_run_desc) {$/;" f namespace:doris +TEST_F test/runtime/sorter_test.cpp /^TEST_F(SorterTest, sorter_run_desc_with_quick_sort) {$/;" f namespace:doris +TEST_F test/runtime/stream_load_pipe_test.cpp /^TEST_F(StreamLoadPipeTest, append_buffer) {$/;" f namespace:doris +TEST_F test/runtime/stream_load_pipe_test.cpp /^TEST_F(StreamLoadPipeTest, append_bytes) {$/;" f namespace:doris +TEST_F test/runtime/stream_load_pipe_test.cpp /^TEST_F(StreamLoadPipeTest, append_bytes2) {$/;" f namespace:doris +TEST_F test/runtime/stream_load_pipe_test.cpp /^TEST_F(StreamLoadPipeTest, append_mix) {$/;" f namespace:doris +TEST_F test/runtime/stream_load_pipe_test.cpp /^TEST_F(StreamLoadPipeTest, cancel) {$/;" f namespace:doris +TEST_F test/runtime/stream_load_pipe_test.cpp /^TEST_F(StreamLoadPipeTest, close) {$/;" f namespace:doris +TEST_F test/runtime/tmp_file_mgr_test.cpp /^TEST_F(TmpFileMgrTest, TestAllocateFails) {$/;" f namespace:doris +TEST_F test/runtime/tmp_file_mgr_test.cpp /^TEST_F(TmpFileMgrTest, TestFileAllocation) {$/;" f namespace:doris +TEST_F test/runtime/tmp_file_mgr_test.cpp /^TEST_F(TmpFileMgrTest, TestMultiDirsPerDevice) {$/;" f namespace:doris +TEST_F test/runtime/tmp_file_mgr_test.cpp /^TEST_F(TmpFileMgrTest, TestOneDirPerDevice) {$/;" f namespace:doris +TEST_F test/runtime/tmp_file_mgr_test.cpp /^TEST_F(TmpFileMgrTest, TestReportError) {$/;" f namespace:doris +TEST_F test/runtime/user_function_cache_test.cpp /^TEST_F(UserFunctionCacheTest, bad_so) {$/;" f namespace:doris +TEST_F test/runtime/user_function_cache_test.cpp /^TEST_F(UserFunctionCacheTest, download_fail) {$/;" f namespace:doris +TEST_F test/runtime/user_function_cache_test.cpp /^TEST_F(UserFunctionCacheTest, download_normal) {$/;" f namespace:doris +TEST_F test/runtime/user_function_cache_test.cpp /^TEST_F(UserFunctionCacheTest, load_normal) {$/;" f namespace:doris +TEST_F test/runtime/user_function_cache_test.cpp /^TEST_F(UserFunctionCacheTest, md5_fail) {$/;" f namespace:doris +TEST_F test/runtime/user_function_cache_test.cpp /^TEST_F(UserFunctionCacheTest, process_symbol) {$/;" f namespace:doris +TEST_F test/util/aes_util_test.cpp /^TEST_F(AesUtilTest, aes_test_basic) {$/;" f namespace:doris +TEST_F test/util/aes_util_test.cpp /^TEST_F(AesUtilTest, aes_test_by_case) {$/;" f namespace:doris +TEST_F test/util/arrow/arrow_row_batch_test.cpp /^TEST_F(ArrowRowBatchTest, PrettyPrint) {$/;" f namespace:doris +TEST_F test/util/arrow/arrow_row_block_test.cpp /^TEST_F(ArrowRowBlockTest, Normal) {$/;" f namespace:doris +TEST_F test/util/arrow/arrow_work_flow_test.cpp /^TEST_F(ArrowWorkFlowTest, NormalUse) {$/;" f namespace:doris +TEST_F test/util/bitmap_test.cpp /^TEST_F(BitMapTest, iterator) {$/;" f namespace:doris +TEST_F test/util/bitmap_test.cpp /^TEST_F(BitMapTest, normal) {$/;" f namespace:doris +TEST_F test/util/block_compression_test.cpp /^TEST_F(BlockCompressionTest, multi) {$/;" f namespace:doris +TEST_F test/util/block_compression_test.cpp /^TEST_F(BlockCompressionTest, single) {$/;" f namespace:doris +TEST_F test/util/brpc_stub_cache_test.cpp /^TEST_F(BrpcStubCacheTest, invalid) {$/;" f namespace:doris +TEST_F test/util/brpc_stub_cache_test.cpp /^TEST_F(BrpcStubCacheTest, normal) {$/;" f namespace:doris +TEST_F test/util/byte_buffer_test2.cpp /^TEST_F(ByteBufferTest, normal) {$/;" f namespace:doris +TEST_F test/util/cgroup_util_test.cpp /^TEST_F(CGroupUtilTest, memlimit) {$/;" f namespace:doris +TEST_F test/util/coding_test.cpp /^TEST_F(CodingTest, fixed_le) {$/;" f namespace:doris +TEST_F test/util/coding_test.cpp /^TEST_F(CodingTest, put_varint) {$/;" f namespace:doris +TEST_F test/util/coding_test.cpp /^TEST_F(CodingTest, variant) {$/;" f namespace:doris +TEST_F test/util/coding_test.cpp /^TEST_F(CodingTest, variant_bigvalue) {$/;" f namespace:doris +TEST_F test/util/coding_test.cpp /^TEST_F(CodingTest, variant_fail) {$/;" f namespace:doris +TEST_F test/util/core_local_test.cpp /^TEST_F(CoreLocalTest, CoreDataAllocator) {$/;" f namespace:doris +TEST_F test/util/core_local_test.cpp /^TEST_F(CoreLocalTest, CoreLocalValue) {$/;" f namespace:doris +TEST_F test/util/core_local_test.cpp /^TEST_F(CoreLocalTest, CoreLocalValueController) {$/;" f namespace:doris +TEST_F test/util/core_local_test.cpp /^TEST_F(CoreLocalTest, CoreLocalValueNormal) {$/;" f namespace:doris +TEST_F test/util/counter_cond_variable_test.cpp /^TEST_F(CounterCondVariableTest, test) {$/;" f namespace:doris +TEST_F test/util/decompress_test.cpp /^TEST_F(DecompressorTest, Bzip) {$/;" f namespace:doris +TEST_F test/util/decompress_test.cpp /^TEST_F(DecompressorTest, Default) {$/;" f namespace:doris +TEST_F test/util/decompress_test.cpp /^TEST_F(DecompressorTest, Deflate) {$/;" f namespace:doris +TEST_F test/util/decompress_test.cpp /^TEST_F(DecompressorTest, Gzip) {$/;" f namespace:doris +TEST_F test/util/decompress_test.cpp /^TEST_F(DecompressorTest, Snappy) {$/;" f namespace:doris +TEST_F test/util/decompress_test.cpp /^TEST_F(DecompressorTest, SnappyBlocked) {$/;" f namespace:doris +TEST_F test/util/doris_metrics_test.cpp /^TEST_F(DorisMetricsTest, Normal) {$/;" f namespace:doris +TEST_F test/util/faststring_test.cpp /^TEST_F(FaststringTest, TestAppend_ExponentiallyExpand) {$/;" f namespace:doris +TEST_F test/util/faststring_test.cpp /^TEST_F(FaststringTest, TestAppend_Simple) {$/;" f namespace:doris +TEST_F test/util/faststring_test.cpp /^TEST_F(FaststringTest, TestPushBack) {$/;" f namespace:doris +TEST_F test/util/faststring_test.cpp /^TEST_F(FaststringTest, TestShrinkToFit_Empty) {$/;" f namespace:doris +TEST_F test/util/faststring_test.cpp /^TEST_F(FaststringTest, TestShrinkToFit_Random) {$/;" f namespace:doris +TEST_F test/util/faststring_test.cpp /^TEST_F(FaststringTest, TestShrinkToFit_SmallerThanInitialCapacity) {$/;" f namespace:doris +TEST_F test/util/file_cache_test.cpp /^TEST_F(FileCacheTest, normal) {$/;" f namespace:doris +TEST_F test/util/file_manager_test.cpp /^TEST_F(FileManagerTest, normal) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestBytesAlign) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestHalfFrame) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestInt64) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestOneFrame) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestOneMinValue) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestSkipHalfFrame) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestSkipOneFrame) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestSkipZero) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestTwoFrame) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestTwoHlafFrame) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestValueSeek) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestValueSeekSpecialCase) {$/;" f namespace:doris +TEST_F test/util/frame_of_reference_coding_test.cpp /^TEST_F(TestForCoding, TestZeroValue) {$/;" f namespace:doris +TEST_F test/util/json_util_test.cpp /^TEST_F(JsonUtilTest, normal_fail) {$/;" f namespace:doris +TEST_F test/util/json_util_test.cpp /^TEST_F(JsonUtilTest, normal_fail_str) {$/;" f namespace:doris +TEST_F test/util/json_util_test.cpp /^TEST_F(JsonUtilTest, success) {$/;" f namespace:doris +TEST_F test/util/lru_cache_util_test.cpp /^TEST_F(LruCacheTest, IteratorTest) {$/;" f namespace:doris +TEST_F test/util/lru_cache_util_test.cpp /^TEST_F(LruCacheTest, NormalTest) {$/;" f namespace:doris +TEST_F test/util/lru_cache_util_test.cpp /^TEST_F(LruCacheTest, OverSize) {$/;" f namespace:doris +TEST_F test/util/md5_test.cpp /^TEST_F(Md5Test, empty) {$/;" f namespace:doris +TEST_F test/util/md5_test.cpp /^TEST_F(Md5Test, normal) {$/;" f namespace:doris +TEST_F test/util/new_metrics_test.cpp /^TEST_F(MetricsTest, Counter) {$/;" f namespace:doris +TEST_F test/util/new_metrics_test.cpp /^TEST_F(MetricsTest, CounterPerf) {$/;" f namespace:doris +TEST_F test/util/new_metrics_test.cpp /^TEST_F(MetricsTest, Gauge) {$/;" f namespace:doris +TEST_F test/util/new_metrics_test.cpp /^TEST_F(MetricsTest, MetricCollector) {$/;" f namespace:doris +TEST_F test/util/new_metrics_test.cpp /^TEST_F(MetricsTest, MetricLabel) {$/;" f namespace:doris +TEST_F test/util/new_metrics_test.cpp /^TEST_F(MetricsTest, MetricLabels) {$/;" f namespace:doris +TEST_F test/util/new_metrics_test.cpp /^TEST_F(MetricsTest, MetricRegistry) {$/;" f namespace:doris +TEST_F test/util/new_metrics_test.cpp /^TEST_F(MetricsTest, MetricRegistry2) {$/;" f namespace:doris +TEST_F test/util/path_trie_test.cpp /^TEST_F(PathTrieTest, EmptyTest) {$/;" f namespace:doris +TEST_F test/util/path_trie_test.cpp /^TEST_F(PathTrieTest, ExactTest) {$/;" f namespace:doris +TEST_F test/util/path_trie_test.cpp /^TEST_F(PathTrieTest, MultiInsertTest) {$/;" f namespace:doris +TEST_F test/util/path_trie_test.cpp /^TEST_F(PathTrieTest, MultiPlayTest) {$/;" f namespace:doris +TEST_F test/util/path_trie_test.cpp /^TEST_F(PathTrieTest, MultiTemplateTest) {$/;" f namespace:doris +TEST_F test/util/path_trie_test.cpp /^TEST_F(PathTrieTest, SplitTest) {$/;" f namespace:doris +TEST_F test/util/path_trie_test.cpp /^TEST_F(PathTrieTest, TemplateTest) {$/;" f namespace:doris +TEST_F test/util/radix_sort_test.cpp /^TEST_F(RadixSortTest, TestDoubleSort) {$/;" f namespace:doris +TEST_F test/util/radix_sort_test.cpp /^TEST_F(RadixSortTest, TestFloatSort) {$/;" f namespace:doris +TEST_F test/util/radix_sort_test.cpp /^TEST_F(RadixSortTest, TestInt32Sort) {$/;" f namespace:doris +TEST_F test/util/radix_sort_test.cpp /^TEST_F(RadixSortTest, TestObjectSort) {$/;" f namespace:doris +TEST_F test/util/radix_sort_test.cpp /^TEST_F(RadixSortTest, TestUint32Sort) {$/;" f namespace:doris +TEST_F test/util/rle_encoding_test.cpp /^TEST_F(BitRle, AllSame) {$/;" f namespace:doris +TEST_F test/util/rle_encoding_test.cpp /^TEST_F(BitRle, Flush) {$/;" f namespace:doris +TEST_F test/util/rle_encoding_test.cpp /^TEST_F(BitRle, Random64Bit) {$/;" f namespace:doris +TEST_F test/util/rle_encoding_test.cpp /^TEST_F(BitRle, RandomBools) {$/;" f namespace:doris +TEST_F test/util/rle_encoding_test.cpp /^TEST_F(BitRle, RepeatedPattern) {$/;" f namespace:doris +TEST_F test/util/rle_encoding_test.cpp /^TEST_F(TestRle, TestBulkPut) {$/;" f namespace:doris +TEST_F test/util/rle_encoding_test.cpp /^TEST_F(TestRle, TestGetNextRun) {$/;" f namespace:doris +TEST_F test/util/rle_encoding_test.cpp /^TEST_F(TestRle, TestRoundTripRandomSequencesWithRuns) {$/;" f namespace:doris +TEST_F test/util/rle_encoding_test.cpp /^TEST_F(TestRle, TestSkip) {$/;" f namespace:doris +TEST_F test/util/string_util_test.cpp /^TEST_F(StringUtilTest, normal) {$/;" f namespace:doris +TEST_F test/util/system_metrics_test.cpp /^TEST_F(SystemMetricsTest, no_proc_file) {$/;" f namespace:doris +TEST_F test/util/system_metrics_test.cpp /^TEST_F(SystemMetricsTest, normal) {$/;" f namespace:doris +TEST_F test/util/tdigest_test.cpp /^TEST_F(TDigestTest, CrashAfterMerge) {$/;" f namespace:doris +TEST_F test/util/tdigest_test.cpp /^TEST_F(TDigestTest, EmptyDigest) {$/;" f namespace:doris +TEST_F test/util/tdigest_test.cpp /^TEST_F(TDigestTest, ExtremeQuantiles) {$/;" f namespace:doris +TEST_F test/util/tdigest_test.cpp /^TEST_F(TDigestTest, FewValues) {$/;" f namespace:doris +TEST_F test/util/tdigest_test.cpp /^TEST_F(TDigestTest, MergeTest) {$/;" f namespace:doris +TEST_F test/util/tdigest_test.cpp /^TEST_F(TDigestTest, Montonicity) {$/;" f namespace:doris +TEST_F test/util/tdigest_test.cpp /^TEST_F(TDigestTest, MoreThan2BValues) {$/;" f namespace:doris +TEST_F test/util/tdigest_test.cpp /^TEST_F(TDigestTest, SingleValue) {$/;" f namespace:doris +TEST_F test/util/tdigest_test.cpp /^TEST_F(TDigestTest, TestSorted) {$/;" f namespace:doris +TEST_F test/util/thread_test.cpp /^TEST_F(ThreadTest, TestDoubleJoinIsNoOp) {$/;" f namespace:doris +TEST_F test/util/thread_test.cpp /^TEST_F(ThreadTest, TestFailedJoin) {$/;" f namespace:doris +TEST_F test/util/thread_test.cpp /^TEST_F(ThreadTest, TestJoinAndWarn) {$/;" f namespace:doris +TEST_F test/util/thread_test.cpp /^TEST_F(ThreadTest, TestJoinOnSelf) {$/;" f namespace:doris +TEST_F test/util/thread_test.cpp /^TEST_F(ThreadTest, ThreadStartBenchmark) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestDeadlocks) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestFuzz) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestMaxQueueSize) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestNoTaskOpenClose) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestRace) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestSimpleTasks) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestSlowDestructor) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestThreadPoolWithNoMaxThreads) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestThreadPoolWithNoMinimum) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestTokenConcurrency) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestTokenSubmitsNonSequential) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestTokenSubmitsProcessedSerially) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestVariableSizeThreadPool) {$/;" f namespace:doris +TEST_F test/util/threadpool_test.cpp /^TEST_F(ThreadPoolTest, TestZeroQueueSize) {$/;" f namespace:doris +TEST_F test/util/types_test.cpp /^TEST_F(TypesTest, packed_int128) {$/;" f namespace:doris +TEST_F test/util/uid_util_test.cpp /^TEST_F(UidUtilTest, Hash) {$/;" f namespace:doris +TEST_F test/util/uid_util_test.cpp /^TEST_F(UidUtilTest, UniqueId) {$/;" f namespace:doris +TEST_F test/util/utf8_check_test.cpp /^TEST_F(Utf8CheckTest, abnormal) {$/;" f namespace:doris +TEST_F test/util/utf8_check_test.cpp /^TEST_F(Utf8CheckTest, empty) {$/;" f namespace:doris +TEST_F test/util/utf8_check_test.cpp /^TEST_F(Utf8CheckTest, naive) {$/;" f namespace:doris +TEST_F test/util/utf8_check_test.cpp /^TEST_F(Utf8CheckTest, normal) {$/;" f namespace:doris +TEST_FILE_BUF_SIZE src/olap/data_dir.h /^ static const size_t TEST_FILE_BUF_SIZE = 4096;$/;" m class:doris::DataDir +TEST_IN_LIST_PREDICATE test/olap/in_list_predicate_test.cpp 135;" d file: +TEST_IN_LIST_PREDICATE test/olap/null_predicate_test.cpp 103;" d file: +TEST_IN_LIST_PREDICATE_V2 test/olap/in_list_predicate_test.cpp 194;" d file: +TEST_LESS_PREDICATE test/olap/comparison_predicate_test.cpp 531;" d file: +TEST_P test/util/threadpool_test.cpp /^TEST_P(ThreadPoolTestTokenTypes, TestTokenShutdown) {$/;" f namespace:doris +TEST_P test/util/threadpool_test.cpp /^TEST_P(ThreadPoolTestTokenTypes, TestTokenSubmissionsAdhereToMaxQueueSize) {$/;" f namespace:doris +TEST_P test/util/threadpool_test.cpp /^TEST_P(ThreadPoolTestTokenTypes, TestTokenSubmitAndWait) {$/;" f namespace:doris +TEST_P test/util/threadpool_test.cpp /^TEST_P(ThreadPoolTestTokenTypes, TestTokenSubmitsProcessedConcurrently) {$/;" f namespace:doris +TEST_P test/util/threadpool_test.cpp /^TEST_P(ThreadPoolTestTokenTypes, TestTokenWaitForAll) {$/;" f namespace:doris +TEST_PREDICATE_DEFINITION test/olap/comparison_predicate_test.cpp 91;" d file: +THIRTEEN src/olap/serialize.h /^ THIRTEEN, FOURTEEN, FIFTEEN, SIXTEEN, SEVENTEEN, EIGHTEEN, NINETEEN,$/;" e enum:doris::ser::FixedBitSize +THIRTY src/olap/serialize.h /^ TWENTYEIGHT, THIRTY, THIRTYTWO, FORTY, FORTYEIGHT, FIFTYSIX, SIXTYFOUR$/;" e enum:doris::ser::FixedBitSize +THIRTYTWO src/olap/serialize.h /^ TWENTYEIGHT, THIRTY, THIRTYTWO, FORTY, FORTYEIGHT, FIFTYSIX, SIXTYFOUR$/;" e enum:doris::ser::FixedBitSize +THOUSAND src/util/debug_util.cpp 40;" d file: +THOUSAND src/util/pretty_printer.h /^ static const int64_t THOUSAND = 1000;$/;" m class:doris::PrettyPrinter +THREADED src/util/thrift_server.h /^ enum ServerType { THREAD_POOL = 0, THREADED, NON_BLOCKING };$/;" e enum:doris::ThriftServer::ServerType +THREADS_PER_FLASH_DISK src/runtime/disk_io_mgr.cc /^static const int THREADS_PER_FLASH_DISK = 8;$/;" m namespace:doris file: +THREADS_PER_ROTATIONAL_DISK src/runtime/disk_io_mgr.cc /^static const int THREADS_PER_ROTATIONAL_DISK = 1;$/;" m namespace:doris file: +THREAD_ATOMICOPS_H_ src/gutil/atomicops.h 52;" d +THREAD_INVOLUNTARY_CONTEXT_SWITCHES src/util/runtime_profile.cpp /^static const std::string THREAD_INVOLUNTARY_CONTEXT_SWITCHES = "InvoluntaryContextSwitches";$/;" m namespace:doris file: +THREAD_LOCAL output/udf/include/udf.h /^ THREAD_LOCAL,$/;" e enum:doris_udf::FunctionContext::FunctionStateScope +THREAD_LOCAL src/udf/udf.h /^ THREAD_LOCAL,$/;" e enum:doris_udf::FunctionContext::FunctionStateScope +THREAD_POOL src/util/thrift_server.h /^ enum ServerType { THREAD_POOL = 0, THREADED, NON_BLOCKING };$/;" e enum:doris::ThriftServer::ServerType +THREAD_SYS_TIME src/util/runtime_profile.cpp /^static const std::string THREAD_SYS_TIME = "SysTime";$/;" m namespace:doris file: +THREAD_TOTAL_TIME src/util/runtime_profile.cpp /^static const std::string THREAD_TOTAL_TIME = "TotalWallClockTime";$/;" m namespace:doris file: +THREAD_USER_TIME src/util/runtime_profile.cpp /^static const std::string THREAD_USER_TIME = "UserTime";$/;" m namespace:doris file: +THREAD_VOLUNTARY_CONTEXT_SWITCHES src/util/runtime_profile.cpp /^static const std::string THREAD_VOLUNTARY_CONTEXT_SWITCHES = "VoluntaryContextSwitches";$/;" m namespace:doris file: +THREE src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +THRIFT_ENUM_OUTPUT_FN src/util/debug_util.cpp 57;" d file: +THRIFT_ENUM_OUTPUT_FN_IMPL src/util/debug_util.cpp 46;" d file: +THRIFT_ENUM_PRINT_FN src/util/debug_util.cpp 60;" d file: +TIMEOUT_KEY src/http/action/mini_load.cpp /^const std::string TIMEOUT_KEY = "timeout";$/;" m namespace:doris file: +TIMEOUT_MS src/util/thrift_server.cpp /^ static const int TIMEOUT_MS = 2500;$/;" m class:doris::ThriftServer::ThriftServerEventProcessor file: +TIME_DATE src/runtime/datetime_value.h /^ TIME_DATE = 2,$/;" e enum:doris::TimeType +TIME_DATETIME src/runtime/datetime_value.h /^ TIME_DATETIME = 3$/;" e enum:doris::TimeType +TIME_DURATION_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t TIME_DURATION_SIZE = sizeof(boost::posix_time::time_duration);$/;" m class:doris::AggFnEvaluator +TIME_DURATION_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t TIME_DURATION_SIZE = sizeof(boost::posix_time::time_duration);$/;" m class:doris::NewAggFnEvaluator +TIME_MAX_HOUR src/runtime/datetime_value.h /^const int TIME_MAX_HOUR = 838;$/;" m namespace:doris +TIME_MAX_MINUTE src/runtime/datetime_value.h /^const int TIME_MAX_MINUTE = 59;$/;" m namespace:doris +TIME_MAX_SECOND src/runtime/datetime_value.h /^const int TIME_MAX_SECOND = 59;$/;" m namespace:doris +TIME_MAX_VALUE src/runtime/datetime_value.h /^const int TIME_MAX_VALUE = 10000 * TIME_MAX_HOUR + 100 * TIME_MAX_MINUTE + TIME_MAX_SECOND;$/;" m namespace:doris +TIME_MAX_VALUE_SECONDS src/runtime/datetime_value.h /^const int TIME_MAX_VALUE_SECONDS = 3600 * TIME_MAX_HOUR + 60 * TIME_MAX_MINUTE + TIME_MAX_SECOND;$/;" m namespace:doris +TIME_NS_PRECISION src/util/pretty_printer.h /^ static const int TIME_NS_PRECISION = 3;$/;" m class:doris::PrettyPrinter +TIME_TIME src/runtime/datetime_value.h /^ TIME_TIME = 1,$/;" e enum:doris::TimeType +TINYINT_SIZE src/exprs/agg_fn_evaluator.h /^ static const size_t TINYINT_SIZE = sizeof(int8_t);$/;" m class:doris::AggFnEvaluator +TINYINT_SIZE src/exprs/new_agg_fn_evaluator.h /^ static const size_t TINYINT_SIZE = sizeof(int8_t);$/;" m class:doris::NewAggFnEvaluator +TMP_FILE_MGR_ACTIVE_SCRATCH_DIRS src/runtime/tmp_file_mgr.cc /^const std::string TMP_FILE_MGR_ACTIVE_SCRATCH_DIRS = "tmp_file_mgr.active_scratch_dirs";$/;" m namespace:doris file: +TMP_FILE_MGR_ACTIVE_SCRATCH_DIRS_LIST src/runtime/tmp_file_mgr.cc /^const std::string TMP_FILE_MGR_ACTIVE_SCRATCH_DIRS_LIST = "tmp_file_mgr.active_scratch_dirs.list";$/;" m namespace:doris file: +TNetworkAddressPtrEquals src/util/container_util.hpp /^struct TNetworkAddressPtrEquals : public std::unary_function {$/;" s namespace:doris +TOKEN_PARAMETER src/http/download_action.cpp /^const std::string TOKEN_PARAMETER = "token";$/;" m namespace:doris file: +TOTAL_DATA_SIZE test/runtime/data_stream_test.cpp /^ static const int TOTAL_DATA_SIZE = 8 * 1024;$/;" m class:doris::DataStreamTest file: +TRAILER src/http/http_headers.cpp /^const char* HttpHeaders::TRAILER = "Trailer";$/;" m class:doris::HttpHeaders file: +TRAILER src/http/http_headers.h /^ static const char* TRAILER;$/;" m class:doris::HttpHeaders +TRANSFER_ENCODING src/http/http_headers.cpp /^const char* HttpHeaders::TRANSFER_ENCODING = "Transfer-Encoding";$/;" m class:doris::HttpHeaders file: +TRANSFER_ENCODING src/http/http_headers.h /^ static const char* TRANSFER_ENCODING;$/;" m class:doris::HttpHeaders +TRASH_PREFIX src/olap/olap_define.h /^static const std::string TRASH_PREFIX = "\/trash";$/;" m namespace:doris +TRUNCATE src/runtime/decimal_value.h /^ TRUNCATE = 5$/;" e enum:doris::DecimalRoundMode +TRYFREE src/util/minizip/unzip.c 115;" d file: +TRY_LOCK src/olap/utils.h 46;" d +TRY_LOCK src/util/mutex.h 25;" d +TSlotDescriptorBuilder src/runtime/descriptor_helper.h /^ TSlotDescriptorBuilder() { _slot_desc.isMaterialized = true; }$/;" f class:doris::TSlotDescriptorBuilder +TSlotDescriptorBuilder src/runtime/descriptor_helper.h /^class TSlotDescriptorBuilder {$/;" c namespace:doris +TTupleDescriptorBuilder src/runtime/descriptor_helper.h /^class TTupleDescriptorBuilder {$/;" c namespace:doris +TUPLE_ID_DST test/exec/parquet_scanner_test.cpp 69;" d file: +TUPLE_ID_SRC test/exec/parquet_scanner_test.cpp 70;" d file: +TWELVE src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +TWENTY src/olap/serialize.h /^ TWENTY, TWENTYONE, TWENTYTWO, TWENTYTHREE, TWENTYFOUR, TWENTYSIX,$/;" e enum:doris::ser::FixedBitSize +TWENTYEIGHT src/olap/serialize.h /^ TWENTYEIGHT, THIRTY, THIRTYTWO, FORTY, FORTYEIGHT, FIFTYSIX, SIXTYFOUR$/;" e enum:doris::ser::FixedBitSize +TWENTYFOUR src/olap/serialize.h /^ TWENTY, TWENTYONE, TWENTYTWO, TWENTYTHREE, TWENTYFOUR, TWENTYSIX,$/;" e enum:doris::ser::FixedBitSize +TWENTYONE src/olap/serialize.h /^ TWENTY, TWENTYONE, TWENTYTWO, TWENTYTHREE, TWENTYFOUR, TWENTYSIX,$/;" e enum:doris::ser::FixedBitSize +TWENTYSIX src/olap/serialize.h /^ TWENTY, TWENTYONE, TWENTYTWO, TWENTYTHREE, TWENTYFOUR, TWENTYSIX,$/;" e enum:doris::ser::FixedBitSize +TWENTYTHREE src/olap/serialize.h /^ TWENTY, TWENTYONE, TWENTYTWO, TWENTYTHREE, TWENTYFOUR, TWENTYSIX,$/;" e enum:doris::ser::FixedBitSize +TWENTYTWO src/olap/serialize.h /^ TWENTY, TWENTYONE, TWENTYTWO, TWENTYTHREE, TWENTYFOUR, TWENTYSIX,$/;" e enum:doris::ser::FixedBitSize +TWO src/olap/serialize.h /^ ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,$/;" e enum:doris::ser::FixedBitSize +TWO_LEVEL output/udf/include/uda_test_harness.h /^ TWO_LEVEL = 3,$/;" e enum:doris_udf::UdaExecutionMode +TWO_LEVEL src/udf/uda_test_harness.h /^ TWO_LEVEL = 3,$/;" e enum:doris_udf::UdaExecutionMode +TYPE_ARRAY src/runtime/primitive_type.h /^ TYPE_ARRAY, \/* 17 *\/$/;" e enum:doris::PrimitiveType +TYPE_BIGINT output/udf/include/udf.h /^ TYPE_BIGINT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_BIGINT src/runtime/primitive_type.h /^ TYPE_BIGINT, \/* 6 *\/$/;" e enum:doris::PrimitiveType +TYPE_BIGINT src/udf/udf.h /^ TYPE_BIGINT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_BINARY src/runtime/primitive_type.h /^ TYPE_BINARY, \/* 13 *\/ \/\/ Not implemented$/;" e enum:doris::PrimitiveType +TYPE_BOOLEAN output/udf/include/udf.h /^ TYPE_BOOLEAN,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_BOOLEAN src/runtime/primitive_type.h /^ TYPE_BOOLEAN, \/* 2 *\/$/;" e enum:doris::PrimitiveType +TYPE_BOOLEAN src/udf/udf.h /^ TYPE_BOOLEAN,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_CHAR output/udf/include/udf.h /^ TYPE_CHAR,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_CHAR src/runtime/primitive_type.h /^ TYPE_CHAR, \/* 15 *\/$/;" e enum:doris::PrimitiveType +TYPE_CHAR src/udf/udf.h /^ TYPE_CHAR,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_DATE output/udf/include/udf.h /^ TYPE_DATE,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_DATE src/runtime/primitive_type.h /^ TYPE_DATE, \/* 11 *\/$/;" e enum:doris::PrimitiveType +TYPE_DATE src/udf/udf.h /^ TYPE_DATE,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_DATETIME output/udf/include/udf.h /^ TYPE_DATETIME,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_DATETIME src/runtime/primitive_type.h /^ TYPE_DATETIME, \/* 12 *\/$/;" e enum:doris::PrimitiveType +TYPE_DATETIME src/udf/udf.h /^ TYPE_DATETIME,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_DECIMAL output/udf/include/udf.h /^ TYPE_DECIMAL,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_DECIMAL src/runtime/primitive_type.h /^ TYPE_DECIMAL, \/* 14 *\/$/;" e enum:doris::PrimitiveType +TYPE_DECIMAL src/udf/udf.h /^ TYPE_DECIMAL,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_DECIMALV2 output/udf/include/udf.h /^ TYPE_DECIMALV2,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_DECIMALV2 src/runtime/primitive_type.h /^ TYPE_DECIMALV2, \/* 20 *\/$/;" e enum:doris::PrimitiveType +TYPE_DECIMALV2 src/udf/udf.h /^ TYPE_DECIMALV2,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_DOUBLE output/udf/include/udf.h /^ TYPE_DOUBLE,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_DOUBLE src/runtime/primitive_type.h /^ TYPE_DOUBLE, \/* 9 *\/$/;" e enum:doris::PrimitiveType +TYPE_DOUBLE src/udf/udf.h /^ TYPE_DOUBLE,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_FIXED_BUFFER output/udf/include/udf.h /^ TYPE_FIXED_BUFFER,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_FIXED_BUFFER src/udf/udf.h /^ TYPE_FIXED_BUFFER,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_FLOAT output/udf/include/udf.h /^ TYPE_FLOAT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_FLOAT src/runtime/primitive_type.h /^ TYPE_FLOAT, \/* 8 *\/$/;" e enum:doris::PrimitiveType +TYPE_FLOAT src/udf/udf.h /^ TYPE_FLOAT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_HLL output/udf/include/udf.h /^ TYPE_HLL,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_HLL src/runtime/primitive_type.h /^ TYPE_HLL, \/* 19 *\/$/;" e enum:doris::PrimitiveType +TYPE_HLL src/udf/udf.h /^ TYPE_HLL,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_INT output/udf/include/udf.h /^ TYPE_INT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_INT src/runtime/primitive_type.h /^ TYPE_INT, \/* 5 *\/$/;" e enum:doris::PrimitiveType +TYPE_INT src/udf/udf.h /^ TYPE_INT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_LARGEINT output/udf/include/udf.h /^ TYPE_LARGEINT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_LARGEINT src/runtime/primitive_type.h /^ TYPE_LARGEINT, \/* 7 *\/$/;" e enum:doris::PrimitiveType +TYPE_LARGEINT src/udf/udf.h /^ TYPE_LARGEINT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_MAP src/runtime/primitive_type.h /^ TYPE_MAP, \/* 18 *\/$/;" e enum:doris::PrimitiveType +TYPE_NULL output/udf/include/udf.h /^ TYPE_NULL,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_NULL src/runtime/primitive_type.h /^ TYPE_NULL, \/* 1 *\/$/;" e enum:doris::PrimitiveType +TYPE_NULL src/udf/udf.h /^ TYPE_NULL,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_OBJECT output/udf/include/udf.h /^ TYPE_OBJECT$/;" e enum:doris_udf::FunctionContext::Type +TYPE_OBJECT src/runtime/primitive_type.h /^ TYPE_OBJECT,$/;" e enum:doris::PrimitiveType +TYPE_OBJECT src/udf/udf.h /^ TYPE_OBJECT$/;" e enum:doris_udf::FunctionContext::Type +TYPE_REINTERPRET_CAST src/olap/schema_change.cpp 127;" d file: +TYPE_REINTERPRET_CAST src/olap/schema_change.cpp 439;" d file: +TYPE_SMALLINT output/udf/include/udf.h /^ TYPE_SMALLINT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_SMALLINT src/runtime/primitive_type.h /^ TYPE_SMALLINT, \/* 4 *\/$/;" e enum:doris::PrimitiveType +TYPE_SMALLINT src/udf/udf.h /^ TYPE_SMALLINT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_STRING output/udf/include/udf.h /^ TYPE_STRING,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_STRING src/udf/udf.h /^ TYPE_STRING,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_STRUCT src/runtime/primitive_type.h /^ TYPE_STRUCT, \/* 16 *\/$/;" e enum:doris::PrimitiveType +TYPE_TIME src/runtime/primitive_type.h /^ TYPE_TIME, \/* 21 *\/$/;" e enum:doris::PrimitiveType +TYPE_TINYINT output/udf/include/udf.h /^ TYPE_TINYINT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_TINYINT src/runtime/primitive_type.h /^ TYPE_TINYINT, \/* 3 *\/$/;" e enum:doris::PrimitiveType +TYPE_TINYINT src/udf/udf.h /^ TYPE_TINYINT,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_VARCHAR output/udf/include/udf.h /^ TYPE_VARCHAR,$/;" e enum:doris_udf::FunctionContext::Type +TYPE_VARCHAR src/runtime/primitive_type.h /^ TYPE_VARCHAR, \/* 10 *\/$/;" e enum:doris::PrimitiveType +TYPE_VARCHAR src/udf/udf.h /^ TYPE_VARCHAR,$/;" e enum:doris_udf::FunctionContext::Type +Table src/olap/memtable.h /^ typedef SkipList Table;$/;" t class:doris::MemTable +TableDescriptor src/runtime/descriptors.cpp /^TableDescriptor::TableDescriptor(const TTableDescriptor& tdesc)$/;" f class:doris::TableDescriptor +TableDescriptor src/runtime/descriptors.h /^class TableDescriptor {$/;" c namespace:doris +TableDescriptorMap src/runtime/descriptors.h /^ typedef std::tr1::unordered_map TableDescriptorMap;$/;" t class:doris::DescriptorTbl +TableId src/common/global_types.h /^typedef int TableId;$/;" t namespace:doris +TableInstances src/olap/tablet_manager.h /^ struct TableInstances {$/;" s class:doris::TabletManager +TableKey src/olap/memtable.h /^ typedef Table::key_type TableKey;$/;" t class:doris::MemTable +Tablet src/olap/tablet.cpp /^Tablet::Tablet(TabletMetaSharedPtr tablet_meta, DataDir* data_dir) :$/;" f class:doris::Tablet +Tablet src/olap/tablet.h /^class Tablet : public std::enable_shared_from_this {$/;" c namespace:doris +TabletColumn src/olap/tablet_schema.cpp /^TabletColumn::TabletColumn() :$/;" f class:doris::TabletColumn +TabletColumn src/olap/tablet_schema.cpp /^TabletColumn::TabletColumn(FieldAggregationMethod agg, FieldType filed_type, bool is_nullable) {$/;" f class:doris::TabletColumn +TabletColumn src/olap/tablet_schema.cpp /^TabletColumn::TabletColumn(FieldAggregationMethod agg, FieldType type) {$/;" f class:doris::TabletColumn +TabletColumn src/olap/tablet_schema.h /^class TabletColumn {$/;" c namespace:doris +TabletDesc src/runtime/dpp_sink_internal.h /^struct TabletDesc {$/;" s namespace:doris +TabletInfo src/olap/olap_common.h /^ TabletInfo(TTabletId in_tablet_id, TSchemaHash in_schema_hash, UniqueId in_uid) :$/;" f struct:doris::TabletInfo +TabletInfo src/olap/olap_common.h /^struct TabletInfo {$/;" s namespace:doris +TabletManager src/olap/tablet_manager.cpp /^TabletManager::TabletManager(int32_t tablet_map_lock_shard_size)$/;" f class:doris::TabletManager +TabletManager src/olap/tablet_manager.h /^class TabletManager {$/;" c namespace:doris +TabletMeta src/olap/tablet_meta.cpp /^TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id,$/;" f class:doris::TabletMeta +TabletMeta src/olap/tablet_meta.h /^class TabletMeta {$/;" c namespace:doris +TabletMetaManager src/olap/tablet_meta_manager.h /^class TabletMetaManager {$/;" c namespace:doris +TabletMetaManagerTest test/olap/tablet_meta_manager_test.cpp /^class TabletMetaManagerTest : public testing::Test {$/;" c namespace:doris file: +TabletMgrTest test/olap/tablet_mgr_test.cpp /^class TabletMgrTest : public testing::Test {$/;" c namespace:doris file: +TabletSchema src/olap/tablet_schema.cpp /^TabletSchema::TabletSchema()$/;" f class:doris::TabletSchema +TabletSchema src/olap/tablet_schema.h /^class TabletSchema {$/;" c namespace:doris +TabletState src/olap/tablet_meta.h /^enum TabletState {$/;" g namespace:doris +TabletSyncService src/olap/tablet_sync_service.cpp /^TabletSyncService::TabletSyncService() {$/;" f class:doris::TabletSyncService +TabletSyncService src/olap/tablet_sync_service.h /^class TabletSyncService {$/;" c namespace:doris +TabletTxnInfo src/olap/txn_manager.h /^ TabletTxnInfo($/;" f struct:doris::TabletTxnInfo +TabletTxnInfo src/olap/txn_manager.h /^struct TabletTxnInfo {$/;" s namespace:doris +TabletUid src/olap/olap_common.h /^typedef UniqueId TabletUid;$/;" t namespace:doris +TabletVars src/olap/push_handler.h /^struct TabletVars {$/;" s namespace:doris +TabletsChannel src/runtime/tablets_channel.cpp /^TabletsChannel::TabletsChannel(const TabletsChannelKey& key, MemTracker* mem_tracker):$/;" f class:doris::TabletsChannel +TabletsChannel src/runtime/tablets_channel.h /^class TabletsChannel {$/;" c namespace:doris +TabletsChannelKey src/runtime/tablets_channel.h /^ TabletsChannelKey(const PUniqueId& pid, int64_t index_id_)$/;" f struct:doris::TabletsChannelKey +TabletsChannelKey src/runtime/tablets_channel.h /^struct TabletsChannelKey {$/;" s namespace:doris +TagunzFile__ src/util/minizip/unzip.h /^typedef struct TagunzFile__ { int unused; } unzFile__;$/;" s +TakeState src/gutil/gscoped_ptr.h /^ void TakeState(gscoped_ptr_impl* other) {$/;" f class:doris::internal::gscoped_ptr_impl +Task src/util/priority_thread_pool.hpp /^ struct Task {$/;" s class:doris::PriorityThreadPool +Task src/util/threadpool.h /^ struct Task {$/;" s class:doris::ThreadPool +TaskWorkerPool src/agent/task_worker_pool.cpp /^TaskWorkerPool::TaskWorkerPool(TaskWorkerType task_worker_type,$/;" f class:doris::TaskWorkerPool +TaskWorkerPool src/agent/task_worker_pool.h /^class TaskWorkerPool {$/;" c namespace:doris +TaskWorkerType src/agent/task_worker_pool.h /^ enum TaskWorkerType {$/;" g class:doris::TaskWorkerPool +TearDown test/exec/broker_reader_test.cpp /^ virtual void TearDown() {$/;" f class:doris::BrokerReaderTest +TearDown test/exec/broker_scan_node_test.cpp /^ virtual void TearDown() {$/;" f class:doris::BrokerScanNodeTest +TearDown test/exec/broker_scanner_test.cpp /^ virtual void TearDown() {$/;" f class:doris::BrokerScannerTest +TearDown test/exec/csv_scan_bench_test.cpp /^ virtual void TearDown() {$/;" f class:doris::CsvScanNodeBenchTest +TearDown test/exec/csv_scan_node_test.cpp /^ virtual void TearDown() {$/;" f class:doris::CsvScanNodeTest +TearDown test/exec/csv_scanner_test.cpp /^ virtual void TearDown() {$/;" f class:doris::CsvScannerTest +TearDown test/exec/es_http_scan_node_test.cpp /^ virtual void TearDown() {$/;" f class:doris::EsHttpScanNodeTest +TearDown test/exec/es_scan_node_test.cpp /^ virtual void TearDown() {$/;" f class:doris::EsScanNodeTest +TearDown test/exec/mysql_scan_node_test.cpp /^ virtual void TearDown() {$/;" f class:doris::MysqlScanNodeTest +TearDown test/exec/new_olap_scan_node_test.cpp /^ void TearDown() {$/;" f class:doris::TestOlapScanNode +TearDown test/exec/olap_common_test.cpp /^ virtual void TearDown() {$/;" f class:doris::ColumnValueRangeTest +TearDown test/exec/olap_common_test.cpp /^ virtual void TearDown() {$/;" f class:doris::OlapScanKeysTest +TearDown test/exec/olap_common_test.cpp /^ void TearDown() {$/;" f class:doris::DorisScanRangeTest +TearDown test/exec/olap_scan_node_test.cpp /^ virtual void TearDown() {$/;" f class:doris::OlapScanNodeTest +TearDown test/exec/orc_scanner_test.cpp /^ virtual void TearDown() {$/;" f class:doris::OrcScannerTest +TearDown test/exec/parquet_scanner_test.cpp /^ virtual void TearDown() {$/;" f class:doris::ParquetSannerTest +TearDown test/exec/partitioned_hash_table_test.cpp /^ virtual void TearDown() {$/;" f class:doris::PartitionedHashTableTest +TearDown test/exec/plain_text_line_reader_bzip_test.cpp /^ virtual void TearDown() {$/;" f class:doris::PlainTextLineReaderTest +TearDown test/exec/plain_text_line_reader_gzip_test.cpp /^ virtual void TearDown() {$/;" f class:doris::PlainTextLineReaderTest +TearDown test/exec/plain_text_line_reader_lz4frame_test.cpp /^ virtual void TearDown() {$/;" f class:doris::PlainTextLineReaderTest +TearDown test/exec/plain_text_line_reader_lzop_test.cpp /^ virtual void TearDown() {$/;" f class:doris::PlainTextLineReaderTest +TearDown test/exec/plain_text_line_reader_uncompressed_test.cpp /^ virtual void TearDown() {$/;" f class:doris::PlainTextLineReaderTest +TearDown test/exec/schema_scan_node_test.cpp /^ virtual void TearDown() {$/;" f class:doris::SchemaScanNodeTest +TearDown test/exprs/binary_predicate_test.cpp /^ virtual void TearDown() {$/;" f class:doris::BinaryOpTest +TearDown test/exprs/bitmap_function_test.cpp /^ void TearDown() {$/;" f class:doris::BitmapFunctionsTest +TearDown test/exprs/hll_function_test.cpp /^ void TearDown() {$/;" f class:doris::HllFunctionsTest +TearDown test/exprs/in_op_test.cpp /^ virtual void TearDown() {$/;" f class:doris::InOpTest +TearDown test/exprs/in_predicate_test.cpp /^ virtual void TearDown() {$/;" f class:doris::InPredicateTest +TearDown test/exprs/timestamp_functions_test.cpp /^ void TearDown() {$/;" f class:doris::TimestampFunctionsTest +TearDown test/olap/bit_field_test.cpp /^ void TearDown() {$/;" f class:doris::TestBitField +TearDown test/olap/bloom_filter_index_test.cpp /^ virtual void TearDown() {}$/;" f class:doris::TestBloomFilterIndex +TearDown test/olap/bloom_filter_test.cpp /^ virtual void TearDown() {}$/;" f class:doris::TestBloomFilter +TearDown test/olap/byte_buffer_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TestByteBuffer +TearDown test/olap/column_reader_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TestColumn +TearDown test/olap/delete_handler_test.cpp /^ void TearDown() {$/;" f class:doris::TestDeleteConditionHandler +TearDown test/olap/delete_handler_test.cpp /^ void TearDown() {$/;" f class:doris::TestDeleteConditionHandler2 +TearDown test/olap/delete_handler_test.cpp /^ void TearDown() {$/;" f class:doris::TestDeleteHandler +TearDown test/olap/delta_writer_test.cpp /^ void TearDown(){$/;" f class:doris::TestDeltaWriter +TearDown test/olap/file_helper_test.cpp /^ virtual void TearDown() {$/;" f class:doris::FileHandlerTest +TearDown test/olap/file_utils_test.cpp /^ virtual void TearDown() {$/;" f class:doris::FileUtilsTest +TearDown test/olap/lru_cache_test.cpp /^ void TearDown() {$/;" f class:doris::CacheTest +TearDown test/olap/memtable_flush_executor_test.cpp /^ void TearDown(){$/;" f class:doris::TestMemTableFlushExecutor +TearDown test/olap/olap_meta_test.cpp /^ virtual void TearDown() {$/;" f class:doris::OlapMetaTest +TearDown test/olap/olap_reader_test.cpp /^ void TearDown() {$/;" f class:doris::TestOLAPReaderColumn +TearDown test/olap/olap_reader_test.cpp /^ void TearDown() {$/;" f class:doris::TestOLAPReaderColumnDeleteCondition +TearDown test/olap/olap_reader_test.cpp /^ void TearDown() {$/;" f class:doris::TestOLAPReaderRow +TearDown test/olap/olap_snapshot_converter_test.cpp /^ virtual void TearDown() {$/;" f class:doris::OlapSnapshotConverterTest +TearDown test/olap/row_block_test.cpp /^ void TearDown() {$/;" f class:doris::TestRowBlock +TearDown test/olap/row_block_v2_test.cpp /^ void TearDown() {$/;" f class:doris::TestRowBlockV2 +TearDown test/olap/row_cursor_test.cpp /^ virtual void TearDown() {}$/;" f class:doris::TestRowCursor +TearDown test/olap/rowset/alpha_rowset_test.cpp /^ virtual void TearDown() {$/;" f class:doris::AlphaRowsetTest +TearDown test/olap/rowset/rowset_converter_test.cpp /^ virtual void TearDown() {$/;" f class:doris::RowsetConverterTest +TearDown test/olap/rowset/rowset_meta_manager_test.cpp /^ virtual void TearDown() {$/;" f class:doris::RowsetMetaManagerTest +TearDown test/olap/rowset/rowset_meta_test.cpp /^ virtual void TearDown() {$/;" f class:doris::RowsetMetaTest +TearDown test/olap/run_length_byte_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TestRunLengthByte +TearDown test/olap/run_length_integer_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TestRunLengthSignInteger +TearDown test/olap/run_length_integer_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TestRunLengthUnsignInteger +TearDown test/olap/schema_change_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TestColumn +TearDown test/olap/stream_index_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TestStreamIndex +TearDown test/olap/tablet_meta_manager_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TabletMetaManagerTest +TearDown test/olap/tablet_mgr_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TabletMgrTest +TearDown test/olap/txn_manager_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TxnManagerTest +TearDown test/olap/vectorized_olap_reader_test.cpp /^ void TearDown() {$/;" f class:doris::TestVectorizedOLAPReader +TearDown test/runtime/buffered_block_mgr2_test.cpp /^ virtual void TearDown() {$/;" f class:doris::BufferedBlockMgrTest +TearDown test/runtime/buffered_tuple_stream2_test.cpp /^ virtual void TearDown() {$/;" f class:doris::SimpleTupleStreamTest +TearDown test/runtime/data_spliter_test.cpp /^ virtual void TearDown() { }$/;" f class:doris::DataSplitTest +TearDown test/runtime/data_stream_test.cpp /^ virtual void TearDown() {$/;" f class:doris::DataStreamTest +TearDown test/runtime/datetime_value_test.cpp /^ virtual void TearDown() {$/;" f class:doris::DateTimeValueTest +TearDown test/runtime/decimal_value_test.cpp /^ virtual void TearDown() {$/;" f class:doris::DecimalValueTest +TearDown test/runtime/decimalv2_value_test.cpp /^ virtual void TearDown() {$/;" f class:doris::DecimalV2ValueTest +TearDown test/runtime/dpp_sink_internal_test.cpp /^ virtual void TearDown() {$/;" f class:doris::DppSinkInternalTest +TearDown test/runtime/dpp_sink_test.cpp /^ virtual void TearDown() {$/;" f class:doris::DppSinkTest +TearDown test/runtime/dpp_writer_test.cpp /^ virtual void TearDown() {$/;" f class:doris::DppWriterTest +TearDown test/runtime/fragment_mgr_test.cpp /^ virtual void TearDown() {$/;" f class:doris::FragmentMgrTest +TearDown test/runtime/large_int_value_test.cpp /^ virtual void TearDown() {$/;" f class:doris::LargeIntValueTest +TearDown test/runtime/memory_scratch_sink_test.cpp /^ virtual void TearDown() {$/;" f class:doris::MemoryScratchSinkTest +TearDown test/runtime/mysql_table_writer_test.cpp /^ virtual void TearDown() {$/;" f class:doris::MysqlTableWriterTest +TearDown test/runtime/qsorter_test.cpp /^ virtual void TearDown() {$/;" f class:doris::QSorterTest +TearDown test/runtime/tmp_file_mgr_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TmpFileMgrTest +TearDown test/util/arrow/arrow_work_flow_test.cpp /^ virtual void TearDown() {$/;" f class:doris::ArrowWorkFlowTest +TearDown test/util/radix_sort_test.cpp /^ virtual void TearDown() {$/;" f class:doris::RadixSortTest +TearDown test/util/string_parser_test.cpp /^ virtual void TearDown() {$/;" f class:doris::StringParserTest +TearDown test/util/tdigest_test.cpp /^ virtual void TearDown() {$/;" f class:doris::TDigestTest +TearDown test/util/thread_test.cpp /^ virtual void TearDown() {}$/;" f class:doris::ThreadTest +TearDownMgrs test/runtime/buffered_block_mgr2_test.cpp /^ void TearDownMgrs() {$/;" f class:doris::BufferedBlockMgrTest +TearDownTestCase test/agent/cgroups_mgr_test.cpp /^ static void TearDownTestCase() {$/;" f class:doris::CgroupsMgrTest +TearDownTestCase test/exec/es_scan_reader_test.cpp /^ static void TearDownTestCase() {$/;" f class:doris::MockESServerTest +TearDownTestCase test/http/http_client_test.cpp /^ static void TearDownTestCase() {$/;" f class:doris::HttpClientTest +TearDownTestCase test/runtime/small_file_mgr_test.cpp /^ static void TearDownTestCase() {$/;" f class:doris::SmallFileMgrTest +TearDownTestCase test/runtime/user_function_cache_test.cpp /^ static void TearDownTestCase() {$/;" f class:doris::UserFunctionCacheTest +Tell src/exec/parquet_reader.cpp /^arrow::Status ParquetFile::Tell(int64_t* position) const {$/;" f class:doris::ParquetFile +TempDisable src/util/cpu_info.h /^ TempDisable(int64_t feature)$/;" f struct:doris::CpuInfo::TempDisable +TempDisable src/util/cpu_info.h /^ struct TempDisable {$/;" s class:doris::CpuInfo +TemplatedElementDeleter src/gutil/stl_util.h /^ explicit TemplatedElementDeleter(STLContainer *ptr)$/;" f class:TemplatedElementDeleter +TemplatedElementDeleter src/gutil/stl_util.h /^class TemplatedElementDeleter : public BaseDeleter {$/;" c +TemplatedValueDeleter src/gutil/stl_util.h /^ explicit TemplatedValueDeleter(STLContainer *ptr)$/;" f class:TemplatedValueDeleter +TemplatedValueDeleter src/gutil/stl_util.h /^class TemplatedValueDeleter : public BaseDeleter {$/;" c +TenHexDigitsToEightBase32Digits src/gutil/strings/escaping.cc /^void TenHexDigitsToEightBase32Digits(const char *in, char *out) {$/;" f namespace:strings +TermQueryBuilder src/exec/es/es_query_builder.cpp /^TermQueryBuilder::TermQueryBuilder(const ExtBinaryPredicate& binary_predicate) : _field(binary_predicate.col.name), _term(binary_predicate.value.to_string()) {$/;" f class:doris::TermQueryBuilder +TermQueryBuilder src/exec/es/es_query_builder.cpp /^TermQueryBuilder::TermQueryBuilder(const std::string& field, const std::string& term) : _field(field), _term(term) {$/;" f class:doris::TermQueryBuilder +TermQueryBuilder src/exec/es/es_query_builder.h /^class TermQueryBuilder : public QueryBuilder {$/;" c namespace:doris +TermsInSetQueryBuilder src/exec/es/es_query_builder.cpp /^TermsInSetQueryBuilder::TermsInSetQueryBuilder(const ExtInPredicate& in_predicate) : _field(in_predicate.col.name) {$/;" f class:doris::TermsInSetQueryBuilder +TermsInSetQueryBuilder src/exec/es/es_query_builder.h /^class TermsInSetQueryBuilder : public QueryBuilder {$/;" c namespace:doris +Test src/gutil/strings/charset.h /^ bool Test(unsigned char c) const { return bits_[Word(c)] & BitMask(c); }$/;" f class:strings::CharSet +TestBinarySeekByValueSmallPage test/olap/rowset/segment_v2/binary_plain_page_test.cpp /^ void TestBinarySeekByValueSmallPage() {$/;" f class:doris::segment_v2::BinaryPlainPageTest +TestBitArrayValues test/util/bit_stream_utils_test.cpp /^void TestBitArrayValues(int bit_width, int num_vals) {$/;" f namespace:doris +TestBitField test/olap/bit_field_test.cpp /^ TestBitField() {$/;" f class:doris::TestBitField +TestBitField test/olap/bit_field_test.cpp /^class TestBitField : public testing::Test {$/;" c namespace:doris file: +TestBitStreamUtil test/util/bit_stream_utils_test.cpp /^class TestBitStreamUtil : public testing::Test {};$/;" c namespace:doris file: +TestBloomFilter test/olap/bloom_filter_test.cpp /^class TestBloomFilter : public testing::Test {$/;" c namespace:doris file: +TestBloomFilterIndex test/olap/bloom_filter_index_test.cpp /^class TestBloomFilterIndex : public testing::Test {$/;" c namespace:doris file: +TestByteBuffer test/olap/byte_buffer_test.cpp /^class TestByteBuffer : public testing::Test {$/;" c namespace:doris file: +TestColumn test/olap/column_reader_test.cpp /^ TestColumn() : $/;" f class:doris::TestColumn +TestColumn test/olap/column_reader_test.cpp /^class TestColumn : public testing::Test {$/;" c namespace:doris file: +TestColumn test/olap/schema_change_test.cpp /^ TestColumn() : $/;" f class:doris::TestColumn +TestColumn test/olap/schema_change_test.cpp /^class TestColumn : public testing::Test {$/;" c namespace:doris file: +TestComparator test/olap/skiplist_test.cpp /^struct TestComparator {$/;" s namespace:doris file: +TestDataTuple test/runtime/dpp_writer_test.cpp /^struct TestDataTuple {$/;" s namespace:doris file: +TestDataTuple test/runtime/mysql_table_writer_test.cpp /^struct TestDataTuple {$/;" s namespace:doris file: +TestDeleteConditionHandler test/olap/delete_handler_test.cpp /^class TestDeleteConditionHandler : public testing::Test {$/;" c namespace:doris file: +TestDeleteConditionHandler2 test/olap/delete_handler_test.cpp /^class TestDeleteConditionHandler2 : public testing::Test {$/;" c namespace:doris file: +TestDeleteHandler test/olap/delete_handler_test.cpp /^class TestDeleteHandler : public testing::Test {$/;" c namespace:doris file: +TestDeltaWriter test/olap/delta_writer_test.cpp /^ TestDeltaWriter() { }$/;" f class:doris::TestDeltaWriter +TestDeltaWriter test/olap/delta_writer_test.cpp /^class TestDeltaWriter : public ::testing::Test {$/;" c namespace:doris file: +TestEnv src/runtime/test_env.cc /^TestEnv::TestEnv() {$/;" f class:doris::TestEnv +TestEnv src/runtime/test_env.h /^class TestEnv {$/;" c namespace:doris +TestEvictionImpl test/runtime/buffered_block_mgr2_test.cpp /^ void TestEvictionImpl(int block_size) {$/;" f class:doris::BufferedBlockMgrTest +TestFloatBruteForce test/util/string_parser_test.cpp /^void TestFloatBruteForce() {$/;" f namespace:doris +TestForCoding test/util/frame_of_reference_coding_test.cpp /^class TestForCoding : public testing::Test {$/;" c namespace:doris file: +TestGetNewBlockImpl test/runtime/buffered_block_mgr2_test.cpp /^ void TestGetNewBlockImpl(int block_size) {$/;" f class:doris::BufferedBlockMgrTest +TestHll test/olap/hll_test.cpp /^class TestHll : public testing::Test {$/;" c namespace:doris file: +TestInListPredicate test/olap/in_list_predicate_test.cpp /^ TestInListPredicate() : _vectorized_batch(NULL) {$/;" f class:doris::TestInListPredicate +TestInListPredicate test/olap/in_list_predicate_test.cpp /^class TestInListPredicate : public testing::Test {$/;" c namespace:doris file: +TestIntValuesInterleaved test/runtime/buffered_tuple_stream2_test.cpp /^ void TestIntValuesInterleaved(int num_batches, int num_batches_before_read) {$/;" f class:doris::SimpleTupleStreamTest +TestInternalService test/exec/tablet_sink_test.cpp /^ TestInternalService() { }$/;" f class:doris::stream_load::TestInternalService +TestInternalService test/exec/tablet_sink_test.cpp /^class TestInternalService : public palo::PInternalService {$/;" c namespace:doris::stream_load file: +TestMemTableFlushExecutor test/olap/memtable_flush_executor_test.cpp /^ TestMemTableFlushExecutor() { }$/;" f class:doris::TestMemTableFlushExecutor +TestMemTableFlushExecutor test/olap/memtable_flush_executor_test.cpp /^class TestMemTableFlushExecutor : public ::testing::Test {$/;" c namespace:doris file: +TestMetricsVisitor test/util/doris_metrics_test.cpp /^class TestMetricsVisitor : public MetricsVisitor {$/;" c namespace:doris file: +TestMetricsVisitor test/util/new_metrics_test.cpp /^class TestMetricsVisitor : public MetricsVisitor {$/;" c namespace:doris file: +TestMetricsVisitor test/util/system_metrics_test.cpp /^class TestMetricsVisitor : public MetricsVisitor {$/;" c namespace:doris file: +TestNullPredicate test/olap/null_predicate_test.cpp /^ TestNullPredicate() : _vectorized_batch(NULL) {$/;" f class:doris::TestNullPredicate +TestNullPredicate test/olap/null_predicate_test.cpp /^class TestNullPredicate : public testing::Test {$/;" c namespace:doris file: +TestOLAPReaderColumn test/olap/olap_reader_test.cpp /^ TestOLAPReaderColumn() : _runtime_stat("test") { $/;" f class:doris::TestOLAPReaderColumn +TestOLAPReaderColumn test/olap/olap_reader_test.cpp /^class TestOLAPReaderColumn : public testing::Test {$/;" c namespace:doris file: +TestOLAPReaderColumnDeleteCondition test/olap/olap_reader_test.cpp /^ TestOLAPReaderColumnDeleteCondition() : _runtime_stat("test") {$/;" f class:doris::TestOLAPReaderColumnDeleteCondition +TestOLAPReaderColumnDeleteCondition test/olap/olap_reader_test.cpp /^class TestOLAPReaderColumnDeleteCondition : public testing::Test {$/;" c namespace:doris file: +TestOLAPReaderRow test/olap/olap_reader_test.cpp /^ TestOLAPReaderRow() : _runtime_stat("test") { $/;" f class:doris::TestOLAPReaderRow +TestOLAPReaderRow test/olap/olap_reader_test.cpp /^class TestOLAPReaderRow : public testing::Test {$/;" c namespace:doris file: +TestObject test/util/radix_sort_test.cpp /^struct TestObject {$/;" s namespace:doris file: +TestOlapScanNode test/exec/new_olap_scan_node_test.cpp /^ TestOlapScanNode() : _runtime_stat("test") { }$/;" f class:doris::TestOlapScanNode +TestOlapScanNode test/exec/new_olap_scan_node_test.cpp /^class TestOlapScanNode : public testing::Test {$/;" c namespace:doris file: +TestRandomInternalImpl test/runtime/buffered_block_mgr2_test.cpp /^ void TestRandomInternalImpl(RuntimeState* state, BufferedBlockMgr2* block_mgr,$/;" f class:doris::BufferedBlockMgrTest +TestRandomInternalMulti test/runtime/buffered_block_mgr2_test.cpp /^ void TestRandomInternalMulti(int num_threads, int block_size) {$/;" f class:doris::BufferedBlockMgrTest +TestRandomInternalSingle test/runtime/buffered_block_mgr2_test.cpp /^ void TestRandomInternalSingle(int block_size) {$/;" f class:doris::BufferedBlockMgrTest +TestRle test/util/rle_encoding_test.cpp /^class TestRle : public testing::Test {};$/;" c namespace:doris file: +TestRleValues test/util/rle_encoding_test.cpp /^void TestRleValues(int bit_width, int num_vals, int value = -1) {$/;" f namespace:doris +TestRowBlock test/olap/row_block_test.cpp /^ TestRowBlock() {}$/;" f class:doris::TestRowBlock +TestRowBlock test/olap/row_block_test.cpp /^class TestRowBlock : public testing::Test {$/;" c namespace:doris file: +TestRowBlockV2 test/olap/row_block_v2_test.cpp /^ TestRowBlockV2() {}$/;" f class:doris::TestRowBlockV2 +TestRowBlockV2 test/olap/row_block_v2_test.cpp /^class TestRowBlockV2 : public testing::Test {$/;" c namespace:doris file: +TestRowCursor test/olap/row_cursor_test.cpp /^ TestRowCursor() {$/;" f class:doris::TestRowCursor +TestRowCursor test/olap/row_cursor_test.cpp /^class TestRowCursor : public testing::Test {$/;" c namespace:doris file: +TestRunLengthByte test/olap/run_length_byte_test.cpp /^ TestRunLengthByte() {$/;" f class:doris::TestRunLengthByte +TestRunLengthByte test/olap/run_length_byte_test.cpp /^class TestRunLengthByte : public testing::Test {$/;" c namespace:doris file: +TestRunLengthSignInteger test/olap/run_length_integer_test.cpp /^ TestRunLengthSignInteger() {$/;" f class:doris::TestRunLengthSignInteger +TestRunLengthSignInteger test/olap/run_length_integer_test.cpp /^class TestRunLengthSignInteger : public testing::Test {$/;" c namespace:doris file: +TestRunLengthUnsignInteger test/olap/run_length_integer_test.cpp /^ TestRunLengthUnsignInteger() {$/;" f class:doris::TestRunLengthUnsignInteger +TestRunLengthUnsignInteger test/olap/run_length_integer_test.cpp /^class TestRunLengthUnsignInteger : public testing::Test {$/;" c namespace:doris file: +TestState test/olap/skiplist_test.cpp /^ explicit TestState(int s)$/;" f class:doris::TestState +TestState test/olap/skiplist_test.cpp /^class TestState {$/;" c namespace:doris file: +TestStreamIndex test/olap/stream_index_test.cpp /^class TestStreamIndex : public testing::Test {$/;" c namespace:doris file: +TestValues test/runtime/buffered_tuple_stream2_test.cpp /^ void TestValues(int num_batches, RowDescriptor* desc, bool gen_null) {$/;" f class:doris::SimpleTupleStreamTest +TestVectorizedOLAPReader test/olap/vectorized_olap_reader_test.cpp /^ TestVectorizedOLAPReader() : _runtime_stat("test") { $/;" f class:doris::TestVectorizedOLAPReader +TestVectorizedOLAPReader test/olap/vectorized_olap_reader_test.cpp /^class TestVectorizedOLAPReader : public testing::Test {$/;" c namespace:doris file: +Testable src/gutil/gscoped_ptr.h /^ gscoped_ptr::*Testable;$/;" t class:gscoped_ptr::gscoped_ptr +Testable src/gutil/gscoped_ptr.h /^ typedef C* gscoped_array::*Testable;$/;" t class:gscoped_array::gscoped_array +Testable src/gutil/gscoped_ptr.h /^ typedef C* gscoped_ptr_malloc::*Testable;$/;" t class:gscoped_ptr_malloc::gscoped_ptr_malloc +Testable src/gutil/ref_counted.h /^ typedef T* scoped_refptr::*Testable;$/;" t class:scoped_refptr::scoped_refptr +Testx src/gutil/utf/rune.c /^ Testx = Maskx ^ 0xFF, \/* 1100 0000 *\/$/;" e enum:__anon8 file: +TextConverter src/exec/text_converter.cpp /^TextConverter::TextConverter(char escape_char)$/;" f class:doris::TextConverter +TextConverter src/exec/text_converter.h /^class TextConverter {$/;" c namespace:doris +Thread src/util/thread.h /^ Thread(const std::string& category, const std::string& name, ThreadFunctor functor)$/;" f class:doris::Thread +Thread src/util/thread.h /^class Thread : public RefCountedThreadSafe {$/;" c namespace:doris +ThreadCategory src/util/thread.cpp /^ typedef std::map ThreadCategory;$/;" t class:doris::ThreadMgr file: +ThreadCategoryMap src/util/thread.cpp /^ typedef std::map ThreadCategoryMap;$/;" t class:doris::ThreadMgr file: +ThreadCollisionWarner src/gutil/threading/thread_collision_warner.h /^ explicit ThreadCollisionWarner(AsserterBase* asserter = new DCheckAsserter())$/;" f class:base::ThreadCollisionWarner +ThreadCollisionWarner src/gutil/threading/thread_collision_warner.h /^class BASE_EXPORT ThreadCollisionWarner {$/;" c namespace:base +ThreadCounters src/util/runtime_profile.h /^ class ThreadCounters {$/;" c class:doris::RuntimeProfile +ThreadDescriptor src/util/thread.cpp /^ ThreadDescriptor() { }$/;" f class:doris::ThreadMgr::ThreadDescriptor +ThreadDescriptor src/util/thread.cpp /^ ThreadDescriptor(std::string category, std::string name, int64_t thread_id)$/;" f class:doris::ThreadMgr::ThreadDescriptor +ThreadDescriptor src/util/thread.cpp /^ class ThreadDescriptor {$/;" c class:doris::ThreadMgr file: +ThreadFunctor src/util/thread.h /^ typedef std::function ThreadFunctor;$/;" t class:doris::Thread +ThreadJoiner src/util/thread.cpp /^ThreadJoiner::ThreadJoiner(Thread* thr)$/;" f class:doris::ThreadJoiner +ThreadJoiner src/util/thread.h /^class ThreadJoiner {$/;" c namespace:doris +ThreadMgr src/util/thread.cpp /^ ThreadMgr()$/;" f class:doris::ThreadMgr +ThreadMgr src/util/thread.cpp /^class ThreadMgr {$/;" c namespace:doris file: +ThreadPool src/util/threadpool.cpp /^ThreadPool::ThreadPool(const ThreadPoolBuilder& builder)$/;" f class:doris::ThreadPool +ThreadPool src/util/threadpool.h /^class ThreadPool {$/;" c namespace:doris +ThreadPoolBuilder src/util/threadpool.cpp /^ThreadPoolBuilder::ThreadPoolBuilder(string name) :$/;" f class:doris::ThreadPoolBuilder +ThreadPoolBuilder src/util/threadpool.h /^class ThreadPoolBuilder {$/;" c namespace:doris +ThreadPoolTest test/util/threadpool_test.cpp /^class ThreadPoolTest : public ::testing::Test {$/;" c namespace:doris file: +ThreadPoolTestTokenTypes test/util/threadpool_test.cpp /^class ThreadPoolTestTokenTypes : public ThreadPoolTest,$/;" c namespace:doris file: +ThreadPoolToken src/util/threadpool.cpp /^ThreadPoolToken::ThreadPoolToken(ThreadPool* pool,$/;" f class:doris::ThreadPoolToken +ThreadPoolToken src/util/threadpool.h /^class ThreadPoolToken {$/;" c namespace:doris +ThreadResourceMgr src/runtime/thread_resource_mgr.cpp /^ThreadResourceMgr::ThreadResourceMgr() {$/;" f class:doris::ThreadResourceMgr +ThreadResourceMgr src/runtime/thread_resource_mgr.cpp /^ThreadResourceMgr::ThreadResourceMgr(int threads_quota) {$/;" f class:doris::ThreadResourceMgr +ThreadResourceMgr src/runtime/thread_resource_mgr.h /^class ThreadResourceMgr {$/;" c namespace:doris +ThreadTest test/util/thread_test.cpp /^class ThreadTest : public ::testing::Test {$/;" c namespace:doris file: +ThreadVector test/util/blocking_queue_test.cpp /^ typedef std::vector > ThreadVector;$/;" t class:doris::MultiThreadTest file: +ThriftClient src/util/thrift_client.h /^ThriftClient::ThriftClient($/;" f class:doris::ThriftClient +ThriftClient src/util/thrift_client.h /^class ThriftClient : public ThriftClientImpl {$/;" c namespace:doris +ThriftClientImpl src/util/thrift_client.h /^ ThriftClientImpl(const std::string& ipaddress, int port) : $/;" f class:doris::ThriftClientImpl +ThriftClientImpl src/util/thrift_client.h /^class ThriftClientImpl {$/;" c namespace:doris +ThriftDeserializer src/util/thrift_util.h /^class ThriftDeserializer {$/;" c namespace:doris +ThriftRpcError src/common/status.h /^ static Status ThriftRpcError(const Slice& msg, int16_t precise_code = 1, const Slice& msg2 = Slice()) {$/;" f class:doris::Status +ThriftRpcHelper src/util/thrift_rpc_helper.h /^class ThriftRpcHelper {$/;" c namespace:doris +ThriftSerializer src/util/thrift_util.cpp /^ThriftSerializer::ThriftSerializer(bool compact, int initial_buffer_size) :$/;" f class:doris::ThriftSerializer +ThriftSerializer src/util/thrift_util.h /^class ThriftSerializer {$/;" c namespace:doris +ThriftServer src/util/thrift_server.cpp /^ThriftServer::ThriftServer($/;" f class:doris::ThriftServer +ThriftServer src/util/thrift_server.h /^class ThriftServer {$/;" c namespace:doris +ThriftServerEventProcessor src/util/thrift_server.cpp /^ ThriftServerEventProcessor(ThriftServer* thrift_server) :$/;" f class:doris::ThriftServer::ThriftServerEventProcessor +ThriftServerEventProcessor src/util/thrift_server.cpp /^class ThriftServer::ThriftServerEventProcessor$/;" c class:doris::ThriftServer file: +TimeInterval src/runtime/datetime_value.h /^ TimeInterval() :$/;" f struct:doris::TimeInterval +TimeInterval src/runtime/datetime_value.h /^ TimeInterval(TimeUnit unit, int count, bool is_neg_param) :$/;" f struct:doris::TimeInterval +TimeInterval src/runtime/datetime_value.h /^struct TimeInterval {$/;" s namespace:doris +TimeOperators src/exprs/time_operators.h /^class TimeOperators {$/;" c namespace:doris +TimePrecision src/util/time.h /^enum TimePrecision {$/;" g namespace:doris +TimeType src/runtime/datetime_value.h /^enum TimeType {$/;" g namespace:doris +TimeUnit src/runtime/datetime_value.h /^enum TimeUnit {$/;" g namespace:doris +TimedOut src/common/status.h /^ static Status TimedOut(const Slice& msg, int16_t precise_code = 1, const Slice& msg2 = Slice()) {$/;" f class:doris::Status +TimeoutMap src/runtime/result_buffer_mgr.h /^ typedef std::map > TimeoutMap;$/;" t class:doris::ResultBufferMgr +TimepointFromUnix src/util/time.cpp /^static chrono::system_clock::time_point TimepointFromUnix(int64_t s) {$/;" f file: +TimepointFromUnixMicros src/util/time.cpp /^static chrono::system_clock::time_point TimepointFromUnixMicros(int64_t us) {$/;" f file: +TimepointFromUnixMillis src/util/time.cpp /^static chrono::system_clock::time_point TimepointFromUnixMillis(int64_t ms) {$/;" f file: +TimepointToString src/util/time.cpp /^static string TimepointToString(const chrono::system_clock::time_point& t,$/;" f file: +TimestampFunctions src/exprs/timestamp_functions.h /^class TimestampFunctions {$/;" c namespace:doris +TimestampFunctionsTest test/exprs/timestamp_functions_test.cpp /^ TimestampFunctionsTest() { }$/;" f class:doris::TimestampFunctionsTest +TimestampFunctionsTest test/exprs/timestamp_functions_test.cpp /^class TimestampFunctionsTest : public testing::Test {$/;" c namespace:doris file: +TimezoneDatabase src/exprs/timezone_db.h /^class TimezoneDatabase {$/;" c namespace:doris +TinyColumnReader src/olap/rowset/column_reader.cpp /^TinyColumnReader::TinyColumnReader(uint32_t column_id, uint32_t column_unique_id) :$/;" f class:doris::TinyColumnReader +TinyColumnReader src/olap/rowset/column_reader.h /^class TinyColumnReader : public ColumnReader {$/;" c namespace:doris +TinyIntVal output/udf/include/udf.h /^ TinyIntVal() : val(0) {}$/;" f struct:doris_udf::TinyIntVal +TinyIntVal output/udf/include/udf.h /^ TinyIntVal(int8_t val) : val(val) { }$/;" f struct:doris_udf::TinyIntVal +TinyIntVal output/udf/include/udf.h /^struct TinyIntVal : public AnyVal {$/;" s namespace:doris_udf +TinyIntVal src/udf/udf.h /^ TinyIntVal() : val(0) {}$/;" f struct:doris_udf::TinyIntVal +TinyIntVal src/udf/udf.h /^ TinyIntVal(int8_t val) : val(val) { }$/;" f struct:doris_udf::TinyIntVal +TinyIntVal src/udf/udf.h /^struct TinyIntVal : public AnyVal {$/;" s namespace:doris_udf +TinyIntWrapper src/exprs/scalar_fn_call.cpp /^typedef TinyIntVal (*TinyIntWrapper)(ExprContext*, TupleRow*);$/;" t namespace:doris file: +TmpFileMgr src/runtime/tmp_file_mgr.cc /^TmpFileMgr::TmpFileMgr(ExecEnv* exec_env) :$/;" f class:doris::TmpFileMgr +TmpFileMgr src/runtime/tmp_file_mgr.h /^ TmpFileMgr() { }$/;" f class:doris::TmpFileMgr +TmpFileMgr src/runtime/tmp_file_mgr.h /^class TmpFileMgr {$/;" c namespace:doris +TmpFileMgrTest test/runtime/tmp_file_mgr_test.cpp /^class TmpFileMgrTest : public ::testing::Test {$/;" c namespace:doris file: +ToContainer src/gutil/strings/split_internal.h /^ Container ToContainer() {$/;" f class:strings::internal::Splitter +ToHost128 src/gutil/endian.h /^ static unsigned __int128 ToHost128(unsigned __int128 x) { return gbswap_128(x); }$/;" f class:BigEndian +ToHost128 src/gutil/endian.h /^ static unsigned __int128 ToHost128(unsigned __int128 x) { return x; }$/;" f class:LittleEndian +ToHost16 src/gutil/endian.h /^ static uint16 ToHost16(uint16 x) { return bswap_16(x); }$/;" f class:BigEndian +ToHost16 src/gutil/endian.h /^ static uint16 ToHost16(uint16 x) { return x; }$/;" f class:LittleEndian +ToHost24 src/gutil/endian.h /^ static uint32 ToHost24(uint32 x) { return bswap_24(x); }$/;" f class:BigEndian +ToHost32 src/gutil/endian.h /^ static uint32 ToHost32(uint32 x) { return bswap_32(x); }$/;" f class:BigEndian +ToHost32 src/gutil/endian.h /^ static uint32 ToHost32(uint32 x) { return x; }$/;" f class:LittleEndian +ToHost64 src/gutil/endian.h /^ static uint64 ToHost64(uint64 x) { return gbswap_64(x); }$/;" f class:BigEndian +ToHost64 src/gutil/endian.h /^ static uint64 ToHost64(uint64 x) { return x; }$/;" f class:LittleEndian +ToJsonValue src/util/json_util.h /^ToJsonValue(const T& value, const TUnit::type unit, rapidjson::Document* document,$/;" f namespace:doris +ToMap src/gutil/strings/split_internal.h /^ Map ToMap() {$/;" f class:strings::internal::Splitter +ToMicroseconds src/util/monotime.cpp /^int64_t MonoDelta::ToMicroseconds() const {$/;" f class:doris::MonoDelta +ToMilliseconds src/util/monotime.cpp /^int64_t MonoDelta::ToMilliseconds() const {$/;" f class:doris::MonoDelta +ToNanoseconds src/util/monotime.cpp /^int64_t MonoDelta::ToNanoseconds() const {$/;" f class:doris::MonoDelta +ToOlapFilterVisitor src/exec/olap_scan_node.h /^ class ToOlapFilterVisitor : public boost::static_visitor {$/;" c class:doris::OlapScanNode +ToPair src/gutil/strings/split_internal.h /^ std::pair ToPair() {$/;" f class:strings::internal::Splitter +ToRowBatchConverter src/util/arrow/row_batch.cpp /^ ToRowBatchConverter(const arrow::RecordBatch& batch,$/;" f class:doris::ToRowBatchConverter +ToRowBatchConverter src/util/arrow/row_batch.cpp /^class ToRowBatchConverter : public arrow::ArrayVisitor {$/;" c namespace:doris file: +ToRowBlockConverter src/util/arrow/row_block.cpp /^ ToRowBlockConverter(const arrow::RecordBatch& batch,$/;" f class:doris::ToRowBlockConverter +ToRowBlockConverter src/util/arrow/row_block.cpp /^class ToRowBlockConverter : public arrow::ArrayVisitor {$/;" c namespace:doris file: +ToSeconds src/util/monotime.cpp /^double MonoDelta::ToSeconds() const {$/;" f class:doris::MonoDelta +ToSeconds src/util/monotime.cpp /^double MonoTime::ToSeconds() const {$/;" f class:doris::MonoTime +ToString src/gutil/strings/stringpiece.h /^ std::string ToString() const {$/;" f class:StringPiece +ToString src/util/faststring.h /^ std::string ToString() const {$/;" f class:doris::faststring +ToString src/util/monotime.cpp /^std::string MonoDelta::ToString() const {$/;" f class:doris::MonoDelta +ToString src/util/monotime.cpp /^std::string MonoTime::ToString() const {$/;" f class:doris::MonoTime +ToString src/util/time.cpp /^static string ToString(const chrono::system_clock::time_point& t, TimePrecision p,$/;" f file: +ToStringFromUnix src/util/time.cpp /^string doris::ToStringFromUnix(int64_t s, TimePrecision p) {$/;" f class:doris +ToStringFromUnixMicros src/util/time.cpp /^string doris::ToStringFromUnixMicros(int64_t us, TimePrecision p) {$/;" f class:doris +ToStringFromUnixMillis src/util/time.cpp /^string doris::ToStringFromUnixMillis(int64_t ms, TimePrecision p) {$/;" f class:doris +ToTimeSpec src/util/monotime.cpp /^void MonoDelta::ToTimeSpec(struct timespec *ts) const {$/;" f class:doris::MonoDelta +ToTimeSpec src/util/monotime.cpp /^void MonoTime::ToTimeSpec(struct timespec* ts) const {$/;" f class:doris::MonoTime +ToTimeVal src/util/monotime.cpp /^void MonoDelta::ToTimeVal(struct timeval *tv) const {$/;" f class:doris::MonoDelta +ToUtcStringFromUnix src/util/time.cpp /^string doris::ToUtcStringFromUnix(int64_t s, TimePrecision p) {$/;" f class:doris +ToUtcStringFromUnixMicros src/util/time.cpp /^string doris::ToUtcStringFromUnixMicros(int64_t us, TimePrecision p) {$/;" f class:doris +ToUtcStringFromUnixMillis src/util/time.cpp /^string doris::ToUtcStringFromUnixMillis(int64_t ms, TimePrecision p) {$/;" f class:doris +TooManyTasks src/common/status.h /^ static Status TooManyTasks(const Slice& msg, int16_t precise_code = 1, const Slice& msg2 = Slice()) {$/;" f class:doris::Status +TopNNode src/exec/topn_node.cpp /^TopNNode::TopNNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) :$/;" f class:doris::TopNNode +TopNNode src/exec/topn_node.h /^class TopNNode : public ExecNode {$/;" c namespace:doris +TopicListener src/agent/topic_listener.h /^class TopicListener {$/;" c namespace:doris +TopicSubscriber src/agent/topic_subscriber.cpp /^TopicSubscriber::TopicSubscriber() {$/;" f class:doris::TopicSubscriber +TopicSubscriber src/agent/topic_subscriber.h /^class TopicSubscriber {$/;" c namespace:doris +TrailingBits src/util/bit_util.h /^ static inline uint64_t TrailingBits(uint64_t v, int num_bits) {$/;" f class:doris::BitUtil +TransferBuffer src/runtime/bufferpool/buffer_pool.cc /^Status BufferPool::TransferBuffer(ClientHandle* src_client, BufferHandle* src,$/;" f class:doris::BufferPool +TransferReservationFrom src/runtime/bufferpool/buffer_pool.cc /^bool BufferPool::ClientHandle::TransferReservationFrom($/;" f class:doris::BufferPool::ClientHandle +TransferReservationTo src/runtime/bufferpool/buffer_pool.cc /^bool BufferPool::ClientHandle::TransferReservationTo($/;" f class:doris::BufferPool::ClientHandle +TransferReservationTo src/runtime/bufferpool/reservation_tracker.cc /^bool ReservationTracker::TransferReservationTo(ReservationTracker* other, int64_t bytes) {$/;" f class:doris::ReservationTracker +TransferStatus src/exec/olap_scan_node.h /^enum TransferStatus {$/;" g namespace:doris +Translator src/runtime/dpp_sink.cpp /^Translator::Translator($/;" f class:doris::Translator +Translator src/runtime/dpp_sink.cpp /^class Translator {$/;" c namespace:doris file: +TrieNode src/util/path_trie.hpp /^ TrieNode(const std::string& key, const T& value, const std::string& wildcard) :$/;" f class:doris::PathTrie::TrieNode +TrieNode src/util/path_trie.hpp /^ TrieNode(const std::string& key, const std::string& wildcard) :$/;" f class:doris::PathTrie::TrieNode +TrieNode src/util/path_trie.hpp /^ class TrieNode {$/;" c class:doris::PathTrie +TrimRunsInString src/gutil/strings/strip.cc /^void TrimRunsInString(string* s, StringPiece remove) {$/;" f +TrimString src/gutil/strings/strip.h /^inline int TrimString(string* s, const StringPiece& remove) {$/;" f +TrimStringLeft src/gutil/strings/strip.cc /^int TrimStringLeft(string* s, const StringPiece& remove) {$/;" f +TrimStringRight src/gutil/strings/strip.cc /^int TrimStringRight(string* s, const StringPiece& remove) {$/;" f +TruncateIfNecessary src/exprs/anyval_util.h /^ static void TruncateIfNecessary(const FunctionContext::TypeDesc& type, StringVal *val) {$/;" f class:doris::AnyValUtil +TryAddToHashTable src/exec/new_partitioned_aggregation_node_ir.cc /^bool NewPartitionedAggregationNode::TryAddToHashTable($/;" f class:NewPartitionedAggregationNode +TryConsumeFromMemTracker src/runtime/bufferpool/reservation_tracker.cc /^bool ReservationTracker::TryConsumeFromMemTracker(int64_t reservation_increase) {$/;" f class:doris::ReservationTracker +TryJoinOnSelf test/util/thread_test.cpp /^static void TryJoinOnSelf() {$/;" f namespace:doris +TryStripPrefixString src/gutil/strings/strip.cc /^bool TryStripPrefixString(StringPiece str, const StringPiece& prefix,$/;" f +TryStripSuffixString src/gutil/strings/strip.cc /^bool TryStripSuffixString(StringPiece str, const StringPiece& suffix,$/;" f +Tuple src/runtime/tuple.h /^class Tuple {$/;" c namespace:doris +TupleDescBuilder src/testutil/desc_tbl_builder.h /^class TupleDescBuilder {$/;" c namespace:doris +TupleDescriptor src/runtime/descriptors.cpp /^TupleDescriptor::TupleDescriptor(const PTupleDescriptor& pdesc)$/;" f class:doris::TupleDescriptor +TupleDescriptor src/runtime/descriptors.cpp /^TupleDescriptor::TupleDescriptor(const TTupleDescriptor& tdesc) :$/;" f class:doris::TupleDescriptor +TupleDescriptor src/runtime/descriptors.h /^class TupleDescriptor {$/;" c namespace:doris +TupleDescriptorMap src/runtime/descriptors.h /^ typedef std::tr1::unordered_map TupleDescriptorMap;$/;" t class:doris::DescriptorTbl +TupleId src/common/global_types.h /^typedef int TupleId;$/;" t namespace:doris +TupleIsNullPredicate src/exprs/tuple_is_null_predicate.cpp /^TupleIsNullPredicate::TupleIsNullPredicate(const TExprNode& node) : $/;" f class:doris::TupleIsNullPredicate +TupleIsNullPredicate src/exprs/tuple_is_null_predicate.h /^class TupleIsNullPredicate: public Predicate {$/;" c namespace:doris +TupleIterator src/runtime/merge_sorter.cpp /^ TupleIterator(TupleSorter* parent, int64_t index)$/;" f class:doris::MergeSorter::TupleSorter::TupleIterator +TupleIterator src/runtime/merge_sorter.cpp /^ class TupleIterator {$/;" c class:doris::MergeSorter::TupleSorter file: +TupleIterator src/runtime/spill_sorter.cc /^ TupleIterator(TupleSorter* parent, int64_t index) :$/;" f class:doris::SpillSorter::TupleSorter::TupleIterator +TupleIterator src/runtime/spill_sorter.cc /^ class TupleIterator {$/;" c class:doris::SpillSorter::TupleSorter file: +TupleRow src/runtime/tuple_row.h /^class TupleRow {$/;" c namespace:doris +TupleRowComparator src/util/tuple_row_compare.h /^ TupleRowComparator($/;" f class:doris::TupleRowComparator +TupleRowComparator src/util/tuple_row_compare.h /^ TupleRowComparator(const SortExecExprs& sort_key_exprs, bool is_asc, bool nulls_first) :$/;" f class:doris::TupleRowComparator +TupleRowComparator src/util/tuple_row_compare.h /^class TupleRowComparator {$/;" c namespace:doris +TupleRowIterator src/exec/row_batch_list.h /^ TupleRowIterator() : _list(NULL), _row_idx(0) { }$/;" f class:doris::RowBatchList::TupleRowIterator +TupleRowIterator src/exec/row_batch_list.h /^ TupleRowIterator(RowBatchList* list)$/;" f class:doris::RowBatchList::TupleRowIterator +TupleRowIterator src/exec/row_batch_list.h /^ class TupleRowIterator {$/;" c class:doris::RowBatchList +TupleRowLessThan src/runtime/qsorter.cpp /^TupleRowLessThan::TupleRowLessThan($/;" f class:doris::TupleRowLessThan +TupleRowLessThan src/runtime/qsorter.cpp /^class TupleRowLessThan {$/;" c namespace:doris file: +TupleSorter src/runtime/merge_sorter.cpp /^MergeSorter::TupleSorter::TupleSorter(const TupleRowComparator& comp, int64_t block_size,$/;" f class:doris::MergeSorter::TupleSorter +TupleSorter src/runtime/merge_sorter.cpp /^class MergeSorter::TupleSorter {$/;" c class:doris::MergeSorter file: +TupleSorter src/runtime/spill_sorter.cc /^SpillSorter::TupleSorter::TupleSorter($/;" f class:doris::SpillSorter::TupleSorter +TupleSorter src/runtime/spill_sorter.cc /^class SpillSorter::TupleSorter {$/;" c class:doris::SpillSorter file: +Tx src/gutil/utf/rune.c /^ Tx = ((1<<(Bitx+1))-1) ^ 0xFF, \/* 1000 0000 *\/$/;" e enum:__anon8 file: +TxnManager src/olap/txn_manager.cpp /^TxnManager::TxnManager() {$/;" f class:doris::TxnManager +TxnManager src/olap/txn_manager.h /^class TxnManager {$/;" c namespace:doris +TxnManagerTest test/olap/txn_manager_test.cpp /^class TxnManagerTest : public testing::Test {$/;" c namespace:doris file: +Type output/udf/include/udf.h /^ enum Type {$/;" g class:doris_udf::FunctionContext +Type src/udf/udf.h /^ enum Type {$/;" g class:doris_udf::FunctionContext +TypeDesc output/udf/include/udf.h /^ struct TypeDesc {$/;" s class:doris_udf::FunctionContext +TypeDesc src/udf/udf.h /^ struct TypeDesc {$/;" s class:doris_udf::FunctionContext +TypeDescriptor src/runtime/types.cpp /^TypeDescriptor::TypeDescriptor($/;" f class:doris::TypeDescriptor +TypeDescriptor src/runtime/types.cpp /^TypeDescriptor::TypeDescriptor(const std::vector& types, int* idx) : $/;" f class:doris::TypeDescriptor +TypeDescriptor src/runtime/types.h /^ TypeDescriptor() :$/;" f struct:doris::TypeDescriptor +TypeDescriptor src/runtime/types.h /^ TypeDescriptor(PrimitiveType type) :$/;" f struct:doris::TypeDescriptor +TypeDescriptor src/runtime/types.h /^struct TypeDescriptor {$/;" s namespace:doris +TypeEncodingTraits src/olap/rowset/segment_v2/encoding_info.cpp /^struct TypeEncodingTraits { };$/;" s namespace:doris::segment_v2 file: +TypeEncodingTraits src/olap/rowset/segment_v2/encoding_info.cpp /^struct TypeEncodingTraits {$/;" s namespace:doris::segment_v2 file: +TypeEncodingTraits src/olap/rowset/segment_v2/encoding_info.cpp /^struct TypeEncodingTraits::CppType> {$/;" s namespace:doris::segment_v2 file: +TypeEncodingTraits src/olap/rowset/segment_v2/encoding_info.cpp /^struct TypeEncodingTraits {$/;" s namespace:doris::segment_v2 file: +TypeEncodingTraits src/olap/rowset/segment_v2/encoding_info.cpp /^struct TypeEncodingTraits {$/;" s namespace:doris::segment_v2 file: +TypeEncodingTraits src/olap/rowset/segment_v2/encoding_info.cpp /^struct TypeEncodingTraits {$/;" s namespace:doris::segment_v2 file: +TypeEncodingTraits src/olap/rowset/segment_v2/encoding_info.cpp /^struct TypeEncodingTraits {$/;" s namespace:doris::segment_v2 file: +TypeInfo src/olap/types.cpp /^TypeInfo::TypeInfo(TypeTraitsClass t)$/;" f class:doris::TypeInfo +TypeInfo src/olap/types.h /^class TypeInfo {$/;" c namespace:doris +TypeInfoResolver src/olap/types.cpp /^TypeInfoResolver::TypeInfoResolver() {$/;" f class:doris::TypeInfoResolver +TypeInfoResolver src/olap/types.cpp /^class TypeInfoResolver {$/;" c namespace:doris file: +TypeTraits src/olap/types.h /^struct TypeTraits : public FieldTypeTraits {$/;" s namespace:doris +TypesTest test/olap/storage_types_test.cpp /^ TypesTest() { }$/;" f class:doris::TypesTest +TypesTest test/olap/storage_types_test.cpp /^class TypesTest : public testing::Test {$/;" c namespace:doris file: +TypesTest test/util/types_test.cpp /^ TypesTest() {$/;" f class:doris::TypesTest +TypesTest test/util/types_test.cpp /^class TypesTest : public ::testing::Test {$/;" c namespace:doris file: +UInt64ToString src/gutil/strings/numbers.cc /^string UInt64ToString(uint64 ui64, const char* format) {$/;" f +UNALIGNED_LOAD16 src/gutil/port.h /^inline uint16 UNALIGNED_LOAD16(const void *p) {$/;" f +UNALIGNED_LOAD16 src/gutil/port.h 1030;" d +UNALIGNED_LOAD16 src/gutil/port.h 1060;" d +UNALIGNED_LOAD32 src/gutil/port.h /^inline uint32 UNALIGNED_LOAD32(const void *p) {$/;" f +UNALIGNED_LOAD32 src/gutil/port.h 1031;" d +UNALIGNED_LOAD32 src/gutil/port.h 1061;" d +UNALIGNED_LOAD64 src/gutil/port.h /^inline uint64 UNALIGNED_LOAD64(const void *p) {$/;" f +UNALIGNED_LOAD64 src/gutil/port.h 1032;" d +UNALIGNED_LOADW src/gutil/port.h 1120;" d +UNALIGNED_LOADW src/gutil/port.h 1123;" d +UNALIGNED_STORE16 src/gutil/port.h /^inline void UNALIGNED_STORE16(void *p, uint16 v) {$/;" f +UNALIGNED_STORE16 src/gutil/port.h 1034;" d +UNALIGNED_STORE16 src/gutil/port.h 1063;" d +UNALIGNED_STORE32 src/gutil/port.h /^inline void UNALIGNED_STORE32(void *p, uint32 v) {$/;" f +UNALIGNED_STORE32 src/gutil/port.h 1035;" d +UNALIGNED_STORE32 src/gutil/port.h 1064;" d +UNALIGNED_STORE64 src/gutil/port.h /^inline void UNALIGNED_STORE64(void *p, uint64 v) {$/;" f +UNALIGNED_STORE64 src/gutil/port.h 1036;" d +UNALIGNED_STOREW src/gutil/port.h 1121;" d +UNALIGNED_STOREW src/gutil/port.h 1124;" d +UNAUTHORIZED src/http/http_status.h /^ UNAUTHORIZED = 401,$/;" e enum:doris::HttpStatus +UNCOMPRESSED src/exec/decompressor.h /^ UNCOMPRESSED,$/;" e enum:doris::CompressType +UNCOMPRESSED src/olap/out_stream.h /^ UNCOMPRESSED = 0,$/;" e enum:doris::StreamHead::StreamType +UNKNOWN src/http/http_method.h /^ UNKNOWN$/;" e enum:doris::HttpMethod +UNLIKELY src/common/compiler_util.h 33;" d +UNLIKELY src/common/compiler_util.h 37;" d +UNLIKELY src/util/thrift_util.cpp 38;" d file: +UNLIMITED_DEPTH src/runtime/mem_tracker.h /^ static const int UNLIMITED_DEPTH = INT_MAX;$/;" m class:doris::MemTracker +UNPIN_ALL src/runtime/buffered_tuple_stream3.h /^ UNPIN_ALL,$/;" e enum:doris::BufferedTupleStream3::UnpinMode +UNPIN_ALL_EXCEPT_CURRENT src/runtime/buffered_tuple_stream3.h /^ UNPIN_ALL_EXCEPT_CURRENT,$/;" e enum:doris::BufferedTupleStream3::UnpinMode +UNSUPPORTED_MEDIA_TYPE src/http/http_status.h /^ UNSUPPORTED_MEDIA_TYPE = 415,$/;" e enum:doris::HttpStatus +UNUSED_PREFIX src/olap/olap_define.h /^static const std::string UNUSED_PREFIX = "\/unused";$/;" m namespace:doris +UNZ_BADZIPFILE src/util/minizip/unzip.h 79;" d +UNZ_BUFSIZE src/util/minizip/unzip.c 104;" d file: +UNZ_CRCERROR src/util/minizip/unzip.h 81;" d +UNZ_END_OF_LIST_OF_FILE src/util/minizip/unzip.h 75;" d +UNZ_EOF src/util/minizip/unzip.h 77;" d +UNZ_ERRNO src/util/minizip/unzip.h 76;" d +UNZ_INTERNALERROR src/util/minizip/unzip.h 80;" d +UNZ_MAXFILENAMEINZIP src/util/minizip/unzip.c 108;" d file: +UNZ_OK src/util/minizip/unzip.h 74;" d +UNZ_PARAMERROR src/util/minizip/unzip.h 78;" d +UPDATE_TABLET_META_INFO src/agent/task_worker_pool.h /^ UPDATE_TABLET_META_INFO$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +UPGRADE src/http/http_headers.cpp /^const char* HttpHeaders::UPGRADE = "Upgrade";$/;" m class:doris::HttpHeaders file: +UPGRADE src/http/http_headers.h /^ static const char* UPGRADE;$/;" m class:doris::HttpHeaders +UPLOAD src/agent/task_worker_pool.h /^ UPLOAD,$/;" e enum:doris::TaskWorkerPool::TaskWorkerType +USERINFO src/util/url_parser.h /^ USERINFO$/;" e enum:doris::UrlParser::UrlPart +USER_AGENT src/http/http_headers.cpp /^const char* HttpHeaders::USER_AGENT = "User-Agent";$/;" m class:doris::HttpHeaders file: +USER_AGENT src/http/http_headers.h /^ static const char* USER_AGENT;$/;" m class:doris::HttpHeaders +USE_PROXY src/http/http_status.h /^ USE_PROXY = 305,$/;" e enum:doris::HttpStatus +USING_DORIS_UDF src/exprs/expr.h 41;" d +USING_DORIS_UDF src/exprs/expr.h 42;" d +USING_DORIS_UDF src/exprs/expr_context.h 28;" d +USING_DORIS_UDF src/exprs/expr_context.h 29;" d +UTFmax src/gutil/utf/utf.h /^ UTFmax = 4, \/* maximum bytes per rune *\/$/;" e enum:__anon7 +UTIL_ENDIAN_ENDIAN_H_ src/gutil/endian.h 31;" d +UTIL_GTL_CHARMAP_H_ src/gutil/charmap.h 17;" d +UTIL_GTL_MAP_UTIL_H_ src/gutil/map-util.h 63;" d +UTIL_GTL_STL_UTIL_H_ src/gutil/stl_util.h 29;" d +UTIL_HASH_BUILTIN_TYPE_HASH_H_ src/gutil/hash/builtin_type_hash.h 8;" d +UTIL_HASH_CITY_H_ src/gutil/hash/city.h 22;" d +UTIL_HASH_HASH128TO64_H_ src/gutil/hash/hash128to64.h 5;" d +UTIL_HASH_HASH_H_ src/gutil/hash/hash.h 74;" d +UTIL_HASH_JENKINS_H_ src/gutil/hash/jenkins.h 11;" d +UTIL_HASH_JENKINS_LOOKUP2_H_ src/gutil/hash/jenkins_lookup2.h 16;" d +UTIL_HASH_LEGACY_HASH_H_ src/gutil/hash/legacy_hash.h 11;" d +UTIL_HASH_STRING_HASH_H_ src/gutil/hash/string_hash.h 12;" d +UUIDGenerator src/util/uuid_generator.h /^class UUIDGenerator {$/;" c namespace:doris +UdaExecutionMode output/udf/include/uda_test_harness.h /^enum UdaExecutionMode {$/;" g namespace:doris_udf +UdaExecutionMode src/udf/uda_test_harness.h /^enum UdaExecutionMode {$/;" g namespace:doris_udf +UdaFinalize output/udf/include/udf.h /^typedef ResultType(*UdaFinalize)(FunctionContext* context, const IntermediateType& v);$/;" t namespace:doris_udf +UdaFinalize src/udf/udf.h /^typedef ResultType(*UdaFinalize)(FunctionContext* context, const IntermediateType& v);$/;" t namespace:doris_udf +UdaInit output/udf/include/udf.h /^typedef void (*UdaInit)(FunctionContext* context, IntermediateType* result);$/;" t namespace:doris_udf +UdaInit src/udf/udf.h /^typedef void (*UdaInit)(FunctionContext* context, IntermediateType* result);$/;" t namespace:doris_udf +UdaMerge output/udf/include/udf.h /^typedef void (*UdaMerge)(FunctionContext* context, const IntermediateType& src,$/;" t namespace:doris_udf +UdaMerge src/udf/udf.h /^typedef void (*UdaMerge)(FunctionContext* context, const IntermediateType& src,$/;" t namespace:doris_udf +UdaSerialize output/udf/include/udf.h /^typedef const IntermediateType(*UdaSerialize)(FunctionContext* context,$/;" t namespace:doris_udf +UdaSerialize src/udf/udf.h /^typedef const IntermediateType(*UdaSerialize)(FunctionContext* context,$/;" t namespace:doris_udf +UdaTestHarness output/udf/include/uda_test_harness.h /^ UdaTestHarness($/;" f class:doris_udf::UdaTestHarness +UdaTestHarness output/udf/include/uda_test_harness.h /^class UdaTestHarness : public UdaTestHarnessBase {$/;" c namespace:doris_udf +UdaTestHarness src/udf/uda_test_harness.h /^ UdaTestHarness($/;" f class:doris_udf::UdaTestHarness +UdaTestHarness src/udf/uda_test_harness.h /^class UdaTestHarness : public UdaTestHarnessBase {$/;" c namespace:doris_udf +UdaTestHarness2 output/udf/include/uda_test_harness.h /^ UdaTestHarness2($/;" f class:doris_udf::UdaTestHarness2 +UdaTestHarness2 output/udf/include/uda_test_harness.h /^class UdaTestHarness2 : public UdaTestHarnessBase {$/;" c namespace:doris_udf +UdaTestHarness2 src/udf/uda_test_harness.h /^ UdaTestHarness2($/;" f class:doris_udf::UdaTestHarness2 +UdaTestHarness2 src/udf/uda_test_harness.h /^class UdaTestHarness2 : public UdaTestHarnessBase {$/;" c namespace:doris_udf +UdaTestHarness3 output/udf/include/uda_test_harness.h /^ UdaTestHarness3($/;" f class:doris_udf::UdaTestHarness3 +UdaTestHarness3 output/udf/include/uda_test_harness.h /^class UdaTestHarness3 : public UdaTestHarnessBase {$/;" c namespace:doris_udf +UdaTestHarness3 src/udf/uda_test_harness.h /^ UdaTestHarness3($/;" f class:doris_udf::UdaTestHarness3 +UdaTestHarness3 src/udf/uda_test_harness.h /^class UdaTestHarness3 : public UdaTestHarnessBase {$/;" c namespace:doris_udf +UdaTestHarness4 output/udf/include/uda_test_harness.h /^ UdaTestHarness4($/;" f class:doris_udf::UdaTestHarness4 +UdaTestHarness4 output/udf/include/uda_test_harness.h /^class UdaTestHarness4 : public UdaTestHarnessBase {$/;" c namespace:doris_udf +UdaTestHarness4 src/udf/uda_test_harness.h /^ UdaTestHarness4($/;" f class:doris_udf::UdaTestHarness4 +UdaTestHarness4 src/udf/uda_test_harness.h /^class UdaTestHarness4 : public UdaTestHarnessBase {$/;" c namespace:doris_udf +UdaTestHarnessBase output/udf/include/uda_test_harness.h /^ UdaTestHarnessBase(InitFn init_fn, MergeFn merge_fn,$/;" f class:doris_udf::UdaTestHarnessBase +UdaTestHarnessBase output/udf/include/uda_test_harness.h /^class UdaTestHarnessBase {$/;" c namespace:doris_udf +UdaTestHarnessBase src/udf/uda_test_harness.h /^ UdaTestHarnessBase(InitFn init_fn, MergeFn merge_fn,$/;" f class:doris_udf::UdaTestHarnessBase +UdaTestHarnessBase src/udf/uda_test_harness.h /^class UdaTestHarnessBase {$/;" c namespace:doris_udf +UdaUpdate output/udf/include/udf.h /^typedef void (*UdaUpdate)(FunctionContext* context, const InputType& input,$/;" t namespace:doris_udf +UdaUpdate src/udf/udf.h /^typedef void (*UdaUpdate)(FunctionContext* context, const InputType& input,$/;" t namespace:doris_udf +UdaUpdate2 output/udf/include/udf.h /^typedef void (*UdaUpdate2)(FunctionContext* context, const InputType& input,$/;" t namespace:doris_udf +UdaUpdate2 src/udf/udf.h /^typedef void (*UdaUpdate2)(FunctionContext* context, const InputType& input,$/;" t namespace:doris_udf +UdfBuiltins src/exprs/udf_builtins.h /^class UdfBuiltins {$/;" c namespace:doris +UdfClose output/udf/include/udf.h /^typedef void (*UdfClose)(FunctionContext* context,$/;" t namespace:doris_udf +UdfClose src/udf/udf.h /^typedef void (*UdfClose)(FunctionContext* context,$/;" t namespace:doris_udf +UdfPrepare output/udf/include/udf.h /^typedef void (*UdfPrepare)(FunctionContext* context,$/;" t namespace:doris_udf +UdfPrepare src/udf/udf.h /^typedef void (*UdfPrepare)(FunctionContext* context,$/;" t namespace:doris_udf +UdfPrepareFn output/udf/include/udf.h /^typedef void (*UdfPrepareFn)(FunctionContext* context);$/;" t namespace:doris_udf +UdfPrepareFn src/udf/udf.h /^typedef void (*UdfPrepareFn)(FunctionContext* context);$/;" t namespace:doris_udf +UidUtilTest test/util/uid_util_test.cpp /^ UidUtilTest() { }$/;" f class:doris::UidUtilTest +UidUtilTest test/util/uid_util_test.cpp /^class UidUtilTest : public testing::Test {$/;" c namespace:doris file: +Uint128High64 src/gutil/int128.h /^inline uint64 Uint128High64(const uint128& v) { return v.hi_; }$/;" f +Uint128Low64 src/gutil/int128.h /^inline uint64 Uint128Low64(const uint128& v) { return v.lo_; }$/;" f +Uint128ToHexString src/gutil/strings/numbers.cc /^string Uint128ToHexString(uint128 ui128) {$/;" f +UnalignedCopy16 src/gutil/port.h /^inline void UnalignedCopy16(const void *src, void *dst) {$/;" f +UnalignedCopy32 src/gutil/port.h /^inline void UnalignedCopy32(const void *src, void *dst) {$/;" f +UnalignedCopy64 src/gutil/port.h /^inline void UnalignedCopy64(const void *src, void *dst) {$/;" f +UnaryOperate1st src/gutil/stl_util.h /^UnaryOperateOnFirst UnaryOperate1st(const UnaryOp& f) {$/;" f +UnaryOperate2nd src/gutil/stl_util.h /^UnaryOperateOnSecond UnaryOperate2nd(const UnaryOp& f) {$/;" f +UnaryOperateOnFirst src/gutil/stl_util.h /^ UnaryOperateOnFirst() {$/;" f class:UnaryOperateOnFirst +UnaryOperateOnFirst src/gutil/stl_util.h /^ UnaryOperateOnFirst(const UnaryOp& f) : f_(f) { \/\/ TODO(user): explicit?$/;" f class:UnaryOperateOnFirst +UnaryOperateOnFirst src/gutil/stl_util.h /^class UnaryOperateOnFirst$/;" c +UnaryOperateOnSecond src/gutil/stl_util.h /^ UnaryOperateOnSecond() {$/;" f class:UnaryOperateOnSecond +UnaryOperateOnSecond src/gutil/stl_util.h /^ UnaryOperateOnSecond(const UnaryOp& f) : f_(f) { \/\/ TODO(user): explicit?$/;" f class:UnaryOperateOnSecond +UnaryOperateOnSecond src/gutil/stl_util.h /^class UnaryOperateOnSecond$/;" c +UnescapeCEscapeSequences src/gutil/strings/escaping.cc /^int UnescapeCEscapeSequences(const char* source, char* dest) {$/;" f namespace:strings +UnescapeCEscapeSequences src/gutil/strings/escaping.cc /^int UnescapeCEscapeSequences(const char* source, char* dest,$/;" f namespace:strings +UnescapeCEscapeString src/gutil/strings/escaping.cc /^int UnescapeCEscapeString(const string& src, string* dest) {$/;" f namespace:strings +UnescapeCEscapeString src/gutil/strings/escaping.cc /^int UnescapeCEscapeString(const string& src, string* dest,$/;" f namespace:strings +UnescapeCEscapeString src/gutil/strings/escaping.cc /^string UnescapeCEscapeString(const string& src) {$/;" f namespace:strings +UnescapeFileName src/gutil/strings/escaping.cc /^void UnescapeFileName(const StringPiece& src_piece, string* dst) {$/;" f namespace:strings +UnescapeFileName src/gutil/strings/escaping.h /^inline string UnescapeFileName(const StringPiece& src) {$/;" f namespace:strings +UnflattenTupleRow src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::UnflattenTupleRow(uint8_t** data, TupleRow* row) const {$/;" f class:BufferedTupleStream3 +Uniform src/util/random.h /^ uint32_t Uniform(int n) { return Next() % n; }$/;" f class:doris::Random +UniformInsertString src/gutil/strings/util.cc /^void UniformInsertString(string* s, int interval, const char* separator) {$/;" f +Uninitialized src/common/status.h /^ static Status Uninitialized(const Slice& msg,$/;" f class:doris::Status +UnionIterator src/olap/generic_iterators.cpp /^ UnionIterator(std::vector iters)$/;" f class:doris::UnionIterator +UnionIterator src/olap/generic_iterators.cpp /^class UnionIterator : public RowwiseIterator {$/;" c namespace:doris file: +UnionMaterializeBatchFn src/exec/union_node.h /^ typedef void (*UnionMaterializeBatchFn)(UnionNode*, RowBatch*, uint8_t**);$/;" t class:doris::UnionNode +UnionNode src/exec/union_node.cpp /^UnionNode::UnionNode(ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::UnionNode +UnionNode src/exec/union_node.h /^class UnionNode : public ExecNode {$/;" c namespace:doris +UniqueId output/udf/include/udf.h /^ struct UniqueId {$/;" s class:doris_udf::FunctionContext +UniqueId src/udf/udf.h /^ struct UniqueId {$/;" s class:doris_udf::FunctionContext +UniqueId src/util/uid_util.h /^ UniqueId(const PUniqueId& puid) : hi(puid.hi()), lo(puid.lo()) { }$/;" f struct:doris::UniqueId +UniqueId src/util/uid_util.h /^ UniqueId(const TUniqueId& tuid) : hi(tuid.hi), lo(tuid.lo) { }$/;" f struct:doris::UniqueId +UniqueId src/util/uid_util.h /^ UniqueId(const std::string& hi_str, const std::string& lo_str) {$/;" f struct:doris::UniqueId +UniqueId src/util/uid_util.h /^ UniqueId(int64_t hi_, int64_t lo_) : hi(hi_), lo(lo_) { }$/;" f struct:doris::UniqueId +UniqueId src/util/uid_util.h /^struct UniqueId {$/;" s namespace:doris +UniqueIdEncodingMap src/olap/rowset/column_reader.h /^typedef std::map UniqueIdEncodingMap;$/;" t namespace:doris +UniqueIdPositionProviderMap src/olap/rowset/column_reader.h /^typedef std::unordered_map UniqueIdPositionProviderMap;$/;" t namespace:doris +UniqueIdSet src/olap/olap_common.h /^typedef std::set UniqueIdSet;$/;" t namespace:doris +UniqueIdToColumnIdMap src/olap/olap_common.h /^typedef std::map UniqueIdToColumnIdMap;$/;" t namespace:doris +UniqueRowsetIdGenerator src/olap/rowset/unique_rowset_id_generator.cpp /^UniqueRowsetIdGenerator::UniqueRowsetIdGenerator(const UniqueId& backend_uid) :$/;" f class:doris::UniqueRowsetIdGenerator +UniqueRowsetIdGenerator src/olap/rowset/unique_rowset_id_generator.h /^class UniqueRowsetIdGenerator : public RowsetIdGenerator {$/;" c namespace:doris +UniqueRowsetIdGeneratorTest test/olap/rowset/unique_rowset_id_generator_test.cpp /^ UniqueRowsetIdGeneratorTest() { }$/;" f class:doris::UniqueRowsetIdGeneratorTest +UniqueRowsetIdGeneratorTest test/olap/rowset/unique_rowset_id_generator_test.cpp /^class UniqueRowsetIdGeneratorTest : public testing::Test {$/;" c namespace:doris file: +UnixMicros src/util/time.h /^inline int64_t UnixMicros() {$/;" f namespace:doris +UnixMillis src/util/time.h /^inline int64_t UnixMillis() {$/;" f namespace:doris +Unpin src/runtime/bufferpool/buffer_pool.cc /^void BufferPool::Unpin(ClientHandle* client, PageHandle* handle) {$/;" f class:doris::BufferPool +UnpinBlocks test/runtime/buffered_block_mgr2_test.cpp /^ void UnpinBlocks(const vector& blocks) {$/;" f class:doris::BufferedBlockMgrTest +UnpinMode src/runtime/buffered_tuple_stream3.h /^ enum UnpinMode {$/;" g class:doris::BufferedTupleStream3 +UnpinPageIfNeeded src/runtime/buffered_tuple_stream3.cc /^void BufferedTupleStream3::UnpinPageIfNeeded(Page* page, bool stream_pinned) {$/;" f class:BufferedTupleStream3 +Unpoison src/runtime/bufferpool/buffer_pool.h /^ void Unpoison() { ASAN_UNPOISON_MEMORY_REGION(data(), len()); }$/;" f class:doris::BufferPool::BufferHandle +UnsetBit src/util/bit_util.h /^ constexpr static T UnsetBit(T v, int bitpos) {$/;" f class:doris::BitUtil +UnusedClass src/util/static_asserts.cpp /^class UnusedClass {$/;" c namespace:doris file: +Update src/exprs/new_agg_fn_evaluator.cc /^void NewAggFnEvaluator::Update(const TupleRow* row, Tuple* dst, void* fn) {$/;" f class:NewAggFnEvaluator +UpdateCounter src/util/runtime_profile.h /^ void UpdateCounter() {$/;" f class:doris::ScopedTimer +UpdateFn output/udf/include/uda_test_harness.h /^ typedef void (*UpdateFn)(FunctionContext* context, const INPUT& input,$/;" t class:doris_udf::UdaTestHarness +UpdateFn output/udf/include/uda_test_harness.h /^ typedef void (*UpdateFn)(FunctionContext* context, const INPUT1& input1,$/;" t class:doris_udf::UdaTestHarness2 +UpdateFn output/udf/include/uda_test_harness.h /^ typedef void (*UpdateFn)(FunctionContext* context, const INPUT1& input1,$/;" t class:doris_udf::UdaTestHarness3 +UpdateFn output/udf/include/uda_test_harness.h /^ typedef void (*UpdateFn)(FunctionContext* context, const INPUT1& input1,$/;" t class:doris_udf::UdaTestHarness4 +UpdateFn src/udf/uda_test_harness.h /^ typedef void (*UpdateFn)(FunctionContext* context, const INPUT& input,$/;" t class:doris_udf::UdaTestHarness +UpdateFn src/udf/uda_test_harness.h /^ typedef void (*UpdateFn)(FunctionContext* context, const INPUT1& input1,$/;" t class:doris_udf::UdaTestHarness2 +UpdateFn src/udf/uda_test_harness.h /^ typedef void (*UpdateFn)(FunctionContext* context, const INPUT1& input1,$/;" t class:doris_udf::UdaTestHarness3 +UpdateFn src/udf/uda_test_harness.h /^ typedef void (*UpdateFn)(FunctionContext* context, const INPUT1& input1,$/;" t class:doris_udf::UdaTestHarness4 +UpdateFn0 src/exprs/agg_fn_evaluator.cpp /^typedef void (*UpdateFn0)(FunctionContext*, AnyVal*);$/;" t namespace:doris file: +UpdateFn0 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*UpdateFn0)(FunctionContext*, AnyVal*);$/;" t file: +UpdateFn1 src/exprs/agg_fn_evaluator.cpp /^typedef void (*UpdateFn1)(FunctionContext*, const AnyVal&, AnyVal*);$/;" t namespace:doris file: +UpdateFn1 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*UpdateFn1)(FunctionContext*, const AnyVal&, AnyVal*);$/;" t file: +UpdateFn2 src/exprs/agg_fn_evaluator.cpp /^typedef void (*UpdateFn2)(FunctionContext*, const AnyVal&, const AnyVal&, AnyVal*);$/;" t namespace:doris file: +UpdateFn2 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*UpdateFn2)(FunctionContext*, const AnyVal&, const AnyVal&, AnyVal*);$/;" t file: +UpdateFn3 src/exprs/agg_fn_evaluator.cpp /^typedef void (*UpdateFn3)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t namespace:doris file: +UpdateFn3 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*UpdateFn3)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t file: +UpdateFn4 src/exprs/agg_fn_evaluator.cpp /^typedef void (*UpdateFn4)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t namespace:doris file: +UpdateFn4 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*UpdateFn4)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t file: +UpdateFn5 src/exprs/agg_fn_evaluator.cpp /^typedef void (*UpdateFn5)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t namespace:doris file: +UpdateFn5 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*UpdateFn5)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t file: +UpdateFn6 src/exprs/agg_fn_evaluator.cpp /^typedef void (*UpdateFn6)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t namespace:doris file: +UpdateFn6 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*UpdateFn6)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t file: +UpdateFn7 src/exprs/agg_fn_evaluator.cpp /^typedef void (*UpdateFn7)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t namespace:doris file: +UpdateFn7 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*UpdateFn7)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t file: +UpdateFn8 src/exprs/agg_fn_evaluator.cpp /^typedef void (*UpdateFn8)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t namespace:doris file: +UpdateFn8 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*UpdateFn8)(FunctionContext*, const AnyVal&, const AnyVal&,$/;" t file: +UpdateMax src/util/runtime_profile.h /^ void UpdateMax(int64_t v) {$/;" f class:doris::RuntimeProfile::HighWaterMarkCounter +UpdateReservation src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::UpdateReservation(int64_t delta) {$/;" f class:doris::ReservationTracker +UpdateReturnCopy src/gutil/map-util.h /^bool UpdateReturnCopy(Collection* const collection,$/;" f +UpdateTuple src/exec/new_partitioned_aggregation_node.cc /^void NewPartitionedAggregationNode::UpdateTuple(NewAggFnEvaluator** agg_fn_evals,$/;" f class:doris::NewPartitionedAggregationNode +UpdateUsedReservation src/runtime/bufferpool/reservation_tracker.cc /^void ReservationTracker::UpdateUsedReservation(int64_t delta) {$/;" f class:doris::ReservationTracker +UrlParser src/util/url_parser.h /^class UrlParser {$/;" c namespace:doris +UrlPart src/util/url_parser.h /^ enum UrlPart {$/;" g class:doris::UrlParser +UserFunctionCache src/runtime/user_function_cache.cpp /^UserFunctionCache::UserFunctionCache() {$/;" f class:doris::UserFunctionCache +UserFunctionCache src/runtime/user_function_cache.h /^class UserFunctionCache {$/;" c namespace:doris +UserFunctionCacheEntry src/runtime/user_function_cache.cpp /^ UserFunctionCacheEntry(int64_t fid_, const std::string& checksum_,$/;" f struct:doris::UserFunctionCacheEntry +UserFunctionCacheEntry src/runtime/user_function_cache.cpp /^struct UserFunctionCacheEntry {$/;" s namespace:doris file: +UserFunctionCacheTest test/runtime/user_function_cache_test.cpp /^ UserFunctionCacheTest() { }$/;" f class:doris::UserFunctionCacheTest +UserFunctionCacheTest test/runtime/user_function_cache_test.cpp /^class UserFunctionCacheTest : public testing::Test {$/;" c namespace:doris file: +UserFunctionTestHandler test/runtime/user_function_cache_test.cpp /^class UserFunctionTestHandler : public HttpHandler {$/;" c namespace:doris file: +UserResourceListener src/agent/user_resource_listener.cpp /^UserResourceListener::UserResourceListener(ExecEnv* exec_env, $/;" f class:doris::UserResourceListener +UserResourceListener src/agent/user_resource_listener.h /^class UserResourceListener : public TopicListener {$/;" c namespace:doris +Utf8CheckTest test/util/utf8_check_test.cpp /^ Utf8CheckTest() { }$/;" f class:doris::Utf8CheckTest +Utf8CheckTest test/util/utf8_check_test.cpp /^class Utf8CheckTest : public testing::Test {$/;" c namespace:doris file: +Utf8SafeCEscape src/gutil/strings/escaping.cc /^string Utf8SafeCEscape(const StringPiece& src) {$/;" f namespace:strings +Utf8SafeCEscapeString src/gutil/strings/escaping.cc /^int Utf8SafeCEscapeString(const char* src, int src_len, char* dest,$/;" f namespace:strings +Utf8SafeCHexEscape src/gutil/strings/escaping.cc /^string Utf8SafeCHexEscape(const StringPiece& src) {$/;" f namespace:strings +Utf8SafeCHexEscapeString src/gutil/strings/escaping.cc /^int Utf8SafeCHexEscapeString(const char* src, int src_len, char* dest,$/;" f namespace:strings +UtilityFunctions src/exprs/utility_functions.h /^class UtilityFunctions {$/;" c namespace:doris +V2_0 output/udf/include/udf.h /^ V2_0,$/;" e enum:doris_udf::FunctionContext::DorisVersion +V2_0 src/udf/udf.h /^ V2_0,$/;" e enum:doris_udf::FunctionContext::DorisVersion +VALGRIND_BRANCH_AND_LINK_TO_NOREDIR_R11 src/gutil/valgrind.h 337;" d +VALGRIND_BRANCH_AND_LINK_TO_NOREDIR_R11 src/gutil/valgrind.h 402;" d +VALGRIND_BRANCH_AND_LINK_TO_NOREDIR_R11 src/gutil/valgrind.h 474;" d +VALGRIND_BRANCH_AND_LINK_TO_NOREDIR_R11 src/gutil/valgrind.h 546;" d +VALGRIND_CALL_NOREDIR_EAX src/gutil/valgrind.h 223;" d +VALGRIND_CALL_NOREDIR_RAX src/gutil/valgrind.h 277;" d +VALGRIND_COUNT_ERRORS src/gutil/valgrind.h 3773;" d +VALGRIND_CREATE_MEMPOOL src/gutil/valgrind.h 3826;" d +VALGRIND_DESTROY_MEMPOOL src/gutil/valgrind.h 3834;" d +VALGRIND_DISCARD_TRANSLATIONS src/gutil/valgrind.h 3652;" d +VALGRIND_DO_CLIENT_REQUEST src/gutil/valgrind.h 131;" d +VALGRIND_DO_CLIENT_REQUEST src/gutil/valgrind.h 189;" d +VALGRIND_DO_CLIENT_REQUEST src/gutil/valgrind.h 243;" d +VALGRIND_DO_CLIENT_REQUEST src/gutil/valgrind.h 297;" d +VALGRIND_DO_CLIENT_REQUEST src/gutil/valgrind.h 358;" d +VALGRIND_DO_CLIENT_REQUEST src/gutil/valgrind.h 424;" d +VALGRIND_DO_CLIENT_REQUEST src/gutil/valgrind.h 496;" d +VALGRIND_FREELIKE_BLOCK src/gutil/valgrind.h 3818;" d +VALGRIND_GET_NR_CONTEXT src/gutil/valgrind.h 210;" d +VALGRIND_GET_NR_CONTEXT src/gutil/valgrind.h 264;" d +VALGRIND_GET_NR_CONTEXT src/gutil/valgrind.h 323;" d +VALGRIND_GET_NR_CONTEXT src/gutil/valgrind.h 381;" d +VALGRIND_GET_NR_CONTEXT src/gutil/valgrind.h 451;" d +VALGRIND_GET_NR_CONTEXT src/gutil/valgrind.h 523;" d +VALGRIND_GET_ORIG_FN src/gutil/valgrind.h 594;" d +VALGRIND_MALLOCLIKE_BLOCK src/gutil/valgrind.h 3807;" d +VALGRIND_MEMPOOL_ALLOC src/gutil/valgrind.h 3842;" d +VALGRIND_MEMPOOL_CHANGE src/gutil/valgrind.h 3874;" d +VALGRIND_MEMPOOL_EXISTS src/gutil/valgrind.h 3882;" d +VALGRIND_MEMPOOL_FREE src/gutil/valgrind.h 3850;" d +VALGRIND_MEMPOOL_TRIM src/gutil/valgrind.h 3858;" d +VALGRIND_MOVE_MEMPOOL src/gutil/valgrind.h 3866;" d +VALGRIND_NON_SIMD_CALL0 src/gutil/valgrind.h 3728;" d +VALGRIND_NON_SIMD_CALL1 src/gutil/valgrind.h 3738;" d +VALGRIND_NON_SIMD_CALL2 src/gutil/valgrind.h 3748;" d +VALGRIND_NON_SIMD_CALL3 src/gutil/valgrind.h 3758;" d +VALGRIND_PRINTF src/gutil/valgrind.h /^VALGRIND_PRINTF(const char *format, ...)$/;" f +VALGRIND_PRINTF src/gutil/valgrind.h 3665;" d +VALGRIND_PRINTF_BACKTRACE src/gutil/valgrind.h /^VALGRIND_PRINTF_BACKTRACE(const char *format, ...)$/;" f +VALGRIND_PRINTF_BACKTRACE src/gutil/valgrind.h 3666;" d +VALGRIND_STACK_CHANGE src/gutil/valgrind.h 3909;" d +VALGRIND_STACK_DEREGISTER src/gutil/valgrind.h 3901;" d +VALGRIND_STACK_REGISTER src/gutil/valgrind.h 3891;" d +VALIDATE_INTERVAL test/util/internal_queue_test.cpp /^const int VALIDATE_INTERVAL = 10000;$/;" m namespace:doris file: +VARNAME_LINENUM src/gutil/macros.h 202;" d +VARNAME_LINENUM_INTERNAL src/gutil/macros.h 203;" d +VARNAME_LINENUM_INTERNAL2 src/gutil/macros.h 204;" d +VARY src/http/http_headers.cpp /^const char* HttpHeaders::VARY = "Vary";$/;" m class:doris::HttpHeaders file: +VARY src/http/http_headers.h /^ static const char* VARY;$/;" m class:doris::HttpHeaders +VERSION_HASH src/http/action/checksum_action.cpp /^const std::string VERSION_HASH = "version_hash";$/;" m namespace:doris file: +VG_CONTRACT_FRAME_BY src/gutil/valgrind.h 2462;" d +VG_CONTRACT_FRAME_BY src/gutil/valgrind.h 3023;" d +VG_EXPAND_FRAME_BY_trashes_r3 src/gutil/valgrind.h 2457;" d +VG_EXPAND_FRAME_BY_trashes_r3 src/gutil/valgrind.h 3018;" d +VG_IS_TOOL_USERREQ src/gutil/valgrind.h 3581;" d +VG_USERREQ_TOOL_BASE src/gutil/valgrind.h 3579;" d +VG_USERREQ__CLIENT_CALL0 src/gutil/valgrind.h /^ VG_USERREQ__CLIENT_CALL0 = 0x1101,$/;" e enum:__anon15 +VG_USERREQ__CLIENT_CALL1 src/gutil/valgrind.h /^ VG_USERREQ__CLIENT_CALL1 = 0x1102,$/;" e enum:__anon15 +VG_USERREQ__CLIENT_CALL2 src/gutil/valgrind.h /^ VG_USERREQ__CLIENT_CALL2 = 0x1103,$/;" e enum:__anon15 +VG_USERREQ__CLIENT_CALL3 src/gutil/valgrind.h /^ VG_USERREQ__CLIENT_CALL3 = 0x1104,$/;" e enum:__anon15 +VG_USERREQ__COUNT_ERRORS src/gutil/valgrind.h /^ VG_USERREQ__COUNT_ERRORS = 0x1201,$/;" e enum:__anon15 +VG_USERREQ__CREATE_MEMPOOL src/gutil/valgrind.h /^ VG_USERREQ__CREATE_MEMPOOL = 0x1303,$/;" e enum:__anon15 +VG_USERREQ__DESTROY_MEMPOOL src/gutil/valgrind.h /^ VG_USERREQ__DESTROY_MEMPOOL = 0x1304,$/;" e enum:__anon15 +VG_USERREQ__DISCARD_TRANSLATIONS src/gutil/valgrind.h /^ VG_USERREQ__DISCARD_TRANSLATIONS = 0x1002,$/;" e enum:__anon15 +VG_USERREQ__FREELIKE_BLOCK src/gutil/valgrind.h /^ VG_USERREQ__FREELIKE_BLOCK = 0x1302,$/;" e enum:__anon15 +VG_USERREQ__MALLOCLIKE_BLOCK src/gutil/valgrind.h /^ VG_USERREQ__MALLOCLIKE_BLOCK = 0x1301,$/;" e enum:__anon15 +VG_USERREQ__MEMPOOL_ALLOC src/gutil/valgrind.h /^ VG_USERREQ__MEMPOOL_ALLOC = 0x1305,$/;" e enum:__anon15 +VG_USERREQ__MEMPOOL_CHANGE src/gutil/valgrind.h /^ VG_USERREQ__MEMPOOL_CHANGE = 0x1309,$/;" e enum:__anon15 +VG_USERREQ__MEMPOOL_EXISTS src/gutil/valgrind.h /^ VG_USERREQ__MEMPOOL_EXISTS = 0x130a,$/;" e enum:__anon15 +VG_USERREQ__MEMPOOL_FREE src/gutil/valgrind.h /^ VG_USERREQ__MEMPOOL_FREE = 0x1306,$/;" e enum:__anon15 +VG_USERREQ__MEMPOOL_TRIM src/gutil/valgrind.h /^ VG_USERREQ__MEMPOOL_TRIM = 0x1307,$/;" e enum:__anon15 +VG_USERREQ__MOVE_MEMPOOL src/gutil/valgrind.h /^ VG_USERREQ__MOVE_MEMPOOL = 0x1308,$/;" e enum:__anon15 +VG_USERREQ__PRINTF src/gutil/valgrind.h /^ VG_USERREQ__PRINTF = 0x1401,$/;" e enum:__anon15 +VG_USERREQ__PRINTF_BACKTRACE src/gutil/valgrind.h /^ VG_USERREQ__PRINTF_BACKTRACE = 0x1402,$/;" e enum:__anon15 +VG_USERREQ__RUNNING_ON_VALGRIND src/gutil/valgrind.h /^ enum { VG_USERREQ__RUNNING_ON_VALGRIND = 0x1001,$/;" e enum:__anon15 +VG_USERREQ__STACK_CHANGE src/gutil/valgrind.h /^ VG_USERREQ__STACK_CHANGE = 0x1503$/;" e enum:__anon15 +VG_USERREQ__STACK_DEREGISTER src/gutil/valgrind.h /^ VG_USERREQ__STACK_DEREGISTER = 0x1502,$/;" e enum:__anon15 +VG_USERREQ__STACK_REGISTER src/gutil/valgrind.h /^ VG_USERREQ__STACK_REGISTER = 0x1501,$/;" e enum:__anon15 +VIA src/http/http_headers.cpp /^const char* HttpHeaders::VIA = "Via";$/;" m class:doris::HttpHeaders file: +VIA src/http/http_headers.h /^ static const char* VIA;$/;" m class:doris::HttpHeaders +VLOG src/common/logging.h 35;" d +VLOG_CONNECTION src/common/logging.h 51;" d +VLOG_CONNECTION_IS_ON src/common/logging.h 58;" d +VLOG_ETL src/runtime/etl_job_mgr.cpp 40;" d file: +VLOG_EXPORT src/runtime/export_task_mgr.cpp 33;" d file: +VLOG_FILE src/common/logging.h 54;" d +VLOG_FILE_IS_ON src/common/logging.h 61;" d +VLOG_PROGRESS src/common/logging.h 56;" d +VLOG_PROGRESS_IS_ON src/common/logging.h 63;" d +VLOG_QUERY src/common/logging.h 53;" d +VLOG_QUERY_IS_ON src/common/logging.h 60;" d +VLOG_ROW src/common/logging.h 55;" d +VLOG_ROW_IS_ON src/common/logging.h 62;" d +VLOG_RPC src/common/logging.h 52;" d +VLOG_RPC_IS_ON src/common/logging.h 59;" d +ValgrindSlowdown src/gutil/dynamic_annotations.c /^double ValgrindSlowdown(void) {$/;" f +Valid src/olap/skiplist.h /^inline bool SkipList::Iterator::Valid() const {$/;" f class:doris::SkipList::Iterator +ValidBase32Byte src/gutil/strings/escaping.cc /^bool ValidBase32Byte(char c) {$/;" f namespace:strings +ValidateBlock test/runtime/buffered_block_mgr2_test.cpp /^ static void ValidateBlock(BufferedBlockMgr2::Block* block, int32_t data) {$/;" f class:doris::BufferedBlockMgrTest +ValidateCounter test/util/runtime_profile_test.cpp /^void ValidateCounter(RuntimeProfile* profile, const string& name, int64_t value) {$/;" f namespace:impala +ValidateMatch test/exec/partitioned_hash_table_test.cpp /^ void ValidateMatch(TupleRow* probe_row, TupleRow* build_row) {$/;" f class:doris::PartitionedHashTableTest +ValidateRandomSizeData test/runtime/buffered_block_mgr2_test.cpp /^ static void ValidateRandomSizeData(BufferedBlockMgr2::Block* block, int32_t size) {$/;" f class:doris::BufferedBlockMgrTest +ValidateRle test/util/rle_encoding_test.cpp /^void ValidateRle(const vector& values, int bit_width,$/;" f namespace:doris +Value src/util/crc32c.h /^inline uint32_t Value(const char* data, size_t n) {$/;" f namespace:doris::crc32c +Value src/util/crc32c.h /^inline uint32_t Value(const std::vector& slices) {$/;" f namespace:doris::crc32c +ValueDeleter src/gutil/stl_util.h /^ explicit ValueDeleter(STLContainer *ptr)$/;" f class:ValueDeleter +ValueDeleter src/gutil/stl_util.h /^class ValueDeleter {$/;" c +VarStringColumnReader src/olap/rowset/column_reader.h /^ VarStringColumnReader($/;" f class:doris::VarStringColumnReader +VarStringColumnReader src/olap/rowset/column_reader.h /^class VarStringColumnReader : public ColumnReader {$/;" c namespace:doris +VarStringColumnWriter src/olap/rowset/column_writer.cpp /^VarStringColumnWriter::VarStringColumnWriter($/;" f class:doris::VarStringColumnWriter +VarStringColumnWriter src/olap/rowset/column_writer.h /^class VarStringColumnWriter : public ColumnWriter {$/;" c namespace:doris +VarargUpdateFn0 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*VarargUpdateFn0)(FunctionContext*, int num_varargs, const AnyVal*, AnyVal*);$/;" t file: +VarargUpdateFn1 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*VarargUpdateFn1)(FunctionContext*, const AnyVal&, int num_varargs, const AnyVal*, AnyVal*);$/;" t file: +VarargUpdateFn2 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*VarargUpdateFn2)(FunctionContext*, const AnyVal&, const AnyVal&, int num_varargs,$/;" t file: +VarargUpdateFn3 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*VarargUpdateFn3)(FunctionContext*, const AnyVal&, const AnyVal&, const AnyVal&,$/;" t file: +VarargUpdateFn4 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*VarargUpdateFn4)(FunctionContext*, const AnyVal&, const AnyVal&, const AnyVal&,$/;" t file: +VarargUpdateFn5 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*VarargUpdateFn5)(FunctionContext*, const AnyVal&, const AnyVal&, const AnyVal&,$/;" t file: +VarargUpdateFn6 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*VarargUpdateFn6)(FunctionContext*, const AnyVal&, const AnyVal&, const AnyVal&,$/;" t file: +VarargUpdateFn7 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*VarargUpdateFn7)(FunctionContext*, const AnyVal&, const AnyVal&, const AnyVal&,$/;" t file: +VarargUpdateFn8 src/exprs/new_agg_fn_evaluator.cc /^typedef void (*VarargUpdateFn8)(FunctionContext*, const AnyVal&, const AnyVal&, const AnyVal&,$/;" t file: +VarcharField src/olap/field.h /^ explicit VarcharField(const TabletColumn& column) : Field(column) {$/;" f class:doris::VarcharField +VarcharField src/olap/field.h /^class VarcharField: public Field {$/;" c namespace:doris +VariableStruct src/exec/schema_scanner/schema_variables_scanner.h /^ struct VariableStruct {$/;" s class:doris::SchemaVariablesScanner +VectorComputeFn src/exprs/expr.h /^ typedef bool (*VectorComputeFn)(Expr*, VectorizedRowBatch*);$/;" t class:doris::Expr +VectorizedPositionInfo src/olap/rowset/segment_reader.h /^ struct VectorizedPositionInfo {$/;" s class:doris::SegmentReader +VectorizedRowBatch src/runtime/vectorized_row_batch.cpp /^VectorizedRowBatch::VectorizedRowBatch($/;" f class:doris::VectorizedRowBatch +VectorizedRowBatch src/runtime/vectorized_row_batch.h /^class VectorizedRowBatch {$/;" c namespace:doris +VerifyResults test/runtime/buffered_tuple_stream2_test.cpp /^ virtual void VerifyResults(const vector& results, int exp_rows,$/;" f class:doris::SimpleTupleStreamTest +VerifyResults test/runtime/buffered_tuple_stream2_test.cpp /^ virtual void VerifyResults(const vector& results, int exp_rows, bool gen_null) {$/;" f class:doris::SimpleTupleStreamTest +Version src/olap/olap_common.h /^ Version() : first(0), second(0) {}$/;" f struct:doris::Version +Version src/olap/olap_common.h /^ Version(int64_t first_, int64_t second_) : first(first_), second(second_) {}$/;" f struct:doris::Version +Version src/olap/olap_common.h /^struct Version {$/;" s namespace:doris +VersionHash src/olap/olap_common.h /^typedef int64_t VersionHash;$/;" t namespace:doris +Versions src/olap/olap_common.h /^typedef std::vector Versions;$/;" t namespace:doris +Vertex src/olap/olap_common.h /^struct Vertex {$/;" s namespace:doris +VeryLowMemTest test/exec/partitioned_hash_table_test.cpp /^ void VeryLowMemTest(bool quadratic) {$/;" f class:doris::PartitionedHashTableTest +Vg_ClientRequest src/gutil/valgrind.h /^ } Vg_ClientRequest;$/;" t typeref:enum:__anon15 +WARNING src/http/http_headers.cpp /^const char* HttpHeaders::WARNING = "Warning";$/;" m class:doris::HttpHeaders file: +WARNING src/http/http_headers.h /^ static const char* WARNING;$/;" m class:doris::HttpHeaders +WARN_IF_ERROR src/common/status.h 278;" d +WARN_UNUSED_RESULT src/common/status.h 296;" d +WARN_UNUSED_RESULT src/exec/exec_node.h /^ virtual Status QueryMaintenance(RuntimeState* state, const std::string& msg) WARN_UNUSED_RESULT;$/;" m class:doris::ExecNode +WARN_UNUSED_RESULT src/exprs/agg_fn.h /^ WARN_UNUSED_RESULT;$/;" m class:doris::AggFn +WARN_UNUSED_RESULT src/exprs/agg_fn.h /^ virtual Status Init(const RowDescriptor& desc, RuntimeState* state) WARN_UNUSED_RESULT;$/;" m class:doris::AggFn +WARN_UNUSED_RESULT src/exprs/new_agg_fn_evaluator.h /^ MemTracker* tracker, const RowDescriptor& row_desc) WARN_UNUSED_RESULT;$/;" m class:doris::NewAggFnEvaluator +WARN_UNUSED_RESULT src/exprs/new_agg_fn_evaluator.h /^ RuntimeState* state) WARN_UNUSED_RESULT;$/;" m class:doris::NewAggFnEvaluator +WARN_UNUSED_RESULT src/exprs/new_agg_fn_evaluator.h /^ const RowDescriptor& row_desc) WARN_UNUSED_RESULT;$/;" m class:doris::NewAggFnEvaluator +WARN_UNUSED_RESULT src/exprs/new_agg_fn_evaluator.h /^ Status Open(RuntimeState* state) WARN_UNUSED_RESULT;$/;" m class:doris::NewAggFnEvaluator +WARN_UNUSED_RESULT src/gutil/port.h 293;" d +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ RowBatch* batch, bool* eos, std::vector* flat_rows) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ bool delete_on_read, bool* got_reservation) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ bool* got_rows) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ int64_t row_size, bool* got_reservation) noexcept WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ Status GetNext(RowBatch* batch, bool* eos) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ Status Init(int node_id, bool pinned) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ Status NewWritePage(int64_t page_len) noexcept WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ Status NextReadPage() WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ Status PinPage(Page* page) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ Status PinPageIfNeeded(Page* page, bool stream_pinned) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ Status PinStream(bool* pinned) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ Status PrepareForRead(bool delete_on_read, bool* got_reservation) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ Status PrepareForReadInternal(bool delete_on_read) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ Status PrepareForWrite(bool* got_reservation) WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/buffered_tuple_stream3.h /^ bool AddRow(TupleRow* row, Status* status) noexcept WARN_UNUSED_RESULT;$/;" m class:doris::BufferedTupleStream3 +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_allocator.h /^ BufferPool::BufferHandle* buffer) WARN_UNUSED_RESULT;$/;" m struct:doris::BufferPool::BufferAllocator +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_allocator.h /^ int64_t len, BufferPool::BufferHandle* buffer) WARN_UNUSED_RESULT;$/;" m struct:doris::BufferPool::BufferAllocator +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool.h /^ ClientHandle* client) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool.h /^ ClientHandle* client, PageHandle* page_handle, BufferHandle* buffer_handle) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool.h /^ ClientHandle* client, int64_t len, BufferHandle* handle) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool.h /^ ClientHandle* dst_client, BufferHandle* dst) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool.h /^ const BufferHandle** buffer = nullptr) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool.h /^ Status DecreaseReservationTo(int64_t target_bytes) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool::ClientHandle +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool.h /^ Status GetBuffer(const BufferHandle** buffer_handle) const WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool::PageHandle +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool.h /^ Status Pin(ClientHandle* client, PageHandle* handle) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool.h /^ bool IncreaseReservation(int64_t bytes) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool::ClientHandle +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool.h /^ bool IncreaseReservationToFit(int64_t bytes) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool::ClientHandle +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool_internal.h /^ Status DecreaseReservationTo(int64_t target_bytes) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool::Client +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool_internal.h /^ Status PrepareToAllocateBuffer(int64_t len) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool::Client +WARN_UNUSED_RESULT src/runtime/bufferpool/buffer_pool_internal.h /^ Status StartMoveToPinned(ClientHandle* client, Page* page) WARN_UNUSED_RESULT;$/;" m class:doris::BufferPool::Client +WARN_UNUSED_RESULT src/runtime/bufferpool/reservation_tracker.h /^ bool IncreaseReservation(int64_t bytes) WARN_UNUSED_RESULT;$/;" m class:doris::ReservationTracker +WARN_UNUSED_RESULT src/runtime/bufferpool/reservation_tracker.h /^ bool IncreaseReservationToFit(int64_t bytes) WARN_UNUSED_RESULT;$/;" m class:doris::ReservationTracker +WARN_UNUSED_RESULT src/runtime/bufferpool/reservation_tracker.h /^ bool TransferReservationTo(ReservationTracker* other, int64_t bytes) WARN_UNUSED_RESULT;$/;" m class:doris::ReservationTracker +WARN_UNUSED_RESULT src/runtime/bufferpool/system_allocator.h /^ Status Allocate(int64_t len, BufferPool::BufferHandle* buffer) WARN_UNUSED_RESULT;$/;" m class:doris::SystemAllocator +WARN_UNUSED_RESULT src/runtime/initial_reservations.h /^ const TUniqueId& query_id, int64_t query_min_reservation) WARN_UNUSED_RESULT;$/;" m class:doris::InitialReservations +WEBSOCKET_LOCATION src/http/http_headers.cpp /^const char* HttpHeaders::WEBSOCKET_LOCATION = "WebSocket-Location";$/;" m class:doris::HttpHeaders file: +WEBSOCKET_LOCATION src/http/http_headers.h /^ static const char* WEBSOCKET_LOCATION;$/;" m class:doris::HttpHeaders +WEBSOCKET_ORIGIN src/http/http_headers.cpp /^const char* HttpHeaders::WEBSOCKET_ORIGIN = "WebSocket-Origin";$/;" m class:doris::HttpHeaders file: +WEBSOCKET_ORIGIN src/http/http_headers.h /^ static const char* WEBSOCKET_ORIGIN;$/;" m class:doris::HttpHeaders +WEBSOCKET_PROTOCOL src/http/http_headers.cpp /^const char* HttpHeaders::WEBSOCKET_PROTOCOL = "WebSocket-Protocol";$/;" m class:doris::HttpHeaders file: +WEBSOCKET_PROTOCOL src/http/http_headers.h /^ static const char* WEBSOCKET_PROTOCOL;$/;" m class:doris::HttpHeaders +WEEK src/runtime/datetime_value.h /^ WEEK,$/;" e enum:doris::TimeUnit +WEEK_FIRST_WEEKDAY src/runtime/datetime_value.h /^const int WEEK_FIRST_WEEKDAY = 4;$/;" m namespace:doris +WEEK_MONDAY_FIRST src/runtime/datetime_value.h /^const int WEEK_MONDAY_FIRST = 1;$/;" m namespace:doris +WEEK_YEAR src/runtime/datetime_value.h /^const int WEEK_YEAR = 2;$/;" m namespace:doris +WINDOW_BITS src/exec/decompressor.h /^ const static int WINDOW_BITS = 15; \/\/ Maximum window size $/;" m class:doris::GzipDecompressor +WORD_HASH src/gutil/hash/jenkins_lookup2.h 154;" d +WORST_UNCOMPRESSED_SLOP src/olap/rowset/segment_reader.h /^ static const int32_t WORST_UNCOMPRESSED_SLOP = 2 + 8 * 512;$/;" m class:doris::SegmentReader +WRITE src/runtime/disk_io_mgr.h /^ WRITE,$/;" e enum:doris::DiskIoMgr::RequestType::type +WRITER_CREATED src/olap/rowset/alpha_rowset_writer.h /^ WRITER_CREATED,$/;" e enum:doris::WriterState +WRITER_FLUSHED src/olap/rowset/alpha_rowset_writer.h /^ WRITER_FLUSHED$/;" e enum:doris::WriterState +WRITER_INITED src/olap/rowset/alpha_rowset_writer.h /^ WRITER_INITED,$/;" e enum:doris::WriterState +WRITE_CHECK_INTERVAL_MILLIS test/runtime/buffered_block_mgr2_test.cpp /^const static int WRITE_CHECK_INTERVAL_MILLIS = 10;$/;" v file: +WRITE_WAIT_MILLIS test/runtime/buffered_block_mgr2_test.cpp /^const static int WRITE_WAIT_MILLIS = 500;$/;" v file: +WWW_AUTHENTICATE src/http/http_headers.cpp /^const char* HttpHeaders::WWW_AUTHENTICATE = "WWW-Authenticate";$/;" m class:doris::HttpHeaders file: +WWW_AUTHENTICATE src/http/http_headers.h /^ static const char* WWW_AUTHENTICATE;$/;" m class:doris::HttpHeaders +WaitForWrites test/runtime/buffered_block_mgr2_test.cpp /^ static void WaitForWrites(BufferedBlockMgr2* block_mgr) {$/;" f class:doris::BufferedBlockMgrTest +WaitForWrites test/runtime/buffered_block_mgr2_test.cpp /^ static void WaitForWrites(const vector& block_mgrs) {$/;" f class:doris::BufferedBlockMgrTest +WallTime src/gutil/walltime.h /^typedef double WallTime;$/;" t +WarnEnumCastError src/gutil/casts.h /^inline void WarnEnumCastError(int value_of_int) {$/;" f namespace:base::internal +WarnIfFileNotEqual src/util/cpu_info.cpp /^void WarnIfFileNotEqual($/;" f namespace:doris +WeakHashLen32WithSeeds src/gutil/hash/city.cc /^static pair WeakHashLen32WithSeeds($/;" f namespace:util_hash +WebPageHandler src/http/web_page_handler.cpp /^WebPageHandler::WebPageHandler(EvHttpServer* server) :$/;" f class:doris::WebPageHandler +WebPageHandler src/http/web_page_handler.h /^class WebPageHandler : public HttpHandler {$/;" c namespace:doris +WebSafeBase64Escape src/gutil/strings/escaping.cc /^int WebSafeBase64Escape(const unsigned char *src, int szsrc, char *dest,$/;" f namespace:strings +WebSafeBase64Escape src/gutil/strings/escaping.cc /^void WebSafeBase64Escape(const string& src, string* dest) {$/;" f namespace:strings +WebSafeBase64Escape src/gutil/strings/escaping.cc /^void WebSafeBase64Escape(const unsigned char *src, int szsrc,$/;" f namespace:strings +WebSafeBase64EscapeWithPadding src/gutil/strings/escaping.cc /^void WebSafeBase64EscapeWithPadding(const string& src, string* dest) {$/;" f namespace:strings +WebSafeBase64Unescape src/gutil/strings/escaping.cc /^bool WebSafeBase64Unescape(const char *src, int slen, string* dest) {$/;" f namespace:strings +WebSafeBase64Unescape src/gutil/strings/escaping.cc /^int WebSafeBase64Unescape(const char *src, int szsrc, char *dest, int szdest) {$/;" f namespace:strings +WebSafeBase64Unescape src/gutil/strings/escaping.h /^inline bool WebSafeBase64Unescape(const string& src, string* dest) {$/;" f namespace:strings +WildCardQueryBuilder src/exec/es/es_query_builder.cpp /^WildCardQueryBuilder::WildCardQueryBuilder(const ExtLikePredicate& like_predicate) : _field(like_predicate.col.name) {$/;" f class:doris::WildCardQueryBuilder +WildCardQueryBuilder src/exec/es/es_query_builder.h /^class WildCardQueryBuilder : public QueryBuilder {$/;" c namespace:doris +WktParse src/geo/wkt_parse.h /^class WktParse {$/;" c namespace:doris +WktParseContext src/geo/wkt_parse_ctx.h /^struct WktParseContext {$/;" s +WktParseTest test/geo/wkt_parse_test.cpp /^ WktParseTest() { }$/;" f class:doris::WktParseTest +WktParseTest test/geo/wkt_parse_test.cpp /^class WktParseTest : public testing::Test {$/;" c namespace:doris file: +Word src/gutil/strings/charset.h /^ static int Word(unsigned char c) { return c >> 6; }$/;" f class:strings::CharSet +Word32At src/gutil/hash/jenkins_lookup2.h /^static inline uint32 Word32At(const char *ptr) {$/;" f +Word64At src/gutil/hash/jenkins_lookup2.h /^static inline uint64 Word64At(const char *ptr) {$/;" f +WorkFunction src/util/priority_thread_pool.hpp /^ typedef boost::function WorkFunction;$/;" t class:doris::PriorityThreadPool +WrapperField src/olap/wrapper_field.cpp /^WrapperField::WrapperField(Field* rep, size_t variable_len, bool is_string_type)$/;" f class:doris::WrapperField +WrapperField src/olap/wrapper_field.h /^class WrapperField {$/;" c namespace:doris +WritableBlock src/olap/fs/block_manager.h /^class WritableBlock : public Block {$/;" c namespace:doris::fs +WritableFile src/env/env.h /^ WritableFile() { }$/;" f class:doris::WritableFile +WritableFile src/env/env.h /^class WritableFile {$/;" c namespace:doris +WritableFileOptions src/env/env.h /^struct WritableFileOptions {$/;" s namespace:doris +WriteDoneCallback src/runtime/disk_io_mgr.h /^ typedef boost::function WriteDoneCallback;$/;" t class:doris::DiskIoMgr::WriteRange +WriteLock src/util/mutex.h /^ explicit WriteLock(RWMutex* mutex, bool try_lock = false)$/;" f class:doris::WriteLock +WriteLock src/util/mutex.h /^class WriteLock {$/;" c namespace:doris +WriteRange src/runtime/disk_io_mgr.cc /^DiskIoMgr::WriteRange::WriteRange(const string& file, int64_t file_offset, int disk_id,$/;" f class:doris::DiskIoMgr::WriteRange +WriteRange src/runtime/disk_io_mgr.h /^ class WriteRange : public RequestRange {$/;" c class:doris::DiskIoMgr +WriteRequest src/olap/delta_writer.h /^struct WriteRequest {$/;" s namespace:doris +WriteType src/olap/delta_writer.h /^enum WriteType {$/;" g namespace:doris +WriterState src/olap/rowset/alpha_rowset_writer.h /^enum WriterState {$/;" g namespace:doris +XorFinalize test/udf/uda_test.cpp /^BigIntVal XorFinalize(FunctionContext* context, const BigIntVal& val) {$/;" f namespace:doris_udf +XorInit test/udf/uda_test.cpp /^void XorInit(FunctionContext* context, BigIntVal* val) {$/;" f namespace:doris_udf +XorMerge test/udf/uda_test.cpp /^void XorMerge(FunctionContext* context, const BigIntVal& src, BigIntVal* dst) {$/;" f namespace:doris_udf +XorUpdate test/udf/uda_test.cpp /^void XorUpdate(FunctionContext* context, const double* input, BigIntVal* val) {$/;" f namespace:doris_udf +YEAR src/runtime/datetime_value.h /^ YEAR,$/;" e enum:doris::TimeUnit +YEAR_MONTH src/runtime/datetime_value.h /^ YEAR_MONTH$/;" e enum:doris::TimeUnit +YYSTYPE src/geo/wkt_parse.cpp 24;" d file: +YY_EXTRA_TYPE src/geo/wkt_parse.cpp 25;" d file: +YY_PART_YEAR src/runtime/datetime_value.h /^const int YY_PART_YEAR = 70;$/;" m namespace:doris +YesType src/gutil/template_util.h /^typedef small_ YesType;$/;" t namespace:base +ZCALLBACK src/util/minizip/ioapi.h 126;" d +ZCALLBACK src/util/minizip/ioapi.h 128;" d +ZCLOSE64 src/util/minizip/ioapi.h 191;" d +ZERROR64 src/util/minizip/ioapi.h 192;" d +ZLIB_FILEFUNC_MODE_CREATE src/util/minizip/ioapi.h 121;" d +ZLIB_FILEFUNC_MODE_EXISTING src/util/minizip/ioapi.h 120;" d +ZLIB_FILEFUNC_MODE_READ src/util/minizip/ioapi.h 116;" d +ZLIB_FILEFUNC_MODE_READWRITEFILTER src/util/minizip/ioapi.h 118;" d +ZLIB_FILEFUNC_MODE_WRITE src/util/minizip/ioapi.h 117;" d +ZLIB_FILEFUNC_SEEK_CUR src/util/minizip/ioapi.h 112;" d +ZLIB_FILEFUNC_SEEK_END src/util/minizip/ioapi.h 113;" d +ZLIB_FILEFUNC_SEEK_SET src/util/minizip/ioapi.h 114;" d +ZOPEN64 src/util/minizip/ioapi.h 200;" d +ZPOS64_T src/util/minizip/ioapi.h /^typedef 64BIT_INT_CUSTOM_TYPE ZPOS64_T;$/;" t +ZPOS64_T src/util/minizip/ioapi.h /^typedef uint64_t ZPOS64_T;$/;" t +ZPOS64_T src/util/minizip/ioapi.h /^typedef unsigned __int64 ZPOS64_T;$/;" t +ZPOS64_T src/util/minizip/ioapi.h /^typedef unsigned long long int ZPOS64_T;$/;" t +ZREAD64 src/util/minizip/ioapi.h 187;" d +ZSEEK64 src/util/minizip/ioapi.h 202;" d +ZTELL64 src/util/minizip/ioapi.h 201;" d +ZWRITE64 src/util/minizip/ioapi.h 188;" d +Z_BZIP2ED src/util/minizip/unzip.h 62;" d +ZipFile src/util/zip_util.h /^ ZipFile(const std::string& zip_path) : _zip_path(zip_path), _zip_file(nullptr), _open_current_file(false) {}$/;" f class:doris::ZipFile +ZipFile src/util/zip_util.h /^class ZipFile {$/;" c namespace:doris +ZlibBlockCompression src/util/block_compression.cpp /^class ZlibBlockCompression : public BlockCompressionCodec {$/;" c namespace:doris file: +ZoneMap src/olap/rowset/segment_v2/zone_map_index.h /^struct ZoneMap {$/;" s namespace:doris::segment_v2 +ZoneMapIndexReader src/olap/rowset/segment_v2/zone_map_index.h /^ explicit ZoneMapIndexReader(const std::string& filename, const ZoneMapIndexPB* index_meta) :$/;" f class:doris::segment_v2::ZoneMapIndexReader +ZoneMapIndexReader src/olap/rowset/segment_v2/zone_map_index.h /^class ZoneMapIndexReader {$/;" c namespace:doris::segment_v2 +ZoneMapIndexWriter src/olap/rowset/segment_v2/zone_map_index.cpp /^ZoneMapIndexWriter::ZoneMapIndexWriter(Field* field) : _field(field), _pool(&_tracker) {$/;" f class:doris::segment_v2::ZoneMapIndexWriter +ZoneMapIndexWriter src/olap/rowset/segment_v2/zone_map_index.h /^class ZoneMapIndexWriter {$/;" c namespace:doris::segment_v2 +_BASE_STRINGPRINTF_H src/gutil/stringprintf.h 11;" d +_BITS_H_ src/gutil/bits.h 12;" d +_CRT_SECURE_NO_WARNINGS src/util/minizip/ioapi.c 14;" d file: +_FILE_OFFSET_BIT src/util/minizip/ioapi.h 39;" d +_FILE_OFFSET_BITS src/exec/local_file_reader.h 20;" d +_FixedFileHeader src/olap/file_helper.h /^typedef struct _FixedFileHeader {$/;" s namespace:doris +_FixedFileHeaderV2 src/olap/file_helper.h /^typedef struct _FixedFileHeaderV2 {$/;" s namespace:doris +_LARGEFILE64_SOURCE src/util/minizip/ioapi.h 36;" d +_LOGGING_IN_H_ src/gutil/logging-inl.h 26;" d +_LSS_BODY src/gutil/linux_syscall_support.h 1305;" d +_LSS_BODY src/gutil/linux_syscall_support.h 1306;" d +_LSS_RETURN src/gutil/linux_syscall_support.h 1293;" d +_LSS_RETURN src/gutil/linux_syscall_support.h 1294;" d +_M_allocated_capacity src/gutil/stl_util.h /^ size_t _M_allocated_capacity;$/;" m union:InternalStringRepGCC4::__anon26 +_M_data src/gutil/stl_util.h /^ char* _M_data;$/;" m struct:InternalStringRepGCC4 +_M_local_data src/gutil/stl_util.h /^ char _M_local_data[_S_local_capacity + 1];$/;" m union:InternalStringRepGCC4::__anon26 +_M_string_length src/gutil/stl_util.h /^ size_t _M_string_length;$/;" m struct:InternalStringRepGCC4 +_SYSINFO_H_ src/gutil/sysinfo.h 32;" d +_S_local_capacity src/gutil/stl_util.h /^ enum { _S_local_capacity = 15 };$/;" e enum:InternalStringRepGCC4::__anon25 +_UTFH_ src/gutil/utf/utf.h 2;" d +_XOPEN_SOURCE src/common/logging.h 41;" d +_ZLIBIOAPI64_H src/util/minizip/ioapi.h 22;" d +__BIG_ENDIAN src/gutil/port.h 99;" d +__BYTE_ORDER src/gutil/port.h 107;" d +__BYTE_ORDER src/gutil/port.h 97;" d +__CALLER_SAVED_REGS src/gutil/valgrind.h 1019;" d +__CALLER_SAVED_REGS src/gutil/valgrind.h 1478;" d +__CALLER_SAVED_REGS src/gutil/valgrind.h 1924;" d +__CALLER_SAVED_REGS src/gutil/valgrind.h 2448;" d +__CALLER_SAVED_REGS src/gutil/valgrind.h 3009;" d +__CALLER_SAVED_REGS src/gutil/valgrind.h 621;" d +__DECLARE_PLUGIN src/plugin/plugin.h 65;" d +__DYNAMIC_ANNOTATIONS_H__ src/gutil/dynamic_annotations.h 58;" d +__DorisMysql src/exec/mysql_scanner.cpp 20;" d file: +__DorisMysql src/exec/mysql_scanner.h 29;" d +__DorisMysql src/runtime/mysql_table_writer.cpp 20;" d file: +__DorisMysql src/runtime/mysql_table_writer.h 27;" d +__DorisMysql src/util/mysql_load_error_hub.cpp 20;" d file: +__DorisMysql src/util/mysql_load_error_hub.h 30;" d +__DorisMysqlRes src/exec/mysql_scanner.cpp 21;" d file: +__DorisMysqlRes src/exec/mysql_scanner.h 33;" d +__IN_CONFIGBASE_CPP__ src/common/configbase.cpp 25;" d file: +__IN_CONFIGBASE_CPP__ src/common/configbase.cpp 27;" d file: +__LITTLE_ENDIAN src/gutil/port.h 98;" d +__NR__exit src/gutil/linux_syscall_support.h 2464;" d +__NR__gettid src/gutil/linux_syscall_support.h 2465;" d +__NR__mremap src/gutil/linux_syscall_support.h 2466;" d +__NR__sigaction src/gutil/linux_syscall_support.h 2663;" d +__NR__sigprocmask src/gutil/linux_syscall_support.h 2664;" d +__NR_clock_getres src/gutil/linux_syscall_support.h 829;" d +__NR_clock_gettime src/gutil/linux_syscall_support.h 826;" d +__NR_fadvise64 src/gutil/linux_syscall_support.h 882;" d +__NR_fadvise64_64 src/gutil/linux_syscall_support.h 934;" d +__NR_fallocate src/gutil/linux_syscall_support.h 856;" d +__NR_fstat64 src/gutil/linux_syscall_support.h 586;" d +__NR_fstat64 src/gutil/linux_syscall_support.h 630;" d +__NR_fstat64 src/gutil/linux_syscall_support.h 666;" d +__NR_fstat64 src/gutil/linux_syscall_support.h 732;" d +__NR_fstatat src/gutil/linux_syscall_support.h 681;" d +__NR_fstatat src/gutil/linux_syscall_support.h 698;" d +__NR_fstatat src/gutil/linux_syscall_support.h 715;" d +__NR_fstatat src/gutil/linux_syscall_support.h 755;" d +__NR_fstatat64 src/gutil/linux_syscall_support.h 937;" d +__NR_fstatfs64 src/gutil/linux_syscall_support.h 835;" d +__NR_futex src/gutil/linux_syscall_support.h 595;" d +__NR_futex src/gutil/linux_syscall_support.h 639;" d +__NR_futex src/gutil/linux_syscall_support.h 647;" d +__NR_futex src/gutil/linux_syscall_support.h 675;" d +__NR_futex src/gutil/linux_syscall_support.h 692;" d +__NR_futex src/gutil/linux_syscall_support.h 709;" d +__NR_futex src/gutil/linux_syscall_support.h 744;" d +__NR_futex src/gutil/linux_syscall_support.h 814;" d +__NR_getcpu src/gutil/linux_syscall_support.h 601;" d +__NR_getcpu src/gutil/linux_syscall_support.h 684;" d +__NR_getcpu src/gutil/linux_syscall_support.h 701;" d +__NR_getcpu src/gutil/linux_syscall_support.h 718;" d +__NR_getcpu src/gutil/linux_syscall_support.h 750;" d +__NR_getcpu src/gutil/linux_syscall_support.h 853;" d +__NR_getdents64 src/gutil/linux_syscall_support.h 589;" d +__NR_getdents64 src/gutil/linux_syscall_support.h 633;" d +__NR_getdents64 src/gutil/linux_syscall_support.h 650;" d +__NR_getdents64 src/gutil/linux_syscall_support.h 669;" d +__NR_getdents64 src/gutil/linux_syscall_support.h 738;" d +__NR_getdents64 src/gutil/linux_syscall_support.h 784;" d +__NR_getresgid src/gutil/linux_syscall_support.h 873;" d +__NR_getresgid src/gutil/linux_syscall_support.h 907;" d +__NR_getresgid32 src/gutil/linux_syscall_support.h 925;" d +__NR_getresuid src/gutil/linux_syscall_support.h 867;" d +__NR_getresuid src/gutil/linux_syscall_support.h 901;" d +__NR_getresuid32 src/gutil/linux_syscall_support.h 919;" d +__NR_getrlimit src/gutil/linux_syscall_support.h 861;" d +__NR_getrlimit src/gutil/linux_syscall_support.h 889;" d +__NR_gettid src/gutil/linux_syscall_support.h 592;" d +__NR_gettid src/gutil/linux_syscall_support.h 636;" d +__NR_gettid src/gutil/linux_syscall_support.h 644;" d +__NR_gettid src/gutil/linux_syscall_support.h 672;" d +__NR_gettid src/gutil/linux_syscall_support.h 689;" d +__NR_gettid src/gutil/linux_syscall_support.h 706;" d +__NR_gettid src/gutil/linux_syscall_support.h 741;" d +__NR_gettid src/gutil/linux_syscall_support.h 808;" d +__NR_getxattr src/gutil/linux_syscall_support.h 796;" d +__NR_ioprio_get src/gutil/linux_syscall_support.h 841;" d +__NR_ioprio_set src/gutil/linux_syscall_support.h 838;" d +__NR_lgetxattr src/gutil/linux_syscall_support.h 799;" d +__NR_listxattr src/gutil/linux_syscall_support.h 802;" d +__NR_llistxattr src/gutil/linux_syscall_support.h 805;" d +__NR_lsetxattr src/gutil/linux_syscall_support.h 793;" d +__NR_mmap2 src/gutil/linux_syscall_support.h 913;" d +__NR_move_pages src/gutil/linux_syscall_support.h 850;" d +__NR_newfstatat src/gutil/linux_syscall_support.h 885;" d +__NR_openat src/gutil/linux_syscall_support.h 598;" d +__NR_openat src/gutil/linux_syscall_support.h 653;" d +__NR_openat src/gutil/linux_syscall_support.h 678;" d +__NR_openat src/gutil/linux_syscall_support.h 695;" d +__NR_openat src/gutil/linux_syscall_support.h 712;" d +__NR_openat src/gutil/linux_syscall_support.h 747;" d +__NR_openat src/gutil/linux_syscall_support.h 844;" d +__NR_pread64 src/gutil/linux_syscall_support.h 778;" d +__NR_pwrite64 src/gutil/linux_syscall_support.h 781;" d +__NR_quotactl src/gutil/linux_syscall_support.h 760;" d +__NR_readahead src/gutil/linux_syscall_support.h 787;" d +__NR_rt_sigaction src/gutil/linux_syscall_support.h 579;" d +__NR_rt_sigaction src/gutil/linux_syscall_support.h 623;" d +__NR_rt_sigaction src/gutil/linux_syscall_support.h 659;" d +__NR_rt_sigaction src/gutil/linux_syscall_support.h 725;" d +__NR_rt_sigaction src/gutil/linux_syscall_support.h 766;" d +__NR_rt_sigpending src/gutil/linux_syscall_support.h 772;" d +__NR_rt_sigprocmask src/gutil/linux_syscall_support.h 580;" d +__NR_rt_sigprocmask src/gutil/linux_syscall_support.h 624;" d +__NR_rt_sigprocmask src/gutil/linux_syscall_support.h 660;" d +__NR_rt_sigprocmask src/gutil/linux_syscall_support.h 726;" d +__NR_rt_sigprocmask src/gutil/linux_syscall_support.h 769;" d +__NR_rt_sigreturn src/gutil/linux_syscall_support.h 763;" d +__NR_rt_sigsuspend src/gutil/linux_syscall_support.h 775;" d +__NR_sched_getaffinity src/gutil/linux_syscall_support.h 820;" d +__NR_sched_setaffinity src/gutil/linux_syscall_support.h 817;" d +__NR_set_tid_address src/gutil/linux_syscall_support.h 823;" d +__NR_setfsgid src/gutil/linux_syscall_support.h 879;" d +__NR_setfsgid src/gutil/linux_syscall_support.h 895;" d +__NR_setfsgid32 src/gutil/linux_syscall_support.h 931;" d +__NR_setfsuid src/gutil/linux_syscall_support.h 876;" d +__NR_setfsuid src/gutil/linux_syscall_support.h 892;" d +__NR_setfsuid32 src/gutil/linux_syscall_support.h 928;" d +__NR_setresgid src/gutil/linux_syscall_support.h 870;" d +__NR_setresgid src/gutil/linux_syscall_support.h 904;" d +__NR_setresgid32 src/gutil/linux_syscall_support.h 922;" d +__NR_setresuid src/gutil/linux_syscall_support.h 864;" d +__NR_setresuid src/gutil/linux_syscall_support.h 898;" d +__NR_setresuid32 src/gutil/linux_syscall_support.h 916;" d +__NR_setxattr src/gutil/linux_syscall_support.h 790;" d +__NR_socket src/gutil/linux_syscall_support.h 735;" d +__NR_stat64 src/gutil/linux_syscall_support.h 583;" d +__NR_stat64 src/gutil/linux_syscall_support.h 627;" d +__NR_stat64 src/gutil/linux_syscall_support.h 663;" d +__NR_stat64 src/gutil/linux_syscall_support.h 729;" d +__NR_statfs64 src/gutil/linux_syscall_support.h 832;" d +__NR_tkill src/gutil/linux_syscall_support.h 811;" d +__NR_ugetrlimit src/gutil/linux_syscall_support.h 910;" d +__NR_unlinkat src/gutil/linux_syscall_support.h 847;" d +__PRIS_PREFIX src/gutil/port.h 1156;" d +__PRIS_PREFIX src/gutil/port.h 1158;" d +__SPECIAL_INSTRUCTION_PREAMBLE src/gutil/valgrind.h 185;" d +__SPECIAL_INSTRUCTION_PREAMBLE src/gutil/valgrind.h 239;" d +__SPECIAL_INSTRUCTION_PREAMBLE src/gutil/valgrind.h 293;" d +__SPECIAL_INSTRUCTION_PREAMBLE src/gutil/valgrind.h 354;" d +__SPECIAL_INSTRUCTION_PREAMBLE src/gutil/valgrind.h 420;" d +__SPECIAL_INSTRUCTION_PREAMBLE src/gutil/valgrind.h 492;" d +__STD src/gutil/port.h 1009;" d +__STDC_FORMAT_MACROS src/gutil/port.h 29;" d +__SYS_REG src/gutil/linux_syscall_support.h 607;" d +__SYS_REG src/gutil/linux_syscall_support.h 616;" d +__SYS_REG_LIST src/gutil/linux_syscall_support.h 608;" d +__SYS_REG_LIST src/gutil/linux_syscall_support.h 617;" d +__USE_FILE_OFFSET64 src/util/minizip/ioapi.h 30;" d +__USE_LARGEFILE64 src/util/minizip/ioapi.h 33;" d +__VALGRIND_H src/gutil/valgrind.h 74;" d +__cpuid src/gutil/cpu.cc /^void __cpuid(int cpu_info[4], int info_type) {$/;" f namespace:base::__anon16 +__extension__ src/gutil/valgrind.h 3632;" d +__gnu_cxx src/gutil/hash/hash.cc /^namespace __gnu_cxx {$/;" n file: +__gnu_cxx src/gutil/hash/hash.h /^namespace __gnu_cxx {$/;" n +__pad0 src/gutil/linux_syscall_support.h /^ unsigned __pad0;$/;" m struct:kernel_stat +__pad0 src/gutil/linux_syscall_support.h /^ unsigned __pad0[3];$/;" m struct:kernel_stat +__pad0 src/gutil/linux_syscall_support.h /^ unsigned char __pad0[4];$/;" m struct:kernel_stat64 +__pad1 src/gutil/linux_syscall_support.h /^ unsigned __pad1[3];$/;" m struct:kernel_stat +__pad1 src/gutil/linux_syscall_support.h /^ unsigned int __pad1;$/;" m struct:kernel_stat +__pad1 src/gutil/linux_syscall_support.h /^ unsigned long __pad1;$/;" m struct:kernel_stat +__pad1 src/gutil/linux_syscall_support.h /^ unsigned short __pad1;$/;" m struct:kernel_stat +__pad2 src/gutil/linux_syscall_support.h /^ int __pad2;$/;" m struct:kernel_stat +__pad2 src/gutil/linux_syscall_support.h /^ int __pad2;$/;" m struct:kernel_stat64 +__pad2 src/gutil/linux_syscall_support.h /^ unsigned __pad2;$/;" m struct:kernel_stat +__pad2 src/gutil/linux_syscall_support.h /^ unsigned short __pad2;$/;" m struct:kernel_stat +__pad3 src/gutil/linux_syscall_support.h /^ unsigned char __pad3[4];$/;" m struct:kernel_stat64 +__ptr_t src/gutil/port.h 221;" d +__ptr_t src/gutil/port.h 958;" d +__st_ino src/gutil/linux_syscall_support.h /^ unsigned __st_ino;$/;" m struct:kernel_stat64 +__sync_val_compare_and_swap src/gutil/atomicops-internals-x86.h /^inline Atomic64 __sync_val_compare_and_swap(volatile Atomic64* ptr,$/;" f namespace:base::subtle +__syscall src/gutil/linux_syscall_support.h 609;" d +__syscall src/gutil/linux_syscall_support.h 618;" d +__syscall_safe src/gutil/linux_syscall_support.h 610;" d +__syscall_safe src/gutil/linux_syscall_support.h 619;" d +__unused src/gutil/linux_syscall_support.h /^ int64_t __unused[3];$/;" m struct:kernel_stat +__unused src/gutil/linux_syscall_support.h /^ unsigned long __unused[3];$/;" m struct:kernel_stat +__unused4 src/gutil/linux_syscall_support.h /^ unsigned __unused4;$/;" m struct:kernel_stat +__unused4 src/gutil/linux_syscall_support.h /^ unsigned int __unused4;$/;" m struct:kernel_stat +__unused4 src/gutil/linux_syscall_support.h /^ unsigned long __unused4;$/;" m struct:kernel_stat +__unused4 src/gutil/linux_syscall_support.h /^ unsigned long __unused4;$/;" m struct:kernel_stat64 +__unused5 src/gutil/linux_syscall_support.h /^ unsigned __unused5;$/;" m struct:kernel_stat +__unused5 src/gutil/linux_syscall_support.h /^ unsigned int __unused5;$/;" m struct:kernel_stat +__unused5 src/gutil/linux_syscall_support.h /^ unsigned long __unused5;$/;" m struct:kernel_stat +__unused5 src/gutil/linux_syscall_support.h /^ unsigned long __unused5;$/;" m struct:kernel_stat64 +__unused6 src/gutil/linux_syscall_support.h /^ unsigned long __unused6;$/;" m struct:kernel_stat +__unused6 src/gutil/linux_syscall_support.h /^ unsigned long __unused6;$/;" m struct:kernel_stat64 +_abfd src/util/bfd_parser.h /^ bfd* _abfd;$/;" m class:doris::BfdParser +_abort_on_default_limit_exceeded src/exec/topn_node.h /^ bool _abort_on_default_limit_exceeded;$/;" m class:doris::TopNNode +_accumulated_mem_consumption src/exprs/agg_fn_evaluator.h /^ uint64_t _accumulated_mem_consumption;$/;" m class:doris::AggFnEvaluator +_accumulated_mem_consumption src/exprs/new_agg_fn_evaluator.h /^ uint64_t _accumulated_mem_consumption;$/;" m class:doris::NewAggFnEvaluator +_active_contexts src/runtime/external_scan_context_mgr.h /^ std::map> _active_contexts;$/;" m class:doris::ExternalScanContextMgr +_active_read_thread_counter src/runtime/disk_io_mgr_internal.h /^ RuntimeProfile::Counter* _active_read_thread_counter;$/;" m class:doris::DiskIoMgr::RequestContext +_active_threads src/util/threadpool.h /^ int _active_threads;$/;" m class:doris::ThreadPool +_active_threads src/util/threadpool.h /^ int _active_threads;$/;" m class:doris::ThreadPoolToken +_add_alter_task src/olap/schema_change.cpp /^OLAPStatus SchemaChangeHandler::_add_alter_task($/;" f class:doris::SchemaChangeHandler +_add_batch_closure src/exec/tablet_sink.h /^ RefCountClosure* _add_batch_closure = nullptr;$/;" m class:doris::stream_load::NodeChannel +_add_batch_request src/exec/tablet_sink.h /^ PTabletWriterAddBatchRequest _add_batch_request;$/;" m class:doris::stream_load::NodeChannel +_add_batch_timer src/runtime/dpp_sink.cpp /^ RuntimeProfile::Counter* _add_batch_timer;$/;" m class:doris::Translator file: +_add_map src/olap/rowset/segment_v2/encoding_info.cpp /^ void _add_map() {$/;" f class:doris::segment_v2::EncodingInfoResolver file: +_add_row src/olap/rowset/alpha_rowset_writer.cpp /^OLAPStatus AlphaRowsetWriter::_add_row(const RowType& row) {$/;" f class:doris::AlphaRowsetWriter +_add_row src/olap/rowset/beta_rowset_writer.cpp /^OLAPStatus BetaRowsetWriter::_add_row(const RowType& row) {$/;" f class:doris::BetaRowsetWriter +_add_segment src/olap/rowset/column_data_writer.cpp /^OLAPStatus ColumnDataWriter::_add_segment() {$/;" f class:doris::ColumnDataWriter +_add_tablet_to_map_unlocked src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::_add_tablet_to_map_unlocked(TTabletId tablet_id, SchemaHash schema_hash,$/;" f class:doris::TabletManager +_add_tablet_to_partition src/olap/tablet_manager.cpp /^void TabletManager::_add_tablet_to_partition(const Tablet& tablet) {$/;" f class:doris::TabletManager +_add_tablet_unlocked src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::_add_tablet_unlocked(TTabletId tablet_id, SchemaHash schema_hash,$/;" f class:doris::TabletManager +_add_vertex_to_graph src/olap/rowset_graph.cpp /^OLAPStatus RowsetGraph::_add_vertex_to_graph(int64_t vertex_value) {$/;" f class:doris::RowsetGraph +_addr_idx src/exec/broker_reader.h /^ int _addr_idx;$/;" m class:doris::BrokerReader +_addr_idx src/exec/broker_writer.h /^ int _addr_idx;$/;" m class:doris::BrokerWriter +_address src/util/cidr.h /^ uint32_t _address;$/;" m class:doris::CIDR +_addresses src/exec/broker_reader.h /^ const std::vector& _addresses;$/;" m class:doris::BrokerReader +_addresses src/exec/broker_writer.h /^ const std::vector& _addresses;$/;" m class:doris::BrokerWriter +_addresses src/exec/es_scan_node.h /^ std::vector _addresses;$/;" m class:doris::EsScanNode +_addresses test/exec/broker_reader_test.cpp /^ std::vector _addresses;$/;" m class:doris::BrokerReaderTest file: +_addresses test/exec/broker_scanner_test.cpp /^ std::vector _addresses;$/;" m class:doris::BrokerScannerTest file: +_addresses test/exec/orc_scanner_test.cpp /^ std::vector _addresses;$/;" m class:doris::OrcScannerTest file: +_adj_deltas src/olap/rowset/run_length_integer_writer.h /^ int64_t _adj_deltas[MAX_SCOPE - 1]; \/\/ for delta encoding$/;" m class:doris::RunLengthIntegerWriter +_adler_checksum src/olap/push_handler.h /^ uint32_t _adler_checksum;$/;" m class:doris::IBinaryReader +_aes_key test/util/aes_util_test.cpp /^ std::string _aes_key;$/;" m class:doris::AesUtilTest file: +_agent_server src/service/backend_service.h /^ std::unique_ptr _agent_server;$/;" m class:doris::BackendService +_agent_utils src/agent/task_worker_pool.h /^ std::unique_ptr _agent_utils;$/;" m class:doris::TaskWorkerPool +_agg_fn_ctxs src/exec/aggregation_node.h /^ std::vector _agg_fn_ctxs;$/;" m class:doris::AggregationNode +_agg_fn_ctxs src/exec/partitioned_aggregation_node.h /^ std::vector _agg_fn_ctxs;$/;" m class:doris::PartitionedAggregationNode +_agg_fn_pool src/exec/aggregation_node.h /^ boost::scoped_ptr _agg_fn_pool;$/;" m class:doris::AggregationNode +_agg_fn_pool src/exec/partitioned_aggregation_node.h /^ boost::scoped_ptr _agg_fn_pool;$/;" m class:doris::PartitionedAggregationNode +_agg_info src/olap/field.h /^ const AggregateInfo* _agg_info;$/;" m class:doris::Field +_agg_key_next_row src/olap/reader.cpp /^OLAPStatus Reader::_agg_key_next_row(RowCursor* row_cursor, MemPool* mem_pool, ObjectPool* agg_pool, bool* eof) {$/;" f class:doris::Reader +_agg_method src/olap/aggregate_func.h /^ FieldAggregationMethod _agg_method;$/;" m class:doris::AggregateInfo +_agg_object_pool src/olap/memtable.h /^ ObjectPool _agg_object_pool;$/;" m class:doris::MemTable +_agg_object_pool src/runtime/row_batch.h /^ std::unique_ptr _agg_object_pool;$/;" m class:doris::RowBatch +_agg_op src/exprs/agg_fn_evaluator.h /^ AggregationOp _agg_op;$/;" m class:doris::AggFnEvaluator +_agg_rate_latch src/exec/pre_aggregation_node.h /^ int _agg_rate_latch;$/;" m class:doris::PreAggregationNode +_agg_record_latch src/exec/pre_aggregation_node.h /^ int _agg_record_latch;$/;" m class:doris::PreAggregationNode +_agg_record_num src/exec/pre_aggregation_node.h /^ int _agg_record_num;$/;" m class:doris::PreAggregationNode +_agg_record_num_sum src/exec/pre_aggregation_node.h /^ int _agg_record_num_sum;$/;" m class:doris::PreAggregationNode +_agg_timer src/runtime/dpp_sink.cpp /^ RuntimeProfile::Counter* _agg_timer;$/;" m class:doris::Translator file: +_aggregate_evaluators src/exec/aggregation_node.h /^ std::vector _aggregate_evaluators;$/;" m class:doris::AggregationNode +_aggregate_evaluators src/exec/partitioned_aggregation_node.h /^ std::vector _aggregate_evaluators;$/;" m class:doris::PartitionedAggregationNode +_aggregate_exprs src/exec/pre_aggregation_node.h /^ std::vector _aggregate_exprs;$/;" m class:doris::PreAggregationNode +_aggregate_two_row src/olap/memtable.cpp /^void MemTable::_aggregate_two_row(const ContiguousRow& src_row, TableKey row_in_skiplist) {$/;" f class:doris::MemTable +_aggregated_partitions src/exec/partitioned_aggregation_node.h /^ std::list _aggregated_partitions;$/;" m class:doris::PartitionedAggregationNode +_aggregation src/exec/olap_scanner.h /^ bool _aggregation;$/;" m class:doris::OlapScanner +_aggregation src/olap/reader.h /^ bool _aggregation;$/;" m class:doris::Reader +_aggregation src/olap/tablet_schema.h /^ FieldAggregationMethod _aggregation;$/;" m class:doris::TabletColumn +_all_check_paths src/olap/data_dir.h /^ std::set _all_check_paths;$/;" m class:doris::DataDir +_all_contexts src/runtime/disk_io_mgr.cc /^ list _all_contexts;$/;" m class:doris::DiskIoMgr::RequestContextCache file: +_all_io_buffers src/runtime/buffered_block_mgr2.h /^ std::list _all_io_buffers;$/;" m class:doris::BufferedBlockMgr2 +_all_num_rows src/olap/rowset/column_data_writer.h /^ int64_t _all_num_rows;$/;" m class:doris::ColumnDataWriter +_all_scanners_finished src/exec/broker_scan_node.h /^ bool _all_scanners_finished;$/;" m class:doris::BrokerScanNode +_all_scanners_finished src/exec/es_http_scan_node.h /^ bool _all_scanners_finished;$/;" m class:doris::EsHttpScanNode +_all_slot_ids src/exec/repeat_node.h /^ std::set _all_slot_ids;$/;" m class:doris::RepeatNode +_all_tablets src/exec/tablet_sink.h /^ std::vector _all_tablets;$/;" m class:doris::stream_load::NodeChannel +_all_trackers src/runtime/mem_tracker.h /^ std::vector _all_trackers; \/\/ this tracker plus all of its ancestors$/;" m class:doris::MemTracker +_allocations src/udf/udf_internal.h /^ std::map _allocations;$/;" m class:doris::FunctionContextImpl +_allocator src/util/path_trie.hpp /^ std::allocator _allocator;$/;" m class:doris::PathTrie::TrieNode +_allocator src/util/path_trie.hpp /^ std::allocator _allocator;$/;" m class:doris::PathTrie +_allocators src/util/core_local.h /^ std::map, CoreDataAllocator*> _allocators;$/;" m class:doris::CoreDataAllocatorFactory +_allocators src/util/core_local.h /^ std::vector _allocators;$/;" m class:doris::CoreLocalValueController +_allow_paths src/http/download_action.h /^ std::vector _allow_paths;$/;" m class:doris::DownloadAction +_alpha_rowset test/olap/txn_manager_test.cpp /^ RowsetSharedPtr _alpha_rowset;$/;" m class:doris::TxnManagerTest file: +_alpha_rowset_diff_id test/olap/txn_manager_test.cpp /^ RowsetSharedPtr _alpha_rowset_diff_id;$/;" m class:doris::TxnManagerTest file: +_alpha_rowset_meta src/olap/rowset/alpha_rowset_reader.h /^ AlphaRowsetMeta* _alpha_rowset_meta;$/;" m class:doris::AlphaRowsetReader +_alpha_rowset_same_id test/olap/txn_manager_test.cpp /^ RowsetSharedPtr _alpha_rowset_same_id;$/;" m class:doris::TxnManagerTest file: +_alpha_rowset_writer test/olap/rowset/alpha_rowset_test.cpp /^ std::unique_ptr _alpha_rowset_writer;$/;" m class:doris::AlphaRowsetTest file: +_already_built src/olap/rowset/beta_rowset_writer.h /^ bool _already_built = false;$/;" m class:doris::BetaRowsetWriter +_already_failed src/exec/tablet_sink.h /^ bool _already_failed = false;$/;" m class:doris::stream_load::NodeChannel +_alter_state src/olap/tablet_meta.h /^ AlterTabletState _alter_state;$/;" m class:doris::AlterTabletTask +_alter_tablet src/agent/task_worker_pool.cpp /^void TaskWorkerPool::_alter_tablet($/;" f class:doris::TaskWorkerPool +_alter_tablet_req src/olap/task/engine_alter_tablet_task.h /^ const TAlterTabletReqV2& _alter_tablet_req;$/;" m class:doris::EngineAlterTabletTask +_alter_tablet_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_alter_tablet_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_alter_tablet_workers src/agent/agent_server.h /^ std::unique_ptr _alter_tablet_workers;$/;" m class:doris::AgentServer +_alter_task src/olap/tablet_meta.h /^ AlterTabletTaskSharedPtr _alter_task;$/;" m class:doris::TabletMeta +_alter_type src/olap/tablet_meta.h /^ AlterTabletType _alter_type;$/;" m class:doris::AlterTabletTask +_anti_join_last_pos src/exec/hash_join_node.h /^ HashTable::Iterator* _anti_join_last_pos;$/;" m class:doris::HashJoinNode +_append src/runtime/stream_load/stream_load_pipe.h /^ Status _append(const ByteBufferPtr& buf) {$/;" f class:doris::StreamLoadPipe +_append_data src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::_append_data(const uint8_t** ptr, size_t num_rows) {$/;" f class:doris::segment_v2::ColumnWriter +_apply_bitmap_index src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_apply_bitmap_index() {$/;" f class:doris::segment_v2::SegmentIterator +_arenas src/runtime/memory/chunk_allocator.h /^ std::vector> _arenas;$/;" m class:doris::ChunkAllocator +_arg_types src/udf/udf_internal.h /^ std::vector _arg_types;$/;" m class:doris::FunctionContextImpl +_array src/olap/byte_buffer.h /^ char* _array;$/;" m class:doris::StorageByteBuffer +_arrays src/util/arrow/row_batch.cpp /^ std::vector> _arrays;$/;" m class:doris::FromRowBatchConverter file: +_arrays src/util/arrow/row_block.cpp /^ std::vector> _arrays;$/;" m class:doris::FromRowBlockConverter file: +_arrow_schema src/runtime/memory_scratch_sink.h /^ std::shared_ptr _arrow_schema;$/;" m class:doris::MemoryScratchSink +_assemble_file_name src/runtime/snapshot_loader.cpp /^void SnapshotLoader::_assemble_file_name($/;" f class:doris::SnapshotLoader +_assure_data src/olap/file_stream.cpp /^OLAPStatus ReadOnlyFileStream::_assure_data() {$/;" f class:doris::ReadOnlyFileStream +_assure_data src/olap/in_stream.cpp /^OLAPStatus InStream::_assure_data() {$/;" f class:doris::InStream +_auto_unregister src/runtime/mem_tracker.h /^ bool _auto_unregister;$/;" m class:doris::MemTracker +_auxiliary_mem_usage src/runtime/row_batch.h /^ int64_t _auxiliary_mem_usage;$/;" m class:doris::RowBatch +_available src/util/block_compression.cpp /^ size_t _available;$/;" m class:doris::SnappySlicesSource file: +_available_bytes src/olap/data_dir.h /^ int64_t _available_bytes;$/;" m class:doris::DataDir +_available_storage_medium_type_count src/olap/storage_engine.h /^ uint32_t _available_storage_medium_type_count;$/;" m class:doris::StorageEngine +_average_thread_tokens src/runtime/plan_fragment_executor.h /^ RuntimeProfile::Counter* _average_thread_tokens;$/;" m class:doris::PlanFragmentExecutor +_backend src/agent/task_worker_pool.h /^ TBackend _backend;$/;" m class:doris::TaskWorkerPool +_backend_client_cache src/runtime/exec_env.h /^ ClientCache* _backend_client_cache = nullptr;$/;" m class:doris::ExecEnv +_backend_num src/runtime/fragment_mgr.cpp /^ int _backend_num;$/;" m class:doris::FragmentExecState file: +_backend_uid src/olap/rowset/unique_rowset_id_generator.h /^ UniqueId _backend_uid;$/;" m class:doris::UniqueRowsetIdGenerator +_bad_agg_latch src/exec/pre_aggregation_node.h /^ int _bad_agg_latch;$/;" m class:doris::PreAggregationNode +_bad_agg_num src/exec/pre_aggregation_node.h /^ int _bad_agg_num;$/;" m class:doris::PreAggregationNode +_base_compaction_thread_callback src/olap/olap_server.cpp /^void* StorageEngine::_base_compaction_thread_callback(void* arg, DataDir* data_dir) {$/;" f class:doris::StorageEngine +_base_compaction_threads src/olap/storage_engine.h /^ std::vector _base_compaction_threads;$/;" m class:doris::StorageEngine +_base_lock src/olap/tablet.h /^ Mutex _base_lock;$/;" m class:doris::Tablet +_base_reduced_literals src/olap/rowset/run_length_integer_writer.h /^ int64_t _base_reduced_literals[MAX_SCOPE]; \/\/ for for patched base encoding$/;" m class:doris::RunLengthIntegerWriter +_batch src/exec/orc_scanner.h /^ std::shared_ptr _batch;$/;" m class:doris::ORCScanner +_batch src/exec/parquet_reader.h /^ std::shared_ptr _batch;$/;" m class:doris::ParquetReaderWrap +_batch src/exec/tablet_sink.h /^ std::unique_ptr _batch;$/;" m class:doris::stream_load::NodeChannel +_batch src/runtime/data_stream_sender.cpp /^ boost::scoped_ptr _batch;$/;" m class:doris::DataStreamSender::Channel file: +_batch src/util/arrow/row_batch.cpp /^ const RowBatch& _batch;$/;" m class:doris::FromRowBatchConverter file: +_batch src/util/arrow/row_batch.cpp /^ const arrow::RecordBatch& _batch;$/;" m class:doris::ToRowBatchConverter file: +_batch src/util/arrow/row_block.cpp /^ const arrow::RecordBatch& _batch;$/;" m class:doris::ToRowBlockConverter file: +_batch test/runtime/data_stream_test.cpp /^ scoped_ptr _batch;$/;" m class:doris::DataStreamTest file: +_batch_eof src/exec/es_http_scanner.h /^ bool _batch_eof;$/;" m class:doris::EsHttpScanner +_batch_it src/exec/row_batch_list.h /^ std::vector::iterator _batch_it;$/;" m class:doris::RowBatchList::TupleRowIterator +_batch_map src/runtime/data_spliter.h /^ std::unordered_map _batch_map;$/;" m class:doris::DataSpliter +_batch_queue src/exec/broker_scan_node.h /^ std::deque> _batch_queue;$/;" m class:doris::BrokerScanNode +_batch_queue src/exec/es_http_scan_node.h /^ std::deque> _batch_queue;$/;" m class:doris::EsHttpScanNode +_batch_queue src/runtime/buffer_control_block.h /^ ResultQueue _batch_queue;$/;" m class:doris::BufferControlBlock +_batch_queue src/runtime/data_stream_recvr.cc /^ RowBatchQueue _batch_queue;$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_batch_queue_lock src/exec/broker_scan_node.h /^ std::mutex _batch_queue_lock;$/;" m class:doris::BrokerScanNode +_batch_queue_lock src/exec/es_http_scan_node.h /^ std::mutex _batch_queue_lock;$/;" m class:doris::EsHttpScanNode +_batch_size src/exec/es/es_scan_reader.h /^ int _batch_size;$/;" m class:doris::ESScanReader +_batch_size src/util/batch_process_thread_pool.hpp /^ uint32_t _batch_size;$/;" m class:doris::BatchProcessThreadPool +_batch_to_write src/runtime/dpp_sink.cpp /^ std::unique_ptr _batch_to_write;$/;" m class:doris::Translator file: +_be_number src/runtime/data_stream_sender.cpp /^ int _be_number;$/;" m class:doris::DataStreamSender::Channel file: +_be_number src/runtime/runtime_state.h /^ int _be_number;$/;" m class:doris::RuntimeState +_begin src/exec/csv_scan_node.cpp /^ char const* _begin;$/;" m class:doris::StringRef file: +_begin src/exec/schema_scanner/schema_variables_scanner.h /^ std::map::iterator _begin;$/;" m class:doris::SchemaVariablesScanner +_begin src/exprs/hybird_set.h /^ typename std::unordered_set<_iT>::iterator _begin;$/;" m class:doris::HybirdSet::Iterator +_begin src/exprs/hybird_set.h /^ typename std::unordered_set::iterator _begin;$/;" m class:doris::StringValueSet::Iterator +_begin_include src/exec/olap_common.h /^ bool _begin_include;$/;" m class:doris::OlapScanKeys +_begin_mini_load src/http/action/mini_load.cpp /^Status MiniLoadAction::_begin_mini_load(StreamLoadContext* ctx) {$/;" f class:doris::MiniLoadAction +_begin_scan_keys src/exec/olap_common.h /^ std::vector _begin_scan_keys;$/;" m class:doris::OlapScanKeys +_begin_time src/olap/utils.h /^ struct timeval _begin_time; \/\/ 起始时间戳$/;" m class:doris::OlapStopWatch typeref:struct:doris::OlapStopWatch::timeval +_bf src/olap/rowset/column_writer.h /^ BloomFilter* _bf;$/;" m class:doris::ColumnWriter +_bf_buffer_size src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ uint64_t _bf_buffer_size;$/;" m class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl file: +_bf_filtered_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _bf_filtered_counter = nullptr;$/;" m class:doris::OlapScanNode +_bf_fpp src/olap/rowset/column_writer.h /^ double _bf_fpp;$/;" m class:doris::ColumnWriter +_bf_fpp src/olap/tablet_schema.h /^ double _bf_fpp = 0;$/;" m class:doris::TabletSchema +_bf_index src/olap/rowset/column_writer.h /^ BloomFilterIndexWriter _bf_index;$/;" m class:doris::ColumnWriter +_bf_index_meta src/olap/rowset/segment_v2/column_reader.h /^ const BloomFilterIndexPB* _bf_index_meta = nullptr;$/;" m class:doris::segment_v2::ColumnReader +_bf_index_stream src/olap/rowset/column_writer.h /^ OutStream* _bf_index_stream;$/;" m class:doris::ColumnWriter +_bf_options src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ BloomFilterOptions _bf_options;$/;" m class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl file: +_bfd_mutex src/util/bfd_parser.cpp /^std::mutex BfdParser::_bfd_mutex;$/;" m class:doris::BfdParser file: +_bfd_mutex src/util/bfd_parser.h /^ static std::mutex _bfd_mutex;$/;" m class:doris::BfdParser +_bfd_parser src/runtime/exec_env.h /^ BfdParser* _bfd_parser = nullptr;$/;" m class:doris::ExecEnv +_bfs src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ std::vector> _bfs;$/;" m class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl file: +_bind src/http/ev_http_server.cpp /^Status EvHttpServer::_bind() {$/;" f class:doris::EvHttpServer +_bit_num src/olap/bloom_filter.hpp /^ uint32_t _bit_num;$/;" m class:doris::BloomFilter +_bit_num src/olap/bloom_filter_reader.h /^ uint32_t _bit_num;$/;" m class:doris::BloomFilterIndexReader +_bit_set src/olap/bloom_filter.hpp /^ BitSet _bit_set;$/;" m class:doris::BloomFilter +_bit_width src/olap/rowset/segment_v2/rle_page.h /^ int _bit_width;$/;" m class:doris::segment_v2::RlePageBuilder +_bit_width src/olap/rowset/segment_v2/rle_page.h /^ int _bit_width;$/;" m class:doris::segment_v2::RlePageDecoder +_bit_widths src/util/frame_of_reference_coding.h /^ std::vector _bit_widths;$/;" m class:doris::ForDecoder +_bit_widths src/util/frame_of_reference_coding.h /^ std::vector _bit_widths;$/;" m class:doris::ForEncoder +_bitmap src/olap/selection_vector.h /^ std::unique_ptr _bitmap;$/;" m class:doris::SelectionVector +_bitmap src/util/bitmap_value.h /^ detail::Roaring64Map _bitmap; \/\/ used when _type == BITMAP$/;" m class:doris::BitmapValue +_bitmap_buf src/olap/rowset/segment_v2/column_writer.cpp /^ faststring _bitmap_buf;$/;" m class:doris::segment_v2::NullBitmapBuilder file: +_bitmap_column_iter src/olap/rowset/segment_v2/bitmap_index_reader.h /^ IndexedColumnIterator _bitmap_column_iter;$/;" m class:doris::segment_v2::BitmapIndexIterator +_bitmap_column_reader src/olap/rowset/segment_v2/bitmap_index_reader.h /^ std::unique_ptr _bitmap_column_reader;$/;" m class:doris::segment_v2::BitmapIndexReader +_bitmap_index src/olap/rowset/segment_v2/column_reader.h /^ std::unique_ptr _bitmap_index;$/;" m class:doris::segment_v2::ColumnReader +_bitmap_index_builder src/olap/rowset/segment_v2/column_writer.h /^ std::unique_ptr _bitmap_index_builder;$/;" m class:doris::segment_v2::ColumnWriter +_bitmap_index_filter_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _bitmap_index_filter_counter = nullptr;$/;" m class:doris::OlapScanNode +_bitmap_index_filter_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _bitmap_index_filter_timer = nullptr;$/;" m class:doris::OlapScanNode +_bitmap_index_iterators src/olap/rowset/segment_v2/segment_iterator.h /^ std::vector _bitmap_index_iterators;$/;" m class:doris::segment_v2::SegmentIterator +_bitmap_index_meta src/olap/rowset/segment_v2/bitmap_index_reader.h /^ const BitmapIndexPB* _bitmap_index_meta;$/;" m class:doris::segment_v2::BitmapIndexReader +_bitmap_index_meta src/olap/rowset/segment_v2/column_reader.h /^ const BitmapIndexPB* _bitmap_index_meta = nullptr;$/;" m class:doris::segment_v2::ColumnReader +_bitmaps src/exprs/bitmap_function.cpp /^ std::map _bitmaps;$/;" m struct:doris::BitmapIntersect file: +_bits_delta_max src/olap/rowset/run_length_integer_writer.h /^ uint32_t _bits_delta_max;$/;" m class:doris::RunLengthIntegerWriter +_bits_left src/olap/rowset/bit_field_reader.h /^ uint32_t _bits_left;$/;" m class:doris::BitFieldReader +_bits_left src/olap/rowset/bit_field_writer.h /^ uint8_t _bits_left;$/;" m class:doris::BitFieldWriter +_blacklisted src/runtime/tmp_file_mgr.h /^ bool _blacklisted;$/;" m class:doris::TmpFileMgr::Dir +_blacklisted src/runtime/tmp_file_mgr.h /^ bool _blacklisted;$/;" m class:doris::TmpFileMgr::File +_block src/olap/column_block.h /^ ColumnBlock _block;$/;" m struct:doris::ColumnBlockCell +_block src/olap/column_block.h /^ ColumnBlock* _block;$/;" m class:doris::ColumnBlockView +_block src/olap/generic_iterators.cpp /^ RowBlockV2 _block;$/;" m class:doris::MergeIteratorContext file: +_block src/olap/row_block2.h /^ const RowBlockV2* _block;$/;" m class:doris::RowBlockRow +_block src/util/arrow/row_block.cpp /^ const RowBlockV2& _block;$/;" m class:doris::FromRowBlockConverter file: +_block_capacity src/runtime/merge_sorter.cpp /^ const int _block_capacity;$/;" m class:doris::MergeSorter::TupleSorter file: +_block_capacity src/runtime/spill_sorter.cc /^ const int _block_capacity;$/;" m class:doris::SpillSorter::TupleSorter file: +_block_convert_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _block_convert_timer = nullptr;$/;" m class:doris::OlapScanNode +_block_count src/olap/rowset/segment_reader.h /^ int64_t _block_count; \/\/ 每一列中,index entry的数目应该相等。$/;" m class:doris::SegmentReader +_block_count src/olap/rowset/segment_writer.h /^ uint64_t _block_count; \/\/ 已经写入的block个数$/;" m class:doris::SegmentWriter +_block_fetch_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _block_fetch_timer = nullptr;$/;" m class:doris::OlapScanNode +_block_id src/olap/fs/file_block_manager.cpp /^ const BlockId _block_id;$/;" m class:doris::fs::internal::FileReadableBlock file: +_block_id src/olap/fs/file_block_manager.cpp /^ const BlockId _block_id;$/;" m class:doris::fs::internal::FileWritableBlock file: +_block_id src/olap/rowset/column_data_writer.h /^ uint32_t _block_id; \/\/ 当前Segment内的block编号$/;" m class:doris::ColumnDataWriter +_block_index src/runtime/merge_sorter.cpp /^ int _block_index;$/;" m class:doris::MergeSorter::TupleSorter::TupleIterator file: +_block_index src/runtime/spill_sorter.cc /^ int _block_index;$/;" m class:doris::SpillSorter::TupleSorter::TupleIterator file: +_block_load_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _block_load_counter = nullptr;$/;" m class:doris::OlapScanNode +_block_load_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _block_load_timer = nullptr;$/;" m class:doris::OlapScanNode +_block_manager src/olap/fs/file_block_manager.cpp /^ FileBlockManager* _block_manager;$/;" m class:doris::fs::internal::FileReadableBlock file: +_block_manager src/olap/fs/file_block_manager.cpp /^ FileBlockManager* _block_manager;$/;" m class:doris::fs::internal::FileWritableBlock file: +_block_mgr src/runtime/buffered_block_mgr.h /^ BufferedBlockMgr* _block_mgr;$/;" m class:doris::BufferedBlockMgr::Block +_block_mgr src/runtime/buffered_block_mgr2.h /^ BufferedBlockMgr2* _block_mgr;$/;" m class:doris::BufferedBlockMgr2::Block +_block_mgr src/runtime/buffered_tuple_stream.h /^ BufferedBlockMgr* _block_mgr;$/;" m class:doris::BufferedTupleStream +_block_mgr src/runtime/buffered_tuple_stream2.h /^ BufferedBlockMgr2* _block_mgr;$/;" m class:doris::BufferedTupleStream2 +_block_mgr src/runtime/merge_sorter.h /^ BufferedBlockMgr* _block_mgr;$/;" m class:doris::MergeSorter +_block_mgr src/runtime/runtime_state.h /^ boost::shared_ptr _block_mgr;$/;" m class:doris::RuntimeState +_block_mgr src/runtime/spill_sorter.h /^ BufferedBlockMgr2* _block_mgr;$/;" m class:doris::SpillSorter +_block_mgr2 src/runtime/runtime_state.h /^ boost::shared_ptr _block_mgr2;$/;" m class:doris::RuntimeState +_block_mgr_client src/exec/partitioned_aggregation_node.h /^ BufferedBlockMgr2::Client* _block_mgr_client;$/;" m class:doris::PartitionedAggregationNode +_block_mgr_client src/exec/partitioned_hash_table.h /^ BufferedBlockMgr2::Client* _block_mgr_client;$/;" m class:doris::PartitionedHashTable +_block_mgr_client src/runtime/buffered_tuple_stream2.h /^ BufferedBlockMgr2::Client* _block_mgr_client;$/;" m class:doris::BufferedTupleStream2 +_block_mgr_client src/runtime/spill_sorter.h /^ BufferedBlockMgr2::Client* _block_mgr_client;$/;" m class:doris::SpillSorter +_block_mgr_parent_tracker src/runtime/test_env.h /^ boost::scoped_ptr _block_mgr_parent_tracker;$/;" m class:doris::TestEnv +_block_row_count src/olap/rowset/column_writer.h /^ std::vector _block_row_count;$/;" m class:doris::VarStringColumnWriter +_block_rowids src/olap/rowset/segment_v2/segment_iterator.h /^ std::vector _block_rowids;$/;" m class:doris::segment_v2::SegmentIterator +_block_seek_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _block_seek_counter = nullptr;$/;" m class:doris::OlapScanNode +_block_seek_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _block_seek_timer = nullptr;$/;" m class:doris::OlapScanNode +_block_size src/runtime/merge_sorter.cpp /^ const int _block_size;$/;" m class:doris::MergeSorter::Run file: +_block_size src/runtime/spill_sorter.cc /^ const int _block_size;$/;" m class:doris::SpillSorter::Run file: +_block_size test/runtime/buffered_block_mgr2_test.cpp /^ const static int _block_size = 1024;$/;" m class:doris::BufferedBlockMgrTest file: +_block_size_counter src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _block_size_counter;$/;" m class:doris::BufferedBlockMgr2 +_block_start_idx src/runtime/buffered_tuple_stream.h /^ std::vector _block_start_idx;$/;" m class:doris::BufferedTupleStream +_block_start_idx src/runtime/buffered_tuple_stream2.h /^ std::vector _block_start_idx;$/;" m class:doris::BufferedTupleStream2 +_block_statistics src/olap/rowset/column_writer.h /^ ColumnStatistics _block_statistics;$/;" m class:doris::ColumnWriter +_block_status src/olap/row_block.h /^ uint8_t _block_status = DEL_PARTIAL_SATISFIED;$/;" m class:doris::RowBlock +_block_status src/runtime/vectorized_row_batch.h /^ uint8_t _block_status;$/;" m class:doris::VectorizedRowBatch +_block_write_threshold src/runtime/buffered_block_mgr2.h /^ const int _block_write_threshold;$/;" m class:doris::BufferedBlockMgr2 +_blocked_on_queue src/runtime/disk_io_mgr.h /^ bool _blocked_on_queue;$/;" m class:doris::DiskIoMgr::ScanRange +_blocked_ranges src/runtime/disk_io_mgr_internal.h /^ InternalQueue _blocked_ranges;$/;" m class:doris::DiskIoMgr::RequestContext +_blocks src/runtime/buffered_tuple_stream.h /^ std::list _blocks;$/;" m class:doris::BufferedTupleStream +_blocks src/runtime/buffered_tuple_stream2.h /^ std::list _blocks;$/;" m class:doris::BufferedTupleStream2 +_blocks src/runtime/row_batch.h /^ std::vector _blocks;$/;" m class:doris::RowBatch +_blocks src/util/core_local.cpp /^ std::vector _blocks;$/;" m class:doris::CoreDataAllocatorImpl file: +_bloom_filter_fpp src/olap/rowset/column_data_writer.h /^ double _bloom_filter_fpp;$/;" m class:doris::ColumnDataWriter +_bloom_filter_fpp src/olap/rowset/segment_writer.h /^ double _bloom_filter_fpp;$/;" m class:doris::SegmentWriter +_bloom_filter_index src/olap/rowset/segment_v2/column_reader.h /^ std::unique_ptr _bloom_filter_index;$/;" m class:doris::segment_v2::ColumnReader +_bloom_filter_index_builder src/olap/rowset/segment_v2/column_writer.h /^ std::unique_ptr _bloom_filter_index_builder;$/;" m class:doris::segment_v2::ColumnWriter +_bloom_filter_index_meta src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ const BloomFilterIndexPB* _bloom_filter_index_meta;$/;" m class:doris::segment_v2::BloomFilterIndexReader +_bloom_filter_iter src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ IndexedColumnIterator _bloom_filter_iter;$/;" m class:doris::segment_v2::BloomFilterIndexIterator +_bloom_filter_reader src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ std::unique_ptr _bloom_filter_reader;$/;" m class:doris::segment_v2::BloomFilterIndexReader +_bloom_filters src/olap/bloom_filter_writer.h /^ std::vector _bloom_filters;$/;" m class:doris::BloomFilterIndexWriter +_bloom_filters src/olap/rowset/segment_reader.h /^ std::map _bloom_filters;$/;" m class:doris::SegmentReader +_boost_uuid_generator src/util/uuid_generator.h /^ boost::uuids::basic_random_generator _boost_uuid_generator;$/;" m class:doris::UUIDGenerator +_br_bits_100p src/olap/rowset/run_length_integer_writer.h /^ uint32_t _br_bits_100p;$/;" m class:doris::RunLengthIntegerWriter +_br_bits_95p src/olap/rowset/run_length_integer_writer.h /^ uint32_t _br_bits_95p;$/;" m class:doris::RunLengthIntegerWriter +_broadcast_sink test/runtime/data_stream_test.cpp /^ TDataStreamSink _broadcast_sink;$/;" m class:doris::DataStreamTest file: +_broker_addresses src/exec/broker_scanner.h /^ const std::vector& _broker_addresses;$/;" m class:doris::BrokerScanner +_broker_addresses src/exec/orc_scanner.h /^ const std::vector& _broker_addresses;$/;" m class:doris::ORCScanner +_broker_addresses src/exec/parquet_scanner.h /^ const std::vector& _broker_addresses;$/;" m class:doris::ParquetScanner +_broker_client_cache src/runtime/exec_env.h /^ ClientCache* _broker_client_cache = nullptr;$/;" m class:doris::ExecEnv +_broker_mgr src/runtime/exec_env.h /^ BrokerMgr* _broker_mgr = nullptr;$/;" m class:doris::ExecEnv +_broker_set src/runtime/broker_mgr.h /^ std::unordered_set _broker_set;$/;" m class:doris::BrokerMgr +_broker_writer src/util/broker_load_error_hub.h /^ BrokerWriter* _broker_writer;$/;" m class:doris::BrokerLoadErrorHub +_brokers src/runtime/routine_load/data_consumer.h /^ std::string _brokers;$/;" m class:doris::KafkaDataConsumer +_brpc_dest_addr src/runtime/data_stream_sender.cpp /^ TNetworkAddress _brpc_dest_addr;$/;" m class:doris::DataStreamSender::Channel file: +_brpc_request src/runtime/data_stream_sender.cpp /^ PTransmitDataParams _brpc_request;$/;" m class:doris::DataStreamSender::Channel file: +_brpc_stub src/runtime/data_stream_sender.cpp /^ palo::PInternalService_Stub* _brpc_stub = nullptr;$/;" m class:doris::DataStreamSender::Channel file: +_brpc_stub_cache src/runtime/exec_env.h /^ BrpcStubCache* _brpc_stub_cache = nullptr;$/;" m class:doris::ExecEnv +_brpc_timeout_ms src/runtime/data_stream_sender.cpp /^ int32_t _brpc_timeout_ms = 500;$/;" m class:doris::DataStreamSender::Channel file: +_bucket_idx src/exec/hash_table.h /^ int64_t _bucket_idx;$/;" m class:doris::HashTable::Iterator +_bucket_idx src/exec/partitioned_hash_table.h /^ int64_t _bucket_idx;$/;" m class:doris::PartitionedHashTable::Iterator +_bucketing_counters src/util/runtime_profile.h /^ std::set* > _bucketing_counters;$/;" m class:doris::RuntimeProfile +_buckets src/exec/hash_table.h /^ std::vector _buckets;$/;" m class:doris::HashTable +_buckets src/exec/partitioned_hash_table.h /^ Bucket* _buckets;$/;" m class:doris::PartitionedHashTable +_buf src/olap/byte_buffer.h /^ boost::shared_ptr _buf; \/\/ 托管的内存$/;" m class:doris::StorageByteBuffer +_buf src/olap/rowset/segment_v2/frame_of_reference_page.h /^ faststring _buf;$/;" m class:doris::segment_v2::FrameOfReferencePageBuilder +_buf src/olap/rowset/segment_v2/rle_page.h /^ faststring _buf;$/;" m class:doris::segment_v2::RlePageBuilder +_buf src/olap/rowset/segment_v2/segment_iterator.cpp /^ uint32_t* _buf = nullptr;$/;" m class:doris::segment_v2::SegmentIterator::BitmapRangeIterator file: +_buf src/olap/utils.cpp /^__thread char Errno::_buf[BUF_SIZE]; \/\/\/< buffer instance$/;" m class:doris::Errno file: +_buf src/olap/utils.h /^ static __thread char _buf[BUF_SIZE];$/;" m class:doris::Errno +_buf src/runtime/dpp_writer.h /^ char* _buf;$/;" m class:doris::DppWriter +_buf src/util/mysql_row_buffer.h /^ char* _buf;$/;" m class:doris::MysqlRowBuffer +_buf_len src/olap/hll.h /^ int _buf_len; \/\/ set len$/;" m class:doris::HllSetResolver +_buf_pos src/olap/rowset/segment_v2/segment_iterator.cpp /^ uint32_t _buf_pos;$/;" m class:doris::segment_v2::SegmentIterator::BitmapRangeIterator file: +_buf_queue src/runtime/stream_load/stream_load_pipe.h /^ std::deque _buf_queue;$/;" m class:doris::StreamLoadPipe +_buf_ref src/olap/hll.h /^ char* _buf_ref; \/\/ set$/;" m class:doris::HllSetResolver +_buf_size src/olap/rowset/segment_v2/segment_iterator.cpp /^ uint32_t _buf_size;$/;" m class:doris::segment_v2::SegmentIterator::BitmapRangeIterator file: +_buf_size src/runtime/result_sink.h /^ int _buf_size; \/\/ Allocated from _pool$/;" m class:doris::ResultSink +_buf_size src/util/mysql_row_buffer.h /^ int _buf_size;$/;" m class:doris::MysqlRowBuffer +_buffer src/olap/bloom_filter_reader.h /^ char* _buffer;$/;" m class:doris::BloomFilterIndexReader +_buffer src/olap/rowset/segment_v2/binary_dict_page.h /^ faststring _buffer;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_buffer src/olap/rowset/segment_v2/binary_plain_page.h /^ faststring _buffer;$/;" m class:doris::segment_v2::BinaryPlainPageBuilder +_buffer src/olap/rowset/segment_v2/binary_prefix_page.h /^ faststring _buffer;$/;" m class:doris::segment_v2::BinaryPrefixPageBuilder +_buffer src/olap/rowset/segment_v2/bitshuffle_page.h /^ faststring _buffer;$/;" m class:doris::segment_v2::BitshufflePageBuilder +_buffer src/olap/rowset/segment_v2/index_page.h /^ faststring _buffer;$/;" m class:doris::segment_v2::IndexPageBuilder +_buffer src/olap/rowset/segment_v2/plain_page.h /^ faststring _buffer;$/;" m class:doris::segment_v2::PlainPageBuilder +_buffer src/olap/stream_index_reader.h /^ char* _buffer;$/;" m class:doris::StreamIndexReader +_buffer src/runtime/decimal_value.h /^ int32_t _buffer[DECIMAL_BUFF_LENGTH];$/;" m class:doris::DecimalValue +_buffer src/runtime/disk_io_mgr.h /^ char* _buffer;$/;" m class:doris::DiskIoMgr::BufferDescriptor +_buffer src/util/buffer_builder.hpp /^ uint8_t* _buffer;$/;" m class:doris::BufferBuilder +_buffer src/util/frame_of_reference_coding.h /^ const uint8_t* _buffer = nullptr;$/;" m class:doris::ForDecoder +_buffer src/util/frame_of_reference_coding.h /^ faststring* _buffer;$/;" m class:doris::ForEncoder +_buffer_available_cv src/runtime/buffered_block_mgr2.h /^ boost::condition_variable _buffer_available_cv;$/;" m class:doris::BufferedBlockMgr2 +_buffer_desc src/runtime/buffered_block_mgr.h /^ BufferDescriptor* _buffer_desc;$/;" m class:doris::BufferedBlockMgr::Block +_buffer_desc src/runtime/buffered_block_mgr2.h /^ BufferDescriptor* _buffer_desc;$/;" m class:doris::BufferedBlockMgr2::Block +_buffer_full_total_timer src/runtime/data_stream_recvr.h /^ RuntimeProfile::Counter* _buffer_full_total_timer;$/;" m class:doris::DataStreamRecvr +_buffer_full_wall_timer src/runtime/data_stream_recvr.h /^ RuntimeProfile::Counter* _buffer_full_wall_timer;$/;" m class:doris::DataStreamRecvr +_buffer_len src/runtime/disk_io_mgr.h /^ int64_t _buffer_len;$/;" m class:doris::DiskIoMgr::BufferDescriptor +_buffer_len src/util/frame_of_reference_coding.h /^ size_t _buffer_len = 0;$/;" m class:doris::ForDecoder +_buffer_limit src/runtime/buffer_control_block.h /^ int _buffer_limit;$/;" m class:doris::BufferControlBlock +_buffer_map src/runtime/result_buffer_mgr.h /^ BufferMap _buffer_map;$/;" m class:doris::ResultBufferMgr +_buffer_mem_pool src/olap/memtable.h /^ std::unique_ptr _buffer_mem_pool;$/;" m class:doris::MemTable +_buffer_pool src/runtime/exec_env.h /^ BufferPool* _buffer_pool = nullptr;$/;" m class:doris::ExecEnv +_buffer_pool_client src/exec/exec_node.h /^ BufferPool::ClientHandle _buffer_pool_client;$/;" m class:doris::ExecNode +_buffer_ready_cv src/runtime/disk_io_mgr.h /^ boost::condition_variable _buffer_ready_cv;$/;" m class:doris::DiskIoMgr::ScanRange +_buffer_reservation src/runtime/exec_env.h /^ ReservationTracker* _buffer_reservation = nullptr;$/;" m class:doris::ExecEnv +_buffer_reservation src/runtime/runtime_state.h /^ ReservationTracker* _buffer_reservation = nullptr;$/;" m class:doris::RuntimeState +_buffer_rows src/runtime/buffer_control_block.h /^ int _buffer_rows;$/;" m class:doris::BufferControlBlock +_buffer_size src/olap/bloom_filter_reader.h /^ size_t _buffer_size;$/;" m class:doris::BloomFilterIndexReader +_buffer_size src/olap/out_stream.h /^ uint32_t _buffer_size; \/\/ 压缩块大小$/;" m class:doris::OutStream +_buffer_size src/olap/rowset/segment_reader.h /^ size_t _buffer_size;$/;" m class:doris::SegmentReader +_buffer_size src/olap/stream_index_reader.h /^ size_t _buffer_size;$/;" m class:doris::StreamIndexReader +_buffer_size src/runtime/data_stream_sender.cpp /^ int _buffer_size;$/;" m class:doris::DataStreamSender::Channel file: +_buffer_size src/runtime/string_buffer.hpp /^ int _buffer_size;$/;" m class:doris::StringBuffer +_buffer_start src/runtime/merge_sorter.cpp /^ uint8_t* _buffer_start = nullptr;$/;" m class:doris::MergeSorter::TupleSorter::TupleIterator file: +_buffer_start src/runtime/spill_sorter.cc /^ uint8_t* _buffer_start;$/;" m class:doris::SpillSorter::TupleSorter::TupleIterator file: +_buffer_wait_timer src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _buffer_wait_timer;$/;" m class:doris::BufferedBlockMgr2 +_buffer_wall_timer_lock src/runtime/data_stream_recvr.h /^ boost::try_mutex _buffer_wall_timer_lock;$/;" m class:doris::DataStreamRecvr +_buffered_batch src/runtime/spill_sorter.cc /^ scoped_ptr _buffered_batch;$/;" m class:doris::SpillSorter::Run file: +_buffered_bytes src/exec/olap_scan_node.h /^ int64_t _buffered_bytes;$/;" m class:doris::OlapScanNode +_buffered_bytes src/runtime/stream_load/stream_load_pipe.h /^ size_t _buffered_bytes;$/;" m class:doris::StreamLoadPipe +_buffered_pin_counter src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _buffered_pin_counter;$/;" m class:doris::BufferedBlockMgr2 +_buffered_tuple_desc src/exec/analytic_eval_node.h /^ TupleDescriptor* _buffered_tuple_desc;$/;" m class:doris::AnalyticEvalNode +_buffered_values src/util/frame_of_reference_coding.h /^ T _buffered_values[FRAME_VALUE_NUM];$/;" m class:doris::ForEncoder +_buffered_values_num src/util/frame_of_reference_coding.h /^ uint8_t _buffered_values_num = 0;$/;" m class:doris::ForEncoder +_buffers src/runtime/row_batch.h /^ std::vector _buffers;$/;" m class:doris::RowBatch +_build_batch_pool src/exec/cross_join_node.h /^ boost::scoped_ptr _build_batch_pool;$/;" m class:doris::CrossJoinNode +_build_batches src/exec/cross_join_node.h /^ RowBatchList _build_batches;$/;" m class:doris::CrossJoinNode +_build_buckets_counter src/exec/hash_join_node.h /^ RuntimeProfile::Counter* _build_buckets_counter; \/\/ num buckets in hash table$/;" m class:doris::HashJoinNode +_build_expr test/exec/hash_table_test.cpp /^ vector _build_expr;$/;" m class:doris::HashTableTest file: +_build_expr_ctxs src/exec/aggregation_node.h /^ std::vector _build_expr_ctxs;$/;" m class:doris::AggregationNode +_build_expr_ctxs src/exec/hash_join_node.h /^ std::vector _build_expr_ctxs;$/;" m class:doris::HashJoinNode +_build_expr_ctxs src/exec/hash_table.h /^ const std::vector& _build_expr_ctxs;$/;" m class:doris::HashTable +_build_expr_ctxs src/exec/partitioned_aggregation_node.h /^ std::vector _build_expr_ctxs;$/;" m class:doris::PartitionedAggregationNode +_build_expr_ctxs src/exec/partitioned_hash_table.h /^ const std::vector& _build_expr_ctxs;$/;" m class:doris::PartitionedHashTableCtx +_build_expr_ctxs test/exec/partitioned_hash_table_test.cpp /^ vector _build_expr_ctxs;$/;" m class:doris::PartitionedHashTableTest file: +_build_exprs src/exec/pre_aggregation_node.h /^ std::vector _build_exprs;$/;" m class:doris::PreAggregationNode +_build_pool src/exec/blocking_join_node.h /^ boost::scoped_ptr _build_pool; \/\/ holds everything referenced from build side$/;" m class:doris::BlockingJoinNode +_build_pool src/exec/hash_join_node.h /^ boost::scoped_ptr _build_pool; \/\/ holds everything referenced in _hash_tbl$/;" m class:doris::HashJoinNode +_build_pool src/exec/intersect_node.h /^ std::unique_ptr _build_pool;$/;" m class:doris::IntersectNode +_build_row_counter src/exec/blocking_join_node.h /^ RuntimeProfile::Counter* _build_row_counter; \/\/ num build rows$/;" m class:doris::BlockingJoinNode +_build_rows_counter src/exec/hash_join_node.h /^ RuntimeProfile::Counter* _build_rows_counter; \/\/ num build rows$/;" m class:doris::HashJoinNode +_build_tablet_stat src/olap/tablet_manager.cpp /^void TabletManager::_build_tablet_stat() {$/;" f class:doris::TabletManager +_build_timer src/exec/aggregation_node.h /^ RuntimeProfile::Counter* _build_timer;$/;" m class:doris::AggregationNode +_build_timer src/exec/blocking_join_node.h /^ RuntimeProfile::Counter* _build_timer; \/\/ time to prepare build side$/;" m class:doris::BlockingJoinNode +_build_timer src/exec/hash_join_node.h /^ RuntimeProfile::Counter* _build_timer; \/\/ time to build hash table$/;" m class:doris::HashJoinNode +_build_timer src/exec/partitioned_aggregation_node.h /^ RuntimeProfile::Counter* _build_timer;$/;" m class:doris::PartitionedAggregationNode +_build_timer src/exec/pre_aggregation_node.h /^ RuntimeProfile::Counter* _build_timer;$/;" m class:doris::PreAggregationNode +_build_tuple_idx src/exec/blocking_join_node.h /^ std::vector _build_tuple_idx;$/;" m class:doris::BlockingJoinNode +_build_tuple_idx src/exec/hash_join_node.h /^ std::vector _build_tuple_idx;$/;" m class:doris::HashJoinNode +_build_tuple_idx src/exec/intersect_node.h /^ std::vector _build_tuple_idx;$/;" m class:doris::IntersectNode +_build_tuple_row_size src/exec/blocking_join_node.h /^ int _build_tuple_row_size;$/;" m class:doris::BlockingJoinNode +_build_tuple_row_size src/exec/hash_join_node.h /^ int _build_tuple_row_size;$/;" m class:doris::HashJoinNode +_build_tuple_row_size src/exec/intersect_node.h /^ int _build_tuple_row_size;$/;" m class:doris::IntersectNode +_build_tuple_size src/exec/blocking_join_node.h /^ int _build_tuple_size;$/;" m class:doris::BlockingJoinNode +_build_tuple_size src/exec/hash_join_node.h /^ int _build_tuple_size;$/;" m class:doris::HashJoinNode +_build_tuple_size src/exec/intersect_node.h /^ int _build_tuple_size;$/;" m class:doris::IntersectNode +_build_tuple_size src/exec/pre_aggregation_node.h /^ int _build_tuple_size;$/;" m class:doris::PreAggregationNode +_byte_reader src/olap/rowset/bit_field_reader.h /^ RunLengthByteReader* _byte_reader;$/;" m class:doris::BitFieldReader +_byte_size src/runtime/descriptors.h /^ int _byte_size;$/;" m class:doris::TupleDescriptor +_byte_writer src/olap/rowset/bit_field_writer.h /^ RunLengthByteWriter* _byte_writer;$/;" m class:doris::BitFieldWriter +_bytes_appended src/olap/fs/file_block_manager.cpp /^ size_t _bytes_appended;$/;" m class:doris::fs::internal::FileWritableBlock file: +_bytes_decompress_counter src/exec/plain_text_line_reader.h /^ RuntimeProfile::Counter* _bytes_decompress_counter;$/;" m class:doris::PlainTextLineReader +_bytes_freed_by_last_gc_metric src/runtime/mem_tracker.h /^ IntGauge* _bytes_freed_by_last_gc_metric;$/;" m class:doris::MemTracker +_bytes_over_limit_metric src/runtime/mem_tracker.h /^ IntGauge* _bytes_over_limit_metric;$/;" m class:doris::MemTracker +_bytes_read src/runtime/disk_io_mgr.h /^ int _bytes_read;$/;" m class:doris::DiskIoMgr::ScanRange +_bytes_read_counter src/exec/plain_text_line_reader.h /^ RuntimeProfile::Counter* _bytes_read_counter;$/;" m class:doris::PlainTextLineReader +_bytes_read_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* _bytes_read_counter; \/\/ # bytes read from the scanner$/;" m class:doris::ScanNode +_bytes_read_counter src/runtime/disk_io_mgr_internal.h /^ RuntimeProfile::Counter* _bytes_read_counter;$/;" m class:doris::DiskIoMgr::RequestContext +_bytes_read_dn_cache src/runtime/disk_io_mgr_internal.h /^ AtomicInt _bytes_read_dn_cache;$/;" m class:doris::DiskIoMgr::RequestContext +_bytes_read_local src/runtime/disk_io_mgr_internal.h /^ AtomicInt _bytes_read_local;$/;" m class:doris::DiskIoMgr::RequestContext +_bytes_read_short_circuit src/runtime/disk_io_mgr_internal.h /^ AtomicInt _bytes_read_short_circuit;$/;" m class:doris::DiskIoMgr::RequestContext +_bytes_received_counter src/runtime/data_stream_recvr.h /^ RuntimeProfile::Counter* _bytes_received_counter;$/;" m class:doris::DataStreamRecvr +_bytes_sent_counter src/runtime/data_stream_sender.h /^ RuntimeProfile::Counter* _bytes_sent_counter;$/;" m class:doris::DataStreamSender +_bytes_written_counter src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _bytes_written_counter;$/;" m class:doris::BufferedBlockMgr2 +_bytes_written_counter src/runtime/export_sink.h /^ RuntimeProfile::Counter* _bytes_written_counter;$/;" m class:doris::ExportSink +_bz_strm src/exec/decompressor.h /^ bz_stream _bz_strm;$/;" m class:doris::Bzip2Decompressor +_cache src/olap/page_cache.h /^ Cache* _cache = nullptr;$/;" m class:doris::PageCacheHandle +_cache src/olap/page_cache.h /^ std::unique_ptr _cache = nullptr;$/;" m class:doris::StoragePageCache +_cache src/util/file_cache.h /^ Cache* _cache;$/;" m class:doris::OpenedFileHandle +_cache src/util/file_cache.h /^ std::unique_ptr _cache;$/;" m class:doris::FileCache +_cache test/olap/lru_cache_test.cpp /^ Cache* _cache;$/;" m class:doris::CacheTest file: +_cache_data src/olap/rowset/segment_v2/page_handle.h /^ PageCacheHandle _cache_data;$/;" m class:doris::segment_v2::PageHandle +_cache_entry src/exprs/expr.h /^ UserFunctionCacheEntry* _cache_entry = nullptr;$/;" m class:doris::Expr +_cache_handle src/olap/file_helper.h /^ Cache::Handle* _cache_handle;$/;" m class:doris::FileHandler +_cache_handle src/olap/rowset/segment_reader.h /^ std::vector _cache_handle;$/;" m class:doris::SegmentReader +_cache_items_list src/util/lru_cache.hpp /^ std::list _cache_items_list;$/;" m class:doris::LruCache +_cache_items_map src/util/lru_cache.hpp /^ std::unordered_map _cache_items_map;$/;" m class:doris::LruCache +_cache_lock src/runtime/user_function_cache.h /^ std::mutex _cache_lock;$/;" m class:doris::UserFunctionCache +_cache_name src/util/file_cache.h /^ std::string _cache_name;$/;" m class:doris::FileCache +_cache_threshold src/olap/file_helper.h /^ const int64_t _cache_threshold = 1<<19;$/;" m class:doris::FileHandler +_cached_buffer src/runtime/disk_io_mgr.h /^ struct hadoopRzBuffer* _cached_buffer;$/;" m class:doris::DiskIoMgr::ScanRange typeref:struct:doris::DiskIoMgr::ScanRange::hadoopRzBuffer +_cached_pages_num_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _cached_pages_num_counter = nullptr;$/;" m class:doris::OlapScanNode +_cached_ranges src/runtime/disk_io_mgr_internal.h /^ InternalQueue _cached_ranges;$/;" m class:doris::DiskIoMgr::RequestContext +_cached_read_options src/runtime/disk_io_mgr.h /^ struct hadoopRzOptions* _cached_read_options;$/;" m class:doris::DiskIoMgr typeref:struct:doris::DiskIoMgr::hadoopRzOptions +_cached_response src/exec/es/es_scan_reader.h /^ std::string _cached_response;$/;" m class:doris::ESScanReader +_calc_snapshot_id_path src/olap/snapshot_manager.cpp /^OLAPStatus SnapshotManager::_calc_snapshot_id_path($/;" f class:doris::SnapshotManager +_calculate_row_ranges src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::_calculate_row_ranges(const std::vector& page_indexes, RowRanges* row_ranges) {$/;" f class:doris::segment_v2::ColumnReader +_callback src/http/http_client.h /^ const HttpCallback* _callback = nullptr;$/;" m class:doris::HttpClient +_callback src/runtime/disk_io_mgr.h /^ WriteDoneCallback _callback;$/;" m class:doris::DiskIoMgr::WriteRange +_callback_function src/agent/task_worker_pool.h /^ CALLBACK_FUNCTION _callback_function;$/;" m class:doris::TaskWorkerPool +_callbacks src/http/web_page_handler.h /^ std::vector _callbacks;$/;" m class:doris::WebPageHandler::PageHandlers +_can_dup src/olap/skiplist.h /^ bool _can_dup;$/;" m class:doris::SkipList +_cancel_thread src/runtime/fragment_mgr.h /^ std::thread _cancel_thread;$/;" m class:doris::FragmentMgr +_cancel_thread src/runtime/result_buffer_mgr.h /^ boost::scoped_ptr _cancel_thread;$/;" m class:doris::ResultBufferMgr +_cancelled src/runtime/routine_load/data_consumer.h /^ bool _cancelled;$/;" m class:doris::DataConsumer +_cancelled src/runtime/stream_load/stream_load_pipe.h /^ bool _cancelled;$/;" m class:doris::StreamLoadPipe +_cap src/geo/geo_types.h /^ std::unique_ptr _cap;$/;" m class:doris::GeoCircle +_capacity src/olap/byte_buffer.h /^ uint64_t _capacity;$/;" m class:doris::StorageByteBuffer +_capacity src/olap/lru_cache.h /^ size_t _capacity;$/;" m class:doris::CachePriority::LRUCache +_capacity src/olap/row_block.h /^ uint32_t _capacity;$/;" m class:doris::RowBlock +_capacity src/olap/row_block2.h /^ size_t _capacity;$/;" m class:doris::RowBlockV2 +_capacity src/runtime/row_batch.h /^ int _capacity; \/\/ maximum # of rows$/;" m class:doris::RowBatch +_capacity src/runtime/vectorized_row_batch.h /^ const uint16_t _capacity;$/;" m class:doris::VectorizedRowBatch +_capacity src/util/buffer_builder.hpp /^ int _capacity;$/;" m class:doris::BufferBuilder +_capacity_bytes src/olap/data_dir.h /^ int64_t _capacity_bytes;$/;" m class:doris::DataDir +_capture_consistent_rowsets_unlocked src/olap/tablet.cpp /^OLAPStatus Tablet::_capture_consistent_rowsets_unlocked(const vector& version_path,$/;" f class:doris::Tablet +_capture_rs_readers src/olap/reader.cpp /^OLAPStatus Reader::_capture_rs_readers(const ReaderParams& read_params) {$/;" f class:doris::Reader +_category src/util/thread.cpp /^ std::string _category;$/;" m class:doris::ThreadMgr::ThreadDescriptor file: +_category src/util/thread.h /^ const std::string _category;$/;" m class:doris::Thread +_cb src/runtime/data_stream_recvr.hpp /^ boost::shared_ptr _cb;$/;" m class:doris::DataStreamRecvr +_cgroups_mgr src/agent/user_resource_listener.h /^ CgroupsMgr& _cgroups_mgr; $/;" m class:doris::UserResourceListener +_cgroups_mgr src/runtime/exec_env.h /^ CgroupsMgr* _cgroups_mgr = nullptr;$/;" m class:doris::ExecEnv +_channel_shared_ptrs src/runtime/data_stream_sender.h /^ std::vector> _channel_shared_ptrs;$/;" m class:doris::DataStreamSender +_channels src/exec/tablet_sink.h /^ std::vector _channels;$/;" m class:doris::stream_load::OlapTableSink +_channels src/runtime/data_stream_sender.h /^ std::vector _channels;$/;" m class:doris::DataStreamSender +_channels_by_tablet src/exec/tablet_sink.h /^ std::unordered_map> _channels_by_tablet;$/;" m class:doris::stream_load::IndexChannel +_check_all_root_path_cluster_id src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::_check_all_root_path_cluster_id() {$/;" f class:doris::StorageEngine +_check_consistency_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_check_consistency_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_check_consistency_workers src/agent/agent_server.h /^ std::unique_ptr _check_consistency_workers;$/;" m class:doris::AgentServer +_check_file src/runtime/small_file_mgr.cpp /^Status SmallFileMgr::_check_file(const CacheEntry& entry, const std::string& md5) {$/;" f class:doris::SmallFileMgr +_check_file_descriptor_number src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::_check_file_descriptor_number() {$/;" f class:doris::StorageEngine +_check_file_version src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_check_file_version() {$/;" f class:doris::SegmentReader +_check_local_snapshot_paths src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::_check_local_snapshot_paths($/;" f class:doris::SnapshotLoader +_check_path_mutex src/olap/data_dir.h /^ std::mutex _check_path_mutex;$/;" m class:doris::DataDir +_check_pending_ids src/olap/data_dir.cpp /^bool DataDir::_check_pending_ids(const std::string& id) {$/;" f class:doris::DataDir +_check_rowset_overlapping src/olap/base_compaction.cpp /^OLAPStatus BaseCompaction::_check_rowset_overlapping(const vector& rowsets) {$/;" f class:doris::BaseCompaction +_check_tablet_id_exist_unlocked src/olap/tablet_manager.cpp /^bool TabletManager::_check_tablet_id_exist_unlocked(TTabletId tablet_id) {$/;" f class:doris::TabletManager +_check_version_valid src/olap/delete_handler.cpp /^OLAPStatus DeleteConditionHandler::_check_version_valid(std::vector* all_file_versions,$/;" f class:doris::DeleteConditionHandler +_check_whether_condition_exist src/olap/delete_handler.cpp /^int DeleteConditionHandler::_check_whether_condition_exist(const DelPredicateArray& delete_conditions, int cond_version) {$/;" f class:doris::DeleteConditionHandler +_checksum src/olap/rowset/segment_group.h /^ uint32_t _checksum;$/;" m class:doris::SegmentGroup +_checksum src/olap/task/engine_checksum_task.h /^ uint32_t* _checksum;$/;" m class:doris::EngineChecksumTask +_child_batch src/exec/except_node.h /^ std::unique_ptr _child_batch;$/;" m class:doris::ExceptNode +_child_batch src/exec/union_node.h /^ boost::scoped_ptr _child_batch;$/;" m class:doris::UnionNode +_child_counter_map src/util/runtime_profile.h /^ ChildCounterMap _child_counter_map;$/;" m class:doris::RuntimeProfile +_child_eos src/exec/except_node.h /^ bool _child_eos;$/;" m class:doris::ExceptNode +_child_eos src/exec/merge_node.h /^ bool _child_eos;$/;" m class:doris::MergeNode +_child_eos src/exec/olap_rewrite_node.h /^ bool _child_eos;$/;" m class:doris::OlapRewriteNode +_child_eos src/exec/pre_aggregation_node.h /^ bool _child_eos;$/;" m class:doris::PreAggregationNode +_child_eos src/exec/repeat_node.h /^ bool _child_eos;$/;" m class:doris::RepeatNode +_child_eos src/exec/select_node.h /^ bool _child_eos;$/;" m class:doris::SelectNode +_child_eos src/exec/union_node.h /^ bool _child_eos;$/;" m class:doris::UnionNode +_child_expr_lists src/exec/except_node.h /^ std::vector> _child_expr_lists;$/;" m class:doris::ExceptNode +_child_expr_lists src/exec/intersect_node.h /^ std::vector> _child_expr_lists;$/;" m class:doris::IntersectNode +_child_expr_lists src/exec/union_node.h /^ std::vector> _child_expr_lists;$/;" m class:doris::UnionNode +_child_idx src/exec/except_node.h /^ int _child_idx;$/;" m class:doris::ExceptNode +_child_idx src/exec/merge_node.h /^ int _child_idx;$/;" m class:doris::MergeNode +_child_idx src/exec/union_node.h /^ int _child_idx;$/;" m class:doris::UnionNode +_child_idx src/olap/reader.cpp /^ int _child_idx = 0;$/;" m class:doris::CollectIterator file: +_child_map src/util/runtime_profile.h /^ ChildMap _child_map;$/;" m class:doris::RuntimeProfile +_child_row_batch src/exec/merge_node.h /^ boost::scoped_ptr _child_row_batch;$/;" m class:doris::MergeNode +_child_row_batch src/exec/olap_rewrite_node.h /^ std::unique_ptr _child_row_batch;$/;" m class:doris::OlapRewriteNode +_child_row_batch src/exec/repeat_node.h /^ std::unique_ptr _child_row_batch;$/;" m class:doris::RepeatNode +_child_row_batch src/exec/select_node.h /^ boost::scoped_ptr _child_row_batch;$/;" m class:doris::SelectNode +_child_row_desc test/runtime/buffered_tuple_stream_test.cpp /^ RowDescriptor *_child_row_desc;$/;" m class:doris::BufferedTupleStreamTest file: +_child_row_desc test/runtime/sorter_test.cpp /^ RowDescriptor *_child_row_desc;$/;" m class:doris::SorterTest file: +_child_row_idx src/exec/except_node.h /^ int _child_row_idx;$/;" m class:doris::ExceptNode +_child_row_idx src/exec/merge_node.h /^ int _child_row_idx;$/;" m class:doris::MergeNode +_child_row_idx src/exec/olap_rewrite_node.h /^ int _child_row_idx;$/;" m class:doris::OlapRewriteNode +_child_row_idx src/exec/select_node.h /^ int _child_row_idx;$/;" m class:doris::SelectNode +_child_row_idx src/exec/union_node.h /^ int _child_row_idx;$/;" m class:doris::UnionNode +_child_tracker_it src/runtime/mem_tracker.h /^ std::list::iterator _child_tracker_it;$/;" m class:doris::MemTracker +_child_trackers src/runtime/mem_tracker.h /^ std::list _child_trackers;$/;" m class:doris::MemTracker +_child_trackers_lock src/runtime/mem_tracker.h /^ mutable std::mutex _child_trackers_lock;$/;" m class:doris::MemTracker +_child_tuple_cmp_row src/exec/analytic_eval_node.h /^ TupleRow* _child_tuple_cmp_row;$/;" m class:doris::AnalyticEvalNode +_child_tuple_desc src/exec/analytic_eval_node.h /^ TupleDescriptor* _child_tuple_desc;$/;" m class:doris::AnalyticEvalNode +_children src/exec/exec_node.h /^ std::vector _children;$/;" m class:doris::ExecNode +_children src/exprs/expr.h /^ std::vector _children;$/;" m class:doris::Expr +_children src/olap/reader.cpp /^ std::vector _children;$/;" m class:doris::CollectIterator file: +_children src/util/path_trie.hpp /^ std::map _children;$/;" m class:doris::PathTrie::TrieNode +_children src/util/runtime_profile.h /^ ChildVector _children;$/;" m class:doris::RuntimeProfile +_children_lock src/util/runtime_profile.h /^ mutable boost::mutex _children_lock; \/\/ protects _child_map and _children$/;" m class:doris::RuntimeProfile +_children_tuple src/exec/pre_aggregation_node.h /^ std::vector _children_tuple;$/;" m class:doris::PreAggregationNode +_chunk_lists src/runtime/memory/chunk_allocator.cpp /^ std::vector> _chunk_lists;$/;" m class:doris::ChunkArena file: +_clean_idle_consumer_bg src/runtime/routine_load/data_consumer_pool.cpp /^void DataConsumerPool::_clean_idle_consumer_bg() {$/;" f class:doris::DataConsumerPool +_clean_idle_consumer_thread src/runtime/routine_load/data_consumer_pool.h /^ std::thread _clean_idle_consumer_thread;$/;" m class:doris::DataConsumerPool +_clean_paths src/util/zip_util.h /^ std::vector _clean_paths;$/;" m class:doris::ZipFile +_clean_unfinished_converting_data src/olap/data_dir.cpp /^OLAPStatus DataDir::_clean_unfinished_converting_data() {$/;" f class:doris::DataDir +_clean_unused_rowset_metas src/olap/storage_engine.cpp /^void StorageEngine::_clean_unused_rowset_metas() {$/;" f class:doris::StorageEngine +_clean_unused_txns src/olap/storage_engine.cpp /^void StorageEngine::_clean_unused_txns() {$/;" f class:doris::StorageEngine +_cleaner_id src/runtime/load_path_mgr.h /^ pthread_t _cleaner_id;$/;" m class:doris::LoadPathMgr +_clear src/olap/rowset/run_length_integer_writer.cpp /^void RunLengthIntegerWriter::_clear() {$/;" f class:doris::RunLengthIntegerWriter +_clear_key src/http/action/restore_tablet_action.cpp /^void RestoreTabletAction::_clear_key(const std::string& key) {$/;" f class:doris::RestoreTabletAction +_clear_transaction_task_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_clear_transaction_task_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_clear_transaction_task_workers src/agent/agent_server.h /^ std::unique_ptr _clear_transaction_task_workers;$/;" m class:doris::AgentServer +_clear_txn_partition_map_unlocked src/olap/txn_manager.cpp /^void TxnManager::_clear_txn_partition_map_unlocked(int64_t transaction_id, int64_t partition_id) {$/;" f class:doris::TxnManager +_cleared src/runtime/row_batch.h /^ bool _cleared = false;$/;" m class:doris::RowBatch +_client src/exec/kudu_scan_node.h /^ kudu::client::sp::shared_ptr _client;$/;" m class:doris::KuduScanNode +_client src/runtime/buffered_block_mgr2.h /^ Client* _client;$/;" m class:doris::BufferedBlockMgr2::Block +_client src/runtime/client_cache.h /^ T* _client;$/;" m class:doris::ClientConnection +_client test/runtime/buffered_tuple_stream2_test.cpp /^ BufferedBlockMgr2::Client* _client;$/;" m class:doris::SimpleTupleStreamTest file: +_client_cache src/agent/utils.h /^ FrontendServiceClientCache* _client_cache;$/;" m class:doris::MasterServerClient +_client_cache src/runtime/client_cache.h /^ ClientCache* _client_cache;$/;" m class:doris::ClientConnection +_client_cache src/runtime/client_cache.h /^ ClientCacheMap _client_cache;$/;" m class:doris::ClientCacheHelper +_client_cache_helper src/runtime/client_cache.h /^ ClientCacheHelper _client_cache_helper;$/;" m class:doris::ClientCache +_client_factory src/runtime/client_cache.h /^ ClientCacheHelper::client_factory _client_factory;$/;" m class:doris::ClientCache +_client_id src/runtime/broker_mgr.h /^ std::string _client_id;$/;" m class:doris::BrokerMgr +_client_local src/runtime/buffered_block_mgr2.h /^ bool _client_local;$/;" m class:doris::BufferedBlockMgr2::Block +_client_map src/runtime/client_cache.h /^ ClientMap _client_map;$/;" m class:doris::ClientCacheHelper +_client_tracker test/runtime/buffered_block_mgr2_test.cpp /^ scoped_ptr _client_tracker;$/;" m class:doris::BufferedBlockMgrTest file: +_clone_copy src/olap/task/engine_clone_task.cpp /^AgentStatus EngineCloneTask::_clone_copy($/;" f class:doris::EngineCloneTask +_clone_full_data src/olap/task/engine_clone_task.cpp /^OLAPStatus EngineCloneTask::_clone_full_data(Tablet* tablet, TabletMeta* cloned_tablet_meta) {$/;" f class:doris::EngineCloneTask +_clone_incremental_data src/olap/task/engine_clone_task.cpp /^OLAPStatus EngineCloneTask::_clone_incremental_data(Tablet* tablet, const TabletMeta& cloned_tablet_meta,$/;" f class:doris::EngineCloneTask +_clone_req src/olap/task/engine_clone_task.h /^ const TCloneReq& _clone_req;$/;" m class:doris::EngineCloneTask +_clone_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_clone_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_clone_workers src/agent/agent_server.h /^ std::unique_ptr _clone_workers;$/;" m class:doris::AgentServer +_close src/exec/tablet_sink.cpp /^Status NodeChannel::_close(RuntimeState* state) {$/;" f class:doris::stream_load::NodeChannel +_close src/olap/fs/file_block_manager.cpp /^Status FileWritableBlock::_close(SyncMode mode) {$/;" f class:doris::fs::internal::FileWritableBlock +_close_fn src/exprs/scalar_fn_call.h /^ UdfClose _close_fn;$/;" m class:doris::ScalarFnCall +_close_status src/runtime/tablets_channel.h /^ Status _close_status;$/;" m class:doris::TabletsChannel +_close_timer src/exec/tablet_sink.h /^ RuntimeProfile::Counter* _close_timer = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_closed src/env/env_posix.cpp /^ bool _closed = false;$/;" m class:doris::PosixRandomRWFile file: +_closed src/env/env_posix.cpp /^ bool _closed = false;$/;" m class:doris::PosixWritableFile file: +_closed src/exec/data_sink.h /^ bool _closed;$/;" m class:doris::DataSink +_closed src/exprs/expr_context.h /^ bool _closed;$/;" m class:doris::ExprContext +_closed src/runtime/buffered_tuple_stream.h /^ bool _closed; \/\/ Used for debugging.$/;" m class:doris::BufferedTupleStream +_closed src/runtime/buffered_tuple_stream2.h /^ bool _closed; \/\/ Used for debugging.$/;" m class:doris::BufferedTupleStream2 +_closed src/runtime/data_stream_sender.h /^ bool _closed;$/;" m class:doris::DataStreamSender +_closed src/runtime/plan_fragment_executor.h /^ bool _closed;$/;" m class:doris::PlanFragmentExecutor +_closed src/udf/udf_internal.h /^ bool _closed;$/;" m class:doris::FunctionContextImpl +_closed_senders src/runtime/tablets_channel.h /^ Bitmap _closed_senders;$/;" m class:doris::TabletsChannel +_closure src/runtime/data_stream_sender.cpp /^ RefCountClosure* _closure = nullptr;$/;" m class:doris::DataStreamSender::Channel file: +_cluster_id src/olap/data_dir.h /^ int32_t _cluster_id;$/;" m class:doris::DataDir +_cluster_id_path src/olap/data_dir.h /^ std::string _cluster_id_path() const { return _path + CLUSTER_ID_PREFIX; }$/;" f class:doris::DataDir +_cmp src/olap/types.h /^ int (*_cmp)(const void* left, const void* right);$/;" m class:doris::TypeInfo +_cmp_func src/exec/merge_join_node.h /^ std::vector _cmp_func;$/;" m class:doris::MergeJoinNode +_cmp_tablet_by_create_time src/olap/tablet_manager.cpp /^static bool _cmp_tablet_by_create_time(const TabletSharedPtr& a, const TabletSharedPtr& b) {$/;" f namespace:doris +_code_buf src/olap/rowset/segment_v2/binary_dict_page.h /^ faststring _code_buf;$/;" m class:doris::segment_v2::BinaryDictPageDecoder +_codegend_except_materialize_batch_fns src/exec/except_node.h /^ std::vector _codegend_except_materialize_batch_fns;$/;" m class:doris::ExceptNode +_codegend_union_materialize_batch_fns src/exec/union_node.h /^ std::vector _codegend_union_materialize_batch_fns;$/;" m class:doris::UnionNode +_coder_map src/olap/key_coder.cpp /^ std::unordered_map _coder_map;$/;" m class:doris::KeyCoderResolver file: +_col_data src/runtime/vectorized_row_batch.h /^ void* _col_data = nullptr;$/;" m class:doris::ColumnVector +_col_ids src/olap/schema.h /^ std::vector _col_ids;$/;" m class:doris::Schema +_col_index src/olap/olap_cond.h /^ int32_t _col_index;$/;" m class:doris::CondColumn +_col_name src/olap/tablet_schema.h /^ std::string _col_name;$/;" m class:doris::TabletColumn +_col_name src/runtime/descriptors.h /^ const std::string _col_name;$/;" m class:doris::SlotDescriptor +_col_offsets src/olap/schema.h /^ std::vector _col_offsets;$/;" m class:doris::Schema +_col_pos src/runtime/descriptors.h /^ const int _col_pos;$/;" m class:doris::SlotDescriptor +_col_predicates src/olap/reader.h /^ std::vector _col_predicates;$/;" m class:doris::Reader +_col_predicates src/olap/rowset/column_data.h /^ const std::vector* _col_predicates;$/;" m class:doris::ColumnData +_col_predicates src/olap/rowset/segment_v2/segment_iterator.h /^ std::vector _col_predicates;$/;" m class:doris::segment_v2::SegmentIterator +_col_vector test/olap/column_reader_test.cpp /^ std::unique_ptr _col_vector;$/;" m class:doris::TestColumn file: +_col_vector test/olap/schema_change_test.cpp /^ std::unique_ptr _col_vector;$/;" m class:doris::TestColumn file: +_col_vectors src/runtime/vectorized_row_batch.h /^ std::vector _col_vectors;$/;" m class:doris::VectorizedRowBatch +_collect_iter src/olap/reader.h /^ CollectIterator* _collect_iter = nullptr;$/;" m class:doris::Reader +_collect_query_statistics_with_every_batch src/runtime/plan_fragment_executor.h /^ bool _collect_query_statistics_with_every_batch; $/;" m class:doris::PlanFragmentExecutor +_collectors src/util/metrics.h /^ std::map _collectors;$/;" m class:doris::MetricRegistry +_cols src/olap/schema.h /^ std::vector _cols;$/;" m class:doris::Schema +_cols src/olap/tablet_schema.h /^ std::vector _cols;$/;" m class:doris::TabletSchema +_cols src/runtime/vectorized_row_batch.h /^ const std::vector& _cols;$/;" m class:doris::VectorizedRowBatch +_column src/olap/rowset/column_writer.h /^ const TabletColumn& _column;$/;" m class:doris::ColumnWriter +_column_data_writer src/olap/rowset/alpha_rowset_writer.h /^ ColumnDataWriter* _column_data_writer;$/;" m class:doris::AlphaRowsetWriter +_column_datas src/olap/row_block2.h /^ std::vector _column_datas;$/;" m class:doris::RowBlockV2 +_column_function_map src/exec/csv_scan_node.h /^ std::map _column_function_map;$/;" m class:doris::CsvScanNode +_column_id src/olap/column_predicate.h /^ uint32_t _column_id;$/;" m class:doris::ColumnPredicate +_column_id src/olap/rowset/column_reader.h /^ uint32_t _column_id; \/\/ column在schema内的id$/;" m class:doris::ColumnReader +_column_id src/olap/rowset/column_writer.h /^ uint32_t _column_id;$/;" m class:doris::ColumnWriter +_column_id src/olap/rowset/column_writer.h /^ uint32_t _column_id;$/;" m class:doris::IntegerColumnWriter +_column_id_to_footer_ordinal src/olap/rowset/segment_v2/segment.h /^ std::unordered_map _column_id_to_footer_ordinal;$/;" m class:doris::segment_v2::Segment +_column_index src/exec/schema_scanner/schema_columns_scanner.h /^ int _column_index;$/;" m class:doris::SchemaColumnsScanner +_column_indices src/olap/rowset/segment_reader.h /^ std::vector _column_indices; \/\/ 保存column的index$/;" m class:doris::SegmentReader +_column_iterators src/olap/rowset/segment_v2/segment_iterator.h /^ std::vector _column_iterators;$/;" m class:doris::segment_v2::SegmentIterator +_column_name src/exec/olap_common.h /^ std::string _column_name;$/;" m class:doris::ColumnValueRange +_column_names src/exec/es_http_scan_node.h /^ std::vector _column_names;$/;" m class:doris::EsHttpScanNode +_column_null_bitmaps src/olap/row_block2.h /^ std::vector _column_null_bitmaps;$/;" m class:doris::RowBlockV2 +_column_num src/exec/schema_scanner.h /^ int _column_num;$/;" m class:doris::SchemaScanner +_column_reader test/olap/column_reader_test.cpp /^ ColumnReader *_column_reader;$/;" m class:doris::TestColumn file: +_column_reader test/olap/schema_change_test.cpp /^ ColumnReader *_column_reader;$/;" m class:doris::TestColumn file: +_column_readers src/olap/rowset/segment_reader.h /^ std::vector _column_readers; \/\/ 实际的数据读取器$/;" m class:doris::SegmentReader +_column_readers src/olap/rowset/segment_v2/segment.h /^ std::vector> _column_readers;$/;" m class:doris::segment_v2::Segment +_column_separator src/exec/csv_scan_node.h /^ std::string _column_separator;$/;" m class:doris::CsvScanNode +_column_slot_map src/exec/csv_scan_node.h /^ std::map _column_slot_map;$/;" m class:doris::CsvScanNode +_column_slot_vec src/exec/csv_scan_node.h /^ std::vector _column_slot_vec;$/;" m class:doris::CsvScanNode +_column_type src/exec/olap_common.h /^ PrimitiveType _column_type; \/\/ Column type (eg: TINYINT,SMALLINT,INT,BIGINT)$/;" m class:doris::ColumnValueRange +_column_type_map src/exec/csv_scan_node.h /^ std::map _column_type_map;$/;" m class:doris::CsvScanNode +_column_type_vec src/exec/csv_scan_node.h /^ std::vector _column_type_vec;$/;" m class:doris::CsvScanNode +_column_types src/exec/olap_rewrite_node.h /^ std::vector _column_types;$/;" m class:doris::OlapRewriteNode +_column_unique_id src/olap/rowset/column_reader.h /^ uint32_t _column_unique_id; \/\/ column的唯一id$/;" m class:doris::ColumnReader +_column_unique_id src/olap/rowset/column_reader.h /^ uint32_t _column_unique_id;$/;" m class:doris::IntegerColumnReader +_column_unique_id src/olap/rowset/column_reader.h /^ uint32_t _column_unique_id;$/;" m class:doris::StringColumnDictionaryReader +_column_unique_id src/olap/rowset/column_reader.h /^ uint32_t _column_unique_id;$/;" m class:doris::StringColumnDirectReader +_column_value_ranges src/exec/olap_scan_node.h /^ std::map _column_value_ranges;$/;" m class:doris::OlapScanNode +_column_writer test/olap/column_reader_test.cpp /^ ColumnWriter *_column_writer;$/;" m class:doris::TestColumn file: +_column_writer test/olap/schema_change_test.cpp /^ ColumnWriter *_column_writer;$/;" m class:doris::TestColumn file: +_column_writers src/olap/rowset/segment_v2/segment_writer.h /^ std::vector> _column_writers;$/;" m class:doris::segment_v2::SegmentWriter +_columns src/exec/csv_scan_node.h /^ std::vector _columns;$/;" m class:doris::CsvScanNode +_columns src/exec/mysql_scan_node.h /^ std::vector _columns;$/;" m class:doris::MysqlScanNode +_columns src/exec/olap_rewrite_node.h /^ std::vector _columns;$/;" m class:doris::OlapRewriteNode +_columns src/exec/schema_scanner.h /^ ColumnDesc* _columns;$/;" m class:doris::SchemaScanner +_columns src/olap/olap_cond.h /^ CondColumns _columns; \/\/ list of condition column$/;" m class:doris::Conditions +_compaction_rowset_type src/olap/storage_engine.h /^ RowsetTypePB _compaction_rowset_type;$/;" m class:doris::StorageEngine +_compare src/olap/olap_index.h /^ bool _compare(const iterator_offset_t& index,$/;" f class:doris::IndexComparator +_compare src/olap/olap_index.h /^ bool _compare(const iterator_offset_t& index,$/;" f class:doris::SegmentComparator +_compare src/olap/rowset/column_data.h /^ bool _compare($/;" f class:doris::ColumnDataComparator +_compare_less_than src/runtime/merge_sorter.h /^ TupleRowComparator _compare_less_than;$/;" m class:doris::MergeSorter +_compare_less_than src/runtime/sorted_run_merger.h /^ TupleRowComparator _compare_less_than;$/;" m class:doris::SortedRunMerger +_compare_less_than src/runtime/spill_sorter.h /^ TupleRowComparator _compare_less_than;$/;" m class:doris::SpillSorter +_compress src/olap/out_stream.cpp /^OLAPStatus OutStream::_compress(StorageByteBuffer* input, StorageByteBuffer* output,$/;" f class:doris::OutStream +_compress src/util/block_compression.cpp /^ Status _compress(LZ4F_compressionContext_t ctx,$/;" f class:doris::Lz4fBlockCompression file: +_compress_buffer_size src/olap/file_stream.h /^ size_t _compress_buffer_size;$/;" m class:doris::ReadOnlyFileStream +_compress_buffer_size src/olap/in_stream.h /^ uint32_t _compress_buffer_size;$/;" m class:doris::InStream +_compress_codec src/olap/rowset/segment_v2/column_reader.h /^ const BlockCompressionCodec* _compress_codec = nullptr;$/;" m class:doris::segment_v2::ColumnReader +_compress_codec src/olap/rowset/segment_v2/column_writer.h /^ const BlockCompressionCodec* _compress_codec = nullptr;$/;" m class:doris::segment_v2::ColumnWriter +_compress_codec src/olap/rowset/segment_v2/indexed_column_reader.h /^ const BlockCompressionCodec* _compress_codec = nullptr;$/;" m class:doris::segment_v2::IndexedColumnReader +_compress_codec src/olap/rowset/segment_v2/indexed_column_writer.h /^ const BlockCompressionCodec* _compress_codec;$/;" m class:doris::segment_v2::IndexedColumnWriter +_compress_kind src/olap/out_stream.h /^ CompressKind _compress_kind;$/;" m class:doris::OutStreamFactory +_compress_kind src/olap/rowset/column_data_writer.h /^ CompressKind _compress_kind;$/;" m class:doris::ColumnDataWriter +_compress_kind src/olap/rowset/segment_writer.h /^ CompressKind _compress_kind;$/;" m class:doris::SegmentWriter +_compress_kind src/olap/tablet_schema.h /^ CompressKind _compress_kind;$/;" m class:doris::TabletSchema +_compressed src/olap/in_stream.h /^ StorageByteBuffer* _compressed;$/;" m class:doris::InStream +_compressed src/olap/out_stream.h /^ StorageByteBuffer* _compressed; \/\/ 即将输出到output_buffers中的字节$/;" m class:doris::OutStream +_compressed_bytes_read src/exec/olap_scanner.h /^ int64_t _compressed_bytes_read = 0;$/;" m class:doris::OlapScanner +_compressed_helper src/olap/file_stream.h /^ StorageByteBuffer* _compressed_helper;$/;" m class:doris::ReadOnlyFileStream +_compressed_size src/olap/rowset/segment_v2/bitshuffle_page.h /^ size_t _compressed_size;$/;" m class:doris::segment_v2::BitShufflePageDecoder +_compression src/util/tdigest.h /^ Value _compression;$/;" m class:doris::TDigest +_compression_scratch src/runtime/row_batch.h /^ std::string _compression_scratch;$/;" m class:doris::RowBatch +_compressor src/olap/out_stream.h /^ Compressor _compressor; \/\/ 压缩函数,如果为NULL表示不压缩$/;" m class:doris::OutStream +_compressor src/olap/out_stream.h /^ Compressor _compressor;$/;" m class:doris::OutStreamFactory +_compute_checksum src/olap/task/engine_checksum_task.cpp /^OLAPStatus EngineChecksumTask::_compute_checksum() {$/;" f class:doris::EngineChecksumTask +_compute_dist_hash src/exec/tablet_info.cpp /^uint32_t OlapTablePartitionParam::_compute_dist_hash(Tuple* key) const {$/;" f class:doris::OlapTablePartitionParam +_compute_fn src/exec/olap_scan_node.h /^ CompareLargeFunc _compute_fn;$/;" m class:doris::OlapScanNode::MergeComparison +_compute_layout src/olap/row_block.cpp /^void RowBlock::_compute_layout() {$/;" f class:doris::RowBlock +_concurrency_sem src/olap/compaction.cpp /^Semaphore Compaction::_concurrency_sem;$/;" m class:doris::Compaction file: +_concurrency_sem src/olap/compaction.h /^ static Semaphore _concurrency_sem;$/;" m class:doris::Compaction +_cond src/util/barrier.h /^ ConditionVariable _cond;$/;" m class:doris::Barrier +_condition src/util/condition_variable.h /^ mutable pthread_cond_t _condition;$/;" m class:doris::ConditionVariable +_conditions src/olap/reader.h /^ Conditions _conditions;$/;" m class:doris::Reader +_conditions src/olap/rowset/column_data.h /^ const Conditions* _conditions;$/;" m class:doris::ColumnData +_conditions src/olap/rowset/segment_reader.h /^ const Conditions* _conditions; \/\/ 列过滤条件$/;" m class:doris::SegmentReader +_conds src/olap/olap_cond.h /^ std::vector _conds;$/;" m class:doris::CondColumn +_config_disk_throttle src/agent/cgroups_mgr.cpp /^AgentStatus CgroupsMgr::_config_disk_throttle(std::string user_name,$/;" f class:doris::CgroupsMgr +_config_user_disk_throttle src/agent/cgroups_mgr.cpp /^void CgroupsMgr::_config_user_disk_throttle(std::string user_name,$/;" f class:doris::CgroupsMgr +_conjunct_ctxs src/exec/es_http_scanner.h /^ const std::vector& _conjunct_ctxs; $/;" m class:doris::EsHttpScanner +_conjunct_ctxs src/exec/exec_node.h /^ std::vector _conjunct_ctxs;$/;" m class:doris::ExecNode +_conjunct_ctxs src/exec/kudu_scanner.h /^ std::vector _conjunct_ctxs;$/;" m class:doris::KuduScanner +_conjunct_ctxs src/exec/olap_scanner.h /^ std::vector _conjunct_ctxs;$/;" m class:doris::OlapScanner +_conjuncts src/exec/exec_node.h /^ std::vector _conjuncts;$/;" m class:doris::ExecNode +_conn_info src/runtime/mysql_table_sink.h /^ MysqlConnInfo _conn_info;$/;" m class:doris::MysqlTableSink +_conn_info test/runtime/mysql_table_writer_test.cpp /^ MysqlConnInfo _conn_info;$/;" m class:doris::MysqlTableWriterTest file: +_connections_total src/util/thrift_server.h /^ std::unique_ptr _connections_total;$/;" m class:doris::ThriftServer +_const_expr_list_idx src/exec/except_node.h /^ int _const_expr_list_idx;$/;" m class:doris::ExceptNode +_const_expr_list_idx src/exec/union_node.h /^ int _const_expr_list_idx;$/;" m class:doris::UnionNode +_const_expr_lists src/exec/except_node.h /^ std::vector> _const_expr_lists;$/;" m class:doris::ExceptNode +_const_expr_lists src/exec/union_node.h /^ std::vector> _const_expr_lists;$/;" m class:doris::UnionNode +_const_result_expr_ctx_lists src/exec/merge_node.h /^ std::vector > _const_result_expr_ctx_lists;$/;" m class:doris::MergeNode +_const_result_expr_idx src/exec/merge_node.h /^ int _const_result_expr_idx;$/;" m class:doris::MergeNode +_constant_args src/udf/udf_internal.h /^ std::vector _constant_args;$/;" m class:doris::FunctionContextImpl +_constant_val src/exprs/expr.h /^ std::shared_ptr _constant_val;$/;" m class:doris::Expr +_construct_err_sg_data_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::_construct_err_sg_data_file_path(const std::string& path_prefix, int32_t segment_id) const {$/;" f class:doris::SegmentGroup +_construct_err_sg_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::_construct_err_sg_file_path(const std::string& path_prefix, int32_t segment_id, const std::string& suffix) const {$/;" f class:doris::SegmentGroup +_construct_err_sg_index_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::_construct_err_sg_index_file_path(const std::string& path_prefix, int32_t segment_id) const {$/;" f class:doris::SegmentGroup +_construct_fail src/exec/pre_aggregation_node.h /^ bool _construct_fail;$/;" m class:doris::PreAggregationNode +_construct_file_name src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::_construct_file_name(const RowsetId& rowset_id, int32_t segment_id, const string& suffix) const {$/;" f class:doris::SegmentGroup +_construct_file_name src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::_construct_file_name(int32_t segment_id, const string& suffix) const {$/;" f class:doris::SegmentGroup +_construct_index_stream_key src/olap/rowset/segment_reader.cpp /^CacheKey SegmentReader::_construct_index_stream_key($/;" f class:doris::SegmentReader +_construct_old_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::_construct_old_file_path(const std::string& path_prefix, int32_t segment_id, const std::string& suffix) const {$/;" f class:doris::SegmentGroup +_construct_old_pending_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::_construct_old_pending_file_path(const std::string& path_prefix, int32_t segment_id,$/;" f class:doris::SegmentGroup +_consumers src/runtime/routine_load/data_consumer_group.h /^ std::vector> _consumers;$/;" m class:doris::DataConsumerGroup +_consumption src/runtime/mem_tracker.h /^ RuntimeProfile::HighWaterMarkCounter* _consumption;$/;" m class:doris::MemTracker +_consumption_metric src/runtime/mem_tracker.h /^ UIntGauge* _consumption_metric;$/;" m class:doris::MemTracker +_contains_rowset src/olap/tablet.cpp /^bool Tablet::_contains_rowset(const RowsetId rowset_id) {$/;" f class:doris::Tablet +_contains_var_len_grouping_exprs src/exec/partitioned_aggregation_node.h /^ bool _contains_var_len_grouping_exprs;$/;" m class:doris::PartitionedAggregationNode +_contains_version src/olap/tablet.cpp /^OLAPStatus Tablet::_contains_version(const Version& version) {$/;" f class:doris::Tablet +_content src/http/http_response.h /^ const std::string* _content;$/;" m class:doris::HttpResponse +_content_adler32 src/runtime/dpp_writer.h /^ uint32_t _content_adler32;$/;" m class:doris::DppWriter +_content_len src/olap/push_handler.h /^ size_t _content_len;$/;" m class:doris::IBinaryReader +_content_type src/http/http_response.h /^ std::string _content_type;$/;" m class:doris::HttpResponse +_context src/exec/es/es_predicate.h /^ ExprContext* _context; $/;" m class:doris::EsPredicate +_context src/olap/rowset/beta_rowset_reader.h /^ RowsetReaderContext* _context;$/;" m class:doris::BetaRowsetReader +_context src/olap/rowset/beta_rowset_writer.h /^ RowsetWriterContext _context;$/;" m class:doris::BetaRowsetWriter +_context src/udf/udf_internal.h /^ doris_udf::FunctionContext* _context;$/;" m class:doris::FunctionContextImpl +_convert src/olap/push_handler.cpp /^OLAPStatus PushHandler::_convert(TabletSharedPtr cur_tablet,$/;" f class:doris::PushHandler +_convert_batch src/exec/tablet_sink.cpp /^void OlapTableSink::_convert_batch(RuntimeState* state, RowBatch* input_batch, RowBatch* output_batch) {$/;" f class:doris::stream_load::OlapTableSink +_convert_batch_ns src/exec/tablet_sink.h /^ int64_t _convert_batch_ns = 0;$/;" m class:doris::stream_load::OlapTableSink +_convert_batch_timer src/exec/tablet_sink.h /^ RuntimeProfile::Counter* _convert_batch_timer = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_convert_beta_rowsets_to_alpha src/olap/snapshot_manager.cpp /^OLAPStatus SnapshotManager::_convert_beta_rowsets_to_alpha(const TabletMetaSharedPtr& new_tablet_meta,$/;" f class:doris::SnapshotManager +_convert_explicit_to_register src/olap/hll.cpp /^void HyperLogLog::_convert_explicit_to_register() {$/;" f class:doris::HyperLogLog +_convert_from src/olap/types.h /^ OLAPStatus (*_convert_from)(void* dest, const void* src, const TypeInfo* src_type, MemPool* mem_pool);$/;" m class:doris::TypeInfo +_convert_historical_rowsets src/olap/schema_change.cpp /^OLAPStatus SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangeParams& sc_params) {$/;" f class:doris::SchemaChangeHandler +_convert_old_data_success src/olap/data_dir.h /^ bool _convert_old_data_success;$/;" m class:doris::DataDir +_convert_old_tablet src/olap/data_dir.cpp /^OLAPStatus DataDir::_convert_old_tablet() {$/;" f class:doris::DataDir +_convert_row_batch_timer src/exec/exchange_node.h /^ RuntimeProfile::Counter* _convert_row_batch_timer;$/;" m class:doris::ExchangeNode +_convert_row_to_tuple src/exec/olap_scanner.cpp /^void OlapScanner::_convert_row_to_tuple(Tuple* tuple) {$/;" f class:doris::OlapScanner +_convert_rowset src/olap/rowset/rowset_converter.cpp /^OLAPStatus RowsetConverter::_convert_rowset(const RowsetMetaSharedPtr& src_rowset_meta,$/;" f class:doris::RowsetConverter +_convert_to_new_snapshot src/olap/task/engine_clone_task.cpp /^OLAPStatus EngineCloneTask::_convert_to_new_snapshot(const string& clone_dir, int64_t tablet_id) {$/;" f class:doris::EngineCloneTask +_convert_to_smaller_type src/util/bitmap_value.h /^ void _convert_to_smaller_type() {$/;" f class:doris::BitmapValue +_coord_addr src/runtime/fragment_mgr.cpp /^ TNetworkAddress _coord_addr;$/;" m class:doris::FragmentExecState file: +_copy_current_to_output src/olap/rowset/segment_v2/binary_prefix_page.cpp /^Status BinaryPrefixPageDecoder::_copy_current_to_output(MemPool* mem_pool, Slice* output) {$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +_copy_from src/olap/schema.cpp /^void Schema::_copy_from(const Schema& other) {$/;" f class:doris::Schema +_copy_index_and_data_files src/olap/task/engine_storage_migration_task.cpp /^OLAPStatus EngineStorageMigrationTask::_copy_index_and_data_files($/;" f class:doris::EngineStorageMigrationTask +_copy_next_values src/olap/rowset/segment_v2/bitshuffle_page.h /^ void _copy_next_values(size_t n, void* data) {$/;" f class:doris::segment_v2::BitShufflePageDecoder +_copy_object src/olap/types.h /^ void (*_copy_object)(void* dest, const void* src, MemPool* mem_pool);$/;" m class:doris::TypeInfo +_copy_size src/olap/task/engine_clone_task.h /^ int64_t _copy_size;$/;" m class:doris::EngineCloneTask +_copy_time_ms src/olap/task/engine_clone_task.h /^ int64_t _copy_time_ms;$/;" m class:doris::EngineCloneTask +_copy_value_at src/olap/rowset/segment_v2/binary_plain_page.h /^ void _copy_value_at(size_t idx, faststring* value) const {$/;" f class:doris::segment_v2::BinaryPlainPageBuilder +_count src/olap/rowset/segment_v2/binary_prefix_page.h /^ size_t _count = 0;$/;" m class:doris::segment_v2::BinaryPrefixPageBuilder +_count src/olap/rowset/segment_v2/bitshuffle_page.h /^ uint32_t _count;$/;" m class:doris::segment_v2::BitshufflePageBuilder +_count src/olap/rowset/segment_v2/frame_of_reference_page.h /^ size_t _count;$/;" m class:doris::segment_v2::FrameOfReferencePageBuilder +_count src/olap/rowset/segment_v2/index_page.h /^ uint32_t _count = 0;$/;" m class:doris::segment_v2::IndexPageBuilder +_count src/olap/rowset/segment_v2/plain_page.h /^ size_t _count;$/;" m class:doris::segment_v2::PlainPageBuilder +_count src/olap/rowset/segment_v2/rle_page.h /^ size_t _count;$/;" m class:doris::segment_v2::RlePageBuilder +_count src/olap/rowset/segment_v2/row_ranges.h /^ size_t _count;$/;" m class:doris::segment_v2::RowRanges +_count src/util/barrier.h /^ int _count;$/;" m class:doris::Barrier +_count src/util/counter_cond_variable.hpp /^ int _count;$/;" m class:doris::CounterCondVariable +_count src/util/semaphore.hpp /^ int _count;$/;" m class:__anon35::Semaphore +_count_none_nulls src/olap/rowset/column_reader.cpp /^uint64_t ColumnReader::_count_none_nulls(uint64_t rows) {$/;" f class:doris::ColumnReader +_counter src/exec/base_scanner.h /^ ScannerCounter* _counter;$/;" m class:doris::BaseScanner +_counter src/exec/es_http_scanner.h /^ EsScanCounter* _counter;$/;" m class:doris::EsHttpScanner +_counter src/runtime/routine_load/data_consumer_group.h /^ int _counter;$/;" m class:doris::DataConsumerGroup +_counter src/util/runtime_profile.h /^ RuntimeProfile::Counter* _counter;$/;" m class:doris::ScopedCounter +_counter src/util/runtime_profile.h /^ RuntimeProfile::Counter* _counter;$/;" m class:doris::ScopedTimer +_counter src/util/runtime_profile.h /^ int64_t* _counter;$/;" m class:doris::ScopedRawTimer +_counter test/exec/broker_scanner_test.cpp /^ ScannerCounter _counter;$/;" m class:doris::BrokerScannerTest file: +_counter test/exec/orc_scanner_test.cpp /^ ScannerCounter _counter;$/;" m class:doris::OrcScannerTest file: +_counter test/runtime/thread_resource_mgr_test.cpp /^ int _counter;$/;" m class:doris::NotifiedCounter file: +_counter test/util/threadpool_test.cpp /^ std::atomic* _counter;$/;" m class:doris::SimpleTask file: +_counter_fn src/util/runtime_profile.h /^ DerivedCounterFunction _counter_fn;$/;" m class:doris::RuntimeProfile::DerivedCounter +_counter_map src/util/runtime_profile.h /^ CounterMap _counter_map;$/;" m class:doris::RuntimeProfile +_counter_map_lock src/util/runtime_profile.h /^ mutable boost::mutex _counter_map_lock;$/;" m class:doris::RuntimeProfile +_counter_names src/util/perf_counters.h /^ std::vector _counter_names;$/;" m class:doris::PerfCounters +_counter_total_time src/util/runtime_profile.h /^ Counter _counter_total_time;$/;" m class:doris::RuntimeProfile +_counters src/util/perf_counters.h /^ std::vector _counters;$/;" m class:doris::PerfCounters +_cpu_total src/util/system_metrics.h /^ std::unique_ptr _cpu_total;$/;" m class:doris::SystemMetrics +_create_column_readers src/olap/rowset/segment_v2/segment.cpp /^Status Segment::_create_column_readers() {$/;" f class:doris::segment_v2::Segment +_create_hard_link_recursive src/http/action/restore_tablet_action.cpp /^Status RestoreTabletAction::_create_hard_link_recursive(const std::string& src, const std::string& dst) {$/;" f class:doris::RestoreTabletAction +_create_inital_rowset_unlocked src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::_create_inital_rowset_unlocked(const TCreateTabletReq& request,$/;" f class:doris::TabletManager +_create_new_input_buffer src/olap/out_stream.cpp /^OLAPStatus OutStream::_create_new_input_buffer() {$/;" f class:doris::OutStream +_create_partition_key src/exec/tablet_info.cpp /^Status OlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, Tuple* tuple, SlotDescriptor* slot_desc) {$/;" f class:doris::OlapTablePartitionParam +_create_partition_keys src/exec/tablet_info.cpp /^Status OlapTablePartitionParam::_create_partition_keys(const std::vector& t_exprs, Tuple** part_key) {$/;" f class:doris::OlapTablePartitionParam +_create_reader src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_create_reader(size_t* buffer_size) {$/;" f class:doris::SegmentReader +_create_segment_writer src/olap/rowset/beta_rowset_writer.cpp /^OLAPStatus BetaRowsetWriter::_create_segment_writer() {$/;" f class:doris::BetaRowsetWriter +_create_snapshot_files src/olap/snapshot_manager.cpp /^OLAPStatus SnapshotManager::_create_snapshot_files($/;" f class:doris::SnapshotManager +_create_tablet test/olap/delete_handler_test.cpp /^ TCreateTabletReq _create_tablet;$/;" m class:doris::TestDeleteConditionHandler file: +_create_tablet test/olap/delete_handler_test.cpp /^ TCreateTabletReq _create_tablet;$/;" m class:doris::TestDeleteConditionHandler2 file: +_create_tablet test/olap/delete_handler_test.cpp /^ TCreateTabletReq _create_tablet;$/;" m class:doris::TestDeleteHandler file: +_create_tablet test/olap/olap_reader_test.cpp /^ TCreateTabletReq _create_tablet;$/;" m class:doris::TestOLAPReaderColumn file: +_create_tablet test/olap/olap_reader_test.cpp /^ TCreateTabletReq _create_tablet;$/;" m class:doris::TestOLAPReaderColumnDeleteCondition file: +_create_tablet test/olap/olap_reader_test.cpp /^ TCreateTabletReq _create_tablet;$/;" m class:doris::TestOLAPReaderRow file: +_create_tablet test/olap/vectorized_olap_reader_test.cpp /^ TCreateTabletReq _create_tablet;$/;" m class:doris::TestVectorizedOLAPReader file: +_create_tablet_meta_and_dir_unlocked src/olap/tablet_manager.cpp /^TabletSharedPtr TabletManager::_create_tablet_meta_and_dir_unlocked($/;" f class:doris::TabletManager +_create_tablet_meta_unlocked src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::_create_tablet_meta_unlocked(const TCreateTabletReq& request,$/;" f class:doris::TabletManager +_create_tablet_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_create_tablet_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_create_tablet_workers src/agent/agent_server.h /^ std::unique_ptr _create_tablet_workers;$/;" m class:doris::AgentServer +_created_block_counter src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _created_block_counter;$/;" m class:doris::BufferedBlockMgr2 +_created_tmp_dirs test/runtime/buffered_block_mgr2_test.cpp /^ vector _created_tmp_dirs;$/;" m class:doris::BufferedBlockMgrTest file: +_creation_time src/olap/tablet_meta.h /^ int64_t _creation_time = 0;$/;" m class:doris::TabletMeta +_csv_scanner src/exec/csv_scan_node.h /^ boost::scoped_ptr _csv_scanner;$/;" m class:doris::CsvScanNode +_ctor_status src/exec/olap_scanner.h /^ Status _ctor_status;$/;" m class:doris::OlapScanner +_ctype src/exec/decompressor.h /^ CompressType _ctype;$/;" m class:doris::Decompressor +_cumulative src/util/tdigest.h /^ std::vector _cumulative;$/;" m class:doris::TDigest +_cumulative_compaction_thread_callback src/olap/olap_server.cpp /^void* StorageEngine::_cumulative_compaction_thread_callback(void* arg, DataDir* data_dir) {$/;" f class:doris::StorageEngine +_cumulative_compaction_threads src/olap/storage_engine.h /^ std::vector _cumulative_compaction_threads;$/;" m class:doris::StorageEngine +_cumulative_layer_point src/olap/tablet_meta.h /^ int64_t _cumulative_layer_point = 0;$/;" m class:doris::TabletMeta +_cumulative_lock src/olap/tablet.h /^ Mutex _cumulative_lock;$/;" m class:doris::Tablet +_cumulative_point src/olap/tablet.h /^ std::atomic _cumulative_point;$/;" m class:doris::Tablet +_cumulative_rowset_size_threshold src/olap/cumulative_compaction.h /^ int64_t _cumulative_rowset_size_threshold;$/;" m class:doris::CumulativeCompaction +_cur_child src/olap/reader.cpp /^ ChildCtx* _cur_child = nullptr;$/;" m class:doris::CollectIterator file: +_cur_decompressor src/exec/broker_scanner.h /^ Decompressor* _cur_decompressor;$/;" m class:doris::BrokerScanner +_cur_field_idx src/util/arrow/row_batch.cpp /^ size_t _cur_field_idx;$/;" m class:doris::FromRowBatchConverter file: +_cur_field_idx src/util/arrow/row_block.cpp /^ size_t _cur_field_idx;$/;" m class:doris::FromRowBlockConverter file: +_cur_field_idx src/util/arrow/row_block.cpp /^ size_t _cur_field_idx;$/;" m class:doris::ToRowBlockConverter file: +_cur_file_eof src/exec/orc_scanner.h /^ bool _cur_file_eof;$/;" m class:doris::ORCScanner +_cur_file_eof src/exec/parquet_scanner.h /^ bool _cur_file_eof; \/\/ is read over?$/;" m class:doris::ParquetScanner +_cur_file_reader src/exec/broker_scanner.h /^ FileReader* _cur_file_reader;$/;" m class:doris::BrokerScanner +_cur_file_reader src/exec/parquet_scanner.h /^ ParquetReaderWrap* _cur_file_reader;$/;" m class:doris::ParquetScanner +_cur_idx src/olap/rowset/segment_v2/binary_plain_page.h /^ uint32_t _cur_idx;$/;" m class:doris::segment_v2::BinaryPlainPageDecoder +_cur_idx src/olap/rowset/segment_v2/ordinal_page_index.h /^ int32_t _cur_idx;$/;" m class:doris::segment_v2::OrdinalPageIndexIterator +_cur_idx src/olap/rowset/segment_v2/plain_page.h /^ uint32_t _cur_idx;$/;" m class:doris::segment_v2::PlainPageDecoder +_cur_index src/olap/rowset/segment_v2/bitshuffle_page.h /^ size_t _cur_index;$/;" m class:doris::segment_v2::BitShufflePageDecoder +_cur_index src/olap/rowset/segment_v2/frame_of_reference_page.h /^ size_t _cur_index;$/;" m class:doris::segment_v2::FrameOfReferencePageDecoder +_cur_index src/olap/rowset/segment_v2/rle_page.h /^ size_t _cur_index;$/;" m class:doris::segment_v2::RlePageDecoder +_cur_kudu_batch src/exec/kudu_scanner.h /^ kudu::client::KuduScanBatch _cur_kudu_batch;$/;" m class:doris::KuduScanner +_cur_kudu_batch_num_read src/exec/kudu_scanner.h /^ int _cur_kudu_batch_num_read;$/;" m class:doris::KuduScanner +_cur_line_reader src/exec/broker_scanner.h /^ LineReader* _cur_line_reader;$/;" m class:doris::BrokerScanner +_cur_line_reader_eof src/exec/broker_scanner.h /^ bool _cur_line_reader_eof;$/;" m class:doris::BrokerScanner +_cur_offset src/exec/broker_reader.h /^ int64_t _cur_offset;$/;" m class:doris::BrokerReader +_cur_offset src/exec/broker_writer.h /^ int64_t _cur_offset;$/;" m class:doris::BrokerWriter +_cur_pos src/olap/rowset/segment_v2/binary_prefix_page.h /^ uint32_t _cur_pos = 0;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +_cur_row_expr_ctxs src/runtime/dpp_sink.cpp /^ std::vector _cur_row_expr_ctxs;$/;" m class:doris::Translator file: +_cur_rowid src/olap/rowset/segment_v2/segment_iterator.h /^ rowid_t _cur_rowid;$/;" m class:doris::segment_v2::SegmentIterator +_cur_rowset src/olap/delta_writer.h /^ RowsetSharedPtr _cur_rowset;$/;" m class:doris::DeltaWriter +_cur_seg src/olap/olap_index.h /^ iterator_offset_t _cur_seg;$/;" m class:doris::IndexComparator +_cur_segment_group src/olap/rowset/alpha_rowset_writer.h /^ SegmentGroup* _cur_segment_group;$/;" m class:doris::AlphaRowsetWriter +_cur_slice src/util/block_compression.cpp /^ size_t _cur_slice;$/;" m class:doris::SnappySlicesSource file: +_cur_slot_ref src/util/arrow/row_batch.cpp /^ std::unique_ptr _cur_slot_ref;$/;" m class:doris::FromRowBatchConverter file: +_cur_slot_ref src/util/arrow/row_batch.cpp /^ std::unique_ptr _cur_slot_ref;$/;" m class:doris::ToRowBatchConverter file: +_cur_version src/agent/cgroups_mgr.h /^ int64_t _cur_version;$/;" m class:doris::CgroupsMgr +_curl src/http/http_client.h /^ CURL* _curl = nullptr;$/;" m class:doris::HttpClient +_curr src/olap/push_handler.h /^ size_t _curr;$/;" m class:doris::IBinaryReader +_curr_child_batch src/exec/analytic_eval_node.h /^ boost::scoped_ptr _curr_child_batch;$/;" m class:doris::AnalyticEvalNode +_curr_partition_idx src/exec/analytic_eval_node.h /^ int64_t _curr_partition_idx;$/;" m class:doris::AnalyticEvalNode +_curr_tuple src/exec/analytic_eval_node.h /^ Tuple* _curr_tuple;$/;" m class:doris::AnalyticEvalNode +_curr_tuple_pool src/exec/analytic_eval_node.h /^ boost::scoped_ptr _curr_tuple_pool;$/;" m class:doris::AnalyticEvalNode +_current src/olap/out_stream.h /^ StorageByteBuffer* _current; \/\/ 缓存未压缩的数据$/;" m class:doris::OutStream +_current src/olap/rowset/bit_field_reader.h /^ char _current;$/;" m class:doris::BitFieldReader +_current src/olap/rowset/bit_field_writer.h /^ char _current;$/;" m class:doris::BitFieldWriter +_current test/olap/skiplist_test.cpp /^ State _current;$/;" m class:doris::ConcurrentTest file: +_current_batch src/runtime/data_stream_recvr.cc /^ scoped_ptr _current_batch;$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_current_block src/olap/rowset/column_data.h /^ uint32_t _current_block = 0;$/;" m class:doris::ColumnData +_current_block_id src/olap/rowset/segment_reader.h /^ int64_t _current_block_id = 0; \/\/ 当前读取到的块$/;" m class:doris::SegmentReader +_current_build_row src/exec/cross_join_node.h /^ RowBatchList::TupleRowIterator _current_build_row;$/;" m class:doris::CrossJoinNode +_current_channel_idx src/runtime/data_stream_sender.h /^ int _current_channel_idx; \/\/ index of current channel to send to if _random == true$/;" m class:doris::DataStreamSender +_current_compress_position src/olap/file_stream.h /^ size_t _current_compress_position;$/;" m class:doris::ReadOnlyFileStream +_current_connections src/util/thrift_server.h /^ std::unique_ptr _current_connections;$/;" m class:doris::ThriftServer +_current_decoded_frame src/util/frame_of_reference_coding.h /^ uint32_t _current_decoded_frame = -1;$/;" m class:doris::ForDecoder +_current_file src/exec/csv_scanner.h /^ std::ifstream* _current_file;$/;" m class:doris::CsvScanner +_current_file_handler src/olap/rowset/segment_group.h /^ FileHandler _current_file_handler;$/;" m class:doris::SegmentGroup +_current_file_idx src/exec/csv_scanner.h /^ int32_t _current_file_idx;$/;" m class:doris::CsvScanner +_current_group src/exec/orc_scanner.h /^ int _current_group;$/;" m class:doris::ORCScanner +_current_group src/exec/parquet_reader.h /^ int _current_group;$/;" m class:doris::ParquetReaderWrap +_current_index src/util/frame_of_reference_coding.h /^ uint32_t _current_index = 0;$/;" m class:doris::ForDecoder +_current_index_row src/olap/rowset/segment_group.h /^ RowCursor _current_index_row;$/;" m class:doris::SegmentGroup +_current_iter src/olap/rowset/segment_v2/indexed_column_reader.h /^ IndexPageIterator* _current_iter = nullptr;$/;" m class:doris::segment_v2::IndexedColumnIterator +_current_left_child_row src/exec/blocking_join_node.h /^ TupleRow* _current_left_child_row;$/;" m class:doris::BlockingJoinNode +_current_line_of_batch src/exec/parquet_reader.h /^ int _current_line_of_batch;$/;" m class:doris::ParquetReaderWrap +_current_line_of_group src/exec/orc_scanner.h /^ int64_t _current_line_of_group;$/;" m class:doris::ORCScanner +_current_line_of_group src/exec/parquet_reader.h /^ int _current_line_of_group;$/;" m class:doris::ParquetReaderWrap +_current_load src/http/action/mini_load.h /^ std::set _current_load;$/;" m class:doris::MiniLoadAction +_current_num_rows_per_row_block src/olap/rowset/segment_group.h /^ size_t _current_num_rows_per_row_block;$/;" m class:doris::SegmentGroup +_current_offset src/exec/local_file_reader.h /^ int64_t _current_offset;$/;" m class:doris::LocalFileReader +_current_offset src/olap/in_stream.h /^ uint64_t _current_offset;$/;" m class:doris::InStream +_current_ordinal src/olap/rowset/segment_v2/column_reader.h /^ ordinal_t _current_ordinal = 0;$/;" m class:doris::segment_v2::FileColumnIterator +_current_ordinal src/olap/rowset/segment_v2/indexed_column_reader.h /^ ordinal_t _current_ordinal = 0;$/;" m class:doris::segment_v2::IndexedColumnIterator +_current_pb_batch src/runtime/data_stream_sender.h /^ PRowBatch* _current_pb_batch = nullptr;$/;" m class:doris::DataStreamSender +_current_probe_row src/exec/hash_join_node.h /^ TupleRow* _current_probe_row;$/;" m class:doris::HashJoinNode +_current_process_handle src/runtime/user_function_cache.h /^ void* _current_process_handle = nullptr;$/;" m class:doris::UserFunctionCache +_current_range src/olap/in_stream.h /^ uint64_t _current_range;$/;" m class:doris::InStream +_current_read_context src/olap/rowset/alpha_rowset_reader.h /^ RowsetReaderContext* _current_read_context;$/;" m class:doris::AlphaRowsetReader +_current_row src/olap/reader.cpp /^ const RowCursor* _current_row = nullptr;$/;" m class:doris::CollectIterator::ChildCtx file: +_current_row src/olap/rowset/column_data.h /^ const RowCursor* _current_row() {$/;" f class:doris::ColumnData +_current_rowid src/olap/rowset/segment_v2/bitmap_index_reader.h /^ rowid_t _current_rowid;$/;" m class:doris::segment_v2::BitmapIndexIterator +_current_rowid src/olap/rowset/segment_v2/column_reader.h /^ ordinal_t _current_rowid = 0;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_current_rowset_meta src/olap/rowset/alpha_rowset_writer.h /^ std::shared_ptr _current_rowset_meta;$/;" m class:doris::AlphaRowsetWriter +_current_segment src/olap/rowset/column_data.h /^ uint32_t _current_segment;$/;" m class:doris::ColumnData +_current_shard src/olap/data_dir.h /^ uint64_t _current_shard;$/;" m class:doris::DataDir +_current_size src/runtime/tmp_file_mgr.h /^ int64_t _current_size;$/;" m class:doris::TmpFileMgr::File +_current_tuple src/runtime/merge_sorter.cpp /^ uint8_t* _current_tuple = nullptr;$/;" m class:doris::MergeSorter::TupleSorter::TupleIterator file: +_current_tuple src/runtime/spill_sorter.cc /^ uint8_t* _current_tuple;$/;" m class:doris::SpillSorter::TupleSorter::TupleIterator file: +_current_value src/olap/rowset/segment_v2/binary_prefix_page.h /^ faststring _current_value;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +_cursor src/olap/rowset/column_data.h /^ RowCursor _cursor;$/;" m class:doris::ColumnData +_cursor src/olap/rowset/column_data_writer.h /^ RowCursor _cursor;$/;" m class:doris::ColumnDataWriter +_cursor src/olap/schema_change.h /^ RowCursor* _cursor;$/;" m class:doris::SchemaChangeDirectly +_custom_headers src/http/http_response.h /^ std::map> _custom_headers;$/;" m class:doris::HttpResponse +_custom_properties src/runtime/routine_load/data_consumer.h /^ std::unordered_map _custom_properties;$/;" m class:doris::KafkaDataConsumer +_cv src/util/counter_cond_variable.hpp /^ std::condition_variable _cv;$/;" m class:doris::CounterCondVariable +_cv src/util/semaphore.hpp /^ std::condition_variable _cv;$/;" m class:__anon35::Semaphore +_cv_state test/olap/skiplist_test.cpp /^ ConditionVariable _cv_state;$/;" m class:doris::TestState file: +_cycle_count src/util/barrier.h /^ uint32_t _cycle_count = 0;$/;" m class:doris::Barrier +_data src/olap/bloom_filter.hpp /^ uint64_t* _data;$/;" m class:doris::BitSet +_data src/olap/column_block.h /^ uint8_t* _data;$/;" m class:doris::ColumnBlock +_data src/olap/lru_cache.h /^ const char* _data;$/;" m class:doris::CacheKey +_data src/olap/rowset/segment_v2/binary_dict_page.h /^ Slice _data;$/;" m class:doris::segment_v2::BinaryDictPageDecoder +_data src/olap/rowset/segment_v2/binary_plain_page.h /^ Slice _data;$/;" m class:doris::segment_v2::BinaryPlainPageDecoder +_data src/olap/rowset/segment_v2/binary_prefix_page.h /^ Slice _data;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +_data src/olap/rowset/segment_v2/bitshuffle_page.h /^ Slice _data;$/;" m class:doris::segment_v2::BitShufflePageDecoder +_data src/olap/rowset/segment_v2/bitshuffle_page.h /^ faststring _data;$/;" m class:doris::segment_v2::BitshufflePageBuilder +_data src/olap/rowset/segment_v2/bloom_filter.h /^ char* _data;$/;" m class:doris::segment_v2::BloomFilter +_data src/olap/rowset/segment_v2/frame_of_reference_page.h /^ Slice _data;$/;" m class:doris::segment_v2::FrameOfReferencePageDecoder +_data src/olap/rowset/segment_v2/page_handle.h /^ Slice _data;$/;" m class:doris::segment_v2::PageHandle +_data src/olap/rowset/segment_v2/plain_page.h /^ Slice _data;$/;" m class:doris::segment_v2::PlainPageDecoder +_data src/olap/rowset/segment_v2/rle_page.h /^ Slice _data;$/;" m class:doris::segment_v2::RlePageDecoder +_data src/runtime/disk_io_mgr.h /^ const uint8_t* _data;$/;" m class:doris::DiskIoMgr::WriteRange +_data src/runtime/tuple.h /^ void* _data;$/;" m class:doris::Tuple +_data test/exprs/in_predicate_test.cpp /^ int _data[2];$/;" m class:doris::InPredicateTest file: +_data_arriaval src/runtime/buffer_control_block.h /^ boost::condition_variable _data_arriaval;$/;" m class:doris::BufferControlBlock +_data_arrival_cv src/runtime/data_stream_recvr.cc /^ condition_variable _data_arrival_cv;$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_data_buffers test/olap/column_reader_test.cpp /^ std::vector _data_buffers;$/;" m class:doris::TestColumn file: +_data_buffers test/olap/schema_change_test.cpp /^ std::vector _data_buffers;$/;" m class:doris::TestColumn file: +_data_consumer_pool src/runtime/routine_load/routine_load_task_executor.h /^ DataConsumerPool _data_consumer_pool;$/;" m class:doris::RoutineLoadTaskExecutor +_data_dir src/olap/tablet.h /^ DataDir* _data_dir;$/;" m class:doris::Tablet +_data_dir test/olap/olap_snapshot_converter_test.cpp /^ DataDir* _data_dir;$/;" m class:doris::OlapSnapshotConverterTest file: +_data_dir test/olap/tablet_meta_manager_test.cpp /^ DataDir* _data_dir;$/;" m class:doris::TabletMetaManagerTest file: +_data_dir test/olap/tablet_mgr_test.cpp /^ DataDir* _data_dir;$/;" m class:doris::TabletMgrTest file: +_data_len src/olap/bloom_filter.hpp /^ uint32_t _data_len;$/;" m class:doris::BitSet +_data_page src/olap/rowset/segment_v2/indexed_column_reader.h /^ std::unique_ptr _data_page;$/;" m class:doris::segment_v2::IndexedColumnIterator +_data_page_builder src/olap/rowset/segment_v2/binary_dict_page.h /^ std::unique_ptr _data_page_builder;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_data_page_builder src/olap/rowset/segment_v2/indexed_column_writer.h /^ std::unique_ptr _data_page_builder;$/;" m class:doris::segment_v2::IndexedColumnWriter +_data_page_decoder src/olap/rowset/segment_v2/binary_dict_page.h /^ std::unique_ptr _data_page_decoder;$/;" m class:doris::segment_v2::BinaryDictPageDecoder +_data_pages src/exec/partitioned_hash_table.h /^ std::vector _data_pages;$/;" m class:doris::PartitionedHashTable +_data_reader src/olap/rowset/column_reader.h /^ RunLengthByteReader* _data_reader;$/;" m class:doris::TinyColumnReader +_data_reader src/olap/rowset/column_reader.h /^ RunLengthIntegerReader* _data_reader; \/\/ 用来读实际的数据(用一个integer表示)$/;" m class:doris::StringColumnDictionaryReader +_data_reader src/olap/rowset/column_reader.h /^ RunLengthIntegerReader* _data_reader;$/;" m class:doris::IntegerColumnReader +_data_removal src/runtime/buffer_control_block.h /^ boost::condition_variable _data_removal;$/;" m class:doris::BufferControlBlock +_data_removal_cv src/runtime/data_stream_recvr.cc /^ condition_variable _data_removal_cv;$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_data_row_cursor test/olap/delete_handler_test.cpp /^ RowCursor _data_row_cursor;$/;" m class:doris::TestDeleteHandler file: +_data_saved_path src/http/action/stream_load.cpp /^Status StreamLoadAction::_data_saved_path(HttpRequest* req, std::string* file_path) {$/;" f class:doris::StreamLoadAction +_data_size src/olap/olap_index.h /^ size_t _data_size;$/;" m class:doris::MemIndex +_data_size src/olap/rowset/segment_v2/column_writer.h /^ uint64_t _data_size;$/;" m class:doris::segment_v2::ColumnWriter +_data_stream src/olap/rowset/column_reader.h /^ ReadOnlyFileStream* _data_stream;$/;" m class:doris::FloatintPointColumnReader +_data_stream src/olap/rowset/column_reader.h /^ ReadOnlyFileStream* _data_stream;$/;" m class:doris::StringColumnDirectReader +_data_stream src/olap/rowset/column_writer.h /^ OutStream* _data_stream;$/;" m class:doris::VarStringColumnWriter +_data_stream_recvrs_lock src/runtime/runtime_state.h /^ boost::mutex _data_stream_recvrs_lock;$/;" m class:doris::RuntimeState +_data_stream_recvrs_pool src/runtime/runtime_state.h /^ boost::scoped_ptr _data_stream_recvrs_pool;$/;" m class:doris::RuntimeState +_database src/runtime/descriptors.h /^ std::string _database;$/;" m class:doris::TableDescriptor +_day src/runtime/datetime_value.h /^ uint8_t _day;$/;" m class:doris::DateTimeValue +_db src/olap/olap_meta.h /^ rocksdb::DB* _db;$/;" m class:doris::OlapMeta +_db test/exec/schema_scanner/schema_authors_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaAuthorScannerTest file: +_db test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaCharsetsScannerTest file: +_db test/exec/schema_scanner/schema_collations_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaCollationsScannerTest file: +_db test/exec/schema_scanner/schema_columns_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaColumnsScannerTest file: +_db test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaCreateTableScannerTest file: +_db test/exec/schema_scanner/schema_engines_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaEnginesScannerTest file: +_db test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaOpenTablesScannerTest file: +_db test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaSchemataScannerTest file: +_db test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaTableNamesScannerTest file: +_db test/exec/schema_scanner/schema_tables_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaTablesScannerTest file: +_db test/exec/schema_scanner/schema_variables_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaVariablesScannerTest file: +_db test/exec/schema_scanner_test.cpp /^ std::string _db;$/;" m class:doris::SchemaScannerTest file: +_db_id src/exec/tablet_info.h /^ int64_t _db_id;$/;" m class:doris::OlapTableSchemaParam +_db_id src/exec/tablet_sink.h /^ int64_t _db_id = -1;$/;" m class:doris::stream_load::OlapTableSink +_db_index src/exec/schema_scanner/schema_columns_scanner.h /^ int _db_index;$/;" m class:doris::SchemaColumnsScanner +_db_index src/exec/schema_scanner/schema_schemata_scanner.h /^ int _db_index;$/;" m class:doris::SchemaSchemataScanner +_db_index src/exec/schema_scanner/schema_tables_scanner.h /^ int _db_index;$/;" m class:doris::SchemaTablesScanner +_db_name src/exec/tablet_sink.h /^ std::string _db_name;$/;" m class:doris::stream_load::OlapTableSink +_db_name src/runtime/runtime_state.h /^ std::string _db_name;$/;" m class:doris::RuntimeState +_db_result src/exec/schema_scanner/schema_columns_scanner.h /^ TGetDbsResult _db_result;$/;" m class:doris::SchemaColumnsScanner +_db_result src/exec/schema_scanner/schema_schemata_scanner.h /^ TGetDbsResult _db_result;$/;" m class:doris::SchemaSchemataScanner +_db_result src/exec/schema_scanner/schema_tables_scanner.h /^ TGetDbsResult _db_result;$/;" m class:doris::SchemaTablesScanner +_dctx src/exec/decompressor.h /^ LZ4F_dctx* _dctx;$/;" m class:doris::Lz4FrameDecompressor +_debug src/udf/udf_internal.h /^ bool _debug;$/;" m class:doris::FunctionContextImpl +_debug_action src/exec/exec_node.h /^ TDebugAction::type _debug_action;$/;" m class:doris::ExecNode +_debug_phase src/exec/exec_node.h /^ TExecNodePhase::type _debug_phase;$/;" m class:doris::ExecNode +_debug_version_list src/olap/push_handler.cpp /^string PushHandler::_debug_version_list(const Versions& versions) const {$/;" f class:doris::PushHandler +_decode src/olap/rowset/segment_v2/bitshuffle_page.h /^ Status _decode() {$/;" f class:doris::segment_v2::BitShufflePageDecoder +_decode_ascending src/olap/key_coder.h /^ DecodeAscendingFunc _decode_ascending;$/;" m class:doris::KeyCoder +_decode_fixed32 src/olap/lru_cache.h /^ uint32_t _decode_fixed32(const char* ptr) const {$/;" f class:doris::CacheKey +_decode_value_lengths src/olap/rowset/segment_v2/binary_prefix_page.cpp /^const uint8_t* BinaryPrefixPageDecoder::_decode_value_lengths(const uint8_t* ptr,$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +_decoded src/olap/rowset/segment_v2/bitshuffle_page.h /^ faststring _decoded;$/;" m class:doris::segment_v2::BitShufflePageDecoder +_decoder src/olap/rowset/segment_v2/frame_of_reference_page.h /^ std::unique_ptr> _decoder;$/;" m class:doris::segment_v2::FrameOfReferencePageDecoder +_decoder src/olap/short_key_index.h /^ const ShortKeyIndexDecoder* _decoder;$/;" m class:doris::ShortKeyIndexIterator +_decompress src/util/block_compression.cpp /^ Status _decompress(LZ4F_decompressionContext_t ctx,$/;" f class:doris::Lz4fBlockCompression file: +_decompress_timer src/exec/plain_text_line_reader.h /^ RuntimeProfile::Counter* _decompress_timer;$/;" m class:doris::PlainTextLineReader +_decompressor src/exec/plain_text_line_reader.h /^ Decompressor* _decompressor;$/;" m class:doris::PlainTextLineReader +_decompressor src/olap/file_stream.h /^ Decompressor _decompressor;$/;" m class:doris::ReadOnlyFileStream +_decompressor src/olap/in_stream.h /^ Decompressor _decompressor;$/;" m class:doris::InStream +_decompressor src/olap/rowset/segment_reader.h /^ Decompressor _decompressor; \/\/根据压缩格式,设置的解压器$/;" m class:doris::SegmentReader +_decompressor_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _decompressor_timer = nullptr;$/;" m class:doris::OlapScanNode +_deep_copy src/olap/types.h /^ void (*_deep_copy)(void* dest, const void* src, MemPool* mem_pool);$/;" m class:doris::TypeInfo +_deep_copy_input src/runtime/sorted_run_merger.h /^ bool _deep_copy_input;$/;" m class:doris::SortedRunMerger +_default_buf src/util/mysql_row_buffer.h /^ char _default_buf[4096];$/;" m class:doris::MysqlRowBuffer +_default_encoding_type_map src/olap/rowset/segment_v2/encoding_info.cpp /^ std::unordered_map> _default_encoding_type_map;$/;" m class:doris::segment_v2::EncodingInfoResolver file: +_default_level src/agent/cgroups_mgr.h /^ std::string _default_level = "normal";$/;" m class:doris::CgroupsMgr +_default_rowset_type src/olap/storage_engine.h /^ RowsetTypePB _default_rowset_type;$/;" m class:doris::StorageEngine +_default_user_name src/agent/cgroups_mgr.h /^ std::string _default_user_name = "default";$/;" m class:doris::CgroupsMgr +_default_value src/olap/rowset/column_reader.h /^ std::string _default_value;$/;" m class:doris::DefaultValueReader +_default_value src/olap/rowset/segment_v2/column_reader.h /^ std::string _default_value;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_default_value src/olap/tablet_schema.h /^ std::string _default_value;$/;" m class:doris::TabletColumn +_default_values src/exec/csv_scan_node.h /^ std::vector _default_values;$/;" m class:doris::CsvScanNode +_del_conds src/olap/delete_handler.h /^ std::vector _del_conds;$/;" m class:doris::DeleteHandler +_del_filtered_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _del_filtered_counter = nullptr;$/;" m class:doris::OlapScanNode +_del_pred_array src/olap/tablet_meta.h /^ DelPredicateArray _del_pred_array;$/;" m class:doris::TabletMeta +_delete_block src/olap/fs/file_block_manager.cpp /^Status FileBlockManager::_delete_block(const string& path) {$/;" f class:doris::fs::FileBlockManager +_delete_cache_file_descriptor src/olap/file_helper.h /^ static void _delete_cache_file_descriptor(const CacheKey& key, void* value) {$/;" f class:doris::FileHandler +_delete_cached_index_stream src/olap/rowset/segment_reader.cpp /^void SegmentReader::_delete_cached_index_stream(const CacheKey& key, void* value) {$/;" f class:doris::SegmentReader +_delete_condition_handler test/olap/delete_handler_test.cpp /^ DeleteConditionHandler _delete_condition_handler;$/;" m class:doris::TestDeleteConditionHandler file: +_delete_condition_handler test/olap/delete_handler_test.cpp /^ DeleteConditionHandler _delete_condition_handler;$/;" m class:doris::TestDeleteConditionHandler2 file: +_delete_condition_handler test/olap/delete_handler_test.cpp /^ DeleteConditionHandler _delete_condition_handler;$/;" m class:doris::TestDeleteHandler file: +_delete_data src/olap/task/engine_batch_load_task.cpp /^OLAPStatus EngineBatchLoadTask::_delete_data($/;" f class:doris::EngineBatchLoadTask +_delete_flag src/olap/rowset/segment_group.h /^ bool _delete_flag;$/;" m class:doris::SegmentGroup +_delete_handler src/olap/reader.h /^ DeleteHandler _delete_handler;$/;" m class:doris::Reader +_delete_handler src/olap/rowset/column_data.h /^ const DeleteHandler*_delete_handler = nullptr;$/;" m class:doris::ColumnData +_delete_handler src/olap/rowset/segment_reader.h /^ const DeleteHandler* _delete_handler = nullptr;$/;" m class:doris::SegmentReader +_delete_handler src/olap/schema_change.h /^ DeleteHandler _delete_handler;$/;" m class:doris::RowBlockChanger +_delete_handler test/olap/delete_handler_test.cpp /^ DeleteHandler _delete_handler;$/;" m class:doris::TestDeleteHandler file: +_delete_handlers src/http/ev_http_server.h /^ PathTrie _delete_handlers;$/;" m class:doris::EvHttpServer +_delete_inc_rowset_by_version src/olap/tablet.cpp /^void Tablet::_delete_inc_rowset_by_version(const Version& version,$/;" f class:doris::Tablet +_delete_on_read src/runtime/buffered_tuple_stream.h /^ const bool _delete_on_read;$/;" m class:doris::BufferedTupleStream +_delete_on_read src/runtime/buffered_tuple_stream2.h /^ bool _delete_on_read;$/;" m class:doris::BufferedTupleStream2 +_delete_partial_statisfied_pages src/olap/rowset/segment_v2/column_reader.h /^ std::vector _delete_partial_statisfied_pages;$/;" m class:doris::segment_v2::FileColumnIterator +_delete_req test/olap/olap_reader_test.cpp /^ TPushReq _delete_req;$/;" m class:doris::TestOLAPReaderColumnDeleteCondition file: +_delete_state src/olap/column_block.h /^ DelCondSatisfied _delete_state;$/;" m class:doris::ColumnBlock +_delete_state src/olap/row_block2.h /^ DelCondSatisfied _delete_state;$/;" m class:doris::RowBlockV2 +_delete_status src/olap/rowset/column_data.h /^ DelCondSatisfied _delete_status;$/;" m class:doris::ColumnData +_delete_status src/olap/rowset/segment_reader.h /^ DelCondSatisfied _delete_status;$/;" m class:doris::SegmentReader +_delete_tablets_on_unused_root_path src/olap/storage_engine.cpp /^bool StorageEngine::_delete_tablets_on_unused_root_path() {$/;" f class:doris::StorageEngine +_delete_workers src/agent/agent_server.h /^ std::unique_ptr _delete_workers;$/;" m class:doris::AgentServer +_deleted_keys test/olap/lru_cache_test.cpp /^ std::vector _deleted_keys;$/;" m class:doris::CacheTest file: +_deleted_values test/olap/lru_cache_test.cpp /^ std::vector _deleted_values;$/;" m class:doris::CacheTest file: +_delta_written_success src/olap/delta_writer.h /^ bool _delta_written_success;$/;" m class:doris::DeltaWriter +_deregister_locked src/util/metrics.cpp /^void MetricRegistry::_deregister_locked(Metric* metric) {$/;" f class:doris::MetricRegistry +_desc src/runtime/buffered_tuple_stream.h /^ const RowDescriptor& _desc;$/;" m class:doris::BufferedTupleStream +_desc src/runtime/buffered_tuple_stream2.h /^ const RowDescriptor& _desc;$/;" m class:doris::BufferedTupleStream2 +_desc_result src/exec/schema_scanner/schema_columns_scanner.h /^ TDescribeTableResult _desc_result;$/;" m class:doris::SchemaColumnsScanner +_desc_tbl src/runtime/descriptor_helper.h /^ TDescriptorTable _desc_tbl;$/;" m class:doris::TDescriptorTableBuilder +_desc_tbl src/runtime/runtime_state.h /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::RuntimeState +_desc_tbl test/exec/broker_scan_node_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::BrokerScanNodeTest file: +_desc_tbl test/exec/broker_scanner_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::BrokerScannerTest file: +_desc_tbl test/exec/csv_scan_bench_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::CsvScanNodeBenchTest file: +_desc_tbl test/exec/csv_scan_node_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::CsvScanNodeTest file: +_desc_tbl test/exec/es_http_scan_node_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::EsHttpScanNodeTest file: +_desc_tbl test/exec/es_predicate_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::EsPredicateTest file: +_desc_tbl test/exec/es_scan_node_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::EsScanNodeTest file: +_desc_tbl test/exec/mysql_scan_node_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::MysqlScanNodeTest file: +_desc_tbl test/exec/new_olap_scan_node_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::TestOlapScanNode file: +_desc_tbl test/exec/olap_scan_node_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::OlapScanNodeTest file: +_desc_tbl test/exec/orc_scanner_test.cpp /^ DescriptorTbl *_desc_tbl;$/;" m class:doris::OrcScannerTest file: +_desc_tbl test/exec/parquet_scanner_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::ParquetSannerTest file: +_desc_tbl test/exec/schema_scan_node_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::SchemaScanNodeTest file: +_desc_tbl test/olap/olap_reader_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::TestOLAPReaderColumn file: +_desc_tbl test/olap/olap_reader_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::TestOLAPReaderColumnDeleteCondition file: +_desc_tbl test/olap/olap_reader_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::TestOLAPReaderRow file: +_desc_tbl test/olap/vectorized_olap_reader_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::TestVectorizedOLAPReader file: +_desc_tbl test/runtime/buffered_tuple_stream_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::BufferedTupleStreamTest file: +_desc_tbl test/runtime/data_spliter_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::DataSplitTest file: +_desc_tbl test/runtime/data_stream_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::DataStreamTest file: +_desc_tbl test/runtime/dpp_sink_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::DppSinkTest file: +_desc_tbl test/runtime/memory_scratch_sink_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::MemoryScratchSinkTest file: +_desc_tbl test/runtime/qsorter_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::QSorterTest file: +_desc_tbl test/runtime/sorter_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::SorterTest file: +_desc_tbl test/util/arrow/arrow_work_flow_test.cpp /^ DescriptorTbl* _desc_tbl;$/;" m class:doris::ArrowWorkFlowTest file: +_deserialize_from_pb src/olap/rowset/rowset_meta.h /^ bool _deserialize_from_pb(const std::string& value) {$/;" f class:doris::RowsetMeta +_deserialize_row_batch_timer src/runtime/data_stream_recvr.h /^ RuntimeProfile::Counter* _deserialize_row_batch_timer;$/;" m class:doris::DataStreamRecvr +_desired_num_rows src/exec/assert_num_rows_node.h /^ int64_t _desired_num_rows;$/;" m class:doris::AssertNumRowsNode +_dest test/runtime/data_stream_test.cpp /^ vector _dest;$/;" m class:doris::DataStreamTest file: +_dest_expr_ctx src/exec/base_scanner.h /^ std::vector _dest_expr_ctx;$/;" m class:doris::BaseScanner +_dest_node_id src/runtime/data_stream_recvr.h /^ PlanNodeId _dest_node_id;$/;" m class:doris::DataStreamRecvr +_dest_node_id src/runtime/data_stream_sender.cpp /^ PlanNodeId _dest_node_id;$/;" m class:doris::DataStreamSender::Channel file: +_dest_node_id src/runtime/data_stream_sender.h /^ PlanNodeId _dest_node_id;$/;" m class:doris::DataStreamSender +_dest_tuple src/exec/schema_scan_node.h /^ Tuple* _dest_tuple;$/;" m class:doris::SchemaScanNode +_dest_tuple_desc src/exec/base_scanner.h /^ const TupleDescriptor* _dest_tuple_desc;$/;" m class:doris::BaseScanner +_dest_tuple_desc src/exec/schema_scan_node.h /^ const TupleDescriptor* _dest_tuple_desc;$/;" m class:doris::SchemaScanNode +_destory src/runtime/exec_env_init.cpp /^void ExecEnv::_destory() {$/;" f class:doris::ExecEnv +_destroy_cache_entry src/runtime/user_function_cache.cpp /^void UserFunctionCache::_destroy_cache_entry(UserFunctionCacheEntry* entry) {$/;" f class:doris::UserFunctionCache +_destruct_array src/olap/utils.h /^void _destruct_array(const void* array, void*) {$/;" f namespace:doris +_destruct_object src/olap/utils.h /^void _destruct_object(const void* obj, void*) {$/;" f namespace:doris +_determined_encoding src/olap/rowset/run_length_integer_writer.cpp /^void RunLengthIntegerWriter::_determined_encoding() {$/;" f class:doris::RunLengthIntegerWriter +_device_id src/runtime/tmp_file_mgr.h /^ DeviceId _device_id;$/;" m class:doris::TmpFileMgr::File +_dict_builder src/olap/rowset/segment_v2/binary_dict_page.h /^ std::unique_ptr _dict_builder;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_dict_column_iter src/olap/rowset/segment_v2/bitmap_index_reader.h /^ IndexedColumnIterator _dict_column_iter;$/;" m class:doris::segment_v2::BitmapIndexIterator +_dict_column_reader src/olap/rowset/segment_v2/bitmap_index_reader.h /^ std::unique_ptr _dict_column_reader;$/;" m class:doris::segment_v2::BitmapIndexReader +_dict_decoder src/olap/rowset/segment_v2/binary_dict_page.h /^ const BinaryPlainPageDecoder* _dict_decoder = nullptr;$/;" m class:doris::segment_v2::BinaryDictPageDecoder +_dict_decoder src/olap/rowset/segment_v2/column_reader.h /^ std::unique_ptr _dict_decoder;$/;" m class:doris::segment_v2::FileColumnIterator +_dict_items src/olap/rowset/segment_v2/binary_dict_page.h /^ std::vector _dict_items;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_dict_page_handle src/olap/rowset/segment_v2/column_reader.h /^ PageHandle _dict_page_handle;$/;" m class:doris::segment_v2::FileColumnIterator +_dict_stream src/olap/rowset/column_writer.h /^ OutStream* _dict_stream;$/;" m class:doris::VarStringColumnWriter +_dict_total_size src/olap/rowset/column_writer.h /^ uint64_t _dict_total_size;$/;" m class:doris::VarStringColumnWriter +_dictionary src/olap/rowset/column_reader.h /^ std::vector _dictionary;$/;" m class:doris::StringColumnDictionaryReader +_dictionary src/olap/rowset/segment_v2/binary_dict_page.h /^ std::unordered_map _dictionary;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_dictionary_buffers test/olap/column_reader_test.cpp /^ std::vector _dictionary_buffers;$/;" m class:doris::TestColumn file: +_dictionary_buffers test/olap/schema_change_test.cpp /^ std::vector _dictionary_buffers;$/;" m class:doris::TestColumn file: +_dictionary_size src/olap/rowset/column_reader.h /^ uint32_t _dictionary_size;$/;" m class:doris::StringColumnDictionaryReader +_dir_status_lock src/runtime/tmp_file_mgr.h /^ SpinLock _dir_status_lock;$/;" m class:doris::TmpFileMgr +_direct_conjunct_size src/exec/olap_scan_node.h /^ size_t _direct_conjunct_size;$/;" m class:doris::OlapScanNode +_direct_conjunct_size src/exec/olap_scanner.h /^ int _direct_conjunct_size = 0;$/;" m class:doris::OlapScanner +_direct_copy src/olap/types.h /^ void (*_direct_copy)(void* dest, const void* src);$/;" m class:doris::TypeInfo +_dirty_dirs src/olap/fs/file_block_manager.h /^ std::unordered_set _dirty_dirs;$/;" m class:doris::fs::FileBlockManager +_disable_spill src/runtime/buffered_block_mgr2.h /^ const bool _disable_spill;$/;" m class:doris::BufferedBlockMgr2 +_disjuncts src/exec/es/es_predicate.h /^ std::vector _disjuncts;$/;" m class:doris::EsPredicate +_disjuncts_num src/exec/es/es_predicate.h /^ int _disjuncts_num;$/;" m class:doris::EsPredicate +_disk_capacity_bytes src/olap/data_dir.h /^ int64_t _disk_capacity_bytes;$/;" m class:doris::DataDir +_disk_id src/runtime/disk_io_mgr.h /^ int _disk_id;$/;" m class:doris::DiskIoMgr::RequestRange +_disk_id src/runtime/tmp_file_mgr.h /^ int _disk_id;$/;" m class:doris::TmpFileMgr::File +_disk_io_mgr src/runtime/exec_env.h /^ DiskIoMgr* _disk_io_mgr = nullptr;$/;" m class:doris::ExecEnv +_disk_metrics src/util/system_metrics.h /^ std::map _disk_metrics;$/;" m class:doris::SystemMetrics +_disk_queues src/runtime/disk_io_mgr.h /^ std::vector _disk_queues;$/;" m class:doris::DiskIoMgr +_disk_read_timer src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _disk_read_timer;$/;" m class:doris::BufferedBlockMgr2 +_disk_stat_monitor_thread src/olap/storage_engine.h /^ std::thread _disk_stat_monitor_thread;$/;" m class:doris::StorageEngine +_disk_stat_monitor_thread_callback src/olap/olap_server.cpp /^void* StorageEngine::_disk_stat_monitor_thread_callback(void* arg) {$/;" f class:doris::StorageEngine +_disk_states src/runtime/disk_io_mgr_internal.h /^ std::vector _disk_states;$/;" m class:doris::DiskIoMgr::RequestContext +_disk_thread_group src/runtime/disk_io_mgr.h /^ boost::thread_group _disk_thread_group;$/;" m class:doris::DiskIoMgr +_disks_accessed_bitmap src/runtime/disk_io_mgr_internal.h /^ RuntimeProfile::Counter* _disks_accessed_bitmap;$/;" m class:doris::DiskIoMgr::RequestContext +_disks_complete_cond_var src/runtime/disk_io_mgr_internal.h /^ boost::condition_variable _disks_complete_cond_var;$/;" m class:doris::DiskIoMgr::RequestContext +_distributed_bucket src/runtime/dpp_sink_internal.h /^ int32_t _distributed_bucket;$/;" m class:doris::PartitionInfo +_distributed_expr_ctxs src/runtime/dpp_sink_internal.h /^ std::vector _distributed_expr_ctxs;$/;" m class:doris::PartitionInfo +_distributed_slot_descs src/exec/tablet_info.h /^ std::vector _distributed_slot_descs;$/;" m class:doris::OlapTablePartitionParam +_do_process_alter_tablet_v2 src/olap/schema_change.cpp /^OLAPStatus SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2& request) {$/;" f class:doris::SchemaChangeHandler +_do_streaming_ingestion src/olap/push_handler.cpp /^OLAPStatus PushHandler::_do_streaming_ingestion($/;" f class:doris::PushHandler +_do_sweep src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::_do_sweep($/;" f class:doris::StorageEngine +_document_node src/exec/es/es_scroll_parser.h /^ rapidjson::Document _document_node;$/;" m class:doris::ScrollParser +_docvalue_context src/exec/es_http_scan_node.h /^ std::map _docvalue_context;$/;" m class:doris::EsHttpScanNode +_done src/exec/kudu_scan_node.h /^ volatile bool _done;$/;" m class:doris::KuduScanNode +_done src/runtime/disk_io_mgr_internal.h /^ bool _done;$/;" m class:doris::DiskIoMgr::RequestContext::PerDiskState +_done src/runtime/plan_fragment_executor.h /^ bool _done;$/;" m class:doris::PlanFragmentExecutor +_done src/util/runtime_profile.h /^ volatile bool _done;$/;" m struct:doris::RuntimeProfile::PeriodicCounterUpdateState +_done src/util/thread.h /^ CountDownLatch _done;$/;" m class:doris::Thread +_download_file src/runtime/small_file_mgr.cpp /^Status SmallFileMgr::_download_file($/;" f class:doris::SmallFileMgr +_download_files src/olap/task/engine_clone_task.cpp /^Status EngineCloneTask::_download_files($/;" f class:doris::EngineCloneTask +_download_lib src/runtime/user_function_cache.cpp /^Status UserFunctionCache::_download_lib($/;" f class:doris::UserFunctionCache +_download_status src/olap/task/engine_batch_load_task.h /^ AgentStatus _download_status;$/;" m class:doris::EngineBatchLoadTask +_download_type src/http/download_action.h /^ DOWNLOAD_TYPE _download_type;$/;" m class:doris::DownloadAction +_download_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_download_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_download_workers src/agent/agent_server.h /^ std::unique_ptr _download_workers;$/;" m class:doris::AgentServer +_dpp_sink_vec src/runtime/data_spliter.h /^ std::vector _dpp_sink_vec;$/;" m class:doris::DataSpliter +_drop_retry_times src/agent/cgroups_mgr.h /^ int32_t _drop_retry_times = 10;$/;" m class:doris::CgroupsMgr +_drop_tablet_directly_unlocked src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::_drop_tablet_directly_unlocked($/;" f class:doris::TabletManager +_drop_tablet_unlocked src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::_drop_tablet_unlocked($/;" f class:doris::TabletManager +_drop_tablet_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_drop_tablet_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_drop_tablet_workers src/agent/agent_server.h /^ std::unique_ptr _drop_tablet_workers;$/;" m class:doris::AgentServer +_dst_cursor src/olap/rowset/alpha_rowset_reader.h /^ RowCursor* _dst_cursor = nullptr;$/;" m class:doris::AlphaRowsetReader +_dummy_mem_limit test/runtime/data_stream_test.cpp /^ MemTracker _dummy_mem_limit;$/;" m class:doris::DataStreamTest file: +_dummy_mem_tracker test/runtime/data_stream_test.cpp /^ MemTracker _dummy_mem_tracker;$/;" m class:doris::DataStreamTest file: +_dummy_result_tuple src/exec/analytic_eval_node.h /^ Tuple* _dummy_result_tuple;$/;" m class:doris::AnalyticEvalNode +_dup_key_next_row src/olap/reader.cpp /^OLAPStatus Reader::_dup_key_next_row(RowCursor* row_cursor, MemPool* mem_pool, ObjectPool* agg_pool, bool* eof) {$/;" f class:doris::Reader +_echo_cmd_to_cgroup src/agent/cgroups_mgr.cpp /^void CgroupsMgr::_echo_cmd_to_cgroup(stringstream& ctrl_cmd, string& cgroups_path) {$/;" f class:doris::CgroupsMgr +_effective_cluster_id src/olap/storage_engine.h /^ int32_t _effective_cluster_id;$/;" m class:doris::StorageEngine +_elems src/olap/lru_cache.h /^ uint32_t _elems;$/;" m class:doris::CachePriority::HandleTable +_empty src/olap/rowset/segment_group.h /^ bool _empty;$/;" m class:doris::SegmentGroup +_empty_cv src/util/batch_process_thread_pool.hpp /^ boost::condition_variable _empty_cv;$/;" m class:doris::BatchProcessThreadPool +_empty_cv src/util/priority_thread_pool.hpp /^ boost::condition_variable _empty_cv;$/;" m class:doris::PriorityThreadPool +_encode_ascending src/olap/key_coder.h /^ EncodeAscendingFunc _encode_ascending;$/;" m class:doris::KeyCoder +_encoder src/olap/rowset/segment_v2/frame_of_reference_page.h /^ std::unique_ptr> _encoder;$/;" m class:doris::segment_v2::FrameOfReferencePageBuilder +_encoding src/olap/rowset/run_length_integer_writer.h /^ EncodingType _encoding;$/;" m class:doris::RunLengthIntegerWriter +_encoding src/olap/rowset/segment_v2/encoding_info.h /^ EncodingTypePB _encoding;$/;" m class:doris::segment_v2::EncodingInfo +_encoding_info src/olap/rowset/segment_v2/column_reader.h /^ const EncodingInfo* _encoding_info = nullptr;$/;" m class:doris::segment_v2::ColumnReader +_encoding_info src/olap/rowset/segment_v2/column_writer.h /^ const EncodingInfo* _encoding_info = nullptr;$/;" m class:doris::segment_v2::ColumnWriter +_encoding_info src/olap/rowset/segment_v2/indexed_column_reader.h /^ const EncodingInfo* _encoding_info = nullptr;$/;" m class:doris::segment_v2::IndexedColumnReader +_encoding_map src/olap/rowset/segment_v2/encoding_info.cpp /^ EncodingInfo*, EncodingMapHash> _encoding_map;$/;" m class:doris::segment_v2::EncodingInfoResolver file: +_encoding_type src/olap/rowset/segment_v2/binary_dict_page.h /^ EncodingTypePB _encoding_type;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_encoding_type src/olap/rowset/segment_v2/binary_dict_page.h /^ EncodingTypePB _encoding_type;$/;" m class:doris::segment_v2::BinaryDictPageDecoder +_encodings_map src/olap/rowset/segment_reader.h /^ UniqueIdEncodingMap _encodings_map; \/\/ 保存encoding$/;" m class:doris::SegmentReader +_encryption_timer src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _encryption_timer;$/;" m class:doris::BufferedBlockMgr2 +_end src/exprs/hybird_set.h /^ typename std::unordered_set<_iT>::iterator _end;$/;" m class:doris::HybirdSet::Iterator +_end src/exprs/hybird_set.h /^ typename std::unordered_set::iterator _end;$/;" m class:doris::StringValueSet::Iterator +_end src/runtime/dpp_writer.h /^ char* _end;$/;" m class:doris::DppWriter +_end_block src/olap/rowset/column_data.h /^ uint32_t _end_block;$/;" m class:doris::ColumnData +_end_block src/olap/rowset/segment_reader.h /^ int64_t _end_block; \/\/ 本次读取的结束块$/;" m class:doris::SegmentReader +_end_include src/exec/olap_common.h /^ bool _end_include;$/;" m class:doris::OlapScanKeys +_end_key src/runtime/dpp_sink_internal.h /^ PartRangeKey _end_key;$/;" m class:doris::PartRange +_end_key_is_set src/olap/rowset/column_data.h /^ bool _end_key_is_set = false;$/;" m class:doris::ColumnData +_end_row_index src/olap/rowset/column_data.h /^ int64_t _end_row_index = 0;$/;" m class:doris::ColumnData +_end_scan_keys src/exec/olap_common.h /^ std::vector _end_scan_keys;$/;" m class:doris::OlapScanKeys +_end_segment src/olap/rowset/column_data.h /^ uint32_t _end_segment;$/;" m class:doris::ColumnData +_end_with src/runtime/snapshot_loader.cpp /^bool SnapshotLoader::_end_with($/;" f class:doris::SnapshotLoader +_engine_data_path test/olap/olap_snapshot_converter_test.cpp /^ std::string _engine_data_path;$/;" m class:doris::OlapSnapshotConverterTest file: +_engine_data_path test/olap/tablet_mgr_test.cpp /^ std::string _engine_data_path;$/;" m class:doris::TabletMgrTest file: +_engine_task_mutex src/olap/storage_engine.h /^ Mutex _engine_task_mutex;$/;" m class:doris::StorageEngine +_ensure_index_loaded src/olap/rowset/segment_v2/column_reader.h /^ Status _ensure_index_loaded() {$/;" f class:doris::segment_v2::ColumnReader +_entries src/util/threadpool.h /^ std::deque _entries;$/;" m class:doris::ThreadPoolToken +_entry src/olap/bloom_filter_reader.h /^ BloomFilter _entry;$/;" m class:doris::BloomFilterIndexReader +_entry src/olap/stream_index_reader.h /^ PositionEntryReader _entry;$/;" m class:doris::StreamIndexReader +_entry src/olap/stream_index_reader.h /^ const PositionEntryReader* _entry;$/;" m class:doris::PositionProvider +_entry_count src/olap/bloom_filter_reader.h /^ size_t _entry_count;$/;" m class:doris::BloomFilterIndexReader +_entry_count src/olap/stream_index_reader.h /^ size_t _entry_count;$/;" m class:doris::StreamIndexReader +_entry_map src/runtime/user_function_cache.h /^ std::unordered_map _entry_map;$/;" m class:doris::UserFunctionCache +_env src/agent/task_worker_pool.h /^ ExecEnv* _env;$/;" m class:doris::TaskWorkerPool +_env src/exec/broker_reader.h /^ ExecEnv* _env;$/;" m class:doris::BrokerReader +_env src/exec/broker_writer.h /^ ExecEnv* _env;$/;" m class:doris::BrokerWriter +_env src/exec/es_scan_node.h /^ ExecEnv* _env;$/;" m class:doris::EsScanNode +_env src/olap/fs/file_block_manager.h /^ Env* _env;$/;" m class:doris::fs::FileBlockManager +_env src/runtime/snapshot_loader.h /^ ExecEnv* _env;$/;" m class:doris::SnapshotLoader +_env src/service/http_service.h /^ ExecEnv* _env;$/;" m class:doris::HttpService +_env src/util/broker_load_error_hub.h /^ ExecEnv* _env;$/;" m class:doris::BrokerLoadErrorHub +_env src/util/file_manager.h /^ Env* _env;$/;" m class:doris::FileManager +_env test/exec/broker_reader_test.cpp /^ ExecEnv* _env;$/;" m class:doris::BrokerReaderTest file: +_env test/exec/csv_scan_node_test.cpp /^ boost::scoped_ptr _env;$/;" m class:doris::CsvScanNodeTest file: +_env test/exec/tablet_sink_test.cpp /^ ExecEnv* _env;$/;" m class:doris::stream_load::OlapTableSinkTest file: +_env test/http/stream_load_test.cpp /^ ExecEnv _env;$/;" m class:doris::StreamLoadActionTest file: +_env test/runtime/routine_load_task_executor_test.cpp /^ ExecEnv _env;$/;" m class:doris::RoutineLoadTaskExecutorTest file: +_eof src/exec/plain_text_line_reader.h /^ bool _eof;$/;" m class:doris::PlainTextLineReader +_eof src/olap/rowset/column_data.h /^ bool _eof;$/;" m class:doris::ColumnData +_eof src/olap/rowset/column_reader.h /^ bool _eof;$/;" m class:doris::DecimalColumnReader +_eof src/olap/rowset/column_reader.h /^ bool _eof;$/;" m class:doris::FixLengthStringColumnReader +_eof src/olap/rowset/column_reader.h /^ bool _eof;$/;" m class:doris::FloatintPointColumnReader +_eof src/olap/rowset/column_reader.h /^ bool _eof;$/;" m class:doris::IntegerColumnReader +_eof src/olap/rowset/column_reader.h /^ bool _eof;$/;" m class:doris::IntegerColumnReaderWrapper +_eof src/olap/rowset/column_reader.h /^ bool _eof;$/;" m class:doris::LargeIntColumnReader +_eof src/olap/rowset/column_reader.h /^ bool _eof;$/;" m class:doris::StringColumnDictionaryReader +_eof src/olap/rowset/column_reader.h /^ bool _eof;$/;" m class:doris::StringColumnDirectReader +_eof src/olap/rowset/column_reader.h /^ bool _eof;$/;" m class:doris::TinyColumnReader +_eof src/olap/rowset/column_reader.h /^ bool _eof;$/;" m class:doris::VarStringColumnReader +_eof src/olap/rowset/segment_reader.h /^ bool _eof; \/\/ eof标志$/;" m class:doris::SegmentReader +_eof src/olap/rowset/segment_v2/segment_iterator.cpp /^ bool _eof;$/;" m class:doris::segment_v2::SegmentIterator::BitmapRangeIterator file: +_eos src/exec/blocking_join_node.h /^ bool _eos; \/\/ if true, nothing left to return in get_next()$/;" m class:doris::BlockingJoinNode +_eos src/exec/es/es_scan_reader.h /^ bool _eos;$/;" m class:doris::ESScanReader +_eos src/exec/es_http_scan_node.h /^ bool _eos;$/;" m class:doris::EsHttpScanNode +_eos src/exec/hash_join_node.h /^ bool _eos; \/\/ if true, nothing left to return in get_next()$/;" m class:doris::HashJoinNode +_eos src/exec/merge_join_node.h /^ bool _eos; \/\/ if true, nothing left to return in get_next()$/;" m class:doris::MergeJoinNode +_eos src/exec/olap_scan_node.h /^ bool _eos;$/;" m class:doris::OlapScanNode +_eosr src/runtime/disk_io_mgr.h /^ bool _eosr;$/;" m class:doris::DiskIoMgr::BufferDescriptor +_eosr_queued src/runtime/disk_io_mgr.h /^ bool _eosr_queued;$/;" m class:doris::DiskIoMgr::ScanRange +_eosr_returned src/runtime/disk_io_mgr.h /^ bool _eosr_returned;$/;" m class:doris::DiskIoMgr::ScanRange +_epoch src/agent/heartbeat_server.h /^ int64_t _epoch;$/;" m class:doris::HeartbeatServer +_equal src/olap/types.h /^ bool (*_equal)(const void* left, const void* right);$/;" m class:doris::TypeInfo +_error_buf src/http/http_client.h /^ char _error_buf[CURL_ERROR_SIZE];$/;" m class:doris::HttpClient +_error_hub src/runtime/runtime_state.h /^ std::unique_ptr _error_hub;$/;" m class:doris::RuntimeState +_error_log src/runtime/runtime_state.h /^ std::vector _error_log;$/;" m class:doris::RuntimeState +_error_log_dir src/runtime/load_path_mgr.h /^ std::string _error_log_dir;$/;" m class:doris::LoadPathMgr +_error_log_file src/runtime/runtime_state.h /^ std::ofstream* _error_log_file; \/\/ error file path, absolute path$/;" m class:doris::RuntimeState +_error_log_file_path src/runtime/runtime_state.h /^ std::string _error_log_file_path;$/;" m class:doris::RuntimeState +_error_log_lock src/runtime/runtime_state.h /^ boost::mutex _error_log_lock;$/;" m class:doris::RuntimeState +_error_log_root_dir src/http/download_action.h /^ std::string _error_log_root_dir;$/;" m class:doris::DownloadAction +_error_msg output/udf/include/uda_test_harness.h /^ std::string _error_msg;$/;" m class:doris_udf::UdaTestHarnessBase +_error_msg src/udf/uda_test_harness.h /^ std::string _error_msg;$/;" m class:doris_udf::UdaTestHarnessBase +_error_msg src/udf/udf_internal.h /^ std::string _error_msg;$/;" m class:doris::FunctionContextImpl +_error_msgs src/olap/task/engine_alter_tablet_task.h /^ vector* _error_msgs;$/;" m class:doris::EngineAlterTabletTask +_error_msgs src/olap/task/engine_clone_task.h /^ vector* _error_msgs;$/;" m class:doris::EngineCloneTask +_error_msgs src/util/broker_load_error_hub.h /^ std::queue _error_msgs;$/;" m class:doris::BrokerLoadErrorHub +_error_msgs src/util/mysql_load_error_hub.h /^ std::queue _error_msgs;$/;" m class:doris::MysqlLoadErrorHub +_error_msgs src/util/null_load_error_hub.h /^ std::queue _error_msgs;$/;" m class:doris::NullLoadErrorHub +_error_path_next_shard src/runtime/load_path_mgr.h /^ uint32_t _error_path_next_shard;$/;" m class:doris::LoadPathMgr +_error_row_number src/runtime/runtime_state.h /^ int64_t _error_row_number;$/;" m class:doris::RuntimeState +_error_status src/exec/mysql_scanner.cpp /^Status MysqlScanner::_error_status(const std::string& prefix) {$/;" f class:doris::MysqlScanner +_error_tablet_ids src/olap/task/engine_publish_version_task.h /^ vector* _error_tablet_ids;$/;" m class:doris::EnginePublishVersionTask +_es_query_status src/exec/es/es_predicate.h /^ Status _es_query_status;$/;" m class:doris::EsPredicate +_es_query_str src/exec/es/es_query_builder.h /^ std::string _es_query_str;$/;" m class:doris::ESQueryBuilder +_es_reader src/exec/es_http_scanner.h /^ std::unique_ptr _es_reader;$/;" m class:doris::EsHttpScanner +_es_scroll_parser src/exec/es_http_scanner.h /^ std::unique_ptr _es_scroll_parser;$/;" m class:doris::EsHttpScanner +_escape_buff src/util/mysql_load_error_hub.h /^ std::array _escape_buff;$/;" m class:doris::MysqlLoadErrorHub +_escape_char src/exec/text_converter.h /^ char _escape_char;$/;" m class:doris::TextConverter +_estimated_size src/olap/rowset/segment_v2/zone_map_index.h /^ uint64_t _estimated_size = 0;$/;" m class:doris::segment_v2::ZoneMapIndexWriter +_etl_job_mgr src/runtime/exec_env.h /^ EtlJobMgr* _etl_job_mgr = nullptr;$/;" m class:doris::ExecEnv +_etl_thread_pool src/runtime/exec_env.h /^ PriorityThreadPool* _etl_thread_pool = nullptr;$/;" m class:doris::ExecEnv +_ev_http_server src/service/http_service.h /^ std::unique_ptr _ev_http_server;$/;" m class:doris::HttpService +_ev_req src/http/http_request.h /^ struct evhttp_request* _ev_req = nullptr; $/;" m class:doris::HttpRequest typeref:struct:doris::HttpRequest::evhttp_request +_eval_conjuncts_fn src/exec/olap_scan_node.h /^ EvalConjunctsFn _eval_conjuncts_fn;$/;" m class:doris::OlapScanNode +_eval_conjuncts_fn src/exec/olap_scanner.h /^ ExecNode::EvalConjunctsFn _eval_conjuncts_fn = nullptr;$/;" m class:doris::OlapScanner +_evaluation_timer src/exec/analytic_eval_node.h /^ RuntimeProfile::Counter* _evaluation_timer;$/;" m class:doris::AnalyticEvalNode +_evaluators src/exec/analytic_eval_node.h /^ std::vector _evaluators;$/;" m class:doris::AnalyticEvalNode +_event_sequence_map src/util/runtime_profile.h /^ EventSequenceMap _event_sequence_map;$/;" m class:doris::RuntimeProfile +_event_sequences_lock src/util/runtime_profile.h /^ mutable boost::mutex _event_sequences_lock;$/;" m class:doris::RuntimeProfile +_events src/util/runtime_profile.h /^ EventList _events;$/;" m class:doris::RuntimeProfile::EventSequence +_evhttp_req test/http/http_utils_test.cpp /^ evhttp_request* _evhttp_req = nullptr;$/;" m class:doris::HttpUtilsTest file: +_evhttp_req test/http/metrics_action_test.cpp /^ evhttp_request* _evhttp_req = nullptr;$/;" m class:doris::MetricsActionTest file: +_evhttp_req test/http/stream_load_test.cpp /^ evhttp_request* _evhttp_req = nullptr;$/;" m class:doris::StreamLoadActionTest file: +_evict_from_lru src/olap/lru_cache.cpp /^void LRUCache::_evict_from_lru(size_t charge, std::vector* deleted) {$/;" f class:doris::LRUCache +_evict_one_entry src/olap/lru_cache.cpp /^void LRUCache::_evict_one_entry(LRUHandle* e) {$/;" f class:doris::LRUCache +_exactly_once src/exec/es/es_scan_reader.h /^ bool _exactly_once;$/;" m class:doris::ESScanReader +_exceeded_limit src/exec/hash_table.h /^ bool _exceeded_limit; \/\/ true if any of _mem_trackers[].limit_exceeded()$/;" m class:doris::HashTable +_exec_env src/agent/agent_server.h /^ ExecEnv* _exec_env;$/;" m class:doris::AgentServer +_exec_env src/agent/cgroups_mgr.h /^ ExecEnv* _exec_env;$/;" m class:doris::CgroupsMgr +_exec_env src/agent/user_resource_listener.h /^ ExecEnv* _exec_env;$/;" m class:doris::UserResourceListener +_exec_env src/http/action/checksum_action.h /^ ExecEnv* _exec_env;$/;" m class:doris::ChecksumAction +_exec_env src/http/action/health_action.h /^ ExecEnv* _exec_env;$/;" m class:doris::HealthAction +_exec_env src/http/action/mini_load.h /^ ExecEnv* _exec_env;$/;" m class:doris::MiniLoadAction +_exec_env src/http/action/reload_tablet_action.h /^ ExecEnv* _exec_env;$/;" m class:doris::ReloadTabletAction +_exec_env src/http/action/restore_tablet_action.h /^ ExecEnv* _exec_env;$/;" m class:doris::RestoreTabletAction +_exec_env src/http/action/snapshot_action.h /^ ExecEnv* _exec_env;$/;" m class:doris::SnapshotAction +_exec_env src/http/action/stream_load.h /^ ExecEnv* _exec_env;$/;" m class:doris::StreamLoadAction +_exec_env src/http/download_action.h /^ ExecEnv* _exec_env;$/;" m class:doris::DownloadAction +_exec_env src/runtime/broker_mgr.h /^ ExecEnv* _exec_env;$/;" m class:doris::BrokerMgr +_exec_env src/runtime/etl_job_mgr.h /^ ExecEnv* _exec_env;$/;" m class:doris::EtlJobMgr +_exec_env src/runtime/export_task_mgr.h /^ ExecEnv* _exec_env;$/;" m class:doris::ExportTaskMgr +_exec_env src/runtime/external_scan_context_mgr.h /^ ExecEnv* _exec_env;$/;" m class:doris::ExternalScanContextMgr +_exec_env src/runtime/fragment_mgr.cpp /^ ExecEnv* _exec_env;$/;" m class:doris::FragmentExecState file: +_exec_env src/runtime/fragment_mgr.h /^ ExecEnv* _exec_env;$/;" m class:doris::FragmentMgr +_exec_env src/runtime/load_path_mgr.h /^ ExecEnv* _exec_env;$/;" m class:doris::LoadPathMgr +_exec_env src/runtime/plan_fragment_executor.h /^ ExecEnv* _exec_env; \/\/ not owned$/;" m class:doris::PlanFragmentExecutor +_exec_env src/runtime/routine_load/routine_load_task_executor.h /^ ExecEnv* _exec_env;$/;" m class:doris::RoutineLoadTaskExecutor +_exec_env src/runtime/runtime_state.h /^ ExecEnv* _exec_env;$/;" m class:doris::RuntimeState +_exec_env src/runtime/small_file_mgr.h /^ ExecEnv* _exec_env;$/;" m class:doris::SmallFileMgr +_exec_env src/runtime/stream_load/stream_load_context.h /^ ExecEnv* _exec_env;$/;" m class:doris::StreamLoadContext +_exec_env src/runtime/stream_load/stream_load_executor.h /^ ExecEnv* _exec_env;$/;" m class:doris::StreamLoadExecutor +_exec_env src/runtime/test_env.h /^ boost::scoped_ptr _exec_env;$/;" m class:doris::TestEnv +_exec_env src/runtime/tmp_file_mgr.h /^ ExecEnv* _exec_env;$/;" m class:doris::TmpFileMgr +_exec_env src/service/backend_service.h /^ ExecEnv* _exec_env;$/;" m class:doris::BackendService +_exec_env src/service/brpc_service.h /^ ExecEnv* _exec_env;$/;" m class:doris::BRpcService +_exec_env src/service/internal_service.h /^ ExecEnv* _exec_env;$/;" m class:doris::PInternalServiceImpl +_exec_env test/runtime/buffered_tuple_stream_test.cpp /^ ExecEnv _exec_env;$/;" m class:doris::BufferedTupleStreamTest file: +_exec_env test/runtime/data_spliter_test.cpp /^ ExecEnv _exec_env;$/;" m class:doris::DataSplitTest file: +_exec_env test/runtime/data_stream_test.cpp /^ ExecEnv _exec_env;$/;" m class:doris::DataStreamTest file: +_exec_env test/runtime/dpp_sink_test.cpp /^ ExecEnv _exec_env;$/;" m class:doris::DppSinkTest file: +_exec_env test/runtime/etl_job_mgr_test.cpp /^ ExecEnv _exec_env;$/;" m class:doris::EtlJobMgrTest file: +_exec_env test/runtime/export_task_mgr_test.cpp /^ ExecEnv _exec_env;$/;" m class:doris::ExportTaskMgrTest file: +_exec_env test/runtime/external_scan_context_mgr_test.cpp /^ ExecEnv _exec_env;$/;" m class:doris::ExternalScanContextMgrTest file: +_exec_env test/runtime/memory_scratch_sink_test.cpp /^ ExecEnv* _exec_env;$/;" m class:doris::MemoryScratchSinkTest file: +_exec_env test/runtime/result_sink_test.cpp /^ ExecEnv _exec_env;$/;" m class:doris::ResultSinkTest file: +_exec_env test/runtime/snapshot_loader_test.cpp /^ ExecEnv* _exec_env;$/;" m class:doris::SnapshotLoaderTest file: +_exec_env test/runtime/sorter_test.cpp /^ ExecEnv _exec_env;$/;" m class:doris::SorterTest file: +_exec_env test/util/arrow/arrow_work_flow_test.cpp /^ ExecEnv* _exec_env;$/;" m class:doris::ArrowWorkFlowTest file: +_exec_options_lock src/exec/exec_node.h /^ boost::mutex _exec_options_lock;$/;" m class:doris::ExecNode +_exec_plan_fragment src/service/internal_service.cpp /^Status PInternalServiceImpl::_exec_plan_fragment(brpc::Controller* cntl) {$/;" f class:doris::PInternalServiceImpl +_exec_status src/runtime/fragment_mgr.cpp /^ Status _exec_status;$/;" m class:doris::FragmentExecState file: +_exec_thread src/runtime/fragment_mgr.cpp /^ std::unique_ptr _exec_thread;$/;" m class:doris::FragmentExecState file: +_execute_plan_for_test src/runtime/routine_load/routine_load_task_executor.cpp /^Status RoutineLoadTaskExecutor::_execute_plan_for_test(StreamLoadContext* ctx) {$/;" f class:doris::RoutineLoadTaskExecutor +_executor src/runtime/fragment_mgr.cpp /^ PlanFragmentExecutor _executor;$/;" m class:doris::FragmentExecState file: +_expect_dec_buf_size src/exec/decompressor.h /^ size_t _expect_dec_buf_size;$/;" m class:doris::Lz4FrameDecompressor +_expected_local src/runtime/disk_io_mgr.h /^ bool _expected_local;$/;" m class:doris::DiskIoMgr::ScanRange +_expected_num test/olap/rowset/segment_v2/block_bloom_filter_test.cpp /^ uint64_t _expected_num = 1024;$/;" m class:doris::segment_v2::BlockBloomFilterTest file: +_explicit_num src/olap/hll.h /^ ExpliclitLengthValueType _explicit_num;$/;" m class:doris::HllSetResolver +_explicit_value src/olap/hll.h /^ uint64_t* _explicit_value;$/;" m class:doris::HllSetResolver +_export_output_files src/runtime/runtime_state.h /^ std::vector _export_output_files;$/;" m class:doris::RuntimeState +_expr_mem_pool src/exec/exec_node.h /^ boost::scoped_ptr _expr_mem_pool;$/;" m class:doris::ExecNode +_expr_mem_tracker src/exec/data_sink.h /^ std::unique_ptr _expr_mem_tracker;$/;" m class:doris::DataSink +_expr_mem_tracker src/exec/exec_node.h /^ boost::scoped_ptr _expr_mem_tracker;$/;" m class:doris::ExecNode +_expr_value_null_bits src/exec/hash_table.h /^ uint8_t* _expr_value_null_bits;$/;" m class:doris::HashTable +_expr_value_null_bits src/exec/partitioned_hash_table.h /^ uint8_t* _expr_value_null_bits;$/;" m class:doris::PartitionedHashTableCtx +_expr_values_buffer src/exec/hash_table.h /^ uint8_t* _expr_values_buffer;$/;" m class:doris::HashTable +_expr_values_buffer src/exec/partitioned_hash_table.h /^ uint8_t* _expr_values_buffer;$/;" m class:doris::PartitionedHashTableCtx +_expr_values_buffer_offsets src/exec/hash_table.h /^ std::vector _expr_values_buffer_offsets;$/;" m class:doris::HashTable +_expr_values_buffer_offsets src/exec/partitioned_hash_table.h /^ std::vector _expr_values_buffer_offsets;$/;" m class:doris::PartitionedHashTableCtx +_exprs test/runtime/memory_scratch_sink_test.cpp /^ std::vector _exprs;$/;" m class:doris::MemoryScratchSinkTest file: +_exprs test/runtime/result_sink_test.cpp /^ std::vector _exprs;$/;" m class:doris::ResultSinkTest file: +_exprs test/runtime/result_writer_test.cpp /^ std::vector _exprs;$/;" m class:doris::ResultWriterTest file: +_extdatasource_client_cache src/runtime/exec_env.h /^ ClientCache* _extdatasource_client_cache = nullptr;$/;" m class:doris::ExecEnv +_external_bytes_tracked src/udf/udf_internal.h /^ int64_t _external_bytes_tracked;$/;" m class:doris::FunctionContextImpl +_external_scan_context_mgr src/runtime/exec_env.h /^ ExternalScanContextMgr* _external_scan_context_mgr = nullptr;$/;" m class:doris::ExecEnv +_external_sorting src/olap/schema_change.cpp /^bool SchemaChangeWithSorting::_external_sorting(vector& src_rowsets,$/;" f class:doris::SchemaChangeWithSorting +_extra_fixed_header src/olap/file_helper.h /^ ExtraType _extra_fixed_header;$/;" m class:doris::FileHeader +_factory src/util/thrift_util.h /^ boost::shared_ptr _factory;$/;" m class:doris::ThriftDeserializer +_failed_jobs src/runtime/etl_job_mgr.h /^ LruCache _failed_jobs;$/;" m class:doris::EtlJobMgr +_failed_tasks src/runtime/export_task_mgr.h /^ LruCache _failed_tasks;$/;" m class:doris::ExportTaskMgr +_fd src/env/env_posix.cpp /^ const int _fd;$/;" m class:doris::PosixRandomRWFile file: +_fd src/env/env_posix.cpp /^ int _fd;$/;" m class:doris::PosixRandomAccessFile file: +_fd src/env/env_posix.cpp /^ int _fd;$/;" m class:doris::PosixWritableFile file: +_fd src/exec/broker_reader.h /^ TBrokerFD _fd;$/;" m class:doris::BrokerReader +_fd src/exec/broker_writer.h /^ TBrokerFD _fd;$/;" m class:doris::BrokerWriter +_fd src/olap/file_helper.h /^ int _fd;$/;" m class:doris::FileHandler +_fd src/runtime/message_body_sink.h /^ int _fd = -1;$/;" m class:doris::MessageBodyFileSink +_fd_cache_clean_callback src/olap/olap_server.cpp /^void* StorageEngine::_fd_cache_clean_callback(void* arg) {$/;" f class:doris::StorageEngine +_fd_cache_clean_thread src/olap/storage_engine.h /^ std::thread _fd_cache_clean_thread;$/;" m class:doris::StorageEngine +_fd_metrics src/util/system_metrics.h /^ std::unique_ptr _fd_metrics;$/;" m class:doris::SystemMetrics +_fetch_rowset_meta_thread src/olap/tablet_sync_service.cpp /^void TabletSyncService::_fetch_rowset_meta_thread(std::vector tasks) {$/;" f class:doris::TabletSyncService +_fetch_rowset_pool src/olap/tablet_sync_service.h /^ BatchProcessThreadPool* _fetch_rowset_pool = nullptr;$/;" m class:doris::TabletSyncService +_fetch_tablet_meta_thread src/olap/tablet_sync_service.cpp /^void TabletSyncService::_fetch_tablet_meta_thread(std::vector tasks) {$/;" f class:doris::TabletSyncService +_fetch_tablet_pool src/olap/tablet_sync_service.h /^ BatchProcessThreadPool* _fetch_tablet_pool = nullptr;$/;" m class:doris::TabletSyncService +_field src/exec/es/es_query_builder.h /^ std::string _field;$/;" m class:doris::ExistsQueryBuilder +_field src/exec/es/es_query_builder.h /^ std::string _field;$/;" m class:doris::RangeQueryBuilder +_field src/exec/es/es_query_builder.h /^ std::string _field;$/;" m class:doris::TermQueryBuilder +_field src/exec/es/es_query_builder.h /^ std::string _field;$/;" m class:doris::TermsInSetQueryBuilder +_field src/exec/es/es_query_builder.h /^ std::string _field;$/;" m class:doris::WildCardQueryBuilder +_field src/olap/rowset/segment_v2/column_writer.h /^ std::unique_ptr _field;$/;" m class:doris::segment_v2::ColumnWriter +_field src/olap/rowset/segment_v2/zone_map_index.h /^ Field* _field;$/;" m class:doris::segment_v2::ZoneMapIndexWriter +_field_buf src/olap/wrapper_field.h /^ char* _field_buf = nullptr;$/;" m class:doris::WrapperField +_field_idx src/runtime/descriptors.h /^ int _field_idx;$/;" m class:doris::SlotDescriptor +_field_num src/exec/mysql_scanner.h /^ int _field_num;$/;" m class:doris::MysqlScanner +_field_offset_in_memory src/olap/row_block.h /^ std::vector _field_offset_in_memory;$/;" m class:doris::RowBlock +_field_type src/olap/stream_index_writer.h /^ FieldType _field_type;$/;" m class:doris::StreamIndexWriter +_field_type src/olap/types.h /^ const FieldType _field_type;$/;" m class:doris::TypeInfo +_file src/env/env_posix.cpp /^ FILE* const _file;$/;" m class:doris::PosixSequentialFile file: +_file src/exec/orc_scanner.cpp /^ FileReader* _file;$/;" m class:doris::ORCFileStream file: +_file src/exec/parquet_reader.h /^ FileReader *_file;$/;" m class:doris::ParquetFile +_file src/olap/push_handler.h /^ BinaryFile* _file;$/;" m class:doris::IBinaryReader +_file src/olap/rowset/segment_v2/indexed_column_reader.h /^ RandomAccessFile* _file;$/;" m class:doris::segment_v2::IndexedColumnIterator +_file src/runtime/disk_io_mgr.h /^ std::string _file;$/;" m class:doris::DiskIoMgr::RequestRange +_file_cache src/runtime/small_file_mgr.h /^ std::unordered_map _file_cache;$/;" m class:doris::SmallFileMgr +_file_cache src/util/file_manager.h /^ std::unique_ptr> _file_cache;$/;" m class:doris::FileManager +_file_cache test/util/file_cache_test.cpp /^ std::unique_ptr> _file_cache;$/;" m class:doris::FileCacheTest file: +_file_cursor src/olap/file_stream.h /^ FileCursor _file_cursor;$/;" m class:doris::ReadOnlyFileStream +_file_descriptor_lru_cache src/olap/storage_engine.h /^ Cache* _file_descriptor_lru_cache; \/\/ TODO(yingchun): add cache$/;" m class:doris::StorageEngine +_file_eof src/exec/plain_text_line_reader.h /^ bool _file_eof;$/;" m class:doris::PlainTextLineReader +_file_exist test/util/file_cache_test.cpp /^ std::string _file_exist;$/;" m class:doris::FileCacheTest file: +_file_exist test/util/file_manager_test.cpp /^ std::string _file_exist;$/;" m class:doris::FileManagerTest file: +_file_handle src/olap/rowset/segment_v2/indexed_column_reader.h /^ OpenedFileHandle _file_handle;$/;" m class:doris::segment_v2::IndexedColumnIterator +_file_handle src/olap/rowset/segment_v2/segment_iterator.h /^ OpenedFileHandle _file_handle;$/;" m class:doris::segment_v2::SegmentIterator +_file_handle_cache src/runtime/disk_io_mgr.h /^ std::multimap _file_handle_cache;$/;" m class:doris::DiskIoMgr +_file_handler src/olap/file_stream.h /^ FileHandler* _file_handler;$/;" m class:doris::ReadOnlyFileStream::FileCursor +_file_handler src/olap/rowset/segment_reader.h /^ doris::FileHandler _file_handler; \/\/ 文件handler$/;" m class:doris::SegmentReader +_file_header src/olap/rowset/segment_group.h /^ FileHeader _file_header;$/;" m class:doris::SegmentGroup +_file_header src/olap/rowset/segment_reader.h /^ const FileHeader* _file_header;$/;" m class:doris::SegmentReader +_file_metadata src/exec/parquet_reader.h /^ std::shared_ptr _file_metadata;$/;" m class:doris::ParquetReaderWrap +_file_name src/olap/file_helper.h /^ std::string _file_name;$/;" m class:doris::FileHandler +_file_name src/olap/file_helper.h /^ std::string _file_name;$/;" m class:doris::FileHandlerWithBuf +_file_name src/olap/rowset/segment_reader.h /^ std::string _file_name; \/\/ 文件名$/;" m class:doris::SegmentReader +_file_name src/olap/rowset/segment_v2/bitmap_index_reader.h /^ std::string _file_name;$/;" m class:doris::segment_v2::BitmapIndexReader +_file_name src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ std::string _file_name;$/;" m class:doris::segment_v2::BloomFilterIndexReader +_file_name src/olap/rowset/segment_v2/column_reader.h /^ std::string _file_name;$/;" m class:doris::segment_v2::ColumnReader +_file_name src/olap/rowset/segment_v2/indexed_column_reader.h /^ std::string _file_name;$/;" m class:doris::segment_v2::IndexedColumnReader +_file_name src/olap/rowset/segment_writer.h /^ std::string _file_name;$/;" m class:doris::SegmentWriter +_file_name src/util/bfd_parser.h /^ std::string _file_name;$/;" m class:doris::BfdParser +_file_path test/olap/bit_field_test.cpp /^ std::string _file_path = ".\/ut_dir\/tmp_file";$/;" m class:doris::TestBitField file: +_file_path test/olap/run_length_byte_test.cpp /^ std::string _file_path = ".\/ut_dir\/tmp_file";$/;" m class:doris::TestRunLengthByte file: +_file_path test/olap/run_length_integer_test.cpp /^ std::string _file_path = ".\/ut_dir\/tmp_file";$/;" m class:doris::TestRunLengthSignInteger file: +_file_path test/olap/run_length_integer_test.cpp /^ std::string _file_path = ".\/ut_dir\/tmp_file";$/;" m class:doris::TestRunLengthUnsignInteger file: +_file_paths src/exec/csv_scan_node.h /^ std::vector _file_paths;$/;" m class:doris::CsvScanNode +_file_paths src/exec/csv_scanner.h /^ std::vector _file_paths;$/;" m class:doris::CsvScanner +_file_paths test/exec/csv_scanner_test.cpp /^ std::vector _file_paths;$/;" m class:doris::CsvScannerTest file: +_file_reader src/exec/plain_text_line_reader.h /^ FileReader* _file_reader;$/;" m class:doris::PlainTextLineReader +_file_size src/exec/broker_reader.h /^ int64_t _file_size;$/;" m class:doris::BrokerReader +_file_size src/exec/local_file_reader.h /^ int64_t _file_size;$/;" m class:doris::LocalFileReader +_file_system src/olap/data_dir.h /^ std::string _file_system;$/;" m class:doris::DataDir +_file_writer src/runtime/export_sink.h /^ std::unique_ptr _file_writer;$/;" m class:doris::ExportSink +_filename src/env/env_posix.cpp /^ const string _filename;$/;" m class:doris::PosixRandomRWFile file: +_filename src/env/env_posix.cpp /^ const string _filename;$/;" m class:doris::PosixSequentialFile file: +_filename src/env/env_posix.cpp /^ std::string _filename;$/;" m class:doris::PosixRandomAccessFile file: +_filename src/env/env_posix.cpp /^ std::string _filename;$/;" m class:doris::PosixWritableFile file: +_filename src/exec/orc_scanner.cpp /^ std::string _filename;$/;" m class:doris::ORCFileStream file: +_filename src/olap/rowset/segment_v2/ordinal_page_index.h /^ std::string _filename;$/;" m class:doris::segment_v2::OrdinalIndexReader +_filename src/olap/rowset/segment_v2/zone_map_index.h /^ std::string _filename;$/;" m class:doris::segment_v2::ZoneMapIndexReader +_filesize src/env/env_posix.cpp /^ uint64_t _filesize = 0;$/;" m class:doris::PosixWritableFile file: +_fill_compressed src/olap/file_stream.cpp /^OLAPStatus ReadOnlyFileStream::_fill_compressed(size_t length) {$/;" f class:doris::ReadOnlyFileStream +_filter_bitmap src/exec/tablet_sink.h /^ Bitmap _filter_bitmap;$/;" m class:doris::stream_load::OlapTableSink +_filter_clauses src/exec/es/es_query_builder.h /^ std::vector _filter_clauses;$/;" m class:doris::BooleanQueryBuilder +_filtered_rows src/olap/schema_change.h /^ uint64_t _filtered_rows;$/;" m class:doris::SchemaChange +_filtered_rows_counter src/exec/tablet_sink.h /^ RuntimeProfile::Counter* _filtered_rows_counter = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_filters src/exec/mysql_scan_node.h /^ std::vector _filters;$/;" m class:doris::MysqlScanNode +_finalize_dict_encoding src/olap/rowset/column_writer.cpp /^OLAPStatus VarStringColumnWriter::_finalize_dict_encoding() {$/;" f class:doris::VarStringColumnWriter +_finalize_direct_encoding src/olap/rowset/column_writer.cpp /^OLAPStatus VarStringColumnWriter::_finalize_direct_encoding() {$/;" f class:doris::VarStringColumnWriter +_finalize_fn output/udf/include/uda_test_harness.h /^ FinalizeFn _finalize_fn;$/;" m class:doris_udf::UdaTestHarnessBase +_finalize_fn src/exprs/agg_fn_evaluator.h /^ void* _finalize_fn;$/;" m class:doris::AggFnEvaluator +_finalize_fn src/olap/aggregate_func.h /^ void (*_finalize_fn)(RowCursorCell* src, MemPool* mem_pool);$/;" m class:doris::AggregateInfo +_finalize_fn src/udf/uda_test_harness.h /^ FinalizeFn _finalize_fn;$/;" m class:doris_udf::UdaTestHarnessBase +_finalize_fn_symbol src/exprs/agg_fn_evaluator.h /^ std::string _finalize_fn_symbol;$/;" m class:doris::AggFnEvaluator +_finalize_segment src/olap/rowset/column_data_writer.cpp /^OLAPStatus ColumnDataWriter::_finalize_segment() {$/;" f class:doris::ColumnDataWriter +_find_handler src/http/ev_http_server.cpp /^HttpHandler* EvHttpServer::_find_handler(HttpRequest* req) {$/;" f class:doris::EvHttpServer +_find_largest_consumption_channel src/runtime/load_channel.cpp /^bool LoadChannel::_find_largest_consumption_channel(std::shared_ptr* channel) {$/;" f class:doris::LoadChannel +_find_nulls src/exec/intersect_node.h /^ std::vector _find_nulls;$/;" m class:doris::IntersectNode +_find_pointer src/olap/lru_cache.cpp /^LRUHandle** HandleTable::_find_pointer(const CacheKey& key, uint32_t hash) {$/;" f class:doris::HandleTable +_find_position_by_full_key src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::_find_position_by_full_key($/;" f class:doris::ColumnData +_find_position_by_short_key src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::_find_position_by_short_key($/;" f class:doris::ColumnData +_finds_nulls src/exec/hash_table.h /^ const std::vector _finds_nulls;$/;" m class:doris::HashTable +_finds_nulls src/exec/partitioned_hash_table.h /^ const bool _finds_nulls;$/;" m class:doris::PartitionedHashTableCtx +_finish src/olap/rowset/segment_v2/bitshuffle_page.h /^ OwnedSlice _finish(int final_size_of_type) {$/;" f class:doris::segment_v2::BitshufflePageBuilder +_finish_clone src/olap/task/engine_clone_task.cpp /^OLAPStatus EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_dir,$/;" f class:doris::EngineCloneTask +_finish_current_data_page src/olap/rowset/segment_v2/indexed_column_writer.cpp /^Status IndexedColumnWriter::_finish_current_data_page() {$/;" f class:doris::segment_v2::IndexedColumnWriter +_finish_current_page src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::_finish_current_page() {$/;" f class:doris::segment_v2::ColumnWriter +_finish_task src/agent/task_worker_pool.cpp /^void TaskWorkerPool::_finish_task(const TFinishTaskRequest& finish_task_request) {$/;" f class:doris::TaskWorkerPool +_finish_timer src/runtime/data_spliter.h /^ RuntimeProfile::Counter* _finish_timer;$/;" m class:doris::DataSpliter +_finished src/olap/rowset/segment_v2/binary_dict_page.h /^ bool _finished;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_finished src/olap/rowset/segment_v2/binary_plain_page.h /^ bool _finished;$/;" m class:doris::segment_v2::BinaryPlainPageBuilder +_finished src/olap/rowset/segment_v2/binary_prefix_page.h /^ bool _finished = false;$/;" m class:doris::segment_v2::BinaryPrefixPageBuilder +_finished src/olap/rowset/segment_v2/bitshuffle_page.h /^ bool _finished;$/;" m class:doris::segment_v2::BitshufflePageBuilder +_finished src/olap/rowset/segment_v2/frame_of_reference_page.h /^ bool _finished;$/;" m class:doris::segment_v2::FrameOfReferencePageBuilder +_finished src/olap/rowset/segment_v2/index_page.h /^ bool _finished = false;$/;" m class:doris::segment_v2::IndexPageBuilder +_finished src/olap/rowset/segment_v2/rle_page.h /^ bool _finished;$/;" m class:doris::segment_v2::RlePageBuilder +_finished src/runtime/stream_load/stream_load_pipe.h /^ bool _finished;$/;" m class:doris::StreamLoadPipe +_finished_channel_ids src/runtime/load_channel.h /^ std::unordered_set _finished_channel_ids;$/;" m class:doris::LoadChannel +_finst_id src/runtime/data_stream_sender.cpp /^ PUniqueId _finst_id;$/;" m class:doris::DataStreamSender::Channel file: +_first_batch_wait_total_timer src/runtime/data_stream_recvr.h /^ RuntimeProfile::Counter* _first_batch_wait_total_timer;$/;" m class:doris::DataStreamRecvr +_first_entry src/olap/rowset/segment_v2/binary_prefix_page.h /^ faststring _first_entry;$/;" m class:doris::segment_v2::BinaryPrefixPageBuilder +_first_len_tbl src/util/utf8_check.cpp /^const uint8_t _first_len_tbl[] = {$/;" m namespace:doris file: +_first_materialized_child_idx src/exec/except_node.h /^ const int _first_materialized_child_idx;$/;" m class:doris::ExceptNode +_first_materialized_child_idx src/exec/union_node.h /^ const int _first_materialized_child_idx;$/;" m class:doris::UnionNode +_first_range_tbl src/util/utf8_check.cpp /^static const uint8_t _first_range_tbl[] = {$/;" m namespace:doris file: +_first_rowid src/olap/rowset/segment_v2/column_writer.h /^ ordinal_t _first_rowid = 0;$/;" m class:doris::segment_v2::ColumnWriter +_first_val src/olap/rowset/segment_v2/frame_of_reference_page.h /^ CppType _first_val;$/;" m class:doris::segment_v2::FrameOfReferencePageBuilder +_first_val_null_offset src/exec/analytic_eval_node.h /^ long _first_val_null_offset;$/;" m class:doris::AnalyticEvalNode +_first_value src/olap/rowset/segment_v2/binary_dict_page.h /^ faststring _first_value;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_first_value src/olap/rowset/segment_v2/binary_plain_page.h /^ faststring _first_value;$/;" m class:doris::segment_v2::BinaryPlainPageBuilder +_first_value src/olap/rowset/segment_v2/bitshuffle_page.h /^ CppType _first_value;$/;" m class:doris::segment_v2::BitshufflePageBuilder +_first_value src/olap/rowset/segment_v2/indexed_column_writer.h /^ faststring _first_value;$/;" m class:doris::segment_v2::IndexedColumnWriter +_first_value src/olap/rowset/segment_v2/plain_page.h /^ faststring _first_value;$/;" m class:doris::segment_v2::PlainPageBuilder +_first_value src/olap/rowset/segment_v2/rle_page.h /^ CppType _first_value;$/;" m class:doris::segment_v2::RlePageBuilder +_fixed_buf src/olap/row_cursor.h /^ char* _fixed_buf = nullptr; \/\/ point to fixed buf$/;" m class:doris::RowCursor +_fixed_buffer_byte_size output/udf/include/uda_test_harness.h /^ int _fixed_buffer_byte_size;$/;" m class:doris_udf::UdaTestHarnessBase +_fixed_buffer_byte_size src/udf/uda_test_harness.h /^ int _fixed_buffer_byte_size;$/;" m class:doris_udf::UdaTestHarnessBase +_fixed_delta src/olap/rowset/run_length_integer_writer.h /^ int64_t _fixed_delta;$/;" m class:doris::RunLengthIntegerWriter +_fixed_file_header src/olap/file_helper.h /^ FixedFileHeaderV2 _fixed_file_header;$/;" m class:doris::FileHeader +_fixed_file_header_size src/olap/file_helper.h /^ uint32_t _fixed_file_header_size;$/;" m class:doris::FileHeader +_fixed_len src/olap/row_cursor.h /^ size_t _fixed_len;$/;" m class:doris::RowCursor +_fixed_len_block_offset src/runtime/merge_sorter.cpp /^ int _fixed_len_block_offset;$/;" m class:doris::MergeSorter::Run file: +_fixed_len_block_offset src/runtime/spill_sorter.cc /^ int _fixed_len_block_offset;$/;" m class:doris::SpillSorter::Run file: +_fixed_len_blocks src/runtime/merge_sorter.cpp /^ std::vector _fixed_len_blocks;$/;" m class:doris::MergeSorter::Run file: +_fixed_len_blocks src/runtime/spill_sorter.cc /^ vector _fixed_len_blocks;$/;" m class:doris::SpillSorter::Run file: +_fixed_len_blocks_index src/runtime/merge_sorter.cpp /^ int _fixed_len_blocks_index;$/;" m class:doris::MergeSorter::Run file: +_fixed_len_blocks_index src/runtime/spill_sorter.cc /^ int _fixed_len_blocks_index;$/;" m class:doris::SpillSorter::Run file: +_fixed_run_length src/olap/rowset/run_length_integer_writer.h /^ uint32_t _fixed_run_length;$/;" m class:doris::RunLengthIntegerWriter +_fixed_tuple_row_size src/runtime/buffered_tuple_stream.h /^ int _fixed_tuple_row_size;$/;" m class:doris::BufferedTupleStream +_fixed_tuple_row_size src/runtime/buffered_tuple_stream2.h /^ int _fixed_tuple_row_size;$/;" m class:doris::BufferedTupleStream2 +_fixed_values src/exec/olap_common.h /^ std::set _fixed_values; \/\/ Column's fixed int value$/;" m class:doris::ColumnValueRange +_flags src/runtime/heartbeat_flags.h /^ std::atomic _flags;$/;" m class:doris::HeartbeatFlags +_flags test/runtime/heartbeat_flags_test.cpp /^ HeartbeatFlags _flags;$/;" m class:doris::HeartbeatFlagsTest file: +_flush src/runtime/row_batch.h /^ FlushMode _flush;$/;" m class:doris::RowBatch +_flush_index src/olap/rowset/segment_v2/indexed_column_writer.cpp /^Status IndexedColumnWriter::_flush_index(IndexPageBuilder* index_builder, BTreeMetaPB* meta) {$/;" f class:doris::segment_v2::IndexedColumnWriter +_flush_memtable src/olap/memtable_flush_executor.cpp /^void FlushToken::_flush_memtable(std::shared_ptr memtable) {$/;" f class:doris::FlushToken +_flush_memtable_async src/olap/delta_writer.cpp /^OLAPStatus DeltaWriter::_flush_memtable_async() {$/;" f class:doris::DeltaWriter +_flush_pool src/olap/memtable_flush_executor.h /^ std::unique_ptr _flush_pool;$/;" m class:doris::MemTableFlushExecutor +_flush_row_block src/olap/rowset/column_data_writer.cpp /^OLAPStatus ColumnDataWriter::_flush_row_block(bool finalize) {$/;" f class:doris::ColumnDataWriter +_flush_segment_with_verfication src/olap/rowset/column_data_writer.cpp /^OLAPStatus ColumnDataWriter::_flush_segment_with_verfication() {$/;" f class:doris::ColumnDataWriter +_flush_segment_writer src/olap/rowset/beta_rowset_writer.cpp /^OLAPStatus BetaRowsetWriter::_flush_segment_writer() {$/;" f class:doris::BetaRowsetWriter +_flush_status src/olap/memtable_flush_executor.h /^ std::atomic _flush_status;$/;" m class:doris::FlushToken +_flush_token src/olap/delta_writer.h /^ std::unique_ptr _flush_token;$/;" m class:doris::DeltaWriter +_flush_token src/olap/memtable_flush_executor.h /^ std::unique_ptr _flush_token;$/;" m class:doris::FlushToken +_fn src/exprs/agg_fn_evaluator.h /^ const TFunction _fn;$/;" m class:doris::AggFnEvaluator +_fn src/exprs/expr.h /^ TFunction _fn;$/;" m class:doris::Expr +_fn_context_index src/exprs/expr.h /^ int _fn_context_index;$/;" m class:doris::Expr +_fn_contexts src/exprs/expr_context.h /^ std::vector _fn_contexts;$/;" m class:doris::ExprContext +_fn_contexts_ptr src/exprs/expr_context.h /^ FunctionContext** _fn_contexts_ptr;$/;" m class:doris::ExprContext +_fn_ctx src/testutil/function_utils.h /^ doris_udf::FunctionContext* _fn_ctx = nullptr;$/;" m class:doris::FunctionUtils +_fn_ctx_idx src/exprs/expr.h /^ int _fn_ctx_idx = -1;$/;" m class:doris::Expr +_fn_ctx_idx_end src/exprs/expr.h /^ int _fn_ctx_idx_end = 0;$/;" m class:doris::Expr +_fn_ctx_idx_start src/exprs/expr.h /^ int _fn_ctx_idx_start = 0;$/;" m class:doris::Expr +_fn_ctxs src/exec/analytic_eval_node.h /^ std::vector _fn_ctxs;$/;" m class:doris::AnalyticEvalNode +_fn_id src/exprs/agg_fn_evaluator.h /^ long _fn_id;$/;" m class:doris::AggFnEvaluator +_fn_scope src/exec/analytic_eval_node.h /^ AnalyticFnScope _fn_scope;$/;" m class:doris::AnalyticEvalNode +_fname src/olap/rowset/segment_v2/segment.h /^ std::string _fname;$/;" m class:doris::segment_v2::Segment +_footer src/olap/rowset/segment_v2/index_page.h /^ IndexPageFooterPB _footer;$/;" m class:doris::segment_v2::IndexPageReader +_footer src/olap/rowset/segment_v2/segment.h /^ SegmentFooterPB _footer;$/;" m class:doris::segment_v2::Segment +_footer src/olap/rowset/segment_v2/segment_writer.h /^ SegmentFooterPB _footer;$/;" m class:doris::segment_v2::SegmentWriter +_footer src/olap/short_key_index.h /^ segment_v2::ShortKeyFooterPB _footer;$/;" m class:doris::ShortKeyIndexDecoder +_footer_start src/olap/rowset/segment_v2/binary_prefix_page.h /^ const uint8_t* _footer_start = nullptr;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +_fp src/exec/local_file_reader.h /^ FILE* _fp;$/;" m class:doris::LocalFileReader +_fp src/exec/local_file_writer.h /^ FILE* _fp;$/;" m class:doris::LocalFileWriter +_fp src/olap/file_helper.h /^ FILE *_fp;$/;" m class:doris::FileHandlerWithBuf +_fp src/runtime/dpp_writer.h /^ FileHandler* _fp;$/;" m class:doris::DppWriter +_fpp test/olap/rowset/segment_v2/block_bloom_filter_test.cpp /^ double _fpp = 0.05;$/;" m class:doris::segment_v2::BlockBloomFilterTest file: +_frac src/olap/tablet_schema.h /^ int32_t _frac;$/;" m class:doris::TabletColumn +_frac_reader src/olap/rowset/column_reader.h /^ RunLengthIntegerReader* _frac_reader;$/;" m class:doris::DecimalColumnReader +_frac_writer src/olap/rowset/column_writer.h /^ RunLengthIntegerWriter* _frac_writer;$/;" m class:doris::DecimalColumnWriter +_fragment_id src/runtime/buffer_control_block.h /^ TUniqueId _fragment_id;$/;" m class:doris::BufferControlBlock +_fragment_instance_id src/runtime/data_stream_recvr.h /^ TUniqueId _fragment_instance_id;$/;" m class:doris::DataStreamRecvr +_fragment_instance_id src/runtime/data_stream_sender.cpp /^ TUniqueId _fragment_instance_id;$/;" m class:doris::DataStreamSender::Channel file: +_fragment_instance_id src/runtime/fragment_mgr.cpp /^ TUniqueId _fragment_instance_id;$/;" m class:doris::FragmentExecState file: +_fragment_instance_id src/runtime/runtime_state.h /^ TUniqueId _fragment_instance_id;$/;" m class:doris::RuntimeState +_fragment_local_fn_state src/udf/udf_internal.h /^ void* _fragment_local_fn_state;$/;" m class:doris::FunctionContextImpl +_fragment_map src/runtime/fragment_mgr.h /^ std::unordered_map> _fragment_map;$/;" m class:doris::FragmentMgr +_fragment_mem_tracker src/runtime/runtime_state.h /^ MemTracker* _fragment_mem_tracker;$/;" m class:doris::RuntimeState +_fragment_mgr src/runtime/exec_env.h /^ FragmentMgr* _fragment_mgr = nullptr;$/;" m class:doris::ExecEnv +_fragment_queue_map src/runtime/result_queue_mgr.h /^ std::unordered_map _fragment_queue_map;$/;" m class:doris::ResultQueueMgr +_fragment_stream_set src/runtime/data_stream_mgr.h /^ FragmentStreamSet _fragment_stream_set;$/;" m class:doris::DataStreamMgr +_frame_count src/util/frame_of_reference_coding.h /^ uint32_t _frame_count = 0;$/;" m class:doris::ForDecoder +_frame_offsets src/util/frame_of_reference_coding.h /^ std::vector _frame_offsets;$/;" m class:doris::ForDecoder +_free_buffer_descs src/runtime/disk_io_mgr.h /^ std::list _free_buffer_descs;$/;" m class:doris::DiskIoMgr +_free_buffers src/runtime/disk_io_mgr.h /^ std::vector > _free_buffers;$/;" m class:doris::DiskIoMgr +_free_buffers_lock src/runtime/disk_io_mgr.h /^ boost::mutex _free_buffers_lock;$/;" m class:doris::DiskIoMgr +_free_ids src/util/core_local.h /^ std::deque _free_ids;$/;" m class:doris::CoreLocalValueController +_free_io_buffers src/runtime/buffered_block_mgr2.h /^ InternalQueue _free_io_buffers;$/;" m class:doris::BufferedBlockMgr2 +_free_pool_objs src/runtime/thread_resource_mgr.h /^ std::list _free_pool_objs;$/;" m class:doris::ThreadResourceMgr +_from src/olap/rowset/segment_v2/row_ranges.h /^ int64_t _from;$/;" m class:doris::segment_v2::RowRange +_from_string src/olap/types.h /^ OLAPStatus (*_from_string)(void* buf, const std::string& scan_key);$/;" m class:doris::TypeInfo +_frontend_client_cache src/runtime/exec_env.h /^ ClientCache* _frontend_client_cache = nullptr;$/;" m class:doris::ExecEnv +_fs src/runtime/disk_io_mgr.h /^ hdfsFS _fs;$/;" m class:doris::DiskIoMgr::HdfsCachedFileHandle +_fs src/runtime/disk_io_mgr.h /^ hdfsFS _fs;$/;" m class:doris::DiskIoMgr::RequestRange +_full_encode_ascending src/olap/key_coder.h /^ FullEncodeAscendingFunc _full_encode_ascending;$/;" m class:doris::KeyCoder +_full_value_position src/olap/hll.h /^ char* _full_value_position;$/;" m class:doris::HllSetResolver +_func src/util/defer_op.h /^ DeferFunction _func;$/;" m class:doris::DeferOp +_func src/util/threadpool.cpp /^ std::function _func;$/;" m class:doris::FunctionRunnable file: +_function_type src/exprs/agg_fn_evaluator.h /^ TFunctionBinaryType::type _function_type;$/;" m class:doris::AggFnEvaluator +_functor src/util/thread.h /^ const ThreadFunctor _functor;$/;" m class:doris::Thread +_gap_vs_patch_list src/olap/rowset/run_length_integer_writer.h /^ int64_t _gap_vs_patch_list[MAX_PATCH_LIST];$/;" m class:doris::RunLengthIntegerWriter +_garbage_collection src/olap/delta_writer.cpp /^void DeltaWriter::_garbage_collection() {$/;" f class:doris::DeltaWriter +_garbage_collection src/olap/rowset/alpha_rowset_writer.cpp /^OLAPStatus AlphaRowsetWriter::_garbage_collection() {$/;" f class:doris::AlphaRowsetWriter +_garbage_sweeper_thread src/olap/storage_engine.h /^ std::thread _garbage_sweeper_thread;$/;" m class:doris::StorageEngine +_garbage_sweeper_thread_callback src/olap/olap_server.cpp /^void* StorageEngine::_garbage_sweeper_thread_callback(void* arg) {$/;" f class:doris::StorageEngine +_gc_functions src/runtime/mem_tracker.h /^ std::vector _gc_functions;$/;" m class:doris::MemTracker +_gc_lock src/runtime/mem_tracker.h /^ std::mutex _gc_lock;$/;" m class:doris::MemTracker +_gc_mutex src/olap/storage_engine.h /^ Mutex _gc_mutex;$/;" m class:doris::StorageEngine +_gen_tablet_dir src/olap/tablet_manager.cpp /^static string _gen_tablet_dir(const string& dir, int16_t shard_id, int64_t tablet_id) {$/;" f namespace:doris +_gen_tablet_path src/olap/tablet.cpp /^void Tablet::_gen_tablet_path() {$/;" f class:doris::Tablet +_generate_new_header src/olap/task/engine_storage_migration_task.cpp /^OLAPStatus EngineStorageMigrationTask::_generate_new_header($/;" f class:doris::EngineStorageMigrationTask +_get_aggregation_by_index src/olap/rowset/segment_reader.h /^ FieldAggregationMethod _get_aggregation_by_index(uint32_t index) {$/;" f class:doris::SegmentReader +_get_block src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::_get_block(bool without_filter, int rows_read) {$/;" f class:doris::ColumnData +_get_block_from_reader src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::_get_block_from_reader($/;" f class:doris::ColumnData +_get_cache_entry src/runtime/user_function_cache.cpp /^Status UserFunctionCache::_get_cache_entry($/;" f class:doris::UserFunctionCache +_get_cache_info src/util/cpu_info.cpp /^void CpuInfo::_get_cache_info(long cache_sizes[NUM_CACHE_LEVELS],$/;" f class:doris::CpuInfo +_get_cond src/runtime/stream_load/stream_load_pipe.h /^ std::condition_variable _get_cond;$/;" m class:doris::StreamLoadPipe +_get_cv src/util/blocking_priority_queue.hpp /^ boost::condition_variable _get_cv; \/\/ 'get' callers wait on this$/;" m class:doris::BlockingPriorityQueue +_get_cv src/util/blocking_queue.hpp /^ std::condition_variable _get_cv; \/\/ 'get' callers wait on this$/;" m class:doris::BlockingQueue +_get_existing_files_from_local src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::_get_existing_files_from_local($/;" f class:doris::SnapshotLoader +_get_existing_files_from_remote src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::_get_existing_files_from_remote($/;" f class:doris::SnapshotLoader +_get_field_index src/olap/delete_handler.h /^ int32_t _get_field_index(const TabletSchema& schema, const std::string& field_name) const {$/;" f class:doris::DeleteConditionHandler +_get_field_index src/olap/olap_cond.h /^ int32_t _get_field_index(const std::string& field_name) const {$/;" f class:doris::Conditions +_get_field_type_by_index src/olap/rowset/segment_reader.h /^ FieldType _get_field_type_by_index(uint32_t index) {$/;" f class:doris::SegmentReader +_get_file_name_from_path src/agent/pusher.cpp /^void Pusher::_get_file_name_from_path(const string& file_path, string* file_name) {$/;" f class:doris::Pusher +_get_file_name_from_path src/olap/task/engine_batch_load_task.cpp /^void EngineBatchLoadTask::_get_file_name_from_path(const string& file_path, string* file_name) {$/;" f class:doris::EngineBatchLoadTask +_get_filtered_pages src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::_get_filtered_pages(CondColumn* cond_column,$/;" f class:doris::segment_v2::ColumnReader +_get_handlers src/http/ev_http_server.h /^ PathTrie _get_handlers;$/;" m class:doris::EvHttpServer +_get_header_full_path src/olap/snapshot_manager.cpp /^string SnapshotManager::_get_header_full_path($/;" f class:doris::SnapshotManager +_get_included_row_index_stream_num src/olap/rowset/segment_reader.h /^ inline const int32_t _get_included_row_index_stream_num() {$/;" f class:doris::SegmentReader +_get_input_num_rows_from_seg_grps src/olap/compaction.cpp /^int64_t Compaction::_get_input_num_rows_from_seg_grps() {$/;" f class:doris::Compaction +_get_latest_tablet_path_from_trash src/http/action/restore_tablet_action.cpp /^bool RestoreTabletAction::_get_latest_tablet_path_from_trash($/;" f class:doris::RestoreTabletAction +_get_new_block_timer src/runtime/buffered_tuple_stream.h /^ RuntimeProfile::Counter* _get_new_block_timer;$/;" m class:doris::BufferedTupleStream +_get_new_block_timer src/runtime/buffered_tuple_stream2.h /^ RuntimeProfile::Counter* _get_new_block_timer;$/;" m class:doris::BufferedTupleStream2 +_get_next_batch_timer src/runtime/sorted_run_merger.h /^ RuntimeProfile::Counter* _get_next_batch_timer;$/;" m class:doris::SortedRunMerger +_get_next_iter src/exec/topn_node.h /^ std::vector::iterator _get_next_iter;$/;" m class:doris::TopNNode +_get_next_task_index src/agent/task_worker_pool.cpp /^uint32_t TaskWorkerPool::_get_next_task_index(int32_t thread_count,$/;" f class:doris::TaskWorkerPool +_get_next_timer src/runtime/sorted_run_merger.h /^ RuntimeProfile::Counter* _get_next_timer;$/;" m class:doris::SortedRunMerger +_get_resource_value src/agent/cgroups_mgr.cpp /^int64_t CgroupsMgr::_get_resource_value(const TResourceType::type resource_type,$/;" f class:doris::CgroupsMgr +_get_restart_point src/olap/rowset/segment_v2/binary_prefix_page.h /^ const uint8_t* _get_restart_point(size_t restart_point_index) const {$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +_get_results_timer src/exec/aggregation_node.h /^ RuntimeProfile::Counter* _get_results_timer;$/;" m class:doris::AggregationNode +_get_results_timer src/exec/partitioned_aggregation_node.h /^ RuntimeProfile::Counter* _get_results_timer;$/;" m class:doris::PartitionedAggregationNode +_get_results_timer src/exec/pre_aggregation_node.h /^ RuntimeProfile::Counter* _get_results_timer;$/;" m class:doris::PreAggregationNode +_get_row_ranges_by_column_conditions src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_get_row_ranges_by_column_conditions() {$/;" f class:doris::segment_v2::SegmentIterator +_get_row_ranges_by_keys src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_get_row_ranges_by_keys() {$/;" f class:doris::segment_v2::SegmentIterator +_get_row_ranges_from_conditions src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_get_row_ranges_from_conditions(RowRanges* condition_row_ranges) {$/;" f class:doris::segment_v2::SegmentIterator +_get_tablet_id_and_schema_hash_from_file_path src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::_get_tablet_id_and_schema_hash_from_file_path($/;" f class:doris::SnapshotLoader +_get_tablet_id_from_remote_path src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::_get_tablet_id_from_remote_path($/;" f class:doris::SnapshotLoader +_get_tablet_info src/agent/task_worker_pool.cpp /^AgentStatus TaskWorkerPool::_get_tablet_info($/;" f class:doris::TaskWorkerPool +_get_tablet_infos src/olap/push_handler.cpp /^void PushHandler::_get_tablet_infos(const vector& tablet_vars,$/;" f class:doris::PushHandler +_get_tablet_map src/olap/tablet_manager.h /^inline TabletManager::tablet_map_t& TabletManager::_get_tablet_map(TTabletId tabletId) {$/;" f class:doris::TabletManager +_get_tablet_map_lock src/olap/tablet_manager.h /^inline RWMutex& TabletManager::_get_tablet_map_lock(TTabletId tabletId) {$/;" f class:doris::TabletManager +_get_tablet_unlocked src/olap/tablet_manager.cpp /^TabletSharedPtr TabletManager::_get_tablet_unlocked(TTabletId tablet_id, SchemaHash schema_hash) {$/;" f class:doris::TabletManager +_get_tablet_unlocked src/olap/tablet_manager.cpp /^TabletSharedPtr TabletManager::_get_tablet_unlocked(TTabletId tablet_id, SchemaHash schema_hash,$/;" f class:doris::TabletManager +_get_timestamp_and_count_from_schema_hash_path src/http/action/restore_tablet_action.cpp /^bool RestoreTabletAction::_get_timestamp_and_count_from_schema_hash_path($/;" f class:doris::RestoreTabletAction +_get_tmp_file_dir src/agent/pusher.cpp /^AgentStatus Pusher::_get_tmp_file_dir(const string& root_path, string* download_path) {$/;" f class:doris::Pusher +_get_tmp_file_dir src/olap/task/engine_batch_load_task.cpp /^AgentStatus EngineBatchLoadTask::_get_tmp_file_dir(const string& root_path, string* download_path) {$/;" f class:doris::EngineBatchLoadTask +_get_txn_lock src/olap/txn_manager.h /^ RWMutex* _get_txn_lock(TTransactionId txn_id) {$/;" f class:doris::TxnManager +_get_value_fn src/exprs/agg_fn_evaluator.h /^ void* _get_value_fn;$/;" m class:doris::AggFnEvaluator +_get_versions_to_be_changed src/olap/schema_change.cpp /^OLAPStatus SchemaChangeHandler::_get_versions_to_be_changed($/;" f class:doris::SchemaChangeHandler +_give_up_after_ms src/util/thread.h /^ int _give_up_after_ms;$/;" m class:doris::ThreadJoiner +_gotten test/util/blocking_queue_test.cpp /^ std::map _gotten;$/;" m class:doris::MultiThreadTest file: +_group src/runtime/fragment_mgr.cpp /^ std::string _group;$/;" m class:doris::FragmentExecState file: +_group_fd src/util/perf_counters.h /^ int _group_fd;$/;" m class:doris::PerfCounters +_grouping_list src/exec/repeat_node.h /^ std::vector> _grouping_list;$/;" m class:doris::RepeatNode +_grp_id src/runtime/routine_load/data_consumer.h /^ UniqueId _grp_id;$/;" m class:doris::DataConsumer +_grp_id src/runtime/routine_load/data_consumer_group.h /^ UniqueId _grp_id;$/;" m class:doris::DataConsumerGroup +_handle src/http/action/mini_load.cpp /^void MiniLoadAction::_handle(HttpRequest* http_req) {$/;" f class:doris::MiniLoadAction +_handle src/http/action/restore_tablet_action.cpp /^Status RestoreTabletAction::_handle(HttpRequest *req) {$/;" f class:doris::RestoreTabletAction +_handle src/http/action/stream_load.cpp /^Status StreamLoadAction::_handle(StreamLoadContext* ctx) {$/;" f class:doris::StreamLoadAction +_handle src/olap/page_cache.h /^ Cache::Handle* _handle = nullptr;$/;" m class:doris::PageCacheHandle +_handle src/util/file_cache.h /^ Cache::Handle* _handle;$/;" m class:doris::OpenedFileHandle +_handle_failed_node src/exec/tablet_sink.cpp /^bool IndexChannel::_handle_failed_node(NodeChannel* channel) {$/;" f class:doris::stream_load::IndexChannel +_handle_header src/http/action/meta_action.cpp /^Status MetaAction::_handle_header(HttpRequest *req, std::string* json_meta) {$/;" f class:doris::MetaAction +_handle_mem_exceed_limit src/runtime/load_channel_mgr.cpp /^void LoadChannelMgr::_handle_mem_exceed_limit() {$/;" f class:doris::LoadChannelMgr +_handle_show_compaction src/http/action/compaction_action.cpp /^Status CompactionAction::_handle_show_compaction(HttpRequest *req, std::string* json_result) {$/;" f class:doris::CompactionAction +_handler src/http/http_request.h /^ HttpHandler* _handler = nullptr;$/;" m class:doris::HttpRequest +_handler_ctx src/http/http_request.h /^ void* _handler_ctx = nullptr;$/;" m class:doris::HttpRequest +_handles src/olap/olap_meta.h /^ std::vector _handles;$/;" m class:doris::OlapMeta +_has_alpha_rowset_extra_meta_pb src/olap/rowset/rowset_meta.h /^ bool _has_alpha_rowset_extra_meta_pb() {$/;" f class:doris::RowsetMeta +_has_bf_fpp src/olap/tablet_schema.h /^ bool _has_bf_fpp = false;$/;" m class:doris::TabletSchema +_has_bitmap_index src/olap/tablet_schema.h /^ bool _has_bitmap_index = false;$/;" m class:doris::TabletColumn +_has_called src/util/once.h /^ std::atomic _has_called;$/;" m class:doris::DorisCallOnce +_has_case_expr src/exprs/case_expr.h /^ const bool _has_case_expr;$/;" m class:doris::CaseExpr +_has_default_value src/olap/rowset/segment_v2/column_reader.h /^ bool _has_default_value;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_has_default_value src/olap/tablet_schema.h /^ bool _has_default_value;$/;" m class:doris::TabletColumn +_has_else_expr src/exprs/case_expr.h /^ const bool _has_else_expr;$/;" m class:doris::CaseExpr +_has_first_val_null_offset src/exec/analytic_eval_node.h /^ bool _has_first_val_null_offset;$/;" m class:doris::AnalyticEvalNode +_has_grp src/runtime/routine_load/data_consumer.h /^ bool _has_grp;$/;" m class:doris::DataConsumer +_has_in_flight_packet src/exec/tablet_sink.h /^ bool _has_in_flight_packet = false;$/;" m class:doris::stream_load::NodeChannel +_has_in_flight_row src/runtime/row_batch.h /^ bool _has_in_flight_row; \/\/ if true, last row hasn't been committed yet$/;" m class:doris::RowBatch +_has_index_page src/olap/rowset/segment_v2/indexed_column_reader.h /^ bool _has_index_page = false;$/;" m class:doris::segment_v2::IndexedColumnReader +_has_matches src/exec/partitioned_hash_table.h /^ bool _has_matches;$/;" m class:doris::PartitionedHashTable +_has_null src/olap/rowset/segment_v2/bitmap_index_reader.h /^ bool _has_null = false;$/;" m class:doris::segment_v2::BitmapIndexReader +_has_null src/olap/rowset/segment_v2/bloom_filter.h /^ bool* _has_null;$/;" m class:doris::segment_v2::BloomFilter +_has_null src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ bool _has_null;$/;" m class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl file: +_has_null src/olap/rowset/segment_v2/column_writer.cpp /^ bool _has_null;$/;" m class:doris::segment_v2::NullBitmapBuilder file: +_has_range_value src/exec/olap_common.h /^ bool _has_range_value;$/;" m class:doris::OlapScanKeys +_has_referenced_column src/olap/tablet_schema.h /^ bool _has_referenced_column;$/;" m class:doris::TabletColumn +_has_thread_token src/runtime/plan_fragment_executor.h /^ bool _has_thread_token;$/;" m class:doris::PlanFragmentExecutor +_has_update_counter src/exec/olap_scanner.h /^ bool _has_update_counter = false;$/;" m class:doris::OlapScanner +_has_var_len_slots src/runtime/merge_sorter.cpp /^ const bool _has_var_len_slots;$/;" m class:doris::MergeSorter::Run file: +_has_var_len_slots src/runtime/merge_sorter.h /^ bool _has_var_len_slots;$/;" m class:doris::MergeSorter +_has_var_len_slots src/runtime/spill_sorter.cc /^ const bool _has_var_len_slots;$/;" m class:doris::SpillSorter::Run file: +_has_var_len_slots src/runtime/spill_sorter.h /^ bool _has_var_len_slots;$/;" m class:doris::SpillSorter +_has_varlen_slots src/runtime/descriptors.h /^ bool _has_varlen_slots;$/;" m class:doris::RowDescriptor +_has_varlen_slots src/runtime/descriptors.h /^ bool _has_varlen_slots;$/;" m class:doris::TupleDescriptor +_hash src/exec/hash_table.h /^ uint32_t _hash; \/\/ Cache of the hash for _data$/;" m struct:doris::HashTable::Node +_hash_code src/olap/types.h /^ uint32_t (*_hash_code)(const void* data, uint32_t seed);$/;" m class:doris::TypeInfo +_hash_func src/olap/rowset/segment_v2/bloom_filter.h /^ std::function _hash_func;$/;" m class:doris::segment_v2::BloomFilter +_hash_function_num src/olap/bloom_filter.hpp /^ uint32_t _hash_function_num;$/;" m class:doris::BloomFilter +_hash_function_num src/olap/bloom_filter_reader.h /^ uint32_t _hash_function_num;$/;" m class:doris::BloomFilterIndexReader +_hash_partitions src/exec/partitioned_aggregation_node.h /^ std::vector _hash_partitions;$/;" m class:doris::PartitionedAggregationNode +_hash_set src/olap/hll.h /^ std::set _hash_set;$/;" m class:doris::HyperLogLog +_hash_sink test/runtime/data_stream_test.cpp /^ TDataStreamSink _hash_sink;$/;" m class:doris::DataStreamTest file: +_hash_slice src/olap/lru_cache.cpp /^inline uint32_t ShardedLRUCache::_hash_slice(const CacheKey& s) {$/;" f class:doris::ShardedLRUCache +_hash_table_buckets_counter src/exec/aggregation_node.h /^ RuntimeProfile::Counter* _hash_table_buckets_counter;$/;" m class:doris::AggregationNode +_hash_table_buckets_counter src/exec/pre_aggregation_node.h /^ RuntimeProfile::Counter* _hash_table_buckets_counter;$/;" m class:doris::PreAggregationNode +_hash_table_load_factor_counter src/exec/aggregation_node.h /^ RuntimeProfile::Counter* _hash_table_load_factor_counter;$/;" m class:doris::AggregationNode +_hash_table_load_factor_counter src/exec/pre_aggregation_node.h /^ RuntimeProfile::Counter* _hash_table_load_factor_counter;$/;" m class:doris::PreAggregationNode +_hash_tbl src/exec/aggregation_node.h /^ boost::scoped_ptr _hash_tbl;$/;" m class:doris::AggregationNode +_hash_tbl src/exec/hash_join_node.h /^ boost::scoped_ptr _hash_tbl;$/;" m class:doris::HashJoinNode +_hash_tbl src/exec/intersect_node.h /^ std::unique_ptr _hash_tbl;$/;" m class:doris::IntersectNode +_hash_tbl src/exec/pre_aggregation_node.h /^ boost::scoped_ptr _hash_tbl;$/;" m class:doris::PreAggregationNode +_hash_tbl_iterator src/exec/hash_join_node.h /^ HashTable::Iterator _hash_tbl_iterator;$/;" m class:doris::HashJoinNode +_hash_tbl_iterator src/exec/intersect_node.h /^ HashTable::Iterator _hash_tbl_iterator;$/;" m class:doris::IntersectNode +_hash_tbl_load_factor_counter src/exec/hash_join_node.h /^ RuntimeProfile::Counter* _hash_tbl_load_factor_counter;$/;" m class:doris::HashJoinNode +_hb_mtx src/agent/heartbeat_server.h /^ std::mutex _hb_mtx;$/;" m class:doris::HeartbeatServer +_hdfs_file src/runtime/disk_io_mgr.h /^ HdfsCachedFileHandle* _hdfs_file;$/;" m union:doris::DiskIoMgr::ScanRange::__anon2 +_hdfs_file src/runtime/disk_io_mgr.h /^ hdfsFile _hdfs_file;$/;" m class:doris::DiskIoMgr::HdfsCachedFileHandle +_hdfs_location src/exprs/agg_fn_evaluator.h /^ std::string _hdfs_location;$/;" m class:doris::AggFnEvaluator +_hdfs_lock src/runtime/disk_io_mgr.h /^ boost::mutex _hdfs_lock;$/;" m class:doris::DiskIoMgr::ScanRange +_head src/runtime/free_list.hpp /^ FreeListNode _head;$/;" m class:doris::FreeList +_head_handlers src/http/ev_http_server.h /^ PathTrie _head_handlers;$/;" m class:doris::EvHttpServer +_header src/olap/bloom_filter_writer.h /^ BloomFilterIndexHeader _header;$/;" m class:doris::BloomFilterIndexWriter +_header src/olap/push_handler.h /^ FileHeader _header;$/;" m class:doris::BinaryFile +_header src/olap/stream_index_writer.h /^ StreamIndexHeader _header;$/;" m class:doris::StreamIndexWriter +_header src/runtime/dpp_writer.h /^ FileHeader _header;$/;" m class:doris::DppWriter +_header_info src/exec/decompressor.h /^ struct HeaderInfo _header_info;$/;" m class:doris::LzopDecompressor typeref:struct:doris::LzopDecompressor::HeaderInfo +_header_length src/olap/rowset/segment_reader.h /^ uint64_t _header_length; \/\/ Header(FixHeader+PB)大小,读数据时需要偏移$/;" m class:doris::SegmentReader +_header_list src/http/http_client.h /^ curl_slist *_header_list = nullptr;$/;" m class:doris::HttpClient +_header_message src/olap/rowset/segment_reader.h /^ inline const ColumnDataHeaderMessage& _header_message() {$/;" f class:doris::SegmentReader +_headers src/http/http_request.h /^ StringCaseUnorderedMap _headers;$/;" m class:doris::HttpRequest +_heap src/olap/reader.cpp /^ MergeHeap _heap;$/;" m class:doris::CollectIterator file: +_heap src/olap/schema_change.cpp /^ std::priority_queue _heap;$/;" m class:doris::RowBlockMerger file: +_heartbeat src/agent/heartbeat_server.cpp /^Status HeartbeatServer::_heartbeat(const TMasterInfo& master_info) {$/;" f class:doris::HeartbeatServer +_heartbeat_flags src/olap/storage_engine.h /^ HeartbeatFlags* _heartbeat_flags;$/;" m class:doris::StorageEngine +_heartbeat_flags src/runtime/exec_env.h /^ HeartbeatFlags* _heartbeat_flags = nullptr;$/;" m class:doris::ExecEnv +_helper test/olap/bit_field_test.cpp /^ FileHandler _helper;$/;" m class:doris::TestBitField file: +_helper_cursor src/olap/olap_index.h /^ RowCursor* _helper_cursor;$/;" m class:doris::IndexComparator +_helper_cursor src/olap/olap_index.h /^ RowCursor* _helper_cursor;$/;" m class:doris::SegmentComparator +_hex src/util/md5.h /^ std::string _hex;$/;" m class:doris::Md5Digest +_high_op src/exec/olap_common.h /^ SQLFilterOp _high_op;$/;" m class:doris::ColumnValueRange +_high_reader src/olap/rowset/column_reader.h /^ RunLengthIntegerReader* _high_reader;$/;" m class:doris::LargeIntColumnReader +_high_value src/exec/olap_common.h /^ T _high_value; \/\/ Column's high value, open interval at right$/;" m class:doris::ColumnValueRange +_high_writer src/olap/rowset/column_writer.h /^ RunLengthIntegerWriter* _high_writer;$/;" m class:doris::LargeIntColumnWriter +_hint src/olap/memtable.h /^ Table::Hint _hint;$/;" m class:doris::MemTable +_hit_count src/olap/lru_cache.h /^ uint64_t _hit_count; \/\/ 命中cache的总次数$/;" m class:doris::CachePriority::LRUCache +_hll_column_num src/exec/csv_scan_node.h /^ int _hll_column_num;$/;" m class:doris::CsvScanNode +_hll_last_row src/runtime/dpp_sink.cpp /^ std::vector _hll_last_row;$/;" m class:doris::HllDppSinkMerge file: +_hll_merge src/runtime/dpp_sink.cpp /^ HllDppSinkMerge _hll_merge;$/;" m class:doris::Translator file: +_hooks src/util/metrics.h /^ std::map> _hooks;$/;" m class:doris::MetricRegistry +_host src/http/ev_http_server.h /^ std::string _host;$/;" m class:doris::EvHttpServer +_host src/runtime/descriptors.h /^ std::string _host;$/;" m class:doris::MySQLTableDescriptor +_ht_ctx src/exec/partitioned_aggregation_node.h /^ boost::scoped_ptr _ht_ctx;$/;" m class:doris::PartitionedAggregationNode +_ht_resize_timer src/exec/partitioned_aggregation_node.h /^ RuntimeProfile::Counter* _ht_resize_timer;$/;" m class:doris::PartitionedAggregationNode +_http_server src/http/web_page_handler.h /^ EvHttpServer* _http_server;$/;" m class:doris::WebPageHandler +_http_timeout_ms src/exec/es/es_scan_reader.h /^ int _http_timeout_ms;$/;" m class:doris::ESScanReader +_hybird_map src/exprs/agg_fn_evaluator.h /^ boost::scoped_ptr _hybird_map;$/;" m class:doris::AggFnEvaluator +_hybird_set src/exprs/in_predicate.h /^ boost::shared_ptr _hybird_set;$/;" m class:doris::InPredicate +_id src/exec/exec_node.h /^ int _id; \/\/ unique w\/in single plan tree$/;" m class:doris::ExecNode +_id src/exec/olap_scanner.h /^ int _id;$/;" m class:doris::OlapScanner +_id src/olap/fs/block_id.h /^ uint64_t _id;$/;" m class:doris::BlockId +_id src/runtime/descriptors.h /^ TableId _id;$/;" m class:doris::TableDescriptor +_id src/runtime/descriptors.h /^ const SlotId _id;$/;" m class:doris::SlotDescriptor +_id src/runtime/descriptors.h /^ const TupleId _id;$/;" m class:doris::TupleDescriptor +_id src/runtime/dpp_sink_internal.h /^ int64_t _id;$/;" m class:doris::PartitionInfo +_id src/runtime/routine_load/data_consumer.h /^ UniqueId _id;$/;" m class:doris::DataConsumer +_id src/util/core_local.h /^ int _id = -1;$/;" m class:doris::CoreLocalValue +_id_generator src/olap/data_dir.h /^ RowsetIdGenerator* _id_generator = nullptr;$/;" m class:doris::DataDir +_id_mutex src/olap/lru_cache.h /^ Mutex _id_mutex;$/;" m class:doris::CachePriority::ShardedLRUCache +_id_writer src/olap/rowset/column_writer.h /^ RunLengthIntegerWriter* _id_writer;$/;" m class:doris::VarStringColumnWriter +_idle_cond src/util/threadpool.h /^ ConditionVariable _idle_cond;$/;" m class:doris::ThreadPool +_idle_threads src/util/threadpool.h /^ boost::intrusive::list _idle_threads; \/\/ NOLINT(build\/include_what_you_use)$/;" m class:doris::ThreadPool +_idle_timeout src/util/threadpool.h /^ MonoDelta _idle_timeout;$/;" m class:doris::ThreadPoolBuilder +_idle_timeout src/util/threadpool.h /^ const MonoDelta _idle_timeout;$/;" m class:doris::ThreadPool +_idx src/olap/column_block.h /^ size_t _idx;$/;" m struct:doris::ColumnBlockCell +_idx src/runtime/load_path_mgr.h /^ int _idx;$/;" m class:doris::LoadPathMgr +_iface src/util/thrift_client.h /^ boost::shared_ptr _iface;$/;" m class:doris::ThriftClient +_ignore_not_found src/runtime/data_stream_sender.h /^ bool _ignore_not_found;$/;" m class:doris::DataStreamSender +_ignore_rows src/runtime/data_stream_sender.h /^ RuntimeProfile::Counter* _ignore_rows;$/;" m class:doris::DataStreamSender +_ignored src/olap/stream_index_common.h /^ bool _ignored;$/;" m class:doris::ColumnStatistics +_impl output/udf/include/udf.h /^ doris::FunctionContextImpl* _impl; \/\/ Owned by this object.$/;" m class:doris_udf::FunctionContext +_impl src/udf/udf.h /^ doris::FunctionContextImpl* _impl; \/\/ Owned by this object.$/;" m class:doris_udf::FunctionContext +_import_label src/runtime/runtime_state.h /^ std::string _import_label;$/;" m class:doris::RuntimeState +_in_flight_ranges src/runtime/disk_io_mgr_internal.h /^ InternalQueue _in_flight_ranges;$/;" m class:doris::DiskIoMgr::RequestContext::PerDiskState +_in_mem_sort_timer src/runtime/merge_sorter.h /^ RuntimeProfile::Counter* _in_mem_sort_timer;$/;" m class:doris::MergeSorter +_in_mem_sort_timer src/runtime/spill_sorter.h /^ RuntimeProfile::Counter* _in_mem_sort_timer;$/;" m class:doris::SpillSorter +_in_mem_tuple_sorter src/runtime/merge_sorter.h /^ boost::scoped_ptr _in_mem_tuple_sorter;$/;" m class:doris::MergeSorter +_in_mem_tuple_sorter src/runtime/spill_sorter.h /^ boost::scoped_ptr _in_mem_tuple_sorter;$/;" m class:doris::SpillSorter +_in_node test/exprs/in_predicate_test.cpp /^ TExprNode _in_node;$/;" m class:doris::InPredicateTest file: +_in_restore_mode src/olap/tablet_meta.h /^ bool _in_restore_mode = false;$/;" m class:doris::TabletMeta +_in_write src/runtime/buffered_block_mgr2.h /^ bool _in_write;$/;" m class:doris::BufferedBlockMgr2::Block +_inactive_contexts src/runtime/disk_io_mgr.cc /^ list _inactive_contexts;$/;" m class:doris::DiskIoMgr::RequestContextCache file: +_inc_id src/olap/rowset/unique_rowset_id_generator.h /^ int64_t _inc_id = 0;$/;" m class:doris::UniqueRowsetIdGenerator +_inc_rs_metas src/olap/tablet_meta.h /^ std::vector _inc_rs_metas;$/;" m class:doris::TabletMeta +_inc_rs_version_map src/olap/tablet.h /^ std::unordered_map _inc_rs_version_map;$/;" m class:doris::Tablet +_include_bf_columns src/olap/rowset/segment_reader.h /^ UniqueIdSet _include_bf_columns;$/;" m class:doris::SegmentReader +_include_blocks src/olap/rowset/segment_reader.h /^ uint8_t* _include_blocks;$/;" m class:doris::SegmentReader +_include_columns src/olap/rowset/segment_reader.h /^ UniqueIdSet _include_columns; \/\/ 用于判断该列是不是被包含$/;" m class:doris::SegmentReader +_include_end_key src/runtime/dpp_sink_internal.h /^ bool _include_end_key;$/;" m class:doris::PartRange +_include_start_key src/runtime/dpp_sink_internal.h /^ bool _include_start_key;$/;" m class:doris::PartRange +_index src/exec/es/es_scan_reader.h /^ std::string _index;$/;" m class:doris::ESScanReader +_index src/exec/schema_scanner/schema_charsets_scanner.h /^ int _index;$/;" m class:doris::SchemaCharsetsScanner +_index src/exec/schema_scanner/schema_collations_scanner.h /^ int _index;$/;" m class:doris::SchemaCollationsScanner +_index src/exec/schema_scanner/schema_variables_scanner.h /^ int _index;$/;" m class:doris::SchemaVariablesScanner +_index src/olap/olap_index.h /^ const MemIndex* _index;$/;" m class:doris::IndexComparator +_index src/olap/olap_index.h /^ const MemIndex* _index;$/;" m class:doris::SegmentComparator +_index src/olap/rowset/column_writer.h /^ StreamIndexWriter _index;$/;" m class:doris::ColumnWriter +_index src/olap/rowset/segment_group.h /^ MemIndex _index;$/;" m class:doris::SegmentGroup +_index src/olap/rowset/segment_v2/ordinal_page_index.h /^ OrdinalIndexReader* _index;$/;" m class:doris::segment_v2::OrdinalPageIndexIterator +_index src/olap/stream_index_reader.h /^ int _index;$/;" m class:doris::PositionProvider +_index src/runtime/merge_sorter.cpp /^ int64_t _index;$/;" m class:doris::MergeSorter::TupleSorter::TupleIterator file: +_index src/runtime/spill_sorter.cc /^ int64_t _index;$/;" m class:doris::SpillSorter::TupleSorter::TupleIterator file: +_index_builder src/olap/rowset/segment_v2/segment_writer.h /^ std::unique_ptr _index_builder;$/;" m class:doris::segment_v2::SegmentWriter +_index_entry src/olap/rowset/column_writer.h /^ PositionEntryWriter _index_entry;$/;" m class:doris::ColumnWriter +_index_id src/exec/tablet_sink.h /^ int64_t _index_id = -1;$/;" m class:doris::stream_load::NodeChannel +_index_id src/exec/tablet_sink.h /^ int64_t _index_id;$/;" m class:doris::stream_load::IndexChannel +_index_id src/runtime/tablets_channel.h /^ int64_t _index_id = -1;$/;" m class:doris::TabletsChannel +_index_in_block src/olap/generic_iterators.cpp /^ size_t _index_in_block = -1;$/;" m class:doris::MergeIteratorContext file: +_index_length src/olap/tablet_schema.h /^ int32_t _index_length;$/;" m class:doris::TabletColumn +_index_load_lock src/olap/rowset/segment_group.h /^ mutable boost::mutex _index_load_lock;$/;" m class:doris::SegmentGroup +_index_load_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _index_load_timer = nullptr;$/;" m class:doris::OlapScanNode +_index_loaded src/olap/rowset/segment_group.h /^ bool _index_loaded; \/\/ whether the segmentgroup has been read$/;" m class:doris::SegmentGroup +_index_map src/exec/schema_scan_node.h /^ std::vector _index_map;$/;" m class:doris::SchemaScanNode +_index_meta src/olap/rowset/segment_v2/ordinal_page_index.h /^ const OrdinalIndexPB* _index_meta;$/;" m class:doris::segment_v2::OrdinalIndexReader +_index_meta src/olap/rowset/segment_v2/zone_map_index.h /^ const ZoneMapIndexPB* _index_meta;$/;" m class:doris::segment_v2::ZoneMapIndexReader +_index_page_size src/olap/rowset/segment_v2/index_page.h /^ const size_t _index_page_size;$/;" m class:doris::segment_v2::IndexPageBuilder +_index_size src/olap/field.h /^ uint16_t _index_size;$/;" m class:doris::Field +_index_size src/olap/olap_index.h /^ size_t _index_size;$/;" m class:doris::MemIndex +_index_stream src/olap/rowset/column_writer.h /^ OutStream* _index_stream; \/\/ 注意对象的所有权是_stream_factory$/;" m class:doris::ColumnWriter +_index_stream_lru_cache src/olap/storage_engine.h /^ Cache* _index_stream_lru_cache;$/;" m class:doris::StorageEngine +_index_to_write src/olap/stream_index_writer.h /^ std::vector _index_to_write;$/;" m class:doris::StreamIndexWriter +_indexes src/exec/tablet_info.h /^ std::vector _indexes;$/;" m class:doris::OlapTableSchemaParam +_indices src/olap/rowset/segment_reader.h /^ std::map _indices;$/;" m class:doris::SegmentReader +_info src/olap/row_block.h /^ RowBlockInfo _info;$/;" m class:doris::RowBlock +_info src/util/broker_load_error_hub.h /^ BrokerInfo _info;$/;" m class:doris::BrokerLoadErrorHub +_info src/util/mysql_load_error_hub.h /^ MysqlInfo _info;$/;" m class:doris::MysqlLoadErrorHub +_info_strings src/util/runtime_profile.h /^ InfoStrings _info_strings;$/;" m class:doris::RuntimeProfile +_info_strings_display_order src/util/runtime_profile.h /^ InfoStringsDisplayOrder _info_strings_display_order;$/;" m class:doris::RuntimeProfile +_info_strings_lock src/util/runtime_profile.h /^ mutable boost::mutex _info_strings_lock;$/;" m class:doris::RuntimeProfile +_infos_mapping src/olap/aggregate_func.cpp /^ std::unordered_map _infos_mapping;$/;" m class:doris::AggregateFuncResolver file: +_ingest_lock src/olap/tablet.h /^ Mutex _ingest_lock;$/;" m class:doris::Tablet +_init src/olap/row_cursor.cpp /^OLAPStatus RowCursor::_init(const std::vector& schema,$/;" f class:doris::RowCursor +_init src/olap/rowset/alpha_rowset_writer.cpp /^OLAPStatus AlphaRowsetWriter::_init() {$/;" f class:doris::AlphaRowsetWriter +_init src/olap/rowset/rowset_meta.h /^ void _init() {$/;" f class:doris::RowsetMeta +_init src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_init() {$/;" f class:doris::segment_v2::SegmentIterator +_init src/olap/schema.cpp /^void Schema::_init(const std::vector& cols,$/;" f class:doris::Schema +_init src/olap/schema.cpp /^void Schema::_init(const std::vector& cols,$/;" f class:doris::Schema +_init src/olap/task/engine_batch_load_task.cpp /^AgentStatus EngineBatchLoadTask::_init() {$/;" f class:doris::EngineBatchLoadTask +_init src/runtime/exec_env_init.cpp /^Status ExecEnv::_init(const std::vector& store_paths) {$/;" f class:doris::ExecEnv +_init src/runtime/routine_load/data_consumer.h /^ bool _init;$/;" m class:doris::DataConsumer +_init_bitmap_index_iterators src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_init_bitmap_index_iterators() {$/;" f class:doris::segment_v2::SegmentIterator +_init_buffer_pool src/runtime/exec_env_init.cpp /^void ExecEnv::_init_buffer_pool(int64_t min_page_size,$/;" f class:doris::ExecEnv +_init_capacity src/olap/data_dir.cpp /^Status DataDir::_init_capacity() {$/;" f class:doris::DataDir +_init_cluster_id src/olap/data_dir.cpp /^Status DataDir::_init_cluster_id() {$/;" f class:doris::DataDir +_init_column_mapping src/olap/schema_change.cpp /^OLAPStatus SchemaChangeHandler::_init_column_mapping(ColumnMapping* column_mapping,$/;" f class:doris::SchemaChangeHandler +_init_conditions_param src/olap/reader.cpp /^OLAPStatus Reader::_init_conditions_param(const ReaderParams& read_params) {$/;" f class:doris::Reader +_init_counter src/exec/olap_scan_node.cpp /^void OlapScanNode::_init_counter(RuntimeState* state) {$/;" f class:doris::OlapScanNode +_init_delete_condition src/olap/reader.cpp /^OLAPStatus Reader::_init_delete_condition(const ReaderParams& read_params) {$/;" f class:doris::Reader +_init_fake_numa_for_test src/util/cpu_info.cpp /^void CpuInfo::_init_fake_numa_for_test($/;" f class:doris::CpuInfo +_init_file_system src/olap/data_dir.cpp /^Status DataDir::_init_file_system() {$/;" f class:doris::DataDir +_init_fn output/udf/include/uda_test_harness.h /^ InitFn _init_fn;$/;" m class:doris_udf::UdaTestHarnessBase +_init_fn src/exprs/agg_fn_evaluator.h /^ void* _init_fn;$/;" m class:doris::AggFnEvaluator +_init_fn src/olap/aggregate_func.h /^ void (*_init_fn)(RowCursorCell* dst, const char* src, bool src_null, MemPool* mem_pool, ObjectPool* agg_pool);$/;" m class:doris::AggregateInfo +_init_fn src/udf/uda_test_harness.h /^ InitFn _init_fn;$/;" m class:doris_udf::UdaTestHarnessBase +_init_fn_symbol src/exprs/agg_fn_evaluator.h /^ std::string _init_fn_symbol;$/;" m class:doris::AggFnEvaluator +_init_include_blocks src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_init_include_blocks(uint32_t first_block, uint32_t last_block) {$/;" f class:doris::SegmentReader +_init_keys_param src/olap/reader.cpp /^OLAPStatus Reader::_init_keys_param(const ReaderParams& read_params) {$/;" f class:doris::Reader +_init_lazy_materialization src/olap/rowset/segment_v2/segment_iterator.cpp /^void SegmentIterator::_init_lazy_materialization() {$/;" f class:doris::segment_v2::SegmentIterator +_init_literals src/olap/rowset/run_length_integer_writer.cpp /^void RunLengthIntegerWriter::_init_literals(int64_t value) {$/;" f class:doris::RunLengthIntegerWriter +_init_load_bf_columns src/olap/reader.cpp /^OLAPStatus Reader::_init_load_bf_columns(const ReaderParams& read_params) {$/;" f class:doris::Reader +_init_mem_tracker src/runtime/exec_env_init.cpp /^Status ExecEnv::_init_mem_tracker() {$/;" f class:doris::ExecEnv +_init_merge_ctxs src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::_init_merge_ctxs(RowsetReaderContext* read_context) {$/;" f class:doris::AlphaRowsetReader +_init_merge_heap src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::_init_merge_heap() {$/;" f class:doris::AlphaRowsetReader +_init_meta src/olap/data_dir.cpp /^Status DataDir::_init_meta() {$/;" f class:doris::DataDir +_init_numa src/util/cpu_info.cpp /^void CpuInfo::_init_numa() {$/;" f class:doris::CpuInfo +_init_numa_node_to_cores src/util/cpu_info.cpp /^void CpuInfo::_init_numa_node_to_cores() {$/;" f class:doris::CpuInfo +_init_once src/olap/tablet.h /^ DorisCallOnce _init_once;$/;" m class:doris::Tablet +_init_once_action src/olap/tablet.cpp /^OLAPStatus Tablet::_init_once_action() {$/;" f class:doris::Tablet +_init_params src/exec/olap_scanner.cpp /^Status OlapScanner::_init_params($/;" f class:doris::OlapScanner +_init_params src/olap/reader.cpp /^OLAPStatus Reader::_init_params(const ReaderParams& read_params) {$/;" f class:doris::Reader +_init_return_column_iterators src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_init_return_column_iterators() {$/;" f class:doris::segment_v2::SegmentIterator +_init_return_columns src/exec/olap_scanner.cpp /^Status OlapScanner::_init_return_columns() {$/;" f class:doris::OlapScanner +_init_return_columns src/olap/reader.cpp /^OLAPStatus Reader::_init_return_columns(const ReaderParams& read_params) {$/;" f class:doris::Reader +_init_scroll_url src/exec/es/es_scan_reader.h /^ std::string _init_scroll_url;$/;" m class:doris::ESScanReader +_init_seek_columns src/olap/reader.cpp /^OLAPStatus Reader::_init_seek_columns() {$/;" f class:doris::Reader +_init_segment src/olap/rowset/column_data_writer.cpp /^OLAPStatus ColumnDataWriter::_init_segment() {$/;" f class:doris::ColumnDataWriter +_inited src/olap/rowset/segment_v2/segment_iterator.h /^ bool _inited;$/;" m class:doris::segment_v2::SegmentIterator +_initial_count src/util/barrier.h /^ const int _initial_count;$/;" m class:doris::Barrier +_initial_queue_capacity src/runtime/disk_io_mgr_internal.h /^ int _initial_queue_capacity;$/;" m class:doris::DiskIoMgr::RequestContext +_initial_reservation_refcnt src/runtime/runtime_state.h /^ AtomicInt32 _initial_reservation_refcnt;$/;" m class:doris::RuntimeState +_initial_reservations src/runtime/runtime_state.h /^ InitialReservations* _initial_reservations = nullptr;$/;" m class:doris::RuntimeState +_initial_runs_counter src/runtime/merge_sorter.h /^ RuntimeProfile::Counter* _initial_runs_counter;$/;" m class:doris::MergeSorter +_initial_runs_counter src/runtime/spill_sorter.h /^ RuntimeProfile::Counter* _initial_runs_counter;$/;" m class:doris::SpillSorter +_initial_seed src/exec/hash_table.h /^ const int32_t _initial_seed;$/;" m class:doris::HashTable +_initialized src/runtime/buffered_block_mgr2.h /^ bool _initialized;$/;" m class:doris::BufferedBlockMgr2 +_initialized src/runtime/tmp_file_mgr.h /^ bool _initialized;$/;" m class:doris::TmpFileMgr +_inner_hits_node src/exec/es/es_scroll_parser.h /^ rapidjson::Value _inner_hits_node;$/;" m class:doris::ScrollParser +_input output/udf/include/uda_test_harness.h /^ std::vector _input;$/;" m class:doris_udf::UdaTestHarness +_input src/olap/rowset/bit_field_reader.h /^ ReadOnlyFileStream* _input;$/;" m class:doris::BitFieldReader +_input src/olap/rowset/run_length_byte_reader.h /^ ReadOnlyFileStream* _input;$/;" m class:doris::RunLengthByteReader +_input src/olap/rowset/run_length_integer_reader.h /^ ReadOnlyFileStream* _input;$/;" m class:doris::RunLengthIntegerReader +_input src/udf/uda_test_harness.h /^ std::vector _input;$/;" m class:doris_udf::UdaTestHarness +_input test/util/decompress_test.cpp /^ uint8_t _input[2 * 26 * 1024];$/;" m class:doris::DecompressorTest file: +_input1 output/udf/include/uda_test_harness.h /^ const std::vector* _input1;$/;" m class:doris_udf::UdaTestHarness2 +_input1 output/udf/include/uda_test_harness.h /^ const std::vector* _input1;$/;" m class:doris_udf::UdaTestHarness3 +_input1 output/udf/include/uda_test_harness.h /^ const std::vector* _input1;$/;" m class:doris_udf::UdaTestHarness4 +_input1 src/udf/uda_test_harness.h /^ const std::vector* _input1;$/;" m class:doris_udf::UdaTestHarness2 +_input1 src/udf/uda_test_harness.h /^ const std::vector* _input1;$/;" m class:doris_udf::UdaTestHarness3 +_input1 src/udf/uda_test_harness.h /^ const std::vector* _input1;$/;" m class:doris_udf::UdaTestHarness4 +_input2 output/udf/include/uda_test_harness.h /^ const std::vector* _input2;$/;" m class:doris_udf::UdaTestHarness2 +_input2 output/udf/include/uda_test_harness.h /^ const std::vector* _input2;$/;" m class:doris_udf::UdaTestHarness3 +_input2 output/udf/include/uda_test_harness.h /^ const std::vector* _input2;$/;" m class:doris_udf::UdaTestHarness4 +_input2 src/udf/uda_test_harness.h /^ const std::vector* _input2;$/;" m class:doris_udf::UdaTestHarness2 +_input2 src/udf/uda_test_harness.h /^ const std::vector* _input2;$/;" m class:doris_udf::UdaTestHarness3 +_input2 src/udf/uda_test_harness.h /^ const std::vector* _input2;$/;" m class:doris_udf::UdaTestHarness4 +_input3 output/udf/include/uda_test_harness.h /^ const std::vector* _input3;$/;" m class:doris_udf::UdaTestHarness3 +_input3 output/udf/include/uda_test_harness.h /^ const std::vector* _input3;$/;" m class:doris_udf::UdaTestHarness4 +_input3 src/udf/uda_test_harness.h /^ const std::vector* _input3;$/;" m class:doris_udf::UdaTestHarness3 +_input3 src/udf/uda_test_harness.h /^ const std::vector* _input3;$/;" m class:doris_udf::UdaTestHarness4 +_input4 output/udf/include/uda_test_harness.h /^ const std::vector* _input4;$/;" m class:doris_udf::UdaTestHarness4 +_input4 src/udf/uda_test_harness.h /^ const std::vector* _input4;$/;" m class:doris_udf::UdaTestHarness4 +_input_batch src/exec/exchange_node.h /^ RowBatch* _input_batch = nullptr;$/;" m class:doris::ExchangeNode +_input_block src/olap/rowset/beta_rowset_reader.h /^ std::unique_ptr _input_block;$/;" m class:doris::BetaRowsetReader +_input_buf src/exec/plain_text_line_reader.h /^ uint8_t* _input_buf;$/;" m class:doris::PlainTextLineReader +_input_buf_limit src/exec/plain_text_line_reader.h /^ size_t _input_buf_limit;$/;" m class:doris::PlainTextLineReader +_input_buf_pos src/exec/plain_text_line_reader.h /^ size_t _input_buf_pos;$/;" m class:doris::PlainTextLineReader +_input_buf_size src/exec/plain_text_line_reader.h /^ size_t _input_buf_size;$/;" m class:doris::PlainTextLineReader +_input_eos src/exec/analytic_eval_node.h /^ bool _input_eos;$/;" m class:doris::AnalyticEvalNode +_input_exprs_ctxs src/exprs/agg_fn_evaluator.h /^ std::vector _input_exprs_ctxs;$/;" m class:doris::AggFnEvaluator +_input_record_num src/exec/pre_aggregation_node.h /^ int _input_record_num;$/;" m class:doris::PreAggregationNode +_input_record_num_sum src/exec/pre_aggregation_node.h /^ int _input_record_num_sum;$/;" m class:doris::PreAggregationNode +_input_row_batch src/runtime/sorted_run_merger.cc /^ RowBatch* _input_row_batch;$/;" m class:doris::SortedRunMerger::BatchedRowSupplier file: +_input_row_batch_index src/runtime/sorted_run_merger.cc /^ int _input_row_batch_index;$/;" m class:doris::SortedRunMerger::BatchedRowSupplier file: +_input_row_desc src/exec/exchange_node.h /^ RowDescriptor _input_row_desc;$/;" m class:doris::ExchangeNode +_input_row_desc src/exec/tablet_sink.h /^ const RowDescriptor& _input_row_desc;$/;" m class:doris::stream_load::OlapTableSink +_input_row_desc src/runtime/sorted_run_merger.h /^ RowDescriptor* _input_row_desc;$/;" m class:doris::SortedRunMerger +_input_row_num src/olap/compaction.h /^ int64_t _input_row_num;$/;" m class:doris::Compaction +_input_rows_counter src/exec/tablet_sink.h /^ RuntimeProfile::Counter* _input_rows_counter = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_input_rowsets src/olap/compaction.h /^ std::vector _input_rowsets;$/;" m class:doris::Compaction +_input_rowsets_size src/olap/compaction.h /^ int64_t _input_rowsets_size;$/;" m class:doris::Compaction +_input_rs_readers src/olap/compaction.h /^ std::vector _input_rs_readers;$/;" m class:doris::Compaction +_input_stream src/exec/analytic_eval_node.h /^ boost::scoped_ptr _input_stream;$/;" m class:doris::AnalyticEvalNode +_inputs src/olap/in_stream.h /^ std::vector _inputs;$/;" m class:doris::InStream +_insert_txn_partition_map_unlocked src/olap/txn_manager.cpp /^void TxnManager::_insert_txn_partition_map_unlocked(int64_t transaction_id, int64_t partition_id) {$/;" f class:doris::TxnManager +_install_cpu_metrics src/util/system_metrics.cpp /^void SystemMetrics::_install_cpu_metrics(MetricRegistry* registry) {$/;" f class:doris::SystemMetrics +_install_disk_metrics src/util/system_metrics.cpp /^void SystemMetrics::_install_disk_metrics(MetricRegistry* registry,$/;" f class:doris::SystemMetrics +_install_fd_metrics src/util/system_metrics.cpp /^void SystemMetrics::_install_fd_metrics(MetricRegistry* registry) {$/;" f class:doris::SystemMetrics +_install_memory_metrics src/util/system_metrics.cpp /^void SystemMetrics::_install_memory_metrics(MetricRegistry* registry) {$/;" f class:doris::SystemMetrics +_install_net_metrics src/util/system_metrics.cpp /^void SystemMetrics::_install_net_metrics(MetricRegistry* registry,$/;" f class:doris::SystemMetrics +_instance_buffer_reservation src/runtime/runtime_state.h /^ boost::scoped_ptr _instance_buffer_reservation;$/;" m class:doris::RuntimeState +_instance_mem_tracker src/runtime/runtime_state.h /^ boost::scoped_ptr _instance_mem_tracker;$/;" m class:doris::RuntimeState +_int_desc test/runtime/buffered_tuple_stream2_test.cpp /^ RowDescriptor* _int_desc;$/;" m class:doris::SimpleTupleStreamTest file: +_int_reader src/olap/rowset/column_reader.h /^ RunLengthIntegerReader* _int_reader;$/;" m class:doris::DecimalColumnReader +_int_value src/exprs/info_func.h /^ int64_t _int_value;$/;" m class:doris::InfoFunc +_int_writer src/olap/rowset/column_writer.h /^ RunLengthIntegerWriter* _int_writer;$/;" m class:doris::DecimalColumnWriter +_integrity_check_timer src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _integrity_check_timer;$/;" m class:doris::BufferedBlockMgr2 +_intermediate_row_desc src/exec/partitioned_aggregation_node.h /^ boost::scoped_ptr _intermediate_row_desc;$/;" m class:doris::PartitionedAggregationNode +_intermediate_slot_desc src/exprs/agg_fn_evaluator.h /^ const SlotDescriptor* _intermediate_slot_desc;$/;" m class:doris::AggFnEvaluator +_intermediate_tuple_desc src/exec/aggregation_node.h /^ TupleDescriptor* _intermediate_tuple_desc;$/;" m class:doris::AggregationNode +_intermediate_tuple_desc src/exec/analytic_eval_node.h /^ const TupleDescriptor* _intermediate_tuple_desc;$/;" m class:doris::AnalyticEvalNode +_intermediate_tuple_desc src/exec/partitioned_aggregation_node.h /^ TupleDescriptor* _intermediate_tuple_desc;$/;" m class:doris::PartitionedAggregationNode +_intermediate_tuple_id src/exec/aggregation_node.h /^ TupleId _intermediate_tuple_id;$/;" m class:doris::AggregationNode +_intermediate_tuple_id src/exec/partitioned_aggregation_node.h /^ TupleId _intermediate_tuple_id;$/;" m class:doris::PartitionedAggregationNode +_intermediate_type src/exprs/agg_fn_evaluator.h /^ const TypeDescriptor _intermediate_type;$/;" m class:doris::AggFnEvaluator +_intermediate_type src/udf/udf_internal.h /^ doris_udf::FunctionContext::TypeDesc _intermediate_type;$/;" m class:doris::FunctionContextImpl +_internal_create_tablet_unlocked src/olap/tablet_manager.cpp /^TabletSharedPtr TabletManager::_internal_create_tablet_unlocked($/;" f class:doris::TabletManager +_internal_sorting src/olap/schema_change.cpp /^bool SchemaChangeWithSorting::_internal_sorting(const vector& row_block_arr,$/;" f class:doris::SchemaChangeWithSorting +_involuntary_context_switches src/util/runtime_profile.h /^ Counter* _involuntary_context_switches;$/;" m class:doris::RuntimeProfile::ThreadCounters +_io_buffers src/runtime/row_batch.h /^ std::vector _io_buffers;$/;" m class:doris::RowBatch +_io_mgr src/runtime/buffered_block_mgr2.h /^ DiskIoMgr* _io_mgr;$/;" m class:doris::BufferedBlockMgr2 +_io_mgr src/runtime/disk_io_mgr.cc /^ DiskIoMgr* _io_mgr;$/;" m class:doris::DiskIoMgr::RequestContextCache file: +_io_mgr src/runtime/disk_io_mgr.h /^ DiskIoMgr* _io_mgr;$/;" m class:doris::DiskIoMgr::BufferDescriptor +_io_mgr src/runtime/disk_io_mgr.h /^ DiskIoMgr* _io_mgr;$/;" m class:doris::DiskIoMgr::ScanRange +_io_mgr_tracker src/runtime/test_env.h /^ boost::scoped_ptr _io_mgr_tracker;$/;" m class:doris::TestEnv +_io_request_context src/runtime/buffered_block_mgr2.h /^ DiskIoMgr::RequestContext* _io_request_context;$/;" m class:doris::BufferedBlockMgr2 +_io_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _io_timer = nullptr;$/;" m class:doris::OlapScanNode +_ipaddress src/util/thrift_client.h /^ std::string _ipaddress;$/;" m class:doris::ThriftClientImpl +_is_all_cluster_id_exist src/olap/storage_engine.h /^ bool _is_all_cluster_id_exist;$/;" m class:doris::StorageEngine +_is_analytic_fn src/exprs/agg_fn_evaluator.h /^ const bool _is_analytic_fn;$/;" m class:doris::AggFnEvaluator +_is_asc src/util/tuple_row_compare.h /^ std::vector _is_asc;$/;" m class:doris::TupleRowComparator +_is_asc_order src/exec/exchange_node.h /^ std::vector _is_asc_order;$/;" m class:doris::ExchangeNode +_is_asc_order src/exec/sort_node.h /^ std::vector _is_asc_order;$/;" m class:doris::SortNode +_is_asc_order src/exec/spill_sort_node.h /^ std::vector _is_asc_order;$/;" m class:doris::SpillSortNode +_is_asc_order src/exec/topn_node.h /^ std::vector _is_asc_order;$/;" m class:doris::TopNNode +_is_asc_order test/runtime/buffered_tuple_stream_test.cpp /^ std::vector _is_asc_order;$/;" m class:doris::BufferedTupleStreamTest file: +_is_asc_order test/runtime/sorter_test.cpp /^ std::vector _is_asc_order;$/;" m class:doris::SorterTest file: +_is_averaged_profile src/util/runtime_profile.h /^ bool _is_averaged_profile;$/;" m class:doris::RuntimeProfile +_is_bad src/olap/tablet.h /^ std::atomic _is_bad;$/;" m class:doris::Tablet +_is_bf_column src/olap/tablet_schema.h /^ bool _is_bf_column;$/;" m class:doris::TabletColumn +_is_bf_column_included src/olap/rowset/segment_reader.h /^ inline bool _is_bf_column_included(ColumnId column_unique_id) {$/;" f class:doris::SegmentReader +_is_bfd_inited src/util/bfd_parser.cpp /^bool BfdParser::_is_bfd_inited = false;$/;" m class:doris::BfdParser file: +_is_bfd_inited src/util/bfd_parser.h /^ static bool _is_bfd_inited;$/;" m class:doris::BfdParser +_is_cancelled src/runtime/buffer_control_block.h /^ bool _is_cancelled;$/;" m class:doris::BufferControlBlock +_is_cancelled src/runtime/buffered_block_mgr2.h /^ bool _is_cancelled;$/;" m class:doris::BufferedBlockMgr2 +_is_cancelled src/runtime/data_stream_recvr.cc /^ bool _is_cancelled;$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_is_cancelled src/runtime/disk_io_mgr.h /^ bool _is_cancelled;$/;" m class:doris::DiskIoMgr::ScanRange +_is_cancelled src/runtime/runtime_state.h /^ bool _is_cancelled;$/;" m class:doris::RuntimeState +_is_cgroups_init_success src/agent/cgroups_mgr.h /^ bool _is_cgroups_init_success;$/;" m class:doris::CgroupsMgr +_is_clone src/exprs/expr_context.h /^ bool _is_clone;$/;" m class:doris::ExprContext +_is_close src/runtime/buffer_control_block.h /^ bool _is_close;$/;" m class:doris::BufferControlBlock +_is_closed src/exec/broker_writer.h /^ bool _is_closed;$/;" m class:doris::BrokerWriter +_is_closed src/exec/exec_node.h /^ bool _is_closed;$/;" m class:doris::ExecNode +_is_closed src/exec/olap_scanner.h /^ bool _is_closed = false;$/;" m class:doris::OlapScanner +_is_column_included src/olap/rowset/segment_reader.h /^ inline bool _is_column_included(ColumnId column_unique_id) {$/;" f class:doris::SegmentReader +_is_convertible src/exec/olap_common.h /^ bool _is_convertible;$/;" m class:doris::OlapScanKeys +_is_cumulative src/olap/rowset/rowset.h /^ bool _is_cumulative; \/\/ rowset is cumulative iff it's visible and start version < end version$/;" m class:doris::Rowset +_is_data_loaded src/olap/rowset/segment_reader.h /^ bool _is_data_loaded;$/;" m class:doris::SegmentReader +_is_data_owner src/olap/rowset/segment_v2/page_handle.h /^ bool _is_data_owner = false;$/;" m class:doris::segment_v2::PageHandle +_is_decimal src/olap/tablet_schema.h /^ bool _is_decimal;$/;" m class:doris::TabletColumn +_is_default_value_null src/olap/rowset/segment_v2/column_reader.h /^ bool _is_default_value_null;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_is_deflate src/exec/decompressor.h /^ bool _is_deflate;$/;" m class:doris::GzipDecompressor +_is_delete src/olap/reader.cpp /^ bool _is_delete = false;$/;" m class:doris::CollectIterator::ChildCtx file: +_is_deleted src/runtime/buffered_block_mgr2.h /^ bool _is_deleted;$/;" m class:doris::BufferedBlockMgr2::Block +_is_fd_valid src/exec/broker_reader.h /^ bool _is_fd_valid;$/;" m class:doris::BrokerReader +_is_first src/exec/es/es_scan_reader.h /^ bool _is_first;$/;" m class:doris::ESScanReader +_is_fixed_delta src/olap/rowset/run_length_integer_writer.h /^ bool _is_fixed_delta;$/;" m class:doris::RunLengthIntegerWriter +_is_found_nulls src/olap/rowset/column_writer.h /^ bool _is_found_nulls;$/;" m class:doris::ColumnWriter +_is_header_loaded src/exec/decompressor.h /^ bool _is_header_loaded;$/;" m class:doris::LzopDecompressor +_is_in_memory src/olap/tablet_schema.h /^ bool _is_in_memory = false;$/;" m class:doris::TabletSchema +_is_init src/exec/csv_scan_node.h /^ bool _is_init;$/;" m class:doris::CsvScanNode +_is_init src/exec/mysql_scan_node.h /^ bool _is_init;$/;" m class:doris::MysqlScanNode +_is_init src/exec/pre_aggregation_node.h /^ bool _is_init;$/;" m class:doris::PreAggregationNode +_is_init src/exec/schema_scan_node.h /^ bool _is_init;$/;" m class:doris::SchemaScanNode +_is_init src/exec/schema_scanner.h /^ bool _is_init;$/;" m class:doris::SchemaScanner +_is_init src/olap/delta_writer.h /^ bool _is_init = false;$/;" m class:doris::DeltaWriter +_is_init src/olap/task/engine_batch_load_task.h /^ bool _is_init = false;$/;" m class:doris::EngineBatchLoadTask +_is_inited src/olap/delete_handler.h /^ bool _is_inited;$/;" m class:doris::DeleteHandler +_is_int128 src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ bool _is_int128() const {$/;" f class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl file: +_is_key src/olap/olap_cond.h /^ bool _is_key;$/;" m class:doris::CondColumn +_is_key src/olap/tablet_schema.h /^ bool _is_key;$/;" m class:doris::TabletColumn +_is_lead_fn src/exec/analytic_eval_node.h /^ std::vector _is_lead_fn;$/;" m class:doris::AnalyticEvalNode +_is_leaf src/olap/rowset/segment_v2/index_page.h /^ const bool _is_leaf;$/;" m class:doris::segment_v2::IndexPageBuilder +_is_lower_keys_included src/olap/reader.h /^ std::vector _is_lower_keys_included;$/;" m class:doris::Reader +_is_materialized src/runtime/descriptors.h /^ const bool _is_materialized;$/;" m class:doris::SlotDescriptor +_is_merge src/exprs/agg_fn_evaluator.h /^ const bool _is_merge;$/;" m class:doris::AggFnEvaluator +_is_merging src/exec/exchange_node.h /^ bool _is_merging;$/;" m class:doris::ExchangeNode +_is_merging src/runtime/data_stream_recvr.h /^ bool _is_merging;$/;" m class:doris::DataStreamRecvr +_is_mmap src/olap/byte_buffer.h /^ bool _is_mmap; \/\/ 是否使用mmap$/;" m class:doris::StorageByteBuffer::BufDeleter +_is_mmap src/olap/byte_buffer.h /^ bool _is_mmap;$/;" m class:doris::StorageByteBuffer +_is_multi_distinct src/exprs/agg_fn_evaluator.h /^ bool _is_multi_distinct;$/;" m class:doris::AggFnEvaluator +_is_multi_distinct src/exprs/new_agg_fn_evaluator.h /^ bool _is_multi_distinct;$/;" m class:doris::NewAggFnEvaluator +_is_normal_read src/olap/rowset/column_data.h /^ bool _is_normal_read = false;$/;" m class:doris::ColumnData +_is_not_in src/exprs/in_predicate.h /^ const bool _is_not_in;$/;" m class:doris::InPredicate +_is_null src/olap/null_predicate.h /^ bool _is_null; \/\/true for null, false for not null$/;" m class:doris::NullPredicate +_is_null src/olap/rowset/column_reader.h /^ bool* _is_null;$/;" m class:doris::ColumnReader +_is_null src/runtime/vectorized_row_batch.h /^ bool* _is_null = nullptr;$/;" m class:doris::ColumnVector +_is_null_safe_eq_join src/exec/hash_join_node.h /^ std::vector _is_null_safe_eq_join;$/;" m class:doris::HashJoinNode +_is_null_vector src/exec/olap_scan_node.h /^ std::vector _is_null_vector;$/;" m class:doris::OlapScanNode +_is_nullable src/exprs/slot_ref.h /^ bool _is_nullable;$/;" m class:doris::SlotRef +_is_nullable src/olap/field.h /^ bool _is_nullable;$/;" m class:doris::Field +_is_nullable src/olap/rowset/segment_v2/column_reader.h /^ bool _is_nullable;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_is_nullable src/olap/rowset/segment_v2/column_writer.h /^ bool _is_nullable;$/;" m class:doris::segment_v2::ColumnWriter +_is_nullable src/olap/tablet_schema.h /^ bool _is_nullable;$/;" m class:doris::TabletColumn +_is_on_queue src/runtime/disk_io_mgr_internal.h /^ bool _is_on_queue;$/;" m class:doris::DiskIoMgr::RequestContext::PerDiskState +_is_open src/exec/csv_scanner.h /^ bool _is_open;$/;" m class:doris::CsvScanner +_is_open src/exec/mysql_scanner.h /^ bool _is_open;$/;" m class:doris::MysqlScanner +_is_open src/exec/olap_scanner.h /^ bool _is_open;$/;" m class:doris::OlapScanner +_is_pending src/olap/rowset/beta_rowset_writer.h /^ bool _is_pending = false;$/;" m class:doris::BetaRowsetWriter +_is_pending src/olap/rowset/rowset.h /^ bool _is_pending; \/\/ rowset is pending iff it's not in visible state$/;" m class:doris::Rowset +_is_pending src/olap/rowset/segment_group.h /^ bool _is_pending;$/;" m class:doris::SegmentGroup +_is_pending_rowset src/olap/rowset/alpha_rowset_writer.h /^ bool _is_pending_rowset;$/;" m class:doris::AlphaRowsetWriter +_is_pinned src/runtime/buffered_block_mgr2.h /^ bool _is_pinned;$/;" m class:doris::BufferedBlockMgr2::Block +_is_pinned src/runtime/spill_sorter.cc /^ bool _is_pinned;$/;" m class:doris::SpillSorter::Run file: +_is_prepare src/exprs/in_predicate.h /^ bool _is_prepare;$/;" m class:doris::InPredicate +_is_present src/olap/rowset/column_writer.h /^ BitFieldWriter* _is_present; \/\/ 对于允许NULL的列记录NULL Bits$/;" m class:doris::ColumnWriter +_is_present_stream src/olap/rowset/column_writer.h /^ OutStream* _is_present_stream;$/;" m class:doris::ColumnWriter +_is_push_down src/exec/hash_join_node.h /^ bool _is_push_down;$/;" m class:doris::HashJoinNode +_is_push_write src/olap/rowset/column_data_writer.h /^ bool _is_push_write;$/;" m class:doris::ColumnDataWriter +_is_removed_from_rowset_meta src/olap/rowset/rowset_meta.h /^ bool _is_removed_from_rowset_meta = false;$/;" m class:doris::RowsetMeta +_is_report_on_cancel src/runtime/plan_fragment_executor.h /^ bool _is_report_on_cancel;$/;" m class:doris::PlanFragmentExecutor +_is_report_success src/runtime/plan_fragment_executor.h /^ bool _is_report_success;$/;" m class:doris::PlanFragmentExecutor +_is_result_order src/exec/olap_scan_node.h /^ bool _is_result_order;$/;" m class:doris::OlapScanNode +_is_segments_overlapping src/olap/rowset/alpha_rowset_reader.h /^ bool _is_segments_overlapping;$/;" m class:doris::AlphaRowsetReader +_is_signed src/olap/rowset/column_writer.h /^ bool _is_signed;$/;" m class:doris::IntegerColumnWriter +_is_signed src/olap/rowset/run_length_integer_writer.h /^ bool _is_signed;$/;" m class:doris::RunLengthIntegerWriter +_is_slice_type src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ bool _is_slice_type() const {$/;" f class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl file: +_is_slotref src/exprs/expr.h /^ const bool _is_slotref;$/;" m class:doris::Expr +_is_sorted src/runtime/merge_sorter.cpp /^ bool _is_sorted;$/;" m class:doris::MergeSorter::Run file: +_is_sorted src/runtime/spill_sorter.cc /^ bool _is_sorted;$/;" m class:doris::SpillSorter::Run file: +_is_stop src/runtime/external_scan_context_mgr.h /^ bool _is_stop;$/;" m class:doris::ExternalScanContextMgr +_is_stop src/runtime/result_buffer_mgr.h /^ bool _is_stop;$/;" m class:doris::ResultBufferMgr +_is_stopped src/runtime/load_channel_mgr.h /^ std::atomic _is_stopped;$/;" m class:doris::LoadChannelMgr +_is_streaming src/http/action/mini_load.cpp /^bool MiniLoadAction::_is_streaming(HttpRequest* req) { $/;" f class:doris::MiniLoadAction +_is_string_type src/olap/wrapper_field.h /^ bool _is_string_type;$/;" m class:doris::WrapperField +_is_suppressed src/olap/out_stream.h /^ bool _is_suppressed; \/\/ 流是否被终止$/;" m class:doris::OutStream +_is_transfer_chain src/runtime/data_stream_sender.cpp /^ bool _is_transfer_chain;$/;" m class:doris::DataStreamSender::Channel file: +_is_upper_keys_included src/olap/reader.h /^ std::vector _is_upper_keys_included;$/;" m class:doris::Reader +_is_used src/olap/data_dir.h /^ bool _is_used;$/;" m class:doris::DataDir +_is_using_cache src/olap/bloom_filter_reader.h /^ bool _is_using_cache;$/;" m class:doris::BloomFilterIndexReader +_is_using_cache src/olap/file_helper.h /^ bool _is_using_cache;$/;" m class:doris::FileHandler +_is_using_cache src/olap/rowset/column_data.h /^ bool _is_using_cache;$/;" m class:doris::ColumnData +_is_using_cache src/olap/stream_index_reader.h /^ bool _is_using_cache;$/;" m class:doris::StreamIndexReader +_is_using_mmap src/olap/rowset/segment_reader.h /^ bool _is_using_mmap; \/\/ 这个标记为true时,使用mmap来读取文件$/;" m class:doris::SegmentReader +_is_valid src/util/load_error_hub.h /^ bool _is_valid = false;$/;" m class:doris::LoadErrorHub +_it src/util/lru_cache.hpp /^ typename std::unordered_map::iterator _it;$/;" m class:doris::LruCache::Iterator +_iter src/olap/generic_iterators.cpp /^ RowwiseIterator* _iter;$/;" m class:doris::MergeIteratorContext file: +_iter src/olap/rowset/segment_v2/segment_iterator.cpp /^ roaring_uint32_iterator_t _iter;$/;" m class:doris::segment_v2::SegmentIterator::BitmapRangeIterator file: +_iter_idx src/olap/generic_iterators.cpp /^ size_t _iter_idx = 0;$/;" m class:doris::UnionIterator file: +_iterations test/util/blocking_queue_test.cpp /^ int _iterations;$/;" m class:doris::MultiThreadTest file: +_iterator src/olap/rowset/beta_rowset_reader.h /^ std::unique_ptr _iterator;$/;" m class:doris::BetaRowsetReader +_job_id src/runtime/snapshot_loader.h /^ int64_t _job_id;$/;" m class:doris::SnapshotLoader +_join_op src/exec/blocking_join_node.h /^ TJoinOp::type _join_op;$/;" m class:doris::BlockingJoinNode +_join_op src/exec/hash_join_node.h /^ TJoinOp::type _join_op;$/;" m class:doris::HashJoinNode +_joinable src/util/thread.h /^ bool _joinable;$/;" m class:doris::Thread +_joined_build_rows src/exec/hash_join_node.h /^ BuildTupleRowSet _joined_build_rows;$/;" m class:doris::HashJoinNode +_json_header test/olap/tablet_meta_manager_test.cpp /^ std::string _json_header;$/;" m class:doris::TabletMetaManagerTest file: +_json_rowset_meta test/olap/olap_snapshot_converter_test.cpp /^ std::string _json_rowset_meta;$/;" m class:doris::OlapSnapshotConverterTest file: +_json_rowset_meta test/olap/rowset/rowset_meta_manager_test.cpp /^ std::string _json_rowset_meta;$/;" m class:doris::RowsetMetaManagerTest file: +_json_rowset_meta test/olap/rowset/rowset_meta_test.cpp /^ std::string _json_rowset_meta;$/;" m class:doris::RowsetMetaTest file: +_json_rowset_meta test/olap/tablet_mgr_test.cpp /^ std::string _json_rowset_meta;$/;" m class:doris::TabletMgrTest file: +_json_rowset_meta test/olap/txn_manager_test.cpp /^ std::string _json_rowset_meta;$/;" m class:doris::TxnManagerTest file: +_judge_and_update_effective_cluster_id src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::_judge_and_update_effective_cluster_id(int32_t cluster_id) {$/;" f class:doris::StorageEngine +_k_consumer src/runtime/routine_load/data_consumer.h /^ RdKafka::KafkaConsumer* _k_consumer = nullptr;$/;" m class:doris::KafkaDataConsumer +_k_consumer_pipe src/runtime/routine_load/data_consumer.h /^ std::shared_ptr _k_consumer_pipe;$/;" m class:doris::KafkaDataConsumer +_k_event_cb src/runtime/routine_load/data_consumer.h /^ KafkaEventCb _k_event_cb;$/;" m class:doris::KafkaDataConsumer +_k_tablet_recorder test/runtime/load_channel_mgr_test.cpp /^std::unordered_map _k_tablet_recorder;$/;" m namespace:doris file: +_keep_alive_reaper src/runtime/external_scan_context_mgr.h /^ std::unique_ptr _keep_alive_reaper;$/;" m class:doris::ExternalScanContextMgr +_kept_in_memory src/olap/rowset/segment_v2/indexed_column_reader.h /^ bool _kept_in_memory;$/;" m class:doris::segment_v2::IndexedColumnReader +_key src/runtime/dpp_sink_internal.h /^ void* _key;$/;" m class:doris::PartRangeKey +_key src/runtime/tablets_channel.h /^ TabletsChannelKey _key;$/;" m class:doris::TabletsChannel +_key_buf src/olap/short_key_index.h /^ faststring _key_buf;$/;" m class:doris::ShortKeyIndexBuilder +_key_cids src/olap/reader.h /^ std::vector _key_cids;$/;" m class:doris::Reader +_key_coder src/olap/field.h /^ const KeyCoder* _key_coder;$/;" m class:doris::Field +_key_ctxs src/runtime/dpp_sink_internal.h /^ std::vector _key_ctxs;$/;" m class:doris::RollupSchema +_key_data src/olap/short_key_index.h /^ Slice _key_data;$/;" m class:doris::ShortKeyIndexDecoder +_key_expr_ctxs_lhs src/util/tuple_row_compare.h /^ const std::vector& _key_expr_ctxs_lhs;$/;" m class:doris::TupleRowComparator +_key_expr_ctxs_rhs src/util/tuple_row_compare.h /^ const std::vector& _key_expr_ctxs_rhs;$/;" m class:doris::TupleRowComparator +_key_length src/olap/olap_index.h /^ size_t _key_length;$/;" m class:doris::MemIndex +_key_num src/olap/olap_index.h /^ size_t _key_num;$/;" m class:doris::MemIndex +_key_range_size src/olap/rowset/alpha_rowset_reader.h /^ int _key_range_size;$/;" m class:doris::AlphaRowsetReader +_keys src/olap/rowset/segment_v2/index_page.h /^ std::vector _keys;$/;" m class:doris::segment_v2::IndexPageReader +_keys_param src/olap/reader.h /^ KeysParam _keys_param;$/;" m class:doris::Reader +_keys_type src/olap/memtable.h /^ KeysType _keys_type;$/;" m class:doris::MemTable +_keys_type src/olap/tablet_schema.h /^ KeysType _keys_type;$/;" m class:doris::TabletSchema +_keys_type src/runtime/dpp_sink_internal.h /^ std::string _keys_type;$/;" m class:doris::RollupSchema +_kind src/olap/stream_name.h /^ StreamInfoMessage::Kind _kind;$/;" m class:doris::StreamName +_kudu_remote_tokens src/exec/kudu_scan_node.h /^ RuntimeProfile::Counter* _kudu_remote_tokens;$/;" m class:doris::KuduScanNode +_kudu_round_trips src/exec/kudu_scan_node.h /^ RuntimeProfile::Counter* _kudu_round_trips;$/;" m class:doris::KuduScanNode +_label src/runtime/mem_tracker.h /^ std::string _label;$/;" m class:doris::MemTracker +_label src/util/progress_updater.h /^ std::string _label;$/;" m class:doris::ProgressUpdater +_last_alive_time_micros src/exec/kudu_scanner.h /^ int64_t _last_alive_time_micros;$/;" m class:doris::KuduScanner +_last_base_compaction_failure_millis src/olap/tablet.h /^ std::atomic _last_base_compaction_failure_millis;$/;" m class:doris::Tablet +_last_base_compaction_success_millis src/olap/tablet.h /^ std::atomic _last_base_compaction_success_millis;$/;" m class:doris::Tablet +_last_checkpoint_time src/olap/tablet.h /^ std::atomic _last_checkpoint_time;$/;" m class:doris::Tablet +_last_cumu_compaction_failure_millis src/olap/tablet.h /^ std::atomic _last_cumu_compaction_failure_millis;$/;" m class:doris::Tablet +_last_cumu_compaction_success_millis src/olap/tablet.h /^ std::atomic _last_cumu_compaction_success_millis;$/;" m class:doris::Tablet +_last_data_page src/olap/rowset/segment_v2/indexed_column_writer.h /^ PagePointer _last_data_page;$/;" m class:doris::segment_v2::IndexedColumnWriter +_last_entry src/olap/rowset/segment_v2/binary_prefix_page.h /^ faststring _last_entry;$/;" m class:doris::segment_v2::BinaryPrefixPageBuilder +_last_frame_size src/util/frame_of_reference_coding.h /^ uint8_t _last_frame_size = 0;$/;" m class:doris::ForDecoder +_last_id src/olap/lru_cache.h /^ uint64_t _last_id;$/;" m class:doris::CachePriority::LRUCache +_last_id src/olap/lru_cache.h /^ uint64_t _last_id;$/;" m class:doris::CachePriority::ShardedLRUCache +_last_output_percentage src/util/progress_updater.h /^ int _last_output_percentage;$/;" m class:doris::ProgressUpdater +_last_pp src/olap/rowset/segment_v2/ordinal_page_index.h /^ PagePointer _last_pp;$/;" m class:doris::segment_v2::OrdinalIndexWriter +_last_result_idx src/exec/analytic_eval_node.h /^ int64_t _last_result_idx;$/;" m class:doris::AnalyticEvalNode +_last_row_expr_ctxs src/runtime/dpp_sink.cpp /^ std::vector _last_row_expr_ctxs;$/;" m class:doris::Translator file: +_last_tuple_block_offset src/runtime/merge_sorter.cpp /^ const int _last_tuple_block_offset;$/;" m class:doris::MergeSorter::TupleSorter file: +_last_tuple_block_offset src/runtime/spill_sorter.cc /^ const int _last_tuple_block_offset;$/;" m class:doris::SpillSorter::TupleSorter file: +_last_update_stat_ms src/olap/tablet_manager.h /^ int64_t _last_update_stat_ms;$/;" m class:doris::TabletManager +_last_updated_time src/runtime/load_channel.h /^ std::atomic _last_updated_time;$/;" m class:doris::LoadChannel +_last_val src/olap/rowset/segment_v2/frame_of_reference_page.h /^ CppType _last_val;$/;" m class:doris::segment_v2::FrameOfReferencePageBuilder +_last_val src/olap/rowset/segment_v2/segment_iterator.cpp /^ uint32_t _last_val;$/;" m class:doris::segment_v2::SegmentIterator::BitmapRangeIterator file: +_last_value src/olap/rowset/segment_v2/binary_plain_page.h /^ faststring _last_value;$/;" m class:doris::segment_v2::BinaryPlainPageBuilder +_last_value src/olap/rowset/segment_v2/bitshuffle_page.h /^ CppType _last_value;$/;" m class:doris::segment_v2::BitshufflePageBuilder +_last_value src/olap/rowset/segment_v2/plain_page.h /^ faststring _last_value;$/;" m class:doris::segment_v2::PlainPageBuilder +_last_value src/olap/rowset/segment_v2/rle_page.h /^ CppType _last_value;$/;" m class:doris::segment_v2::RlePageBuilder +_last_value_size src/olap/rowset/segment_v2/binary_plain_page.h /^ uint32_t _last_value_size = 0;$/;" m class:doris::segment_v2::BinaryPlainPageBuilder +_last_visit_time src/runtime/routine_load/data_consumer.h /^ time_t _last_visit_time;$/;" m class:doris::DataConsumer +_lastest_success_channel src/runtime/load_channel_mgr.h /^ Cache* _lastest_success_channel = nullptr;$/;" m class:doris::LoadChannelMgr +_latch test/util/threadpool_test.cpp /^ CountDownLatch* _latch;$/;" m class:doris::SlowTask file: +_lazy_materialization_read src/olap/rowset/segment_v2/segment_iterator.h /^ bool _lazy_materialization_read;$/;" m class:doris::segment_v2::SegmentIterator +_left_batch src/exec/blocking_join_node.h /^ boost::scoped_ptr _left_batch;$/;" m class:doris::BlockingJoinNode +_left_batch_pos src/exec/blocking_join_node.h /^ int _left_batch_pos; \/\/ current scan pos in _left_batch$/;" m class:doris::BlockingJoinNode +_left_child_ctx src/exec/merge_join_node.h /^ boost::scoped_ptr _left_child_ctx;$/;" m class:doris::MergeJoinNode +_left_child_row_counter src/exec/blocking_join_node.h /^ RuntimeProfile::Counter* _left_child_row_counter; \/\/ num left child rows$/;" m class:doris::BlockingJoinNode +_left_child_timer src/exec/blocking_join_node.h /^ RuntimeProfile::Counter* _left_child_timer; \/\/ time to process left child batch$/;" m class:doris::BlockingJoinNode +_left_expr_ctxs src/exec/merge_join_node.h /^ std::vector _left_expr_ctxs;$/;" m class:doris::MergeJoinNode +_left_side_eos src/exec/blocking_join_node.h /^ bool _left_side_eos; \/\/ if true, left child has no more rows to process$/;" m class:doris::BlockingJoinNode +_left_tuple_size src/exec/merge_join_node.h /^ int _left_tuple_size;$/;" m class:doris::MergeJoinNode +_len src/runtime/disk_io_mgr.h /^ int64_t _len;$/;" m class:doris::DiskIoMgr::BufferDescriptor +_len src/runtime/disk_io_mgr.h /^ int64_t _len;$/;" m class:doris::DiskIoMgr::RequestRange +_length src/olap/field.h /^ uint32_t _length;$/;" m class:doris::Field +_length src/olap/file_stream.h /^ size_t _length; \/\/ length limit$/;" m class:doris::ReadOnlyFileStream::FileCursor +_length src/olap/in_stream.h /^ uint64_t _length;$/;" m class:doris::InStream +_length src/olap/lru_cache.h /^ uint32_t _length;$/;" m class:doris::CachePriority::HandleTable +_length src/olap/rowset/column_reader.h /^ int32_t _length;$/;" m class:doris::DefaultValueReader +_length src/olap/rowset/column_writer.h /^ uint32_t _length;$/;" m class:doris::FixLengthStringColumnWriter +_length src/olap/tablet_schema.h /^ int32_t _length;$/;" m class:doris::TabletColumn +_length src/olap/wrapper_field.h /^ size_t _length;$/;" m class:doris::WrapperField +_length_buffers test/olap/column_reader_test.cpp /^ std::vector _length_buffers;$/;" m class:doris::TestColumn file: +_length_buffers test/olap/schema_change_test.cpp /^ std::vector _length_buffers;$/;" m class:doris::TestColumn file: +_length_reader src/olap/rowset/column_reader.h /^ RunLengthIntegerReader* _length_reader;$/;" m class:doris::StringColumnDirectReader +_length_writer src/olap/rowset/column_writer.h /^ RunLengthIntegerWriter* _length_writer;$/;" m class:doris::VarStringColumnWriter +_less_than test/runtime/data_stream_test.cpp /^ TupleRowComparator* _less_than;$/;" m class:doris::DataStreamTest file: +_less_than_comp src/runtime/merge_sorter.cpp /^ const TupleRowComparator _less_than_comp;$/;" m class:doris::MergeSorter::TupleSorter file: +_less_than_comp src/runtime/spill_sorter.cc /^ const TupleRowComparator _less_than_comp;$/;" m class:doris::SpillSorter::TupleSorter file: +_level src/exec/partitioned_hash_table.h /^ int _level;$/;" m class:doris::PartitionedHashTableCtx +_lhs_expr_ctxs src/runtime/qsorter.cpp /^ std::vector& _lhs_expr_ctxs;$/;" m class:doris::TupleRowLessThan file: +_lhs_expr_ctxs src/runtime/qsorter.h /^ std::vector _lhs_expr_ctxs;$/;" m class:doris::QSorter +_lhs_ordering_expr_ctxs src/exec/sort_exec_exprs.h /^ std::vector _lhs_ordering_expr_ctxs;$/;" m class:doris::SortExecExprs +_lhs_slot_ctx test/runtime/data_stream_test.cpp /^ ExprContext* _lhs_slot_ctx;$/;" m class:doris::DataStreamTest file: +_lib_dir src/runtime/user_function_cache.h /^ std::string _lib_dir;$/;" m class:doris::UserFunctionCache +_like_value src/exec/es/es_query_builder.h /^ std::string _like_value;$/;" m class:doris::WildCardQueryBuilder +_limit src/exec/exec_node.h /^ int64_t _limit; \/\/ -1: no limit$/;" m class:doris::ExecNode +_limit src/olap/byte_buffer.h /^ uint64_t _limit;$/;" m class:doris::StorageByteBuffer +_limit src/olap/row_block.h /^ size_t _limit = 0;$/;" m class:doris::RowBlock +_limit src/runtime/mem_tracker.h /^ int64_t _limit; \/\/ in bytes$/;" m class:doris::MemTracker +_limit src/runtime/vectorized_row_batch.h /^ uint16_t _limit;$/;" m class:doris::VectorizedRowBatch +_limit test/exec/partitioned_hash_table_test.cpp /^ MemTracker _limit;$/;" m class:doris::PartitionedHashTableTest file: +_limit test/runtime/data_stream_test.cpp /^ MemTracker _limit;$/;" m class:doris::DataStreamTest file: +_limit_metric src/runtime/mem_tracker.h /^ IntGauge* _limit_metric;$/;" m class:doris::MemTracker +_limit_trackers src/runtime/mem_tracker.h /^ std::vector _limit_trackers; \/\/ _all_trackers with valid limits$/;" m class:doris::MemTracker +_line_buf_size src/util/system_metrics.h /^ size_t _line_buf_size = 0;$/;" m class:doris::SystemMetrics +_line_delimiter src/exec/broker_scanner.h /^ char _line_delimiter;$/;" m class:doris::BrokerScanner +_line_delimiter src/exec/plain_text_line_reader.h /^ uint8_t _line_delimiter;$/;" m class:doris::PlainTextLineReader +_line_eof src/exec/es_http_scanner.h /^ bool _line_eof;$/;" m class:doris::EsHttpScanner +_line_index src/exec/es/es_scroll_parser.h /^ rapidjson::SizeType _line_index;$/;" m class:doris::ScrollParser +_line_ptr src/util/system_metrics.h /^ char* _line_ptr = nullptr;$/;" m class:doris::SystemMetrics +_link_index_and_data_files src/olap/snapshot_manager.cpp /^OLAPStatus SnapshotManager::_link_index_and_data_files($/;" f class:doris::SnapshotManager +_list src/exec/row_batch_list.h /^ RowBatchList* _list;$/;" m class:doris::RowBatchList::TupleRowIterator +_list src/olap/lru_cache.h /^ LRUHandle** _list;$/;" m class:doris::CachePriority::HandleTable +_list src/util/blocking_queue.hpp /^ std::list _list;$/;" m class:doris::BlockingQueue +_list test/olap/skiplist_test.cpp /^ SkipList _list;$/;" m class:doris::ConcurrentTest file: +_listener_mtx src/agent/topic_subscriber.h /^ boost::shared_mutex _listener_mtx;$/;" m class:doris::TopicSubscriber +_lists src/runtime/free_pool.hpp /^ FreeListNode _lists[NUM_LISTS];$/;" m class:doris::FreePool +_literals src/olap/rowset/run_length_byte_reader.h /^ char _literals[RunLengthByteWriter::MAX_LITERAL_SIZE];$/;" m class:doris::RunLengthByteReader +_literals src/olap/rowset/run_length_byte_writer.h /^ char _literals[MAX_LITERAL_SIZE];$/;" m class:doris::RunLengthByteWriter +_literals src/olap/rowset/run_length_integer_reader.h /^ int64_t _literals[RunLengthIntegerWriter::MAX_SCOPE];$/;" m class:doris::RunLengthIntegerReader +_literals src/olap/rowset/run_length_integer_writer.h /^ int64_t _literals[MAX_SCOPE];$/;" m class:doris::RunLengthIntegerWriter +_load src/http/action/mini_load.cpp /^Status MiniLoadAction::_load($/;" f class:doris::MiniLoadAction +_load_bf_columns src/olap/reader.h /^ std::set _load_bf_columns;$/;" m class:doris::Reader +_load_bf_columns src/olap/rowset/column_data.h /^ std::set _load_bf_columns;$/;" m class:doris::ColumnData +_load_bf_columns src/olap/rowset/segment_reader.h /^ UniqueIdSet _load_bf_columns;$/;" m class:doris::SegmentReader +_load_bitmap_index src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::_load_bitmap_index(bool use_page_cache, bool kept_in_memory) {$/;" f class:doris::segment_v2::ColumnReader +_load_bloom_filter_index src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::_load_bloom_filter_index(bool use_page_cache, bool kept_in_memory) {$/;" f class:doris::segment_v2::ColumnReader +_load_cache_entry src/runtime/user_function_cache.cpp /^Status UserFunctionCache::_load_cache_entry($/;" f class:doris::UserFunctionCache +_load_cache_entry_internal src/runtime/user_function_cache.cpp /^Status UserFunctionCache::_load_cache_entry_internal(UserFunctionCacheEntry* entry) {$/;" f class:doris::UserFunctionCache +_load_cached_lib src/runtime/user_function_cache.cpp /^Status UserFunctionCache::_load_cached_lib() {$/;" f class:doris::UserFunctionCache +_load_channel_mgr src/runtime/exec_env.h /^ LoadChannelMgr* _load_channel_mgr = nullptr;$/;" m class:doris::ExecEnv +_load_channel_timeout_s src/exec/tablet_sink.h /^ int64_t _load_channel_timeout_s = 0;$/;" m class:doris::stream_load::OlapTableSink +_load_channels src/runtime/load_channel_mgr.h /^ std::unordered_map> _load_channels;$/;" m class:doris::LoadChannelMgr +_load_channels_clean_thread src/runtime/load_channel_mgr.h /^ std::thread _load_channels_clean_thread;$/;" m class:doris::LoadChannelMgr +_load_dir src/runtime/runtime_state.h /^ std::string _load_dir;$/;" m class:doris::RuntimeState +_load_entry_from_lib src/runtime/user_function_cache.cpp /^Status UserFunctionCache::_load_entry_from_lib(const std::string& dir, const std::string& file) {$/;" f class:doris::UserFunctionCache +_load_error_hub_info src/runtime/runtime_state.h /^ std::unique_ptr _load_error_hub_info;$/;" m class:doris::RuntimeState +_load_id src/exec/tablet_sink.h /^ PUniqueId _load_id;$/;" m class:doris::stream_load::OlapTableSink +_load_id src/olap/rowset/segment_group.h /^ PUniqueId _load_id; \/\/ load id for segmentgroup$/;" m class:doris::SegmentGroup +_load_id src/runtime/load_channel.h /^ UniqueId _load_id;$/;" m class:doris::LoadChannel +_load_index src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_load_index(bool is_using_cache) {$/;" f class:doris::SegmentReader +_load_index src/olap/rowset/segment_v2/segment.cpp /^Status Segment::_load_index() {$/;" f class:doris::segment_v2::Segment +_load_index_once src/olap/rowset/segment_v2/column_reader.h /^ DorisCallOnce _load_index_once;$/;" m class:doris::segment_v2::ColumnReader +_load_index_once src/olap/rowset/segment_v2/segment.h /^ DorisCallOnce _load_index_once;$/;" m class:doris::segment_v2::Segment +_load_job_id src/runtime/runtime_state.h /^ int64_t _load_job_id;$/;" m class:doris::RuntimeState +_load_local_files src/runtime/small_file_mgr.cpp /^Status SmallFileMgr::_load_local_files() {$/;" f class:doris::SmallFileMgr +_load_mem_limit src/exec/tablet_sink.h /^ int64_t _load_mem_limit = -1;$/;" m class:doris::stream_load::OlapTableSink +_load_next_block src/olap/generic_iterators.cpp /^Status MergeIteratorContext::_load_next_block() {$/;" f class:doris::MergeIteratorContext +_load_next_page src/olap/rowset/segment_v2/column_reader.cpp /^Status FileColumnIterator::_load_next_page(bool* eos) {$/;" f class:doris::segment_v2::FileColumnIterator +_load_ordinal_index src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::_load_ordinal_index(bool use_page_cache, bool kept_in_memory) {$/;" f class:doris::segment_v2::ColumnReader +_load_path_mgr src/runtime/exec_env.h /^ LoadPathMgr* _load_path_mgr = nullptr;$/;" m class:doris::ExecEnv +_load_segment_file src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_load_segment_file() {$/;" f class:doris::SegmentReader +_load_single_file src/runtime/small_file_mgr.cpp /^Status SmallFileMgr::_load_single_file($/;" f class:doris::SmallFileMgr +_load_stream_mgr src/runtime/exec_env.h /^ LoadStreamMgr* _load_stream_mgr = nullptr;$/;" m class:doris::ExecEnv +_load_to_vectorized_row_batch src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_load_to_vectorized_row_batch($/;" f class:doris::SegmentReader +_load_zone_map_index src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::_load_zone_map_index(bool use_page_cache, bool kept_in_memory) {$/;" f class:doris::segment_v2::ColumnReader +_local_allocations src/udf/udf_internal.h /^ std::vector _local_allocations;$/;" m class:doris::FunctionContextImpl +_local_counter src/runtime/mem_tracker.h /^ RuntimeProfile::HighWaterMarkCounter _local_counter;$/;" m class:doris::MemTracker +_local_file src/runtime/disk_io_mgr.h /^ FILE* _local_file;$/;" m union:doris::DiskIoMgr::ScanRange::__anon2 +_local_file_path src/olap/task/engine_batch_load_task.h /^ std::string _local_file_path;$/;" m class:doris::EngineBatchLoadTask +_local_lock src/runtime/external_scan_context_mgr.h /^ std::mutex _local_lock;$/;" m struct:doris::ScanContext +_local_path src/runtime/small_file_mgr.h /^ std::string _local_path;$/;" m class:doris::SmallFileMgr +_local_time_percent src/util/runtime_profile.h /^ double _local_time_percent;$/;" m class:doris::RuntimeProfile +_local_users src/agent/cgroups_mgr.h /^ std::set _local_users;$/;" m class:doris::CgroupsMgr +_location src/exec/tablet_sink.h /^ OlapTableLocationParam* _location = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_lock src/common/object_pool.h /^ SpinLock _lock;$/;" m class:doris::ObjectPool +_lock src/exec/kudu_scan_node.h /^ boost::mutex _lock;$/;" m class:doris::KuduScanNode +_lock src/http/action/mini_load.h /^ std::mutex _lock;$/;" m class:doris::MiniLoadAction +_lock src/olap/rowset/rowset.h /^ std::mutex _lock;$/;" m class:doris::Rowset +_lock src/olap/rowset/unique_rowset_id_generator.h /^ SpinLock _lock;$/;" m class:doris::UniqueRowsetIdGenerator +_lock src/runtime/buffer_control_block.h /^ boost::mutex _lock;$/;" m class:doris::BufferControlBlock +_lock src/runtime/buffered_block_mgr2.h /^ boost::mutex _lock;$/;" m class:doris::BufferedBlockMgr2 +_lock src/runtime/client_cache.h /^ boost::mutex _lock;$/;" m class:doris::ClientCacheHelper +_lock src/runtime/data_stream_mgr.h /^ boost::mutex _lock;$/;" m class:doris::DataStreamMgr +_lock src/runtime/data_stream_recvr.cc /^ mutex _lock;$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_lock src/runtime/disk_io_mgr.cc /^ mutex _lock;$/;" m class:doris::DiskIoMgr::RequestContextCache file: +_lock src/runtime/disk_io_mgr.h /^ boost::mutex _lock;$/;" m class:doris::DiskIoMgr::ScanRange +_lock src/runtime/disk_io_mgr_internal.h /^ boost::mutex _lock;$/;" m class:doris::DiskIoMgr::RequestContext +_lock src/runtime/etl_job_mgr.h /^ std::mutex _lock;$/;" m class:doris::EtlJobMgr +_lock src/runtime/export_task_mgr.h /^ std::mutex _lock;$/;" m class:doris::ExportTaskMgr +_lock src/runtime/external_scan_context_mgr.h /^ std::mutex _lock;$/;" m class:doris::ExternalScanContextMgr +_lock src/runtime/fragment_mgr.h /^ std::mutex _lock;$/;" m class:doris::FragmentMgr +_lock src/runtime/load_channel.h /^ std::mutex _lock;$/;" m class:doris::LoadChannel +_lock src/runtime/load_channel_mgr.h /^ std::mutex _lock;$/;" m class:doris::LoadChannelMgr +_lock src/runtime/load_path_mgr.h /^ std::mutex _lock;$/;" m class:doris::LoadPathMgr +_lock src/runtime/memory/chunk_allocator.cpp /^ SpinLock _lock;$/;" m class:doris::ChunkArena file: +_lock src/runtime/query_statistics.h /^ SpinLock _lock;$/;" m class:doris::QueryStatisticsRecvr +_lock src/runtime/result_buffer_mgr.h /^ boost::mutex _lock;$/;" m class:doris::ResultBufferMgr +_lock src/runtime/result_queue_mgr.h /^ std::mutex _lock;$/;" m class:doris::ResultQueueMgr +_lock src/runtime/routine_load/data_consumer.h /^ std::mutex _lock;$/;" m class:doris::DataConsumer +_lock src/runtime/routine_load/data_consumer_pool.h /^ std::mutex _lock;$/;" m class:doris::DataConsumerPool +_lock src/runtime/routine_load/routine_load_task_executor.h /^ std::mutex _lock;$/;" m class:doris::RoutineLoadTaskExecutor +_lock src/runtime/small_file_mgr.h /^ std::mutex _lock;$/;" m class:doris::SmallFileMgr +_lock src/runtime/stream_load/load_stream_mgr.h /^ std::mutex _lock;$/;" m class:doris::LoadStreamMgr +_lock src/runtime/stream_load/stream_load_pipe.h /^ std::mutex _lock;$/;" m class:doris::StreamLoadPipe +_lock src/runtime/tablets_channel.h /^ std::mutex _lock;$/;" m class:doris::TabletsChannel +_lock src/runtime/thread_resource_mgr.h /^ boost::mutex _lock;$/;" m class:doris::ThreadResourceMgr::ResourcePool +_lock src/runtime/thread_resource_mgr.h /^ boost::mutex _lock;$/;" m class:doris::ThreadResourceMgr +_lock src/util/batch_process_thread_pool.hpp /^ boost::mutex _lock;$/;" m class:doris::BatchProcessThreadPool +_lock src/util/blocking_priority_queue.hpp /^ mutable boost::mutex _lock;$/;" m class:doris::BlockingPriorityQueue +_lock src/util/blocking_queue.hpp /^ mutable std::mutex _lock;$/;" m class:doris::BlockingQueue +_lock src/util/brpc_stub_cache.h /^ SpinLock _lock;$/;" m class:doris::BrpcStubCache +_lock src/util/core_local.h /^ std::mutex _lock;$/;" m class:doris::CoreDataAllocatorFactory +_lock src/util/core_local.h /^ std::mutex _lock;$/;" m class:doris::CoreLocalValueController +_lock src/util/counter_cond_variable.hpp /^ std::mutex _lock;$/;" m class:doris::CounterCondVariable +_lock src/util/metrics.h /^ mutable SpinLock _lock;$/;" m class:doris::LockSimpleMetric +_lock src/util/metrics.h /^ mutable SpinLock _lock;$/;" m class:doris::MetricRegistry +_lock src/util/mutex.h /^ pthread_mutex_t _lock;$/;" m class:doris::Mutex +_lock src/util/mutex.h /^ pthread_rwlock_t _lock;$/;" m class:doris::RWMutex +_lock src/util/priority_thread_pool.hpp /^ boost::mutex _lock;$/;" m class:doris::PriorityThreadPool +_lock src/util/thread.cpp /^ Mutex _lock;$/;" m class:doris::ThreadMgr file: +_lock src/util/threadpool.h /^ mutable Mutex _lock;$/;" m class:doris::ThreadPool +_lock test/exec/tablet_sink_test.cpp /^ std::mutex _lock;$/;" m class:doris::stream_load::TestInternalService file: +_lock test/util/blocking_queue_test.cpp /^ boost::mutex _lock;$/;" m class:doris::MultiThreadTest file: +_locked src/util/mutex.h /^ bool _locked;$/;" m class:doris::MutexLock +_locked src/util/mutex.h /^ bool _locked;$/;" m class:doris::ReadLock +_locked src/util/mutex.h /^ bool _locked;$/;" m class:doris::WriteLock +_locked src/util/spinlock.h /^ bool _locked;$/;" m class:doris::SpinLock +_log_usage_if_zero src/runtime/mem_tracker.h /^ bool _log_usage_if_zero;$/;" m class:doris::MemTracker +_logging_level src/util/progress_updater.h /^ int _logging_level;$/;" m class:doris::ProgressUpdater +_lookup_count src/olap/lru_cache.h /^ uint64_t _lookup_count; \/\/ cache查找总次数$/;" m class:doris::CachePriority::LRUCache +_lookup_ordinal src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_lookup_ordinal(const RowCursor& key, bool is_include,$/;" f class:doris::segment_v2::SegmentIterator +_low_op src/exec/olap_common.h /^ SQLFilterOp _low_op;$/;" m class:doris::ColumnValueRange +_low_reader src/olap/rowset/column_reader.h /^ RunLengthIntegerReader* _low_reader;$/;" m class:doris::LargeIntColumnReader +_low_value src/exec/olap_common.h /^ T _low_value; \/\/ Column's low value, closed interval at left$/;" m class:doris::ColumnValueRange +_low_writer src/olap/rowset/column_writer.h /^ RunLengthIntegerWriter* _low_writer;$/;" m class:doris::LargeIntColumnWriter +_lru src/olap/lru_cache.h /^ LRUHandle _lru;$/;" m class:doris::CachePriority::LRUCache +_lru_append src/olap/lru_cache.cpp /^void LRUCache::_lru_append(LRUHandle* list, LRUHandle* e) {$/;" f class:doris::LRUCache +_lru_cache src/olap/rowset/column_data.h /^ Cache* _lru_cache;$/;" m class:doris::ColumnData +_lru_cache src/olap/rowset/segment_reader.h /^ Cache* _lru_cache;$/;" m class:doris::SegmentReader +_lru_remove src/olap/lru_cache.cpp /^void LRUCache::_lru_remove(LRUHandle* e) {$/;" f class:doris::LRUCache +_make_file_header src/olap/rowset/segment_writer.cpp /^OLAPStatus SegmentWriter::_make_file_header(ColumnDataHeaderMessage* file_header) {$/;" f class:doris::SegmentWriter +_make_heap src/olap/schema_change.cpp /^bool RowBlockMerger::_make_heap(const vector& row_block_arr) {$/;" f class:doris::RowBlockMerger +_make_lib_file src/runtime/user_function_cache.cpp /^std::string UserFunctionCache::_make_lib_file(int64_t function_id, const std::string& checksum) {$/;" f class:doris::UserFunctionCache +_make_snapshot src/olap/task/engine_clone_task.cpp /^Status EngineCloneTask::_make_snapshot($/;" f class:doris::EngineCloneTask +_make_snapshot_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_make_snapshot_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_make_snapshot_workers src/agent/agent_server.h /^ std::unique_ptr _make_snapshot_workers;$/;" m class:doris::AgentServer +_make_sure_output_buffer src/olap/out_stream.cpp /^OLAPStatus OutStream::_make_sure_output_buffer() {$/;" f class:doris::OutStream +_map src/exprs/hybird_map.h /^ std::unordered_map _map;$/;" m class:doris::HybirdMap +_map_column src/exec/parquet_reader.h /^ std::map _map_column; \/\/ column-name <---> column-index$/;" m class:doris::ParquetReaderWrap +_map_in_streams test/olap/column_reader_test.cpp /^ std::map _map_in_streams;$/;" m class:doris::TestColumn file: +_map_in_streams test/olap/schema_change_test.cpp /^ std::map _map_in_streams;$/;" m class:doris::TestColumn file: +_map_lock src/http/web_page_handler.h /^ boost::mutex _map_lock;$/;" m class:doris::WebPageHandler +_mapping src/olap/types.cpp /^ std::hash> _mapping;$/;" m class:doris::TypeInfoResolver file: +_mask src/runtime/string_search.hpp /^ int64_t _mask;$/;" m class:doris::StringSearch +_master_client src/agent/task_worker_pool.h /^ std::unique_ptr _master_client;$/;" m class:doris::TaskWorkerPool +_master_info src/agent/agent_server.h /^ const TMasterInfo& _master_info;$/;" m class:doris::AgentServer +_master_info src/agent/heartbeat_server.h /^ TMasterInfo* _master_info;$/;" m class:doris::HeartbeatServer +_master_info src/agent/task_worker_pool.h /^ const TMasterInfo& _master_info;$/;" m class:doris::TaskWorkerPool +_master_info src/agent/user_resource_listener.h /^ const TMasterInfo& _master_info;$/;" m class:doris::UserResourceListener +_master_info src/agent/utils.h /^ const TMasterInfo& _master_info;$/;" m class:doris::MasterServerClient +_master_info src/olap/task/engine_clone_task.h /^ const TMasterInfo& _master_info;$/;" m class:doris::EngineCloneTask +_master_info src/runtime/exec_env.h /^ TMasterInfo* _master_info = nullptr;$/;" m class:doris::ExecEnv +_master_service_client_cache src/agent/task_worker_pool.cpp /^FrontendServiceClientCache TaskWorkerPool::_master_service_client_cache;$/;" m class:doris::TaskWorkerPool file: +_master_service_client_cache src/agent/task_worker_pool.h /^ static FrontendServiceClientCache _master_service_client_cache;$/;" m class:doris::TaskWorkerPool +_match_all_build src/exec/hash_join_node.h /^ bool _match_all_build; \/\/ output all rows coming from the build input$/;" m class:doris::HashJoinNode +_match_all_probe src/exec/hash_join_node.h /^ bool _match_all_probe; \/\/ output all rows coming from the probe input$/;" m class:doris::HashJoinNode +_match_one_build src/exec/hash_join_node.h /^ bool _match_one_build; \/\/ match at most one build row to each probe row$/;" m class:doris::HashJoinNode +_matched_probe src/exec/hash_join_node.h /^ bool _matched_probe; \/\/ if true, we have matched the current probe row$/;" m class:doris::HashJoinNode +_materialize_slots src/runtime/merge_sorter.cpp /^ const bool _materialize_slots;$/;" m class:doris::MergeSorter::Run file: +_materialize_slots src/runtime/spill_sorter.cc /^ const bool _materialize_slots;$/;" m class:doris::SpillSorter::Run file: +_materialize_timer src/exec/base_scanner.h /^ RuntimeProfile::Counter* _materialize_timer;$/;" m class:doris::BaseScanner +_materialize_timer src/exec/es_http_scanner.h /^ RuntimeProfile::Counter* _materialize_timer;$/;" m class:doris::EsHttpScanner +_materialize_tuple src/exec/sort_exec_exprs.h /^ bool _materialize_tuple;$/;" m class:doris::SortExecExprs +_materialize_tuple_timer src/exec/scan_node.h /^ RuntimeProfile::Counter* _materialize_tuple_timer; \/\/ time writing tuple slots$/;" m class:doris::ScanNode +_materialized_row_batches src/exec/kudu_scan_node.h /^ boost::scoped_ptr _materialized_row_batches;$/;" m class:doris::KuduScanNode +_materialized_row_batches src/exec/olap_scan_node.h /^ std::list _materialized_row_batches;$/;" m class:doris::OlapScanNode +_materialized_slots src/exec/merge_node.h /^ std::vector _materialized_slots;$/;" m class:doris::MergeNode +_materialized_tuple_desc src/exec/topn_node.h /^ TupleDescriptor* _materialized_tuple_desc;$/;" m class:doris::TopNNode +_max src/util/tdigest.h /^ Value _max = std::numeric_limits::min();$/;" m class:doris::TDigest +_max_block_size src/runtime/buffered_block_mgr.h /^ const int64_t _max_block_size;$/;" m class:doris::BufferedBlockMgr +_max_block_size src/runtime/buffered_block_mgr2.h /^ const int64_t _max_block_size;$/;" m class:doris::BufferedBlockMgr2 +_max_buffer_size src/runtime/disk_io_mgr.h /^ const int _max_buffer_size;$/;" m class:doris::DiskIoMgr +_max_buffered_batches src/exec/broker_scan_node.h /^ int _max_buffered_batches;$/;" m class:doris::BrokerScanNode +_max_buffered_batches src/exec/es_http_scan_node.h /^ int _max_buffered_batches;$/;" m class:doris::EsHttpScanNode +_max_buffered_bytes src/runtime/stream_load/stream_load_pipe.h /^ size_t _max_buffered_bytes;$/;" m class:doris::StreamLoadPipe +_max_cache_size_per_host src/runtime/client_cache.h /^ int _max_cache_size_per_host;$/;" m class:doris::ClientCacheHelper +_max_compressed_buf_size src/olap/push_handler.h /^ size_t _max_compressed_buf_size;$/;" m class:doris::LzoBinaryReader +_max_continuous_version_from_begining_unlocked src/olap/tablet.cpp /^OLAPStatus Tablet::_max_continuous_version_from_begining_unlocked(Version* version,$/;" f class:doris::Tablet +_max_decimal_val src/exec/olap_rewrite_node.h /^ std::vector _max_decimal_val;$/;" m class:doris::OlapRewriteNode +_max_decimal_val src/exec/tablet_sink.h /^ std::vector _max_decimal_val;$/;" m class:doris::stream_load::OlapTableSink +_max_decimalv2_val src/exec/olap_rewrite_node.h /^ std::vector _max_decimalv2_val;$/;" m class:doris::OlapRewriteNode +_max_decimalv2_val src/exec/tablet_sink.h /^ std::vector _max_decimalv2_val;$/;" m class:doris::stream_load::OlapTableSink +_max_element src/util/blocking_priority_queue.hpp /^ const int _max_element;$/;" m class:doris::BlockingPriorityQueue +_max_elements src/util/blocking_queue.hpp /^ const int _max_elements;$/;" m class:doris::BlockingQueue +_max_frame_size src/util/frame_of_reference_coding.h /^ uint8_t _max_frame_size = 0;$/;" m class:doris::ForDecoder +_max_length src/olap/rowset/column_reader.h /^ uint32_t _max_length;$/;" m class:doris::VarStringColumnReader +_max_materialized_row_batches src/exec/olap_scan_node.h /^ int _max_materialized_row_batches;$/;" m class:doris::OlapScanNode +_max_num_buckets src/exec/partitioned_hash_table.h /^ const int64_t _max_num_buckets;$/;" m class:doris::PartitionedHashTable +_max_pool_size src/runtime/routine_load/data_consumer_pool.h /^ int64_t _max_pool_size;$/;" m class:doris::DataConsumerPool +_max_processed src/util/tdigest.h /^ Index _max_processed;$/;" m class:doris::TDigest +_max_queue_size src/util/threadpool.h /^ const int _max_queue_size;$/;" m class:doris::ThreadPool +_max_queue_size src/util/threadpool.h /^ int _max_queue_size;$/;" m class:doris::ThreadPoolBuilder +_max_quota src/runtime/thread_resource_mgr.h /^ int _max_quota;$/;" m class:doris::ThreadResourceMgr::ResourcePool +_max_row_buf_size src/olap/push_handler.h /^ size_t _max_row_buf_size;$/;" m class:doris::LzoBinaryReader +_max_row_num src/olap/push_handler.h /^ size_t _max_row_num;$/;" m class:doris::LzoBinaryReader +_max_segment_size src/olap/rowset/column_data_writer.h /^ uint32_t _max_segment_size;$/;" m class:doris::ColumnDataWriter +_max_size src/util/lru_cache.hpp /^ size_t _max_size;$/;" m class:doris::LruCache +_max_threads src/util/threadpool.h /^ const int _max_threads;$/;" m class:doris::ThreadPool +_max_threads src/util/threadpool.h /^ int _max_threads;$/;" m class:doris::ThreadPoolBuilder +_max_unprocessed src/util/tdigest.h /^ Index _max_unprocessed;$/;" m class:doris::TDigest +_maximum src/olap/stream_index_common.h /^ WrapperField* _maximum;$/;" m class:doris::ColumnStatistics +_md5_ctx src/util/md5.h /^ MD5_CTX _md5_ctx;$/;" m class:doris::Md5Digest +_mean src/util/tdigest.h /^ Value _mean = 0;$/;" m class:doris::Centroid +_mem_buf src/olap/row_block.h /^ char* _mem_buf = nullptr;$/;" m class:doris::RowBlock +_mem_buf_bytes src/olap/row_block.h /^ size_t _mem_buf_bytes = 0;$/;" m class:doris::RowBlock +_mem_buffer src/util/thrift_util.h /^ boost::shared_ptr _mem_buffer;$/;" m class:doris::ThriftSerializer +_mem_index src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^ MemoryIndexType _mem_index;$/;" m class:doris::segment_v2::__anon54::BitmapIndexWriterImpl file: +_mem_limit_exceeded src/exec/hash_table.h /^ bool _mem_limit_exceeded;$/;" m class:doris::HashTable +_mem_pool src/exec/analytic_eval_node.h /^ boost::scoped_ptr _mem_pool;$/;" m class:doris::AnalyticEvalNode +_mem_pool src/exec/base_scanner.h /^ MemPool _mem_pool;$/;" m class:doris::BaseScanner +_mem_pool src/exec/es_http_scanner.h /^ MemPool _mem_pool;$/;" m class:doris::EsHttpScanner +_mem_pool src/exec/partitioned_aggregation_node.h /^ boost::scoped_ptr _mem_pool;$/;" m class:doris::PartitionedAggregationNode +_mem_pool src/exec/tablet_info.h /^ std::unique_ptr _mem_pool;$/;" m class:doris::OlapTablePartitionParam +_mem_pool src/olap/olap_index.h /^ std::unique_ptr _mem_pool;$/;" m class:doris::MemIndex +_mem_pool src/olap/row_block.h /^ std::unique_ptr _mem_pool;$/;" m class:doris::RowBlock +_mem_pool src/olap/rowset/segment_reader.h /^ std::unique_ptr _mem_pool;$/;" m class:doris::SegmentReader +_mem_pool src/olap/rowset/segment_v2/indexed_column_writer.h /^ MemPool _mem_pool;$/;" m class:doris::segment_v2::IndexedColumnWriter +_mem_pool src/olap/skiplist.h /^ MemPool* const _mem_pool; \/\/ MemPool used for allocations of nodes$/;" m class:doris::SkipList +_mem_pool src/runtime/free_pool.hpp /^ MemPool* _mem_pool;$/;" m class:doris::FreePool +_mem_pool src/runtime/vectorized_row_batch.h /^ std::unique_ptr _mem_pool;$/;" m class:doris::VectorizedRowBatch +_mem_pool test/exec/hash_table_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::HashTableTest file: +_mem_pool test/exec/partitioned_hash_table_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::PartitionedHashTableTest file: +_mem_pool test/exec/schema_scanner/schema_authors_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaAuthorScannerTest file: +_mem_pool test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaCharsetsScannerTest file: +_mem_pool test/exec/schema_scanner/schema_collations_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaCollationsScannerTest file: +_mem_pool test/exec/schema_scanner/schema_columns_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaColumnsScannerTest file: +_mem_pool test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaCreateTableScannerTest file: +_mem_pool test/exec/schema_scanner/schema_engines_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaEnginesScannerTest file: +_mem_pool test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaOpenTablesScannerTest file: +_mem_pool test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaSchemataScannerTest file: +_mem_pool test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaTableNamesScannerTest file: +_mem_pool test/exec/schema_scanner/schema_tables_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaTablesScannerTest file: +_mem_pool test/exec/schema_scanner/schema_variables_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaVariablesScannerTest file: +_mem_pool test/exec/schema_scanner_test.cpp /^ MemPool _mem_pool;$/;" m class:doris::SchemaScannerTest file: +_mem_pool test/olap/column_reader_test.cpp /^ std::unique_ptr _mem_pool;$/;" m class:doris::TestColumn file: +_mem_pool test/olap/in_list_predicate_test.cpp /^ std::unique_ptr _mem_pool;$/;" m class:doris::TestInListPredicate file: +_mem_pool test/olap/null_predicate_test.cpp /^ std::unique_ptr _mem_pool;$/;" m class:doris::TestNullPredicate file: +_mem_pool test/olap/row_cursor_test.cpp /^ std::unique_ptr _mem_pool;$/;" m class:doris::TestRowCursor file: +_mem_pool test/olap/rowset/alpha_rowset_test.cpp /^ std::unique_ptr _mem_pool;$/;" m class:doris::AlphaRowsetTest file: +_mem_pool test/olap/rowset/rowset_converter_test.cpp /^ std::unique_ptr _mem_pool;$/;" m class:doris::RowsetConverterTest file: +_mem_pool test/olap/schema_change_test.cpp /^ std::unique_ptr _mem_pool;$/;" m class:doris::TestColumn file: +_mem_pool test/olap/skiplist_test.cpp /^ std::unique_ptr _mem_pool;$/;" m class:doris::ConcurrentTest file: +_mem_pool test/runtime/buffered_tuple_stream2_test.cpp /^ scoped_ptr _mem_pool;$/;" m class:doris::SimpleTupleStreamTest file: +_mem_row_bytes src/olap/row_block.h /^ size_t _mem_row_bytes = 0;$/;" m class:doris::RowBlock +_mem_table src/olap/delta_writer.h /^ std::shared_ptr _mem_table;$/;" m class:doris::DeltaWriter +_mem_tracker src/exec/base_scanner.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::BaseScanner +_mem_tracker src/exec/es_http_scanner.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::EsHttpScanner +_mem_tracker src/exec/exec_node.h /^ boost::scoped_ptr _mem_tracker;$/;" m class:doris::ExecNode +_mem_tracker src/exec/hash_table.h /^ MemTracker* _mem_tracker;$/;" m class:doris::HashTable +_mem_tracker src/exec/tablet_info.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::OlapTablePartitionParam +_mem_tracker src/exec/tablet_sink.h /^ MemTracker* _mem_tracker = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_mem_tracker src/exprs/agg_fn_evaluator.h /^ MemTracker* _mem_tracker; \/\/ saved c'tor param$/;" m class:doris::AggFnEvaluator +_mem_tracker src/exprs/new_agg_fn_evaluator.h /^ MemTracker* _mem_tracker; \/\/ saved c'tor param$/;" m class:doris::NewAggFnEvaluator +_mem_tracker src/olap/delta_writer.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::DeltaWriter +_mem_tracker src/olap/fs/file_block_manager.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::fs::FileBlockManager +_mem_tracker src/olap/memtable.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::MemTable +_mem_tracker src/olap/rowset/segment_v2/indexed_column_writer.h /^ MemTracker _mem_tracker;$/;" m class:doris::segment_v2::IndexedColumnWriter +_mem_tracker src/runtime/buffered_block_mgr2.h /^ boost::scoped_ptr _mem_tracker;$/;" m class:doris::BufferedBlockMgr2 +_mem_tracker src/runtime/data_stream_recvr.h /^ boost::scoped_ptr _mem_tracker;$/;" m class:doris::DataStreamRecvr +_mem_tracker src/runtime/data_stream_sender.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::DataStreamSender +_mem_tracker src/runtime/disk_io_mgr.h /^ MemTracker* _mem_tracker;$/;" m class:doris::DiskIoMgr::BufferDescriptor +_mem_tracker src/runtime/disk_io_mgr_internal.h /^ MemTracker* _mem_tracker;$/;" m class:doris::DiskIoMgr::RequestContext +_mem_tracker src/runtime/exec_env.h /^ MemTracker* _mem_tracker = nullptr;$/;" m class:doris::ExecEnv +_mem_tracker src/runtime/export_sink.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::ExportSink +_mem_tracker src/runtime/load_channel.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::LoadChannel +_mem_tracker src/runtime/load_channel_mgr.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::LoadChannelMgr +_mem_tracker src/runtime/mysql_table_sink.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::MysqlTableSink +_mem_tracker src/runtime/plan_fragment_executor.h /^ boost::scoped_ptr _mem_tracker;$/;" m class:doris::PlanFragmentExecutor +_mem_tracker src/runtime/row_batch.h /^ MemTracker* _mem_tracker; \/\/ not owned$/;" m class:doris::RowBatch +_mem_tracker src/runtime/spill_sorter.h /^ MemTracker* _mem_tracker;$/;" m class:doris::SpillSorter +_mem_tracker src/runtime/tablets_channel.h /^ std::unique_ptr _mem_tracker;$/;" m class:doris::TabletsChannel +_mem_tracker src/testutil/function_utils.h /^ MemTracker* _mem_tracker = nullptr;$/;" m class:doris::FunctionUtils +_mem_tracker test/olap/column_reader_test.cpp /^ std::unique_ptr _mem_tracker;$/;" m class:doris::TestColumn file: +_mem_tracker test/olap/in_list_predicate_test.cpp /^ std::unique_ptr _mem_tracker;$/;" m class:doris::TestInListPredicate file: +_mem_tracker test/olap/null_predicate_test.cpp /^ std::unique_ptr _mem_tracker;$/;" m class:doris::TestNullPredicate file: +_mem_tracker test/olap/row_cursor_test.cpp /^ std::unique_ptr _mem_tracker;$/;" m class:doris::TestRowCursor file: +_mem_tracker test/olap/rowset/alpha_rowset_test.cpp /^ std::unique_ptr _mem_tracker;$/;" m class:doris::AlphaRowsetTest file: +_mem_tracker test/olap/rowset/rowset_converter_test.cpp /^ std::unique_ptr _mem_tracker;$/;" m class:doris::RowsetConverterTest file: +_mem_tracker test/olap/schema_change_test.cpp /^ std::unique_ptr _mem_tracker;$/;" m class:doris::TestColumn file: +_mem_tracker test/olap/skiplist_test.cpp /^ std::unique_ptr _mem_tracker;$/;" m class:doris::ConcurrentTest file: +_mem_tracker test/runtime/memory_scratch_sink_test.cpp /^ MemTracker *_mem_tracker;$/;" m class:doris::MemoryScratchSinkTest file: +_mem_tracker test/util/arrow/arrow_work_flow_test.cpp /^ MemTracker *_mem_tracker;$/;" m class:doris::ArrowWorkFlowTest file: +_mem_tracker_counter src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _mem_tracker_counter;$/;" m class:doris::BufferedBlockMgr2 +_mem_trackers src/runtime/runtime_state.h /^ std::vector _mem_trackers;$/;" m class:doris::RuntimeState +_mem_value src/olap/rowset/segment_v2/column_reader.h /^ void* _mem_value = nullptr;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_memory_allocated src/olap/schema_change.h /^ size_t _memory_allocated;$/;" m class:doris::RowBlockAllocator +_memory_limitation src/olap/schema_change.h /^ size_t _memory_limitation;$/;" m class:doris::RowBlockAllocator +_memory_limitation src/olap/schema_change.h /^ size_t _memory_limitation;$/;" m class:doris::SchemaChangeWithSorting +_memory_metrics src/util/system_metrics.h /^ std::unique_ptr _memory_metrics;$/;" m class:doris::SystemMetrics +_memory_pool src/testutil/function_utils.h /^ MemPool* _memory_pool = nullptr;$/;" m class:doris::FunctionUtils +_memory_used_counter src/exec/exec_node.h /^ RuntimeProfile::Counter* _memory_used_counter;$/;" m class:doris::ExecNode +_memtable_flush_executor src/olap/storage_engine.h /^ std::unique_ptr _memtable_flush_executor;$/;" m class:doris::StorageEngine +_merge src/olap/reader.cpp /^ bool _merge = true;$/;" m class:doris::CollectIterator file: +_merge_block src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::_merge_block(RowBlock** block) {$/;" f class:doris::AlphaRowsetReader +_merge_ctxs src/olap/generic_iterators.cpp /^ std::vector _merge_ctxs;$/;" m class:doris::MergeIterator file: +_merge_ctxs src/olap/rowset/alpha_rowset_reader.h /^ std::vector _merge_ctxs;$/;" m class:doris::AlphaRowsetReader +_merge_fn output/udf/include/uda_test_harness.h /^ MergeFn _merge_fn;$/;" m class:doris_udf::UdaTestHarnessBase +_merge_fn src/exprs/agg_fn_evaluator.h /^ void* _merge_fn;$/;" m class:doris::AggFnEvaluator +_merge_fn src/udf/uda_test_harness.h /^ MergeFn _merge_fn;$/;" m class:doris_udf::UdaTestHarnessBase +_merge_fn_symbol src/exprs/agg_fn_evaluator.h /^ std::string _merge_fn_symbol;$/;" m class:doris::AggFnEvaluator +_merge_heap src/olap/rowset/alpha_rowset_reader.h /^ std::priority_queue, AlphaMergeContextComparator> _merge_heap;$/;" m class:doris::AlphaRowsetReader +_merge_next src/olap/reader.cpp /^inline OLAPStatus CollectIterator::_merge_next(const RowCursor** row, bool* delete_flag) {$/;" f class:doris::CollectIterator +_merge_registers src/olap/hll.h /^ void _merge_registers(const uint8_t* other_registers) {$/;" f class:doris::HyperLogLog +_merged_rows src/olap/reader.h /^ uint64_t _merged_rows;$/;" m class:doris::Reader +_merged_rows src/olap/schema_change.h /^ uint64_t _merged_rows;$/;" m class:doris::SchemaChange +_merger src/runtime/data_stream_recvr.h /^ boost::scoped_ptr _merger;$/;" m class:doris::DataStreamRecvr +_merger src/runtime/spill_sorter.h /^ boost::scoped_ptr _merger;$/;" m class:doris::SpillSorter +_merging_runs src/runtime/spill_sorter.h /^ std::deque _merging_runs;$/;" m class:doris::SpillSorter +_meta src/olap/data_dir.h /^ OlapMeta* _meta = nullptr;$/;" m class:doris::DataDir +_meta src/olap/olap_index.h /^ std::vector _meta;$/;" m class:doris::MemIndex +_meta src/olap/rowset/segment_v2/column_reader.h /^ ColumnMetaPB _meta;$/;" m class:doris::segment_v2::ColumnReader +_meta src/olap/rowset/segment_v2/indexed_column_reader.h /^ IndexedColumnMetaPB _meta;$/;" m class:doris::segment_v2::IndexedColumnReader +_meta test/olap/olap_meta_test.cpp /^ OlapMeta* _meta;$/;" m class:doris::OlapMetaTest file: +_meta test/olap/olap_snapshot_converter_test.cpp /^ OlapMeta* _meta;$/;" m class:doris::OlapSnapshotConverterTest file: +_meta test/olap/rowset/rowset_meta_manager_test.cpp /^ OlapMeta* _meta;$/;" m class:doris::RowsetMetaManagerTest file: +_meta test/olap/rowset/rowset_meta_test.cpp /^ OlapMeta* _meta;$/;" m class:doris::RowsetMetaTest file: +_meta test/olap/txn_manager_test.cpp /^ OlapMeta* _meta;$/;" m class:doris::TxnManagerTest file: +_meta_data src/runtime/disk_io_mgr.h /^ void* _meta_data;$/;" m class:doris::DiskIoMgr::ScanRange +_meta_lock src/olap/tablet.h /^ RWMutex _meta_lock;$/;" m class:doris::Tablet +_meta_lock src/olap/tablet_meta.h /^ RWMutex _meta_lock;$/;" m class:doris::TabletMeta +_meta_path test/olap/olap_snapshot_converter_test.cpp /^ std::string _meta_path;$/;" m class:doris::OlapSnapshotConverterTest file: +_meta_store_lock src/olap/tablet.h /^ RWMutex _meta_store_lock;$/;" m class:doris::Tablet +_meta_type src/http/action/meta_action.h /^ META_TYPE _meta_type;$/;" m class:doris::MetaAction +_metadata src/util/runtime_profile.h /^ int64_t _metadata;$/;" m class:doris::RuntimeProfile +_method src/http/http_request.h /^ HttpMethod _method;$/;" m class:doris::HttpRequest +_metrics src/http/action/metrics_action.h /^ MetricRegistry* _metrics;$/;" m class:doris::MetricsAction +_metrics src/olap/fs/file_block_manager.h /^ std::unique_ptr _metrics;$/;" m class:doris::fs::FileBlockManager +_metrics src/runtime/exec_env.h /^ MetricRegistry* _metrics = nullptr;$/;" m class:doris::ExecEnv +_metrics src/runtime/test_env.h /^ boost::scoped_ptr _metrics;$/;" m class:doris::TestEnv +_metrics src/util/doris_metrics.h /^ MetricRegistry* _metrics;$/;" m class:doris::DorisMetrics +_metrics src/util/metrics.h /^ std::map _metrics;$/;" m class:doris::MetricCollector +_metrics test/runtime/tmp_file_mgr_test.cpp /^ boost::scoped_ptr _metrics;$/;" m class:doris::TmpFileMgrTest file: +_metrics_enabled src/runtime/client_cache.h /^ bool _metrics_enabled;$/;" m class:doris::ClientCacheHelper +_metrics_enabled src/util/thrift_server.h /^ bool _metrics_enabled;$/;" m class:doris::ThriftServer +_mgr src/runtime/buffered_block_mgr2.cc /^ BufferedBlockMgr2* _mgr;$/;" m class:doris::BufferedBlockMgr2::Client file: +_mgr src/runtime/data_stream_recvr.h /^ DataStreamMgr* _mgr;$/;" m class:doris::DataStreamRecvr +_mgr src/runtime/data_stream_recvr.hpp /^ DataStreamMgr* _mgr;$/;" m class:doris::DataStreamRecvr +_mgr src/runtime/tmp_file_mgr.h /^ TmpFileMgr* _mgr;$/;" m class:doris::TmpFileMgr::File +_mgr test/runtime/data_stream_test.cpp /^ DataStreamMgr* _mgr;$/;" m class:doris::DorisTestBackend file: +_microsecond src/runtime/datetime_value.h /^ uint64_t _microsecond;$/;" m class:doris::DateTimeValue +_migration_lock src/olap/tablet.h /^ RWMutex _migration_lock;$/;" m class:doris::Tablet +_min src/olap/rowset/run_length_integer_writer.h /^ int64_t _min;$/;" m class:doris::RunLengthIntegerWriter +_min src/util/tdigest.h /^ Value _min = std::numeric_limits::max();$/;" m class:doris::TDigest +_min_buffer_size src/runtime/disk_io_mgr.h /^ const int _min_buffer_size;$/;" m class:doris::DiskIoMgr +_min_chunk_size src/runtime/stream_load/stream_load_pipe.h /^ size_t _min_chunk_size;$/;" m class:doris::StreamLoadPipe +_min_decimal_val src/exec/tablet_sink.h /^ std::vector _min_decimal_val;$/;" m class:doris::stream_load::OlapTableSink +_min_decimalv2_val src/exec/tablet_sink.h /^ std::vector _min_decimalv2_val;$/;" m class:doris::stream_load::OlapTableSink +_min_heap src/runtime/sorted_run_merger.h /^ std::vector _min_heap;$/;" m class:doris::SortedRunMerger +_min_length src/exec/plain_text_line_reader.h /^ size_t _min_length;$/;" m class:doris::PlainTextLineReader +_min_threads src/util/threadpool.h /^ const int _min_threads;$/;" m class:doris::ThreadPool +_min_threads src/util/threadpool.h /^ int _min_threads;$/;" m class:doris::ThreadPoolBuilder +_minimum src/olap/stream_index_common.h /^ WrapperField* _minimum;$/;" m class:doris::ColumnStatistics +_minute src/runtime/datetime_value.h /^ uint8_t _minute;$/;" m class:doris::DateTimeValue +_mlock src/olap/snapshot_manager.cpp /^std::mutex SnapshotManager::_mlock;$/;" m class:doris::SnapshotManager file: +_mlock src/olap/snapshot_manager.h /^ static std::mutex _mlock;$/;" m class:doris::SnapshotManager +_mmap2 src/gutil/linux_syscall_support.h /^ LSS_INLINE void* LSS_NAME(_mmap2)(void *s, size_t l, int p, int f, int d,$/;" f +_mmap_buffer src/olap/rowset/segment_reader.h /^ StorageByteBuffer* _mmap_buffer;$/;" m class:doris::SegmentReader +_mmap_length src/olap/byte_buffer.h /^ size_t _mmap_length; \/\/ 如果使用mmap,记录mmap的长度$/;" m class:doris::StorageByteBuffer::BufDeleter +_mode src/util/threadpool.h /^ const ThreadPool::ExecutionMode _mode;$/;" m class:doris::ThreadPoolToken +_modify_old_segment_group_id src/olap/olap_snapshot_converter.cpp /^void OlapSnapshotConverter::_modify_old_segment_group_id(RowsetMetaPB& rowset_meta) {$/;" f class:doris::OlapSnapshotConverter +_module_by_name src/http/monitor_action.h /^ std::map _module_by_name;$/;" m class:doris::MonitorAction +_month src/runtime/datetime_value.h /^ uint8_t _month;$/;" m class:doris::DateTimeValue +_more_input_bytes src/exec/plain_text_line_reader.h /^ size_t _more_input_bytes;$/;" m class:doris::PlainTextLineReader +_more_output_bytes src/exec/plain_text_line_reader.h /^ size_t _more_output_bytes;$/;" m class:doris::PlainTextLineReader +_move_dir src/agent/task_worker_pool.cpp /^AgentStatus TaskWorkerPool::_move_dir($/;" f class:doris::TaskWorkerPool +_move_dir_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_move_dir_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_move_dir_workers src/agent/agent_server.h /^ std::unique_ptr _move_dir_workers;$/;" m class:doris::AgentServer +_mtime src/runtime/disk_io_mgr.h /^ int64_t _mtime;$/;" m class:doris::DiskIoMgr::HdfsCachedFileHandle +_mtime src/runtime/disk_io_mgr.h /^ int64_t _mtime;$/;" m class:doris::DiskIoMgr::ScanRange +_mtx src/util/broker_load_error_hub.h /^ std::mutex _mtx;$/;" m class:doris::BrokerLoadErrorHub +_mtx src/util/mysql_load_error_hub.h /^ std::mutex _mtx;$/;" m class:doris::MysqlLoadErrorHub +_mtx src/util/null_load_error_hub.h /^ std::mutex _mtx;$/;" m class:doris::NullLoadErrorHub +_mu test/olap/skiplist_test.cpp /^ Mutex _mu;$/;" m class:doris::TestState file: +_must_clauses src/exec/es/es_query_builder.h /^ std::vector _must_clauses;$/;" m class:doris::BooleanQueryBuilder +_must_not_clauses src/exec/es/es_query_builder.h /^ std::vector _must_not_clauses;$/;" m class:doris::BooleanQueryBuilder +_mutable_alpha_rowset_extra_meta_pb src/olap/rowset/rowset_meta.h /^ AlphaRowsetExtraMetaPB* _mutable_alpha_rowset_extra_meta_pb() {$/;" f class:doris::RowsetMeta +_mutex src/olap/data_dir.h /^ std::mutex _mutex;$/;" m class:doris::DataDir +_mutex src/olap/lru_cache.h /^ Mutex _mutex;$/;" m class:doris::CachePriority::LRUCache +_mutex src/runtime/broker_mgr.h /^ std::mutex _mutex;$/;" m class:doris::BrokerMgr +_mutex src/runtime/routine_load/data_consumer_group.h /^ std::mutex _mutex;$/;" m class:doris::DataConsumerGroup +_mutex src/util/barrier.h /^ Mutex _mutex;$/;" m class:doris::Barrier +_mutex src/util/bfd_parser.h /^ std::mutex _mutex;$/;" m class:doris::BfdParser +_mutex src/util/mutex.h /^ Mutex* _mutex;$/;" m class:doris::MutexLock +_mutex src/util/mutex.h /^ RWMutex* _mutex;$/;" m class:doris::ReadLock +_mutex src/util/mutex.h /^ RWMutex* _mutex;$/;" m class:doris::WriteLock +_mutex src/util/semaphore.hpp /^ std::mutex _mutex;$/;" m class:__anon35::Semaphore +_my_conn src/exec/mysql_scanner.h /^ __DorisMysql* _my_conn;$/;" m class:doris::MysqlScanner +_my_param src/exec/mysql_scan_node.h /^ MysqlScannerParam _my_param;$/;" m class:doris::MysqlScanNode +_my_param src/exec/mysql_scanner.h /^ const MysqlScannerParam& _my_param;$/;" m class:doris::MysqlScanner +_my_result src/exec/mysql_scanner.h /^ __DorisMysqlRes* _my_result;$/;" m class:doris::MysqlScanner +_mysql_conn src/runtime/mysql_table_writer.h /^ __DorisMysql* _mysql_conn;$/;" m class:doris::MysqlTableWriter +_mysql_db src/runtime/descriptors.h /^ std::string _mysql_db;$/;" m class:doris::MySQLTableDescriptor +_mysql_scanner src/exec/mysql_scan_node.h /^ std::unique_ptr _mysql_scanner;$/;" m class:doris::MysqlScanNode +_mysql_table src/runtime/descriptors.h /^ std::string _mysql_table;$/;" m class:doris::MySQLTableDescriptor +_mysql_tbl src/runtime/mysql_table_sink.h /^ std::string _mysql_tbl;$/;" m class:doris::MysqlTableSink +_mysql_tbl src/runtime/mysql_table_writer.h /^ std::string _mysql_tbl;$/;" m class:doris::MysqlTableWriter +_n test/util/threadpool_test.cpp /^ int _n;$/;" m class:doris::SimpleTask file: +_n_bytes src/olap/selection_vector.h /^ size_t _n_bytes;$/;" m class:doris::SelectionVector +_n_rows src/olap/selection_vector.h /^ size_t _n_rows;$/;" m class:doris::SelectionVector +_name src/runtime/descriptors.h /^ std::string _name;$/;" m class:doris::TableDescriptor +_name src/util/metrics.h /^ const std::string _name;$/;" m class:doris::MetricRegistry +_name src/util/runtime_profile.h /^ std::string _name;$/;" m class:doris::RuntimeProfile +_name src/util/thread.cpp /^ std::string _name;$/;" m class:doris::ThreadMgr::ThreadDescriptor file: +_name src/util/thread.h /^ const std::string _name;$/;" m class:doris::Thread +_name src/util/threadpool.h /^ const std::string _name;$/;" m class:doris::ThreadPool +_name src/util/threadpool.h /^ const std::string _name;$/;" m class:doris::ThreadPoolBuilder +_name src/util/thrift_server.h /^ const std::string _name;$/;" m class:doris::ThriftServer +_named_wildcard src/util/path_trie.hpp /^ std::string _named_wildcard;$/;" m class:doris::PathTrie::TrieNode +_need_agg_finalize src/exec/olap_scan_node.h /^ bool _need_agg_finalize = true;$/;" m class:doris::OlapScanNode +_need_agg_finalize src/exec/olap_scanner.h /^ bool _need_agg_finalize = true;$/;" m class:doris::OlapScanner +_need_agg_finalize src/olap/reader.h /^ bool _need_agg_finalize = true;$/;" m class:doris::Reader +_need_block_filter src/olap/rowset/segment_reader.h /^ bool _need_block_filter; \/\/与include blocks组合使用,如果全不中,就不再读$/;" m class:doris::SegmentReader +_need_close src/runtime/data_stream_sender.cpp /^ bool _need_close;$/;" m class:doris::DataStreamSender::Channel file: +_need_column_data_writer src/olap/rowset/alpha_rowset_writer.h /^ bool _need_column_data_writer;$/;" m class:doris::AlphaRowsetWriter +_need_delete_file src/olap/rowset/rowset.h /^ bool _need_delete_file = false;$/;" m class:doris::Rowset +_need_eval_predicates src/olap/rowset/column_data.h /^ bool _need_eval_predicates = false;$/;" m class:doris::ColumnData +_need_gen_rollup src/exec/tablet_sink.h /^ bool _need_gen_rollup = false;$/;" m class:doris::stream_load::OlapTableSink +_need_report_disk_stat src/olap/storage_engine.h /^ bool _need_report_disk_stat = false;$/;" m class:doris::StorageEngine +_need_report_tablet src/olap/storage_engine.h /^ bool _need_report_tablet = false;$/;" m class:doris::StorageEngine +_need_to_return src/runtime/row_batch.h /^ bool _need_to_return;$/;" m class:doris::RowBatch +_need_to_seek_block src/olap/rowset/segment_reader.h /^ bool _need_to_seek_block = true;$/;" m class:doris::SegmentReader +_need_validate_data src/exec/tablet_sink.h /^ bool _need_validate_data = false;$/;" m class:doris::stream_load::OlapTableSink +_needs_deep_copy src/runtime/row_batch.h /^ bool _needs_deep_copy;$/;" m class:doris::RowBatch +_needs_finalize src/exec/aggregation_node.h /^ bool _needs_finalize;$/;" m class:doris::AggregationNode +_needs_finalize src/exec/partitioned_aggregation_node.h /^ const bool _needs_finalize;$/;" m class:doris::PartitionedAggregationNode +_needs_serialize src/exec/partitioned_aggregation_node.h /^ bool _needs_serialize;$/;" m class:doris::PartitionedAggregationNode +_net_metrics src/util/system_metrics.h /^ std::map _net_metrics;$/;" m class:doris::SystemMetrics +_netmask src/util/cidr.h /^ uint32_t _netmask;$/;" m class:doris::CIDR +_network_client src/exec/es/es_scan_reader.h /^ HttpClient _network_client;$/;" m class:doris::ESScanReader +_network_throughput src/runtime/data_stream_sender.h /^ RuntimeProfile::Counter* _network_throughput;$/;" m class:doris::DataStreamSender +_new_handle src/http/action/mini_load.cpp /^void MiniLoadAction::_new_handle(HttpRequest* req) {$/;" f class:doris::MiniLoadAction +_new_key_length src/olap/olap_index.h /^ size_t _new_key_length;$/;" m class:doris::MemIndex +_new_rowset src/olap/delta_writer.h /^ RowsetSharedPtr _new_rowset;$/;" m class:doris::DeltaWriter +_new_segment_created src/olap/rowset/column_data_writer.h /^ bool _new_segment_created;$/;" m class:doris::ColumnDataWriter +_new_segment_created src/olap/rowset/segment_group.h /^ bool _new_segment_created;$/;" m class:doris::SegmentGroup +_new_short_key_length src/olap/rowset/segment_group.h /^ size_t _new_short_key_length;$/;" m class:doris::SegmentGroup +_new_tablet src/olap/delta_writer.h /^ TabletSharedPtr _new_tablet;$/;" m class:doris::DeltaWriter +_newly_created_rowset_num src/olap/tablet.h /^ std::atomic _newly_created_rowset_num;$/;" m class:doris::Tablet +_next_block src/olap/push_handler.cpp /^OLAPStatus LzoBinaryReader::_next_block() {$/;" f class:doris::LzoBinaryReader +_next_block src/olap/rowset/alpha_rowset_reader.h /^ OLAPStatus (AlphaRowsetReader::*_next_block)(RowBlock** block) = nullptr;$/;" m class:doris::AlphaRowsetReader +_next_block src/olap/rowset/column_data.h /^ uint32_t _next_block;$/;" m class:doris::ColumnData +_next_block_id src/olap/rowset/segment_reader.h /^ int64_t _next_block_id = 0;$/;" m class:doris::SegmentReader +_next_block_index src/runtime/buffered_block_mgr2.h /^ int _next_block_index;$/;" m class:doris::BufferedBlockMgr2 +_next_column_unique_id src/olap/tablet_schema.h /^ size_t _next_column_unique_id;$/;" m class:doris::TabletSchema +_next_delete_flag src/olap/reader.h /^ bool _next_delete_flag;$/;" m class:doris::Reader +_next_id src/util/core_local.h /^ int _next_id = 0;$/;" m class:doris::CoreLocalValueController +_next_idx src/exec/hash_table.h /^ int64_t _next_idx; \/\/ chain to next node for collisions$/;" m struct:doris::HashTable::Node +_next_instance_id test/runtime/data_stream_test.cpp /^ TUniqueId _next_instance_id;$/;" m class:doris::DataStreamTest file: +_next_iter src/runtime/qsorter.h /^ std::vector::iterator _next_iter;$/;" m class:doris::QSorter +_next_key src/olap/reader.h /^ const RowCursor* _next_key;$/;" m class:doris::Reader +_next_key_index src/olap/reader.h /^ int32_t _next_key_index;$/;" m class:doris::Reader +_next_node src/exec/partitioned_hash_table.h /^ DuplicateNode* _next_node;$/;" m class:doris::PartitionedHashTable +_next_packet_seq src/exec/tablet_sink.h /^ int64_t _next_packet_seq = 0;$/;" m class:doris::stream_load::NodeChannel +_next_ptr src/olap/rowset/segment_v2/binary_prefix_page.h /^ const uint8_t* _next_ptr = nullptr;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +_next_range src/exec/broker_scanner.h /^ int _next_range;$/;" m class:doris::BrokerScanner +_next_range src/exec/es_http_scanner.h /^ int _next_range;$/;" m class:doris::EsHttpScanner +_next_range src/exec/orc_scanner.h /^ int _next_range;$/;" m class:doris::ORCScanner +_next_range src/exec/parquet_scanner.h /^ int _next_range;$/;" m class:doris::ParquetScanner +_next_row src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::_next_row(const RowCursor** row, bool without_filter) {$/;" f class:doris::ColumnData +_next_row_func src/olap/reader.h /^ OLAPStatus (Reader::*_next_row_func)(RowCursor* row_cursor, MemPool* mem_pool, ObjectPool* agg_pool, bool* eof) = nullptr;$/;" m class:doris::Reader +_next_row_idx src/exec/exchange_node.h /^ int _next_row_idx;$/;" m class:doris::ExchangeNode +_next_row_start src/olap/push_handler.h /^ size_t _next_row_start;$/;" m class:doris::LzoBinaryReader +_next_rowid src/olap/rowset/segment_v2/column_writer.h /^ ordinal_t _next_rowid = 0;$/;" m class:doris::segment_v2::ColumnWriter +_next_scan_range_to_start src/runtime/disk_io_mgr_internal.h /^ ScanRange* _next_scan_range_to_start;$/;" m class:doris::DiskIoMgr::RequestContext::PerDiskState +_next_scan_token_idx src/exec/kudu_scan_node.h /^ int _next_scan_token_idx;$/;" m class:doris::KuduScanNode +_next_scroll_url src/exec/es/es_scan_reader.h /^ std::string _next_scroll_url;$/;" m class:doris::ESScanReader +_next_seqs src/runtime/tablets_channel.h /^ std::vector _next_seqs;$/;" m class:doris::TabletsChannel +_next_shard src/runtime/load_path_mgr.h /^ uint32_t _next_shard;$/;" m class:doris::LoadPathMgr +_next_slot_id src/runtime/descriptor_helper.h /^ TSlotId _next_slot_id = 0;$/;" m class:doris::TDescriptorTableBuilder +_next_tuple_id src/runtime/descriptor_helper.h /^ TTupleId _next_tuple_id = 0;$/;" m class:doris::TDescriptorTableBuilder +_next_val test/runtime/data_stream_test.cpp /^ int _next_val;$/;" m class:doris::DataStreamTest file: +_nice src/exec/olap_scan_node.h /^ int _nice;$/;" m class:doris::OlapScanNode +_no_nulls src/runtime/vectorized_row_batch.h /^ bool _no_nulls = false;$/;" m class:doris::ColumnVector +_no_string_slots src/runtime/descriptors.h /^ std::vector _no_string_slots;$/;" m class:doris::TupleDescriptor +_no_threads_cond src/util/threadpool.h /^ ConditionVariable _no_threads_cond;$/;" m class:doris::ThreadPool +_node src/exec/partitioned_hash_table.h /^ DuplicateNode* _node;$/;" m class:doris::PartitionedHashTable::Iterator +_node_add_batch_counter_map src/exec/tablet_sink.h /^ std::unordered_map _node_add_batch_counter_map;$/;" m class:doris::stream_load::OlapTableSink +_node_byte_size src/exec/hash_table.h /^ const int _node_byte_size;$/;" m class:doris::HashTable +_node_channels src/exec/tablet_sink.h /^ std::unordered_map _node_channels;$/;" m class:doris::stream_load::IndexChannel +_node_id src/exec/tablet_sink.h /^ int64_t _node_id = -1;$/;" m class:doris::stream_load::NodeChannel +_node_idx src/exec/hash_table.h /^ int64_t _node_idx;$/;" m class:doris::HashTable::Iterator +_node_idx src/exec/hash_table.h /^ int64_t _node_idx;$/;" m struct:doris::HashTable::Bucket +_node_info src/exec/tablet_sink.h /^ const NodeInfo* _node_info = nullptr;$/;" m class:doris::stream_load::NodeChannel +_node_name src/exec/blocking_join_node.h /^ const std::string _node_name;$/;" m class:doris::BlockingJoinNode +_node_remaining_current_page src/exec/partitioned_hash_table.h /^ int _node_remaining_current_page;$/;" m class:doris::PartitionedHashTable +_node_type src/exprs/expr.h /^ TExprNodeType::type _node_type;$/;" m class:doris::Expr +_nodes src/exec/hash_table.h /^ uint8_t* _nodes;$/;" m class:doris::HashTable +_nodes src/exec/tablet_info.h /^ std::unordered_map _nodes;$/;" m class:doris::DorisNodesInfo +_nodes_capacity src/exec/hash_table.h /^ int64_t _nodes_capacity;$/;" m class:doris::HashTable +_nodes_info src/exec/tablet_sink.h /^ DorisNodesInfo* _nodes_info = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_non_local_outstanding_writes src/runtime/buffered_block_mgr2.h /^ int _non_local_outstanding_writes;$/;" m class:doris::BufferedBlockMgr2 +_non_predicate_columns src/olap/rowset/segment_v2/segment_iterator.h /^ std::vector _non_predicate_columns;$/;" m class:doris::segment_v2::SegmentIterator +_normal_next src/olap/reader.cpp /^inline OLAPStatus CollectIterator::_normal_next(const RowCursor** row, bool* delete_flag) {$/;" f class:doris::CollectIterator +_normal_row_number src/runtime/runtime_state.h /^ int64_t _normal_row_number;$/;" m class:doris::RuntimeState +_not_running_cond src/util/threadpool.h /^ ConditionVariable _not_running_cond;$/;" m class:doris::ThreadPoolToken +_nrows src/olap/column_block.h /^ size_t _nrows;$/;" m class:doris::ColumnBlock +_nthreads test/util/blocking_queue_test.cpp /^ int _nthreads;$/;" m class:doris::MultiThreadTest file: +_null_bitmap src/olap/column_block.h /^ uint8_t* _null_bitmap;$/;" m class:doris::ColumnBlock +_null_bitmap src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^ Roaring _null_bitmap;$/;" m class:doris::segment_v2::__anon54::BitmapIndexWriterImpl file: +_null_bitmap_builder src/olap/rowset/segment_v2/column_writer.h /^ std::unique_ptr _null_bitmap_builder;$/;" m class:doris::segment_v2::ColumnWriter +_null_in_set src/exprs/in_predicate.h /^ bool _null_in_set;$/;" m class:doris::InPredicate +_null_indicator_offset src/exprs/slot_ref.h /^ NullIndicatorOffset _null_indicator_offset; \/\/ within tuple$/;" m class:doris::SlotRef +_null_indicator_offset src/runtime/descriptors.h /^ const NullIndicatorOffset _null_indicator_offset;$/;" m class:doris::SlotDescriptor +_null_indicators_read_block src/runtime/buffered_tuple_stream.h /^ uint32_t _null_indicators_read_block;$/;" m class:doris::BufferedTupleStream +_null_indicators_read_block src/runtime/buffered_tuple_stream2.h /^ uint32_t _null_indicators_read_block;$/;" m class:doris::BufferedTupleStream2 +_null_indicators_write_block src/runtime/buffered_tuple_stream.h /^ uint32_t _null_indicators_write_block;$/;" m class:doris::BufferedTupleStream +_null_indicators_write_block src/runtime/buffered_tuple_stream2.h /^ uint32_t _null_indicators_write_block;$/;" m class:doris::BufferedTupleStream2 +_null_supported src/olap/row_block.h /^ bool _null_supported;$/;" m class:doris::RowBlock +_null_supported src/olap/rowset/segment_reader.h /^ bool _null_supported;$/;" m class:doris::SegmentReader +_null_supported src/olap/stream_index_common.h /^ bool _null_supported;$/;" m class:doris::ColumnStatistics +_null_supported src/olap/stream_index_reader.h /^ bool _null_supported;$/;" m class:doris::StreamIndexReader +_nullable_tuple src/runtime/buffered_tuple_stream.h /^ const bool _nullable_tuple;$/;" m class:doris::BufferedTupleStream +_nullable_tuple src/runtime/buffered_tuple_stream2.h /^ const bool _nullable_tuple;$/;" m class:doris::BufferedTupleStream2 +_nulls src/olap/tuple.h /^ std::vector _nulls;$/;" m class:doris::OlapTuple +_nulls_first src/exec/exchange_node.h /^ std::vector _nulls_first;$/;" m class:doris::ExchangeNode +_nulls_first src/exec/sort_node.h /^ std::vector _nulls_first;$/;" m class:doris::SortNode +_nulls_first src/exec/spill_sort_node.h /^ std::vector _nulls_first;$/;" m class:doris::SpillSortNode +_nulls_first src/exec/topn_node.h /^ std::vector _nulls_first;$/;" m class:doris::TopNNode +_nulls_first src/util/tuple_row_compare.h /^ std::vector _nulls_first;$/;" m class:doris::TupleRowComparator +_nulls_first test/runtime/buffered_tuple_stream_test.cpp /^ std::vector _nulls_first;$/;" m class:doris::BufferedTupleStreamTest file: +_nulls_first test/runtime/sorter_test.cpp /^ std::vector _nulls_first;$/;" m class:doris::SorterTest file: +_num_active_scanners src/exec/kudu_scan_node.h /^ int _num_active_scanners;$/;" m class:doris::KuduScanNode +_num_active_scratch_dirs_metric src/runtime/tmp_file_mgr.h /^ std::unique_ptr _num_active_scratch_dirs_metric;$/;" m class:doris::TmpFileMgr +_num_allocated_buffers src/runtime/disk_io_mgr.h /^ AtomicInt _num_allocated_buffers;$/;" m class:doris::DiskIoMgr +_num_buckets src/exec/hash_table.h /^ int64_t _num_buckets;$/;" m class:doris::HashTable +_num_buckets src/exec/partitioned_hash_table.h /^ int64_t _num_buckets;$/;" m class:doris::PartitionedHashTable +_num_buckets_till_resize src/exec/hash_table.h /^ int64_t _num_buckets_till_resize;$/;" m class:doris::HashTable +_num_buckets_with_duplicates src/exec/partitioned_hash_table.h /^ int64_t _num_buckets_with_duplicates;$/;" m class:doris::PartitionedHashTable +_num_buffered_bytes src/runtime/data_stream_recvr.h /^ AtomicInt _num_buffered_bytes;$/;" m class:doris::DataStreamRecvr +_num_buffers_in_reader src/runtime/disk_io_mgr_internal.h /^ AtomicInt _num_buffers_in_reader;$/;" m class:doris::DiskIoMgr::RequestContext +_num_buffers_in_readers src/runtime/disk_io_mgr.h /^ AtomicInt _num_buffers_in_readers;$/;" m class:doris::DiskIoMgr +_num_build_tuples src/exec/hash_table.h /^ const int _num_build_tuples;$/;" m class:doris::HashTable +_num_build_tuples src/exec/partitioned_hash_table.h /^ const int _num_build_tuples;$/;" m class:doris::PartitionedHashTable +_num_bytes src/olap/rowset/segment_v2/bloom_filter.h /^ uint32_t _num_bytes;$/;" m class:doris::segment_v2::BloomFilter +_num_clustering_cols src/runtime/descriptors.h /^ int _num_clustering_cols;$/;" m class:doris::TableDescriptor +_num_cols src/runtime/descriptors.h /^ int _num_cols;$/;" m class:doris::TableDescriptor +_num_columns src/olap/tablet_schema.h /^ size_t _num_columns;$/;" m class:doris::TabletSchema +_num_complete src/util/progress_updater.h /^ int64_t _num_complete;$/;" m class:doris::ProgressUpdater +_num_data_bytes_sent src/runtime/data_stream_sender.cpp /^ int64_t _num_data_bytes_sent;$/;" m class:doris::DataStreamSender::Channel file: +_num_data_pages src/olap/rowset/segment_v2/indexed_column_writer.h /^ uint32_t _num_data_pages;$/;" m class:doris::segment_v2::IndexedColumnWriter +_num_disks_accessed_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* _num_disks_accessed_counter;$/;" m class:doris::ScanNode +_num_disks_with_ranges src/runtime/disk_io_mgr_internal.h /^ int _num_disks_with_ranges;$/;" m class:doris::DiskIoMgr::RequestContext +_num_duplicate_nodes src/exec/partitioned_hash_table.h /^ int64_t _num_duplicate_nodes;$/;" m class:doris::PartitionedHashTable +_num_element_after_padding src/olap/rowset/segment_v2/bitshuffle_page.h /^ size_t _num_element_after_padding;$/;" m class:doris::segment_v2::BitShufflePageDecoder +_num_elements src/olap/rowset/segment_v2/bitshuffle_page.h /^ size_t _num_elements;$/;" m class:doris::segment_v2::BitShufflePageDecoder +_num_elements src/olap/rowset/segment_v2/frame_of_reference_page.h /^ uint32_t _num_elements;$/;" m class:doris::segment_v2::FrameOfReferencePageDecoder +_num_elements src/olap/rowset/segment_v2/rle_page.h /^ uint32_t _num_elements;$/;" m class:doris::segment_v2::RlePageDecoder +_num_elems src/olap/rowset/segment_v2/binary_plain_page.h /^ uint32_t _num_elems;$/;" m class:doris::segment_v2::BinaryPlainPageDecoder +_num_elems src/olap/rowset/segment_v2/plain_page.h /^ uint32_t _num_elems;$/;" m class:doris::segment_v2::PlainPageDecoder +_num_entries src/olap/olap_index.h /^ size_t _num_entries;$/;" m class:doris::MemIndex +_num_failed_channels src/exec/tablet_sink.h /^ int _num_failed_channels = 0;$/;" m class:doris::stream_load::IndexChannel +_num_failed_probes src/exec/partitioned_hash_table.h /^ int64_t _num_failed_probes;$/;" m class:doris::PartitionedHashTable +_num_filled_buckets src/exec/hash_table.h /^ int64_t _num_filled_buckets;$/;" m class:doris::HashTable +_num_filled_buckets src/exec/partitioned_hash_table.h /^ int64_t _num_filled_buckets;$/;" m class:doris::PartitionedHashTable +_num_finished_ranges src/runtime/disk_io_mgr_internal.h /^ AtomicInt _num_finished_ranges;$/;" m class:doris::DiskIoMgr::RequestContext +_num_gcs_metric src/runtime/mem_tracker.h /^ IntCounter* _num_gcs_metric;$/;" m class:doris::MemTracker +_num_hash_buckets src/exec/partitioned_aggregation_node.h /^ RuntimeProfile::Counter* _num_hash_buckets;$/;" m class:doris::PartitionedAggregationNode +_num_hash_collisions src/exec/partitioned_hash_table.h /^ int64_t _num_hash_collisions;$/;" m class:doris::PartitionedHashTable +_num_input_values output/udf/include/uda_test_harness.h /^ int _num_input_values;$/;" m class:doris_udf::UdaTestHarnessBase +_num_input_values src/udf/uda_test_harness.h /^ int _num_input_values;$/;" m class:doris_udf::UdaTestHarnessBase +_num_inserters test/util/blocking_queue_test.cpp /^ int _num_inserters;$/;" m class:doris::MultiThreadTest file: +_num_items src/olap/short_key_index.h /^ uint32_t _num_items;$/;" m class:doris::ShortKeyIndexBuilder +_num_key_columns src/olap/schema.h /^ size_t _num_key_columns;$/;" m class:doris::Schema +_num_key_columns src/olap/tablet_schema.h /^ size_t _num_key_columns;$/;" m class:doris::TabletSchema +_num_literals src/olap/rowset/run_length_byte_reader.h /^ int32_t _num_literals;$/;" m class:doris::RunLengthByteReader +_num_literals src/olap/rowset/run_length_byte_writer.h /^ int32_t _num_literals;$/;" m class:doris::RunLengthByteWriter +_num_literals src/olap/rowset/run_length_integer_reader.h /^ int32_t _num_literals;$/;" m class:doris::RunLengthIntegerReader +_num_literals src/olap/rowset/run_length_integer_writer.h /^ uint16_t _num_literals;$/;" m class:doris::RunLengthIntegerWriter +_num_materialized_slots src/runtime/descriptors.h /^ int _num_materialized_slots;$/;" m class:doris::TupleDescriptor +_num_merges_counter src/runtime/merge_sorter.h /^ RuntimeProfile::Counter* _num_merges_counter;$/;" m class:doris::MergeSorter +_num_merges_counter src/runtime/spill_sorter.h /^ RuntimeProfile::Counter* _num_merges_counter;$/;" m class:doris::SpillSorter +_num_nodes src/exec/hash_table.h /^ int64_t _num_nodes;$/;" m class:doris::HashTable +_num_null_bytes src/runtime/descriptors.h /^ int _num_null_bytes;$/;" m class:doris::RowDescriptor +_num_null_bytes src/runtime/descriptors.h /^ int _num_null_bytes;$/;" m class:doris::TupleDescriptor +_num_null_bytes src/runtime/dpp_writer.h /^ int _num_null_bytes;$/;" m class:doris::DppWriter +_num_null_columns src/olap/tablet_schema.h /^ size_t _num_null_columns;$/;" m class:doris::TabletSchema +_num_null_slots src/runtime/descriptors.h /^ int _num_null_slots;$/;" m class:doris::RowDescriptor +_num_null_slots src/runtime/descriptors.h /^ int _num_null_slots;$/;" m class:doris::TupleDescriptor +_num_null_slots src/runtime/dpp_writer.h /^ int _num_null_slots;$/;" m class:doris::DppWriter +_num_of_columns_from_file src/exec/orc_scanner.h /^ int _num_of_columns_from_file;$/;" m class:doris::ORCScanner +_num_of_columns_from_file src/exec/parquet_reader.h /^ const int32_t _num_of_columns_from_file;$/;" m class:doris::ParquetReaderWrap +_num_pages src/olap/rowset/segment_v2/ordinal_page_index.h /^ int _num_pages = 0;$/;" m class:doris::segment_v2::OrdinalIndexReader +_num_per_fragment_instances src/runtime/runtime_state.h /^ int _num_per_fragment_instances = 0;$/;" m class:doris::RuntimeState +_num_pinned src/runtime/buffered_tuple_stream2.h /^ int _num_pinned;$/;" m class:doris::BufferedTupleStream2 +_num_pinned_buffers src/runtime/buffered_block_mgr2.cc /^ int _num_pinned_buffers;$/;" m class:doris::BufferedBlockMgr2::Client file: +_num_print_error_rows src/runtime/runtime_state.h /^ std::atomic _num_print_error_rows;$/;" m class:doris::RuntimeState +_num_probes src/exec/partitioned_hash_table.h /^ int64_t _num_probes;$/;" m class:doris::PartitionedHashTable +_num_ranges_written test/runtime/disk_io_mgr_test.cpp /^ int _num_ranges_written;$/;" m class:doris::DiskIoMgrTest file: +_num_ready_buffers src/runtime/disk_io_mgr_internal.h /^ AtomicInt _num_ready_buffers;$/;" m class:doris::DiskIoMgr::RequestContext +_num_remaining_ranges src/runtime/disk_io_mgr_internal.h /^ int _num_remaining_ranges;$/;" m class:doris::DiskIoMgr::RequestContext::PerDiskState +_num_remaining_senders src/runtime/data_stream_recvr.cc /^ int _num_remaining_senders;$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_num_remaining_senders src/runtime/tablets_channel.h /^ int _num_remaining_senders = 0;$/;" m class:doris::TabletsChannel +_num_remote_ranges src/runtime/disk_io_mgr_internal.h /^ AtomicInt _num_remote_ranges;$/;" m class:doris::DiskIoMgr::RequestContext +_num_removes src/udf/udf_internal.h /^ int64_t _num_removes;$/;" m class:doris::FunctionContextImpl +_num_repartitions src/exec/partitioned_aggregation_node.h /^ RuntimeProfile::Counter* _num_repartitions;$/;" m class:doris::PartitionedAggregationNode +_num_repicas src/exec/tablet_sink.h /^ int _num_repicas = -1;$/;" m class:doris::stream_load::OlapTableSink +_num_reserved_buffers src/runtime/buffered_block_mgr2.cc /^ int _num_reserved_buffers;$/;" m class:doris::BufferedBlockMgr2::Client file: +_num_reserved_optional_threads src/runtime/thread_resource_mgr.h /^ int _num_reserved_optional_threads;$/;" m class:doris::ThreadResourceMgr::ResourcePool +_num_resizes src/exec/partitioned_hash_table.h /^ int64_t _num_resizes;$/;" m class:doris::PartitionedHashTable +_num_restarts src/olap/rowset/segment_v2/binary_prefix_page.h /^ uint32_t _num_restarts = 0;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +_num_row_repartitioned src/exec/partitioned_aggregation_node.h /^ RuntimeProfile::Counter* _num_row_repartitioned;$/;" m class:doris::PartitionedAggregationNode +_num_rows src/olap/generic_iterators.cpp /^ size_t _num_rows;$/;" m class:doris::AutoIncrementIterator file: +_num_rows src/olap/olap_index.h /^ size_t _num_rows;$/;" m class:doris::MemIndex +_num_rows src/olap/row_block2.h /^ size_t _num_rows;$/;" m class:doris::RowBlockV2 +_num_rows src/olap/rowset/column_data_writer.h /^ int64_t _num_rows;$/;" m class:doris::ColumnDataWriter +_num_rows src/olap/rowset/segment_v2/column_reader.h /^ uint64_t _num_rows;$/;" m class:doris::segment_v2::ColumnReader +_num_rows src/runtime/buffered_block_mgr.h /^ int _num_rows;$/;" m class:doris::BufferedBlockMgr::Block +_num_rows src/runtime/buffered_block_mgr2.h /^ int _num_rows;$/;" m class:doris::BufferedBlockMgr2::Block +_num_rows src/runtime/buffered_tuple_stream.h /^ int64_t _num_rows;$/;" m class:doris::BufferedTupleStream +_num_rows src/runtime/buffered_tuple_stream2.h /^ int64_t _num_rows;$/;" m class:doris::BufferedTupleStream2 +_num_rows src/runtime/row_batch.h /^ int _num_rows; \/\/ # of committed rows$/;" m class:doris::RowBatch +_num_rows_in_block src/olap/rowset/segment_reader.h /^ uint64_t _num_rows_in_block;$/;" m class:doris::SegmentReader +_num_rows_load_filtered src/exec/csv_scan_node.h /^ int64_t _num_rows_load_filtered = 0L;$/;" m class:doris::CsvScanNode +_num_rows_load_filtered src/runtime/runtime_state.h /^ std::atomic _num_rows_load_filtered; \/\/ unqualified rows$/;" m class:doris::RuntimeState +_num_rows_load_total src/exec/csv_scan_node.h /^ int64_t _num_rows_load_total = 0L;$/;" m class:doris::CsvScanNode +_num_rows_load_total src/runtime/runtime_state.h /^ std::atomic _num_rows_load_total; \/\/ total rows read from source$/;" m class:doris::RuntimeState +_num_rows_load_unselected src/runtime/runtime_state.h /^ std::atomic _num_rows_load_unselected; \/\/ rows filtered by predicates$/;" m class:doris::RuntimeState +_num_rows_per_block src/olap/rowset/column_data.h /^ size_t _num_rows_per_block;$/;" m class:doris::ColumnData +_num_rows_per_block src/olap/short_key_index.h /^ uint32_t _num_rows_per_block;$/;" m class:doris::ShortKeyIndexBuilder +_num_rows_per_row_block src/olap/rowset/alpha_rowset_reader.h /^ int _num_rows_per_row_block;$/;" m class:doris::AlphaRowsetReader +_num_rows_per_row_block src/olap/rowset/column_writer.h /^ size_t _num_rows_per_row_block;$/;" m class:doris::ColumnWriter +_num_rows_per_row_block src/olap/tablet_schema.h /^ size_t _num_rows_per_row_block;$/;" m class:doris::TabletSchema +_num_rows_pushed_cond_filtered src/exec/olap_scanner.h /^ int64_t _num_rows_pushed_cond_filtered = 0;$/;" m class:doris::OlapScanner +_num_rows_read src/exec/olap_scanner.h /^ int64_t _num_rows_read = 0;$/;" m class:doris::OlapScanner +_num_rows_returned src/exec/exec_node.h /^ int64_t _num_rows_returned;$/;" m class:doris::ExecNode +_num_rows_skipped src/exec/exchange_node.h /^ int64_t _num_rows_skipped;$/;" m class:doris::ExchangeNode +_num_rows_skipped src/exec/sort_node.h /^ int64_t _num_rows_skipped;$/;" m class:doris::SortNode +_num_rows_skipped src/exec/spill_sort_node.h /^ int64_t _num_rows_skipped;$/;" m class:doris::SpillSortNode +_num_rows_skipped src/exec/topn_node.h /^ int64_t _num_rows_skipped;$/;" m class:doris::TopNNode +_num_rows_written src/olap/rowset/alpha_rowset_writer.h /^ int64_t _num_rows_written;$/;" m class:doris::AlphaRowsetWriter +_num_rows_written src/olap/rowset/beta_rowset_writer.h /^ int64_t _num_rows_written;$/;" m class:doris::BetaRowsetWriter +_num_running_scanners src/exec/broker_scan_node.h /^ int _num_running_scanners;$/;" m class:doris::BrokerScanNode +_num_running_scanners src/exec/es_http_scan_node.h /^ int _num_running_scanners;$/;" m class:doris::EsHttpScanNode +_num_scanner_threads_started_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* _num_scanner_threads_started_counter;$/;" m class:doris::ScanNode +_num_segment src/olap/rowset/beta_rowset_writer.h /^ int _num_segment;$/;" m class:doris::BetaRowsetWriter +_num_segments src/olap/rowset/segment_group.h /^ int32_t _num_segments; \/\/ number of segments in this segmentgroup$/;" m class:doris::SegmentGroup +_num_senders src/exec/exchange_node.h /^ int _num_senders; \/\/ needed for _stream_recvr construction$/;" m class:doris::ExchangeNode +_num_senders src/exec/pl_task_root.h /^ int _num_senders; \/\/ needed for _stream_recvr construction$/;" m class:doris::PlTaskRoot +_num_senders src/exec/tablet_sink.h /^ int _num_senders = -1;$/;" m class:doris::stream_load::OlapTableSink +_num_short_key_columns src/olap/tablet_schema.h /^ size_t _num_short_key_columns;$/;" m class:doris::TabletSchema +_num_small_blocks src/runtime/buffered_tuple_stream.h /^ int _num_small_blocks;$/;" m class:doris::BufferedTupleStream +_num_small_blocks src/runtime/buffered_tuple_stream2.h /^ int _num_small_blocks;$/;" m class:doris::BufferedTupleStream2 +_num_spilled_partitions src/exec/partitioned_aggregation_node.h /^ RuntimeProfile::Counter* _num_spilled_partitions;$/;" m class:doris::PartitionedAggregationNode +_num_symbols src/util/bfd_parser.h /^ long _num_symbols;$/;" m class:doris::BfdParser +_num_threads src/runtime/thread_resource_mgr.h /^ int64_t _num_threads;$/;" m class:doris::ThreadResourceMgr::ResourcePool +_num_threads src/util/threadpool.h /^ int _num_threads;$/;" m class:doris::ThreadPool +_num_threads_in_op src/runtime/disk_io_mgr_internal.h /^ AtomicInt _num_threads_in_op;$/;" m class:doris::DiskIoMgr::RequestContext::PerDiskState +_num_threads_pending_start src/util/threadpool.h /^ int _num_threads_pending_start;$/;" m class:doris::ThreadPool +_num_threads_per_disk src/runtime/disk_io_mgr.h /^ const int _num_threads_per_disk;$/;" m class:doris::DiskIoMgr +_num_tmp_reserved_buffers src/runtime/buffered_block_mgr2.cc /^ int _num_tmp_reserved_buffers;$/;" m class:doris::BufferedBlockMgr2::Client file: +_num_tuples src/runtime/merge_sorter.cpp /^ int64_t _num_tuples;$/;" m class:doris::MergeSorter::Run file: +_num_tuples src/runtime/spill_sorter.cc /^ int64_t _num_tuples;$/;" m class:doris::SpillSorter::Run file: +_num_tuples_per_row src/runtime/row_batch.h /^ const int _num_tuples_per_row;$/;" m class:doris::RowBatch::Iterator +_num_tuples_per_row src/runtime/row_batch.h /^ int _num_tuples_per_row;$/;" m class:doris::RowBatch +_num_tuples_returned src/runtime/merge_sorter.cpp /^ int64_t _num_tuples_returned;$/;" m class:doris::MergeSorter::Run file: +_num_tuples_returned src/runtime/spill_sorter.cc /^ int64_t _num_tuples_returned;$/;" m class:doris::SpillSorter::Run file: +_num_unstarted_scan_ranges src/runtime/disk_io_mgr_internal.h /^ AtomicInt _num_unstarted_scan_ranges;$/;" m class:doris::DiskIoMgr::RequestContext +_num_updates src/udf/udf_internal.h /^ int64_t _num_updates;$/;" m class:doris::FunctionContextImpl +_num_used_buffers src/runtime/disk_io_mgr_internal.h /^ AtomicInt _num_used_buffers;$/;" m class:doris::DiskIoMgr::RequestContext +_num_values src/olap/rowset/segment_v2/binary_prefix_page.h /^ size_t _num_values = 0;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +_num_values src/olap/rowset/segment_v2/indexed_column_reader.h /^ int64_t _num_values = 0;$/;" m class:doris::segment_v2::IndexedColumnReader +_num_values src/olap/rowset/segment_v2/indexed_column_writer.h /^ ordinal_t _num_values;$/;" m class:doris::segment_v2::IndexedColumnWriter +_num_values src/olap/rowset/segment_v2/ordinal_page_index.h /^ ordinal_t _num_values;$/;" m class:doris::segment_v2::OrdinalIndexReader +_num_warnings src/udf/udf_internal.h /^ int64_t _num_warnings;$/;" m class:doris::FunctionContextImpl +_num_worker_threads src/util/thrift_server.h /^ int _num_worker_threads;$/;" m class:doris::ThriftServer +_num_workers src/http/ev_http_server.h /^ int _num_workers;$/;" m class:doris::EvHttpServer +_number_filtered_rows src/exec/tablet_sink.h /^ int64_t _number_filtered_rows = 0;$/;" m class:doris::stream_load::OlapTableSink +_number_input_rows src/exec/tablet_sink.h /^ int64_t _number_input_rows = 0;$/;" m class:doris::stream_load::OlapTableSink +_number_output_rows src/exec/tablet_sink.h /^ int64_t _number_output_rows = 0;$/;" m class:doris::stream_load::OlapTableSink +_obj_pool src/exec/tablet_info.h /^ ObjectPool _obj_pool;$/;" m class:doris::OlapTablePartitionParam +_obj_pool src/exec/tablet_info.h /^ mutable ObjectPool _obj_pool;$/;" m class:doris::OlapTableSchemaParam +_obj_pool src/runtime/buffered_block_mgr.h /^ ObjectPool _obj_pool;$/;" m class:doris::BufferedBlockMgr +_obj_pool src/runtime/buffered_block_mgr2.h /^ ObjectPool _obj_pool;$/;" m class:doris::BufferedBlockMgr2 +_obj_pool src/runtime/data_spliter.h /^ boost::scoped_ptr _obj_pool;$/;" m class:doris::DataSpliter +_obj_pool src/runtime/dpp_sink.cpp /^ ObjectPool* _obj_pool;$/;" m class:doris::Translator file: +_obj_pool src/runtime/memory_scratch_sink.h /^ ObjectPool* _obj_pool;$/;" m class:doris::MemoryScratchSink +_obj_pool src/runtime/merge_sorter.h /^ ObjectPool _obj_pool;$/;" m class:doris::MergeSorter +_obj_pool src/runtime/result_sink.h /^ ObjectPool* _obj_pool;$/;" m class:doris::ResultSink +_obj_pool src/runtime/runtime_state.h /^ std::shared_ptr _obj_pool;$/;" m class:doris::RuntimeState +_obj_pool src/runtime/spill_sorter.h /^ ObjectPool _obj_pool;$/;" m class:doris::SpillSorter +_obj_pool src/testutil/desc_tbl_builder.h /^ ObjectPool* _obj_pool;$/;" m class:doris::DescriptorTblBuilder +_obj_pool test/exec/broker_scan_node_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::BrokerScanNodeTest file: +_obj_pool test/exec/broker_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::BrokerScannerTest file: +_obj_pool test/exec/csv_scan_bench_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::CsvScanNodeBenchTest file: +_obj_pool test/exec/csv_scan_node_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::CsvScanNodeTest file: +_obj_pool test/exec/es_http_scan_node_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::EsHttpScanNodeTest file: +_obj_pool test/exec/es_predicate_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::EsPredicateTest file: +_obj_pool test/exec/es_scan_node_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::EsScanNodeTest file: +_obj_pool test/exec/mysql_scan_node_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::MysqlScanNodeTest file: +_obj_pool test/exec/new_olap_scan_node_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::TestOlapScanNode file: +_obj_pool test/exec/olap_scan_node_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::OlapScanNodeTest file: +_obj_pool test/exec/orc_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::OrcScannerTest file: +_obj_pool test/exec/parquet_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::ParquetSannerTest file: +_obj_pool test/exec/plain_text_line_reader_bzip_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::PlainTextLineReaderTest file: +_obj_pool test/exec/plain_text_line_reader_gzip_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::PlainTextLineReaderTest file: +_obj_pool test/exec/plain_text_line_reader_lz4frame_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::PlainTextLineReaderTest file: +_obj_pool test/exec/plain_text_line_reader_lzop_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::PlainTextLineReaderTest file: +_obj_pool test/exec/plain_text_line_reader_uncompressed_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::PlainTextLineReaderTest file: +_obj_pool test/exec/schema_scan_node_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaScanNodeTest file: +_obj_pool test/exec/schema_scanner/schema_authors_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaAuthorScannerTest file: +_obj_pool test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaCharsetsScannerTest file: +_obj_pool test/exec/schema_scanner/schema_collations_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaCollationsScannerTest file: +_obj_pool test/exec/schema_scanner/schema_columns_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaColumnsScannerTest file: +_obj_pool test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaCreateTableScannerTest file: +_obj_pool test/exec/schema_scanner/schema_engines_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaEnginesScannerTest file: +_obj_pool test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaOpenTablesScannerTest file: +_obj_pool test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaSchemataScannerTest file: +_obj_pool test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaTableNamesScannerTest file: +_obj_pool test/exec/schema_scanner/schema_tables_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaTablesScannerTest file: +_obj_pool test/exec/schema_scanner/schema_variables_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaVariablesScannerTest file: +_obj_pool test/exec/schema_scanner_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::SchemaScannerTest file: +_obj_pool test/exprs/in_predicate_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::InPredicateTest file: +_obj_pool test/olap/olap_reader_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::TestOLAPReaderColumn file: +_obj_pool test/olap/olap_reader_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::TestOLAPReaderColumnDeleteCondition file: +_obj_pool test/olap/olap_reader_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::TestOLAPReaderRow file: +_obj_pool test/olap/vectorized_olap_reader_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::TestVectorizedOLAPReader file: +_obj_pool test/runtime/data_spliter_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::DataSplitTest file: +_obj_pool test/runtime/data_stream_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::DataStreamTest file: +_obj_pool test/runtime/dpp_sink_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::DppSinkTest file: +_obj_pool test/runtime/memory_scratch_sink_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::MemoryScratchSinkTest file: +_obj_pool test/runtime/qsorter_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::QSorterTest file: +_obj_pool test/util/arrow/arrow_work_flow_test.cpp /^ ObjectPool _obj_pool;$/;" m class:doris::ArrowWorkFlowTest file: +_object_pool test/exprs/binary_predicate_test.cpp /^ ObjectPool* _object_pool;$/;" m class:doris::BinaryOpTest file: +_object_pool test/exprs/in_op_test.cpp /^ ObjectPool* _object_pool;$/;" m class:doris::InOpTest file: +_object_pool test/runtime/buffered_tuple_stream_test.cpp /^ ObjectPool *_object_pool;$/;" m class:doris::BufferedTupleStreamTest file: +_object_pool test/runtime/sorter_test.cpp /^ ObjectPool *_object_pool;$/;" m class:doris::SorterTest file: +_objects src/common/object_pool.h /^ ElementVector _objects;$/;" m class:doris::ObjectPool +_offset src/exec/exchange_node.h /^ int64_t _offset;$/;" m class:doris::ExchangeNode +_offset src/exec/olap_scan_node.h /^ int _offset;$/;" m class:doris::OlapScanNode::MergeComparison +_offset src/exec/sort_node.h /^ int64_t _offset;$/;" m class:doris::SortNode +_offset src/exec/spill_sort_node.h /^ int64_t _offset;$/;" m class:doris::SpillSortNode +_offset src/exec/topn_node.h /^ int64_t _offset;$/;" m class:doris::TopNNode +_offset src/olap/file_stream.h /^ size_t _offset; \/\/ start from where$/;" m class:doris::ReadOnlyFileStream::FileCursor +_offset src/olap/utils.h /^ iterator_offset_t _offset;$/;" m class:doris::BinarySearchIterator +_offset src/runtime/disk_io_mgr.h /^ int64_t _offset;$/;" m class:doris::DiskIoMgr::RequestRange +_offset_buf src/olap/short_key_index.h /^ faststring _offset_buf;$/;" m class:doris::ShortKeyIndexBuilder +_offsets src/exec/es_scan_node.h /^ std::vector _offsets;$/;" m class:doris::EsScanNode +_offsets src/olap/in_stream.h /^ std::vector _offsets;$/;" m class:doris::InStream +_offsets src/olap/rowset/segment_v2/binary_plain_page.h /^ std::vector _offsets;$/;" m class:doris::segment_v2::BinaryPlainPageBuilder +_offsets src/olap/short_key_index.h /^ std::vector _offsets;$/;" m class:doris::ShortKeyIndexDecoder +_offsets test/olap/column_reader_test.cpp /^ std::vector _offsets;$/;" m class:doris::TestColumn file: +_offsets test/olap/schema_change_test.cpp /^ std::vector _offsets;$/;" m class:doris::TestColumn file: +_offsets_pos src/olap/rowset/segment_v2/binary_plain_page.h /^ uint32_t _offsets_pos;$/;" m class:doris::segment_v2::BinaryPlainPageDecoder +_olap_data src/olap/rowset/column_data.h /^ ColumnData* _olap_data;$/;" m class:doris::ColumnDataComparator +_olap_engine src/agent/heartbeat_server.h /^ StorageEngine* _olap_engine;$/;" m class:doris::HeartbeatServer +_olap_filter src/exec/olap_scan_node.h /^ std::vector _olap_filter;$/;" m class:doris::OlapScanNode +_olap_scan_node src/exec/olap_scan_node.h /^ TOlapScanNode _olap_scan_node;$/;" m class:doris::OlapScanNode +_olap_scanners src/exec/olap_scan_node.h /^ std::list _olap_scanners;$/;" m class:doris::OlapScanNode +_on_chunk_data src/http/action/mini_load.cpp /^void MiniLoadAction::_on_chunk_data(HttpRequest* http_req) {$/;" f class:doris::MiniLoadAction +_on_header src/http/action/mini_load.cpp /^Status MiniLoadAction::_on_header(HttpRequest* req) {$/;" f class:doris::MiniLoadAction +_on_header src/http/action/stream_load.cpp /^Status StreamLoadAction::_on_header(HttpRequest* http_req, StreamLoadContext* ctx) {$/;" f class:doris::StreamLoadAction +_on_new_chunk_data src/http/action/mini_load.cpp /^void MiniLoadAction::_on_new_chunk_data(HttpRequest* http_req) {$/;" f class:doris::MiniLoadAction +_on_new_handle src/http/action/mini_load.cpp /^Status MiniLoadAction::_on_new_handle(StreamLoadContext* ctx) {$/;" f class:doris::MiniLoadAction +_on_new_header src/http/action/mini_load.cpp /^Status MiniLoadAction::_on_new_header(HttpRequest* req) {$/;" f class:doris::MiniLoadAction +_once_flag src/util/once.h /^ std::once_flag _once_flag;$/;" m class:doris::DorisCallOnce +_op src/exec/es/es_query_builder.h /^ TExprOpcode::type _op;$/;" m class:doris::RangeQueryBuilder +_opcode src/exprs/expr.h /^ TExprOpcode::type _opcode;$/;" m class:doris::Expr +_open src/olap/rowset/segment_v2/segment.cpp /^Status Segment::_open() {$/;" f class:doris::segment_v2::Segment +_open src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::_open() {$/;" f class:doris::StorageEngine +_open_all_writers src/runtime/tablets_channel.cpp /^Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& params) {$/;" f class:doris::TabletsChannel +_open_closure src/exec/tablet_sink.h /^ RefCountClosure* _open_closure = nullptr;$/;" m class:doris::stream_load::NodeChannel +_open_current_file src/util/zip_util.h /^ bool _open_current_file;$/;" m class:doris::ZipFile +_open_timer src/exec/tablet_sink.h /^ RuntimeProfile::Counter* _open_timer = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_opened src/exprs/expr_context.h /^ bool _opened;$/;" m class:doris::ExprContext +_opened src/runtime/load_channel.h /^ bool _opened = false;$/;" m class:doris::LoadChannel +_opened_clients src/runtime/client_cache.h /^ std::unique_ptr _opened_clients;$/;" m class:doris::ClientCacheHelper +_optimal_bit_num src/olap/bloom_filter.hpp /^ uint32_t _optimal_bit_num(int64_t n, double fpp) {$/;" f class:doris::BloomFilter +_optimal_bit_num src/olap/rowset/segment_v2/bloom_filter.cpp /^uint32_t BloomFilter::_optimal_bit_num(uint64_t n, double fpp) {$/;" f class:doris::segment_v2::BloomFilter +_optimal_hash_function_num src/olap/bloom_filter.hpp /^ uint32_t _optimal_hash_function_num(int64_t n, uint32_t m) {$/;" f class:doris::BloomFilter +_options src/exec/orc_scanner.h /^ orc::ReaderOptions _options;$/;" m class:doris::ORCScanner +_options src/olap/rowset/segment_v2/binary_dict_page.h /^ PageBuilderOptions _options;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_options src/olap/rowset/segment_v2/binary_dict_page.h /^ PageDecoderOptions _options;$/;" m class:doris::segment_v2::BinaryDictPageDecoder +_options src/olap/rowset/segment_v2/binary_plain_page.h /^ PageBuilderOptions _options;$/;" m class:doris::segment_v2::BinaryPlainPageBuilder +_options src/olap/rowset/segment_v2/binary_plain_page.h /^ PageDecoderOptions _options;$/;" m class:doris::segment_v2::BinaryPlainPageDecoder +_options src/olap/rowset/segment_v2/binary_prefix_page.h /^ PageBuilderOptions _options;$/;" m class:doris::segment_v2::BinaryPrefixPageBuilder +_options src/olap/rowset/segment_v2/bitshuffle_page.h /^ PageBuilderOptions _options;$/;" m class:doris::segment_v2::BitshufflePageBuilder +_options src/olap/rowset/segment_v2/bitshuffle_page.h /^ PageDecoderOptions _options;$/;" m class:doris::segment_v2::BitShufflePageDecoder +_options src/olap/rowset/segment_v2/frame_of_reference_page.h /^ PageBuilderOptions _options;$/;" m class:doris::segment_v2::FrameOfReferencePageBuilder +_options src/olap/rowset/segment_v2/indexed_column_writer.h /^ IndexedColumnWriterOptions _options;$/;" m class:doris::segment_v2::IndexedColumnWriter +_options src/olap/rowset/segment_v2/plain_page.h /^ PageBuilderOptions _options;$/;" m class:doris::segment_v2::PlainPageBuilder +_options src/olap/rowset/segment_v2/plain_page.h /^ PageDecoderOptions _options;$/;" m class:doris::segment_v2::PlainPageDecoder +_options src/olap/rowset/segment_v2/rle_page.h /^ PageBuilderOptions _options;$/;" m class:doris::segment_v2::RlePageBuilder +_options src/olap/rowset/segment_v2/rle_page.h /^ PageDecoderOptions _options;$/;" m class:doris::segment_v2::RlePageDecoder +_options src/olap/storage_engine.h /^ EngineOptions _options;$/;" m class:doris::StorageEngine +_options_handlers src/http/ev_http_server.h /^ PathTrie _options_handlers;$/;" m class:doris::EvHttpServer +_opts src/olap/fs/file_block_manager.h /^ const BlockManagerOptions _opts;$/;" m class:doris::fs::FileBlockManager +_opts src/olap/rowset/segment_v2/column_reader.h /^ ColumnIteratorOptions _opts;$/;" m class:doris::segment_v2::ColumnIterator +_opts src/olap/rowset/segment_v2/column_reader.h /^ ColumnReaderOptions _opts;$/;" m class:doris::segment_v2::ColumnReader +_opts src/olap/rowset/segment_v2/column_writer.h /^ ColumnWriterOptions _opts;$/;" m class:doris::segment_v2::ColumnWriter +_opts src/olap/rowset/segment_v2/segment_iterator.h /^ StorageReadOptions _opts;$/;" m class:doris::segment_v2::SegmentIterator +_opts src/olap/rowset/segment_v2/segment_writer.h /^ SegmentWriterOptions _opts;$/;" m class:doris::segment_v2::SegmentWriter +_order_by_eq_expr_ctx src/exec/analytic_eval_node.h /^ ExprContext* _order_by_eq_expr_ctx;$/;" m class:doris::AnalyticEvalNode +_order_expr test/runtime/qsorter_test.cpp /^ std::vector _order_expr;$/;" m class:doris::QSorterTest file: +_order_expr_ctxs src/runtime/qsorter.h /^ const std::vector& _order_expr_ctxs;$/;" m class:doris::QSorter +_ordering_exprs test/runtime/buffered_tuple_stream_test.cpp /^ std::vector _ordering_exprs; $/;" m class:doris::BufferedTupleStreamTest file: +_ordering_exprs test/runtime/sorter_test.cpp /^ std::vector _ordering_exprs; $/;" m class:doris::SorterTest file: +_ordinal src/olap/rowset/alpha_rowset_reader.h /^ size_t _ordinal;$/;" m class:doris::AlphaRowsetReader +_ordinal src/olap/short_key_index.h /^ ssize_t _ordinal;$/;" m class:doris::ShortKeyIndexIterator +_ordinal_index src/olap/rowset/segment_v2/column_reader.h /^ std::unique_ptr _ordinal_index;$/;" m class:doris::segment_v2::ColumnReader +_ordinal_index_builder src/olap/rowset/segment_v2/column_writer.h /^ std::unique_ptr _ordinal_index_builder;$/;" m class:doris::segment_v2::ColumnWriter +_ordinal_index_builder src/olap/rowset/segment_v2/indexed_column_writer.h /^ std::unique_ptr _ordinal_index_builder;$/;" m class:doris::segment_v2::IndexedColumnWriter +_ordinal_index_meta src/olap/rowset/segment_v2/column_reader.h /^ const OrdinalIndexPB* _ordinal_index_meta = nullptr;$/;" m class:doris::segment_v2::ColumnReader +_ordinal_index_page_handle src/olap/rowset/segment_v2/indexed_column_reader.h /^ PageHandle _ordinal_index_page_handle;$/;" m class:doris::segment_v2::IndexedColumnReader +_ordinal_index_reader src/olap/rowset/segment_v2/indexed_column_reader.h /^ IndexPageReader _ordinal_index_reader;$/;" m class:doris::segment_v2::IndexedColumnReader +_ordinal_iter src/olap/rowset/segment_v2/indexed_column_reader.h /^ IndexPageIterator _ordinal_iter;$/;" m class:doris::segment_v2::IndexedColumnIterator +_ordinals src/olap/rowset/segment_v2/ordinal_page_index.h /^ std::vector _ordinals;$/;" m class:doris::segment_v2::OrdinalIndexReader +_origin_iters src/olap/generic_iterators.cpp /^ std::vector _origin_iters;$/;" m class:doris::MergeIterator file: +_origin_iters src/olap/generic_iterators.cpp /^ std::vector _origin_iters;$/;" m class:doris::UnionIterator file: +_other_join_conjunct_ctxs src/exec/hash_join_node.h /^ std::vector _other_join_conjunct_ctxs;$/;" m class:doris::HashJoinNode +_other_join_conjunct_ctxs src/exec/merge_join_node.h /^ std::vector _other_join_conjunct_ctxs;$/;" m class:doris::MergeJoinNode +_out_batch src/exec/merge_join_node.h /^ RowBatch* _out_batch;$/;" m class:doris::MergeJoinNode +_out_buffer src/util/frame_of_reference_coding.h /^ std::vector _out_buffer; \/\/ store values of decoded frame$/;" m class:doris::ForDecoder +_out_stream test/olap/bit_field_test.cpp /^ OutStream* _out_stream;$/;" m class:doris::TestBitField file: +_out_stream test/olap/run_length_byte_test.cpp /^ OutStream* _out_stream;$/;" m class:doris::TestRunLengthByte file: +_out_stream test/olap/run_length_integer_test.cpp /^ OutStream* _out_stream;$/;" m class:doris::TestRunLengthSignInteger file: +_out_stream test/olap/run_length_integer_test.cpp /^ OutStream* _out_stream;$/;" m class:doris::TestRunLengthUnsignInteger file: +_output src/olap/rowset/bit_field_writer.h /^ OutStream* _output;$/;" m class:doris::BitFieldWriter +_output src/olap/rowset/run_length_byte_writer.h /^ OutStream* _output;$/;" m class:doris::RunLengthByteWriter +_output src/olap/rowset/run_length_integer_writer.h /^ OutStream* _output;$/;" m class:doris::RunLengthIntegerWriter +_output src/util/arrow/row_batch.cpp /^ std::shared_ptr _output;$/;" m class:doris::ToRowBatchConverter file: +_output src/util/arrow/row_block.cpp /^ std::shared_ptr _output;$/;" m class:doris::ToRowBlockConverter file: +_output_batch src/exec/tablet_sink.h /^ std::unique_ptr _output_batch;$/;" m class:doris::stream_load::OlapTableSink +_output_block src/olap/rowset/beta_rowset_reader.h /^ std::unique_ptr _output_block;$/;" m class:doris::BetaRowsetReader +_output_buf src/exec/plain_text_line_reader.h /^ uint8_t* _output_buf;$/;" m class:doris::PlainTextLineReader +_output_buf_limit src/exec/plain_text_line_reader.h /^ size_t _output_buf_limit;$/;" m class:doris::PlainTextLineReader +_output_buf_pos src/exec/plain_text_line_reader.h /^ size_t _output_buf_pos;$/;" m class:doris::PlainTextLineReader +_output_buf_size src/exec/plain_text_line_reader.h /^ size_t _output_buf_size;$/;" m class:doris::PlainTextLineReader +_output_buffers src/olap/out_stream.h /^ std::vector _output_buffers;\/\/ 缓冲所有的输出$/;" m class:doris::OutStream +_output_column src/exprs/expr.h /^ int _output_column;$/;" m class:doris::Expr +_output_compressed src/olap/out_stream.cpp /^void OutStream::_output_compressed() {$/;" f class:doris::OutStream +_output_expr_ctxs src/exec/tablet_sink.h /^ std::vector _output_expr_ctxs;$/;" m class:doris::stream_load::OlapTableSink +_output_expr_ctxs src/runtime/dpp_writer.h /^ const std::vector& _output_expr_ctxs;$/;" m class:doris::DppWriter +_output_expr_ctxs src/runtime/export_sink.h /^ std::vector _output_expr_ctxs;$/;" m class:doris::ExportSink +_output_expr_ctxs src/runtime/memory_scratch_sink.h /^ std::vector _output_expr_ctxs;$/;" m class:doris::MemoryScratchSink +_output_expr_ctxs src/runtime/mysql_table_sink.h /^ std::vector _output_expr_ctxs;$/;" m class:doris::MysqlTableSink +_output_expr_ctxs src/runtime/mysql_table_writer.h /^ const std::vector& _output_expr_ctxs;$/;" m class:doris::MysqlTableWriter +_output_expr_ctxs src/runtime/result_sink.h /^ std::vector _output_expr_ctxs;$/;" m class:doris::ResultSink +_output_expr_ctxs src/runtime/result_writer.h /^ const std::vector& _output_expr_ctxs;$/;" m class:doris::ResultWriter +_output_files src/runtime/runtime_state.h /^ std::vector _output_files;$/;" m class:doris::RuntimeState +_output_iterator src/exec/aggregation_node.h /^ HashTable::Iterator _output_iterator;$/;" m class:doris::AggregationNode +_output_iterator src/exec/partitioned_aggregation_node.h /^ PartitionedHashTable::Iterator _output_iterator;$/;" m class:doris::PartitionedAggregationNode +_output_iterator src/exec/pre_aggregation_node.h /^ HashTable::Iterator _output_iterator;$/;" m class:doris::PreAggregationNode +_output_partition src/exec/partitioned_aggregation_node.h /^ Partition* _output_partition;$/;" m class:doris::PartitionedAggregationNode +_output_path src/runtime/dpp_sink.cpp /^ std::string _output_path;$/;" m class:doris::Translator file: +_output_row_desc src/exec/tablet_sink.h /^ RowDescriptor* _output_row_desc = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_output_row_desc src/runtime/merge_sorter.h /^ RowDescriptor* _output_row_desc;$/;" m class:doris::MergeSorter +_output_row_desc src/runtime/spill_sorter.h /^ RowDescriptor* _output_row_desc;$/;" m class:doris::SpillSorter +_output_row_desc test/runtime/buffered_tuple_stream_test.cpp /^ RowDescriptor *_output_row_desc;$/;" m class:doris::BufferedTupleStreamTest file: +_output_row_desc test/runtime/sorter_test.cpp /^ RowDescriptor *_output_row_desc;$/;" m class:doris::SorterTest file: +_output_row_expr_ctxs src/runtime/dpp_sink.cpp /^ std::vector _output_row_expr_ctxs;$/;" m class:doris::Translator file: +_output_rows_counter src/exec/tablet_sink.h /^ RuntimeProfile::Counter* _output_rows_counter = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_output_rowset src/olap/compaction.h /^ RowsetSharedPtr _output_rowset;$/;" m class:doris::Compaction +_output_rs_writer src/olap/compaction.h /^ std::unique_ptr _output_rs_writer;$/;" m class:doris::Compaction +_output_scale src/exprs/expr.h /^ int _output_scale;$/;" m class:doris::Expr +_output_set test/exec/tablet_sink_test.cpp /^ std::set* _output_set;$/;" m class:doris::stream_load::TestInternalService file: +_output_slot_desc src/exprs/agg_fn_evaluator.h /^ const SlotDescriptor* _output_slot_desc;$/;" m class:doris::AggFnEvaluator +_output_tuple_desc src/exec/aggregation_node.h /^ TupleDescriptor* _output_tuple_desc;$/;" m class:doris::AggregationNode +_output_tuple_desc src/exec/olap_rewrite_node.h /^ TupleDescriptor* _output_tuple_desc;$/;" m class:doris::OlapRewriteNode +_output_tuple_desc src/exec/partitioned_aggregation_node.h /^ TupleDescriptor* _output_tuple_desc;$/;" m class:doris::PartitionedAggregationNode +_output_tuple_desc src/exec/tablet_sink.h /^ TupleDescriptor* _output_tuple_desc = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_output_tuple_id src/exec/aggregation_node.h /^ TupleId _output_tuple_id;$/;" m class:doris::AggregationNode +_output_tuple_id src/exec/olap_rewrite_node.h /^ TupleId _output_tuple_id;$/;" m class:doris::OlapRewriteNode +_output_tuple_id src/exec/partitioned_aggregation_node.h /^ TupleId _output_tuple_id;$/;" m class:doris::PartitionedAggregationNode +_output_tuple_id src/exec/repeat_node.h /^ TupleId _output_tuple_id;$/;" m class:doris::RepeatNode +_output_uncompress src/olap/out_stream.cpp /^void OutStream::_output_uncompress() {$/;" f class:doris::OutStream +_output_version src/olap/compaction.h /^ Version _output_version;$/;" m class:doris::Compaction +_output_version_hash src/olap/compaction.h /^ VersionHash _output_version_hash;$/;" m class:doris::Compaction +_outstanding_writes_counter src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _outstanding_writes_counter;$/;" m class:doris::BufferedBlockMgr2 +_overall_throughput src/runtime/data_stream_sender.h /^ RuntimeProfile::Counter* _overall_throughput;$/;" m class:doris::DataStreamSender +_overflow src/olap/out_stream.h /^ StorageByteBuffer* _overflow; \/\/ _output中放不下的字节$/;" m class:doris::OutStream +_own_pool src/util/runtime_profile.h /^ bool _own_pool;$/;" m class:doris::RuntimeProfile +_own_rs_readers src/olap/reader.h /^ std::vector _own_rs_readers;$/;" m class:doris::Reader +_owned_buf src/olap/wrapper_field.h /^ char* _owned_buf = nullptr;$/;" m class:doris::WrapperField +_owned_fixed_buf src/olap/row_cursor.h /^ char* _owned_fixed_buf = nullptr; \/\/ point to buf allocated in init function$/;" m class:doris::RowCursor +_owned_stats src/olap/rowset/alpha_rowset_reader.h /^ OlapReaderStatistics _owned_stats;$/;" m class:doris::AlphaRowsetReader +_owned_stats src/olap/rowset/beta_rowset_reader.h /^ OlapReaderStatistics _owned_stats;$/;" m class:doris::BetaRowsetReader +_packet_num src/runtime/buffer_control_block.h /^ int64_t _packet_num;$/;" m class:doris::BufferControlBlock +_packet_seq src/runtime/data_stream_sender.cpp /^ int64_t _packet_seq;$/;" m class:doris::DataStreamSender::Channel file: +_packet_seq_map src/runtime/data_stream_recvr.cc /^ std::unordered_map _packet_seq_map; \/\/ be_number => packet_seq$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_page src/olap/rowset/segment_v2/column_reader.h /^ std::unique_ptr _page;$/;" m class:doris::segment_v2::FileColumnIterator +_page_builder src/olap/rowset/segment_v2/column_writer.h /^ std::unique_ptr _page_builder;$/;" m class:doris::segment_v2::ColumnWriter +_page_builder src/olap/rowset/segment_v2/ordinal_page_index.h /^ std::unique_ptr _page_builder;$/;" m class:doris::segment_v2::OrdinalIndexWriter +_page_iter src/olap/rowset/segment_v2/column_reader.h /^ OrdinalPageIndexIterator _page_iter;$/;" m class:doris::segment_v2::FileColumnIterator +_page_map src/http/web_page_handler.h /^ PageHandlersMap _page_map;$/;" m class:doris::WebPageHandler +_page_zone_map src/olap/rowset/segment_v2/zone_map_index.h /^ ZoneMap _page_zone_map;$/;" m class:doris::segment_v2::ZoneMapIndexWriter +_page_zone_maps src/olap/rowset/segment_v2/zone_map_index.h /^ std::vector _page_zone_maps;$/;" m class:doris::segment_v2::ZoneMapIndexReader +_pages src/olap/rowset/segment_v2/column_writer.h /^ PageHead _pages;$/;" m class:doris::segment_v2::ColumnWriter +_pages src/olap/rowset/segment_v2/ordinal_page_index.h /^ std::vector _pages;$/;" m class:doris::segment_v2::OrdinalIndexReader +_param src/exec/schema_scanner.h /^ SchemaScannerParam* _param;$/;" m class:doris::SchemaScanner +_param test/exec/mysql_scanner_test.cpp /^ MysqlScannerParam _param;$/;" m class:doris::MysqlScannerTest file: +_param test/exec/schema_scanner/schema_authors_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaAuthorScannerTest file: +_param test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaCharsetsScannerTest file: +_param test/exec/schema_scanner/schema_collations_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaCollationsScannerTest file: +_param test/exec/schema_scanner/schema_columns_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaColumnsScannerTest file: +_param test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaCreateTableScannerTest file: +_param test/exec/schema_scanner/schema_engines_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaEnginesScannerTest file: +_param test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaOpenTablesScannerTest file: +_param test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaSchemataScannerTest file: +_param test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaTableNamesScannerTest file: +_param test/exec/schema_scanner/schema_tables_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaTablesScannerTest file: +_param test/exec/schema_scanner/schema_variables_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaVariablesScannerTest file: +_param test/exec/schema_scanner_test.cpp /^ SchemaScannerParam _param;$/;" m class:doris::SchemaScannerTest file: +_params src/exec/base_scanner.h /^ const TBrokerScanRangeParams& _params;$/;" m class:doris::BaseScanner +_params src/exec/olap_scanner.h /^ ReaderParams _params;$/;" m class:doris::OlapScanner +_params src/http/http_request.h /^ std::map _params;$/;" m class:doris::HttpRequest +_params test/exec/broker_scan_node_test.cpp /^ TBrokerScanRangeParams _params;$/;" m class:doris::BrokerScanNodeTest file: +_params test/exec/broker_scanner_test.cpp /^ TBrokerScanRangeParams _params;$/;" m class:doris::BrokerScannerTest file: +_params test/exec/parquet_scanner_test.cpp /^ TBrokerScanRangeParams _params;$/;" m class:doris::ParquetSannerTest file: +_parent src/exec/olap_scanner.h /^ OlapScanNode* _parent;$/;" m class:doris::OlapScanner +_parent src/exec/tablet_sink.h /^ OlapTableSink* _parent = nullptr;$/;" m class:doris::stream_load::NodeChannel +_parent src/exec/tablet_sink.h /^ OlapTableSink* _parent;$/;" m class:doris::stream_load::IndexChannel +_parent src/runtime/data_stream_sender.cpp /^ DataStreamSender* _parent;$/;" m class:doris::DataStreamSender::Channel file: +_parent src/runtime/descriptors.h /^ const TupleId _parent;$/;" m class:doris::SlotDescriptor +_parent src/runtime/disk_io_mgr_internal.h /^ DiskIoMgr* _parent;$/;" m class:doris::DiskIoMgr::RequestContext +_parent src/runtime/mem_tracker.h /^ MemTracker* _parent;$/;" m class:doris::MemTracker +_parent src/runtime/merge_sorter.cpp /^ TupleSorter* _parent;$/;" m class:doris::MergeSorter::TupleSorter::TupleIterator file: +_parent src/runtime/row_batch.h /^ RowBatch* const _parent;$/;" m class:doris::RowBatch::Iterator +_parent src/runtime/sorted_run_merger.cc /^ SortedRunMerger* _parent;$/;" m class:doris::SortedRunMerger::BatchedRowSupplier file: +_parent src/runtime/spill_sorter.cc /^ TupleSorter* _parent;$/;" m class:doris::SpillSorter::TupleSorter::TupleIterator file: +_parent src/runtime/thread_resource_mgr.h /^ ThreadResourceMgr* _parent;$/;" m class:doris::ThreadResourceMgr::ResourcePool +_parquet src/exec/parquet_reader.h /^ std::shared_ptr _parquet;$/;" m class:doris::ParquetReaderWrap +_parquet_column_ids src/exec/parquet_reader.h /^ std::vector _parquet_column_ids;$/;" m class:doris::ParquetReaderWrap +_parquet_column_type src/exec/parquet_reader.h /^ std::vector _parquet_column_type;$/;" m class:doris::ParquetReaderWrap +_parse_condition src/olap/delete_handler.cpp /^bool DeleteHandler::_parse_condition(const std::string& condition_str, TCondition* condition) {$/;" f class:doris::DeleteHandler +_parse_default_rowset_type src/olap/storage_engine.cpp /^void StorageEngine::_parse_default_rowset_type() {$/;" f class:doris::StorageEngine +_parse_footer src/olap/rowset/segment_v2/segment.cpp /^Status Segment::_parse_footer() {$/;" f class:doris::segment_v2::Segment +_parse_header src/olap/stream_index_reader.cpp /^OLAPStatus StreamIndexReader::_parse_header(FieldType type) {$/;" f class:doris::StreamIndexReader +_parse_request src/olap/schema_change.cpp /^OLAPStatus SchemaChangeHandler::_parse_request(TabletSharedPtr base_tablet,$/;" f class:doris::SchemaChangeHandler +_parse_to_predicate src/olap/reader.cpp /^ColumnPredicate* Reader::_parse_to_predicate(const TCondition& condition) {$/;" f class:doris::Reader +_parsed src/olap/rowset/segment_v2/binary_dict_page.h /^ bool _parsed;$/;" m class:doris::segment_v2::BinaryDictPageDecoder +_parsed src/olap/rowset/segment_v2/binary_plain_page.h /^ bool _parsed;$/;" m class:doris::segment_v2::BinaryPlainPageDecoder +_parsed src/olap/rowset/segment_v2/binary_prefix_page.h /^ bool _parsed = false;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +_parsed src/olap/rowset/segment_v2/bitshuffle_page.h /^ bool _parsed;$/;" m class:doris::segment_v2::BitShufflePageDecoder +_parsed src/olap/rowset/segment_v2/frame_of_reference_page.h /^ bool _parsed;$/;" m class:doris::segment_v2::FrameOfReferencePageDecoder +_parsed src/olap/rowset/segment_v2/index_page.h /^ bool _parsed;$/;" m class:doris::segment_v2::IndexPageReader +_parsed src/olap/rowset/segment_v2/plain_page.h /^ bool _parsed;$/;" m class:doris::segment_v2::PlainPageDecoder +_parsed src/olap/rowset/segment_v2/rle_page.h /^ bool _parsed;$/;" m class:doris::segment_v2::RlePageDecoder +_parsed src/olap/short_key_index.h /^ bool _parsed;$/;" m class:doris::ShortKeyIndexDecoder +_parsed src/util/frame_of_reference_coding.h /^ bool _parsed = false;$/;" m class:doris::ForDecoder +_parser src/http/action/pprof_actions.cpp /^ BfdParser* _parser;$/;" m class:doris::SymbolAction file: +_part_contains src/exec/tablet_info.h /^ bool _part_contains(OlapTablePartition* part, Tuple* key) const {$/;" f class:doris::OlapTablePartitionParam +_part_type src/runtime/data_stream_sender.h /^ TPartitionType::type _part_type;$/;" m class:doris::DataStreamSender +_partition src/exec/tablet_sink.h /^ OlapTablePartitionParam* _partition = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_partition_by_eq_expr_ctx src/exec/analytic_eval_node.h /^ ExprContext* _partition_by_eq_expr_ctx;$/;" m class:doris::AnalyticEvalNode +_partition_expr_ctxs src/exec/broker_scan_node.h /^ std::vector _partition_expr_ctxs;$/;" m class:doris::BrokerScanNode +_partition_expr_ctxs src/runtime/data_spliter.h /^ std::vector _partition_expr_ctxs;$/;" m class:doris::DataSpliter +_partition_expr_ctxs src/runtime/data_stream_sender.h /^ std::vector _partition_expr_ctxs; \/\/ compute per-row partition values$/;" m class:doris::DataStreamSender +_partition_id src/olap/rowset/segment_group.h /^ TPartitionId _partition_id;$/;" m class:doris::SegmentGroup +_partition_id src/olap/tablet_meta.h /^ int64_t _partition_id = 0;$/;" m class:doris::TabletMeta +_partition_ids src/exec/tablet_sink.h /^ std::set _partition_ids;$/;" m class:doris::stream_load::OlapTableSink +_partition_ids src/runtime/tablets_channel.h /^ std::unordered_set _partition_ids;$/;" m class:doris::TabletsChannel +_partition_infos src/exec/broker_scan_node.h /^ std::vector _partition_infos;$/;" m class:doris::BrokerScanNode +_partition_infos src/runtime/data_spliter.h /^ std::vector _partition_infos;$/;" m class:doris::DataSpliter +_partition_infos src/runtime/data_stream_sender.h /^ std::vector _partition_infos;$/;" m class:doris::DataStreamSender +_partition_pool src/exec/partitioned_aggregation_node.h /^ boost::scoped_ptr _partition_pool;$/;" m class:doris::PartitionedAggregationNode +_partition_slot_descs src/exec/tablet_info.h /^ std::vector _partition_slot_descs;$/;" m class:doris::OlapTablePartitionParam +_partition_tablet_map src/olap/tablet_manager.h /^ std::map> _partition_tablet_map;$/;" m class:doris::TabletManager +_partition_tablet_map_lock src/olap/tablet_manager.h /^ RWMutex _partition_tablet_map_lock;$/;" m class:doris::TabletManager +_partitions src/exec/tablet_info.h /^ std::vector _partitions;$/;" m class:doris::OlapTablePartitionParam +_partitions_created src/exec/partitioned_aggregation_node.h /^ RuntimeProfile::Counter* _partitions_created;$/;" m class:doris::PartitionedAggregationNode +_partitions_map src/exec/tablet_info.h /^ std::map> _partitions_map;$/;" m class:doris::OlapTablePartitionParam +_passwd src/exec/es/es_scan_reader.h /^ std::string _passwd;$/;" m class:doris::ESScanReader +_passwd src/runtime/descriptors.h /^ std::string _passwd;$/;" m class:doris::MySQLTableDescriptor +_patch_gap_width src/olap/rowset/run_length_integer_writer.h /^ uint32_t _patch_gap_width;$/;" m class:doris::RunLengthIntegerWriter +_patch_length src/olap/rowset/run_length_integer_writer.h /^ uint32_t _patch_length;$/;" m class:doris::RunLengthIntegerWriter +_patch_width src/olap/rowset/run_length_integer_writer.h /^ uint32_t _patch_width;$/;" m class:doris::RunLengthIntegerWriter +_path src/exec/broker_reader.h /^ const std::string& _path;$/;" m class:doris::BrokerReader +_path src/exec/broker_writer.h /^ std::string _path;$/;" m class:doris::BrokerWriter +_path src/exec/local_file_reader.h /^ std::string _path;$/;" m class:doris::LocalFileReader +_path src/exec/local_file_writer.h /^ std::string _path;$/;" m class:doris::LocalFileWriter +_path src/olap/data_dir.h /^ std::string _path;$/;" m class:doris::DataDir +_path src/olap/fs/file_block_manager.cpp /^ const string _path;$/;" m class:doris::fs::internal::FileReadableBlock file: +_path src/olap/fs/file_block_manager.cpp /^ const string _path;$/;" m class:doris::fs::internal::FileWritableBlock file: +_path src/runtime/message_body_sink.h /^ std::string _path;$/;" m class:doris::MessageBodyFileSink +_path src/runtime/tmp_file_mgr.h /^ std::string _path;$/;" m class:doris::TmpFileMgr::Dir +_path src/runtime/tmp_file_mgr.h /^ std::string _path;$/;" m class:doris::TmpFileMgr::File +_path_gc_thread_callback src/olap/storage_engine.cpp /^void* StorageEngine::_path_gc_thread_callback(void* arg) {$/;" f class:doris::StorageEngine +_path_gc_threads src/olap/storage_engine.h /^ std::vector _path_gc_threads;$/;" m class:doris::StorageEngine +_path_hash src/olap/data_dir.h /^ size_t _path_hash;$/;" m class:doris::DataDir +_path_scan_thread_callback src/olap/storage_engine.cpp /^void* StorageEngine::_path_scan_thread_callback(void* arg) {$/;" f class:doris::StorageEngine +_path_scan_threads src/olap/storage_engine.h /^ std::vector _path_scan_threads;$/;" m class:doris::StorageEngine +_path_vec src/runtime/load_path_mgr.h /^ std::vector _path_vec;$/;" m class:doris::LoadPathMgr +_pattern src/runtime/string_search.hpp /^ const StringValue* _pattern;$/;" m class:doris::StringSearch +_pb_batch src/runtime/data_stream_sender.cpp /^ PRowBatch _pb_batch;$/;" m class:doris::DataStreamSender::Channel file: +_pb_batch1 src/runtime/data_stream_sender.h /^ PRowBatch _pb_batch1;$/;" m class:doris::DataStreamSender +_pb_batch2 src/runtime/data_stream_sender.h /^ PRowBatch _pb_batch2;$/;" m class:doris::DataStreamSender +_pending_closures src/runtime/data_stream_recvr.cc /^ std::deque _pending_closures;$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_pending_path_ids src/olap/data_dir.h /^ std::set _pending_path_ids;$/;" m class:doris::DataDir +_pending_path_mutex src/olap/data_dir.h /^ RWMutex _pending_path_mutex;$/;" m class:doris::DataDir +_pending_sync src/env/env_posix.cpp /^ bool _pending_sync = false;$/;" m class:doris::PosixWritableFile file: +_per_fragment_instance_idx src/runtime/runtime_state.h /^ int _per_fragment_instance_idx;$/;" m class:doris::RuntimeState +_per_pool_quota src/runtime/thread_resource_mgr.h /^ int _per_pool_quota;$/;" m class:doris::ThreadResourceMgr +_per_read_thread_throughput_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* _per_read_thread_throughput_counter;$/;" m class:doris::ScanNode +_perform_base_compaction src/olap/storage_engine.cpp /^void StorageEngine::_perform_base_compaction(DataDir* data_dir) {$/;" f class:doris::StorageEngine +_perform_cumulative_compaction src/olap/storage_engine.cpp /^void StorageEngine::_perform_cumulative_compaction(DataDir* data_dir) {$/;" f class:doris::StorageEngine +_pick_columns src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_pick_columns() {$/;" f class:doris::SegmentReader +_pick_delete_row_groups src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_pick_delete_row_groups(uint32_t first_block, uint32_t last_block) {$/;" f class:doris::SegmentReader +_pick_row_groups src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_pick_row_groups(uint32_t first_block, uint32_t last_block) {$/;" f class:doris::SegmentReader +_pin_next_fixed_len_block src/runtime/spill_sorter.cc /^ bool _pin_next_fixed_len_block;$/;" m class:doris::SpillSorter::Run file: +_pin_next_var_len_block src/runtime/spill_sorter.cc /^ bool _pin_next_var_len_block;$/;" m class:doris::SpillSorter::Run file: +_pin_timer src/runtime/buffered_tuple_stream.h /^ RuntimeProfile::Counter* _pin_timer;$/;" m class:doris::BufferedTupleStream +_pin_timer src/runtime/buffered_tuple_stream2.h /^ RuntimeProfile::Counter* _pin_timer;$/;" m class:doris::BufferedTupleStream2 +_ping_thread src/runtime/broker_mgr.h /^ std::thread _ping_thread;$/;" m class:doris::BrokerMgr +_pinned src/runtime/buffered_tuple_stream2.h /^ bool _pinned;$/;" m class:doris::BufferedTupleStream2 +_plan src/runtime/plan_fragment_executor.h /^ ExecNode* _plan; \/\/ lives in _runtime_state->obj_pool()$/;" m class:doris::PlanFragmentExecutor +_plugins src/plugin/plugin_manager.h /^ std::unordered_map _plugins;$/;" m class:doris::PluginManager +_point src/geo/geo_types.h /^ S2Point _point;$/;" m class:doris::GeoPoint +_polygon src/geo/geo_types.h /^ std::unique_ptr _polygon;$/;" m class:doris::GeoPolygon +_polyline src/geo/geo_types.h /^ std::unique_ptr _polyline;$/;" m class:doris::GeoLine +_pool src/exec/exec_node.h /^ ObjectPool* _pool;$/;" m class:doris::ExecNode +_pool src/exec/tablet_sink.h /^ ObjectPool* _pool;$/;" m class:doris::stream_load::OlapTableSink +_pool src/exprs/expr_context.h /^ std::unique_ptr _pool;$/;" m class:doris::ExprContext +_pool src/exprs/hybird_map.h /^ ObjectPool _pool;$/;" m class:doris::HybirdMap +_pool src/exprs/hybird_set.h /^ ObjectPool _pool;$/;" m class:doris::HybirdSet +_pool src/exprs/hybird_set.h /^ ObjectPool _pool;$/;" m class:doris::StringValueSet +_pool src/olap/column_block.h /^ MemPool* _pool;$/;" m class:doris::ColumnBlock +_pool src/olap/row_block2.h /^ std::unique_ptr _pool;$/;" m class:doris::RowBlockV2 +_pool src/olap/rowset/segment_v2/binary_dict_page.h /^ MemPool _pool;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_pool src/olap/rowset/segment_v2/bitmap_index_reader.h /^ std::unique_ptr _pool;$/;" m class:doris::segment_v2::BitmapIndexIterator +_pool src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^ MemPool _pool;$/;" m class:doris::segment_v2::__anon54::BitmapIndexWriterImpl file: +_pool src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ std::unique_ptr _pool;$/;" m class:doris::segment_v2::BloomFilterIndexIterator +_pool src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ MemPool _pool;$/;" m class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl file: +_pool src/olap/rowset/segment_v2/column_reader.h /^ std::unique_ptr _pool;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_pool src/olap/rowset/segment_v2/zone_map_index.h /^ MemPool _pool;$/;" m class:doris::segment_v2::ZoneMapIndexWriter +_pool src/runtime/data_stream_sender.h /^ ObjectPool* _pool;$/;" m class:doris::DataStreamSender +_pool src/runtime/disk_io_mgr.h /^ ObjectPool _pool;$/;" m class:doris::DiskIoMgr +_pool src/runtime/export_sink.h /^ ObjectPool* _pool;$/;" m class:doris::ExportSink +_pool src/runtime/mysql_table_sink.h /^ ObjectPool* _pool;$/;" m class:doris::MysqlTableSink +_pool src/runtime/routine_load/data_consumer_pool.h /^ std::list> _pool;$/;" m class:doris::DataConsumerPool +_pool src/runtime/sorted_run_merger.h /^ ObjectPool _pool;$/;" m class:doris::SortedRunMerger +_pool src/runtime/string_buffer.hpp /^ MemPool* _pool;$/;" m class:doris::StringBuffer +_pool src/udf/udf_internal.h /^ FreePool* _pool;$/;" m class:doris::FunctionContextImpl +_pool src/util/arrow/row_batch.cpp /^ arrow::MemoryPool* _pool;$/;" m class:doris::FromRowBatchConverter file: +_pool src/util/arrow/row_block.cpp /^ arrow::MemoryPool* _pool;$/;" m class:doris::FromRowBlockConverter file: +_pool src/util/dummy_runtime_profile.h /^ ObjectPool _pool;$/;" m class:doris::DummyProfile +_pool src/util/runtime_profile.h /^ std::unique_ptr _pool;$/;" m class:doris::RuntimeProfile +_pool src/util/threadpool.h /^ ThreadPool* _pool;$/;" m class:doris::ThreadPoolToken +_pool test/exec/hash_table_test.cpp /^ ObjectPool _pool;$/;" m class:doris::HashTableTest file: +_pool test/exec/partitioned_hash_table_test.cpp /^ ObjectPool _pool;$/;" m class:doris::PartitionedHashTableTest file: +_pool test/olap/key_coder_test.cpp /^ MemPool _pool;$/;" m class:doris::KeyCoderTest file: +_pool test/olap/rowset/segment_v2/bitmap_index_test.cpp /^ MemPool _pool;$/;" m class:doris::segment_v2::BitmapIndexTest file: +_pool test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^ MemPool _pool;$/;" m class:doris::segment_v2::ColumnReaderWriterTest file: +_pool test/runtime/buffered_tuple_stream2_test.cpp /^ ObjectPool _pool;$/;" m class:doris::SimpleTupleStreamTest file: +_pool test/runtime/disk_io_mgr_test.cpp /^ scoped_ptr _pool;$/;" m class:doris::DiskIoMgrTest file: +_pool test/util/threadpool_test.cpp /^ unique_ptr _pool;$/;" m class:doris::ThreadPoolTest file: +_pool_mem_trackers src/runtime/exec_env.h /^ PoolMemTrackerRegistry* _pool_mem_trackers = nullptr;$/;" m class:doris::ExecEnv +_pool_name src/runtime/mem_tracker.h /^ std::string _pool_name;$/;" m class:doris::MemTracker +_pool_status src/util/threadpool.h /^ Status _pool_status;$/;" m class:doris::ThreadPool +_pool_to_mem_trackers src/runtime/mem_tracker.h /^ PoolTrackersMap _pool_to_mem_trackers;$/;" m class:doris::PoolMemTrackerRegistry +_pool_to_mem_trackers_lock src/runtime/mem_tracker.h /^ SpinLock _pool_to_mem_trackers_lock;$/;" m class:doris::PoolMemTrackerRegistry +_pools src/runtime/thread_resource_mgr.h /^ Pools _pools;$/;" m class:doris::ThreadResourceMgr +_pop_heap src/olap/schema_change.cpp /^bool RowBlockMerger::_pop_heap() {$/;" f class:doris::RowBlockMerger +_port src/http/ev_http_server.h /^ int _port;$/;" m class:doris::EvHttpServer +_port src/runtime/descriptors.h /^ std::string _port;$/;" m class:doris::MySQLTableDescriptor +_port src/util/thrift_client.h /^ int _port;$/;" m class:doris::ThriftClientImpl +_port src/util/thrift_server.h /^ int _port;$/;" m class:doris::ThriftServer +_pos src/exec/parquet_reader.h /^ int64_t _pos = 0;$/;" m class:doris::ParquetFile +_pos src/olap/row_block.h /^ size_t _pos = 0;$/;" m class:doris::RowBlock +_pos src/olap/rowset/segment_v2/index_page.h /^ size_t _pos;$/;" m class:doris::segment_v2::IndexPageIterator +_pos src/runtime/dpp_writer.h /^ char* _pos;$/;" m class:doris::DppWriter +_pos src/util/mysql_row_buffer.h /^ char* _pos;$/;" m class:doris::MysqlRowBuffer +_position src/olap/byte_buffer.h /^ uint64_t _position;$/;" m class:doris::StorageByteBuffer +_position_in_orc_original src/exec/orc_scanner.h /^ std::vector _position_in_orc_original;$/;" m class:doris::ORCScanner +_positions src/olap/stream_index_reader.h /^ const uint32_t* _positions;$/;" m class:doris::PositionEntryReader +_positions src/olap/stream_index_writer.h /^ uint32_t _positions[MAX_POSITION_SIZE];$/;" m class:doris::PositionEntryWriter +_positions_count src/olap/stream_index_reader.h /^ size_t _positions_count;$/;" m class:doris::PositionEntryReader +_positions_count src/olap/stream_index_writer.h /^ size_t _positions_count;$/;" m class:doris::PositionEntryWriter +_post_handlers src/http/ev_http_server.h /^ PathTrie _post_handlers;$/;" m class:doris::EvHttpServer +_pre_allocated_size src/env/env_posix.cpp /^ uint64_t _pre_allocated_size = 0;$/;" m class:doris::PosixWritableFile file: +_precision src/olap/tablet_schema.h /^ int32_t _precision;$/;" m class:doris::TabletColumn +_predicate_columns src/olap/rowset/segment_v2/segment_iterator.h /^ std::vector _predicate_columns;$/;" m class:doris::segment_v2::SegmentIterator +_predicate_mem_pool src/olap/reader.h /^ std::unique_ptr _predicate_mem_pool;$/;" m class:doris::Reader +_predicate_to_conjunct src/exec/es_http_scan_node.h /^ std::vector _predicate_to_conjunct;$/;" m class:doris::EsHttpScanNode +_predicates src/exec/es_http_scan_node.h /^ std::vector _predicates;$/;" m class:doris::EsHttpScanNode +_preferred_rowset_type src/olap/tablet_meta.h /^ RowsetTypePB _preferred_rowset_type;$/;" m class:doris::TabletMeta +_prepare src/exec/olap_scanner.cpp /^Status OlapScanner::_prepare($/;" f class:doris::OlapScanner +_prepare_fn src/exprs/scalar_fn_call.h /^ UdfPrepare _prepare_fn;$/;" m class:doris::ScalarFnCall +_prepare_patched_blob src/olap/rowset/run_length_integer_writer.cpp /^void RunLengthIntegerWriter::_prepare_patched_blob() {$/;" f class:doris::RunLengthIntegerWriter +_prepare_seek src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_prepare_seek(const StorageReadOptions::KeyRange& key_range) {$/;" f class:doris::segment_v2::SegmentIterator +_prepared src/exprs/expr_context.h /^ bool _prepared;$/;" m class:doris::ExprContext +_prepared src/runtime/plan_fragment_executor.h /^ bool _prepared;$/;" m class:doris::PlanFragmentExecutor +_prepared_size src/olap/rowset/segment_v2/binary_plain_page.h /^ size_t _prepared_size;$/;" m class:doris::segment_v2::BinaryPlainPageBuilder +_present_buffers test/olap/column_reader_test.cpp /^ std::vector _present_buffers;$/;" m class:doris::TestColumn file: +_present_buffers test/olap/schema_change_test.cpp /^ std::vector _present_buffers;$/;" m class:doris::TestColumn file: +_present_reader src/olap/rowset/column_reader.h /^ BitFieldReader* _present_reader; \/\/ NULLable的字段的NULL值$/;" m class:doris::ColumnReader +_prev_child_batch src/exec/analytic_eval_node.h /^ boost::scoped_ptr _prev_child_batch;$/;" m class:doris::AnalyticEvalNode +_prev_delta src/olap/rowset/run_length_integer_writer.h /^ int64_t _prev_delta;$/;" m class:doris::RunLengthIntegerWriter +_prev_input_row src/exec/analytic_eval_node.h /^ TupleRow* _prev_input_row;$/;" m class:doris::AnalyticEvalNode +_prev_pool_last_result_idx src/exec/analytic_eval_node.h /^ int64_t _prev_pool_last_result_idx;$/;" m class:doris::AnalyticEvalNode +_prev_pool_last_window_idx src/exec/analytic_eval_node.h /^ int64_t _prev_pool_last_window_idx;$/;" m class:doris::AnalyticEvalNode +_prev_tuple_pool src/exec/analytic_eval_node.h /^ boost::scoped_ptr _prev_tuple_pool;$/;" m class:doris::AnalyticEvalNode +_print_missed_versions src/olap/tablet.cpp /^void Tablet::_print_missed_versions(const std::vector& missed_versions) const {$/;" f class:doris::Tablet +_priority_queue src/exec/topn_node.h /^ Tuple*, std::vector, TupleRowComparator> > _priority_queue;$/;" m class:doris::TopNNode +_probe_batch src/exec/hash_join_node.h /^ boost::scoped_ptr _probe_batch;$/;" m class:doris::HashJoinNode +_probe_batch src/exec/intersect_node.h /^ std::unique_ptr _probe_batch;$/;" m class:doris::IntersectNode +_probe_batch_pos src/exec/hash_join_node.h /^ int _probe_batch_pos; \/\/ current scan pos in _probe_batch$/;" m class:doris::HashJoinNode +_probe_eos src/exec/hash_join_node.h /^ bool _probe_eos; \/\/ if true, probe child has no more rows to process$/;" m class:doris::HashJoinNode +_probe_expr test/exec/hash_table_test.cpp /^ vector _probe_expr;$/;" m class:doris::HashTableTest file: +_probe_expr_ctxs src/exec/aggregation_node.h /^ std::vector _probe_expr_ctxs;$/;" m class:doris::AggregationNode +_probe_expr_ctxs src/exec/hash_join_node.h /^ std::vector _probe_expr_ctxs;$/;" m class:doris::HashJoinNode +_probe_expr_ctxs src/exec/hash_table.h /^ const std::vector& _probe_expr_ctxs;$/;" m class:doris::HashTable +_probe_expr_ctxs src/exec/partitioned_aggregation_node.h /^ std::vector _probe_expr_ctxs;$/;" m class:doris::PartitionedAggregationNode +_probe_expr_ctxs src/exec/partitioned_hash_table.h /^ const std::vector& _probe_expr_ctxs;$/;" m class:doris::PartitionedHashTableCtx +_probe_expr_ctxs test/exec/partitioned_hash_table_test.cpp /^ vector _probe_expr_ctxs;$/;" m class:doris::PartitionedHashTableTest file: +_probe_exprs src/exec/pre_aggregation_node.h /^ std::vector _probe_exprs;$/;" m class:doris::PreAggregationNode +_probe_rows_counter src/exec/hash_join_node.h /^ RuntimeProfile::Counter* _probe_rows_counter; \/\/ num probe rows$/;" m class:doris::HashJoinNode +_probe_timer src/exec/hash_join_node.h /^ RuntimeProfile::Counter* _probe_timer; \/\/ time to probe$/;" m class:doris::HashJoinNode +_probe_tuple_row_size src/exec/blocking_join_node.h /^ int _probe_tuple_row_size;$/;" m class:doris::BlockingJoinNode +_probe_tuple_row_size src/exec/hash_join_node.h /^ int _probe_tuple_row_size;$/;" m class:doris::HashJoinNode +_proc_net_dev_version src/util/system_metrics.h /^ int _proc_net_dev_version = 0;$/;" m class:doris::SystemMetrics +_process src/olap/task/engine_batch_load_task.cpp /^AgentStatus EngineBatchLoadTask::_process() {$/;" f class:doris::EngineBatchLoadTask +_process_batch_status src/exec/partitioned_aggregation_node.h /^ Status _process_batch_status;$/;" m class:doris::PartitionedAggregationNode +_process_build_batch_fn src/exec/hash_join_node.h /^ ProcessBuildBatchFn _process_build_batch_fn;$/;" m class:doris::HashJoinNode +_process_garbage_path src/olap/data_dir.cpp /^void DataDir::_process_garbage_path(const std::string& path) {$/;" f class:doris::DataDir +_process_mem_tracker src/runtime/disk_io_mgr.h /^ MemTracker* _process_mem_tracker;$/;" m class:doris::DiskIoMgr +_process_name src/olap/task/engine_alter_tablet_task.h /^ const string& _process_name;$/;" m class:doris::EngineAlterTabletTask +_process_probe_batch_fn src/exec/hash_join_node.h /^ ProcessProbeBatchFn _process_probe_batch_fn;$/;" m class:doris::HashJoinNode +_process_put src/http/action/mini_load.cpp /^Status MiniLoadAction::_process_put(HttpRequest* req, StreamLoadContext* ctx) {$/;" f class:doris::MiniLoadAction +_process_put src/http/action/stream_load.cpp /^Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext* ctx) {$/;" f class:doris::StreamLoadAction +_process_row_batch_fn src/exec/aggregation_node.h /^ ProcessRowBatchFn _process_row_batch_fn;$/;" m class:doris::AggregationNode +_process_row_batch_fn src/exec/partitioned_aggregation_node.h /^ ProcessRowBatchFn _process_row_batch_fn;$/;" m class:doris::PartitionedAggregationNode +_process_status src/exec/broker_scan_node.h /^ Status _process_status;$/;" m class:doris::BrokerScanNode +_process_status src/exec/es_http_scan_node.h /^ Status _process_status;$/;" m class:doris::EsHttpScanNode +_process_status src/runtime/runtime_state.h /^ Status _process_status;$/;" m class:doris::RuntimeState +_process_status_lock src/runtime/runtime_state.h /^ boost::mutex _process_status_lock;$/;" m class:doris::RuntimeState +_processed src/util/tdigest.h /^ std::vector _processed;$/;" m class:doris::TDigest +_processed_weight src/util/tdigest.h /^ Value _processed_weight = 0.0;$/;" m class:doris::TDigest +_processor src/util/thrift_server.h /^ boost::shared_ptr _processor;$/;" m class:doris::ThriftServer +_profile src/exec/base_scanner.h /^ RuntimeProfile* _profile;$/;" m class:doris::BaseScanner +_profile src/exec/es_http_scanner.h /^ RuntimeProfile* _profile;$/;" m class:doris::EsHttpScanner +_profile src/exec/olap_scanner.h /^ RuntimeProfile* _profile;$/;" m class:doris::OlapScanner +_profile src/exec/plain_text_line_reader.h /^ RuntimeProfile* _profile;$/;" m class:doris::PlainTextLineReader +_profile src/exec/tablet_sink.h /^ RuntimeProfile* _profile = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_profile src/runtime/buffered_block_mgr2.h /^ boost::scoped_ptr _profile;$/;" m class:doris::BufferedBlockMgr2 +_profile src/runtime/data_spliter.h /^ RuntimeProfile* _profile;$/;" m class:doris::DataSpliter +_profile src/runtime/data_stream_recvr.h /^ RuntimeProfile* _profile;$/;" m class:doris::DataStreamRecvr +_profile src/runtime/data_stream_sender.h /^ RuntimeProfile* _profile; \/\/ Allocated from _pool$/;" m class:doris::DataStreamSender +_profile src/runtime/dpp_sink.cpp /^ RuntimeProfile* _profile;$/;" m class:doris::Translator file: +_profile src/runtime/dpp_sink.h /^ RuntimeProfile* _profile;$/;" m class:doris::DppSink +_profile src/runtime/export_sink.h /^ RuntimeProfile* _profile;$/;" m class:doris::ExportSink +_profile src/runtime/memory_scratch_sink.h /^ RuntimeProfile* _profile; \/\/ Allocated from _pool$/;" m class:doris::MemoryScratchSink +_profile src/runtime/merge_sorter.h /^ RuntimeProfile* _profile;$/;" m class:doris::MergeSorter +_profile src/runtime/mysql_table_sink.h /^ RuntimeProfile* _profile;$/;" m class:doris::MysqlTableSink +_profile src/runtime/result_sink.h /^ RuntimeProfile* _profile; \/\/ Allocated from _pool$/;" m class:doris::ResultSink +_profile src/runtime/runtime_state.h /^ RuntimeProfile _profile;$/;" m class:doris::RuntimeState +_profile src/runtime/spill_sorter.h /^ RuntimeProfile* _profile;$/;" m class:doris::SpillSorter +_profile src/util/dummy_runtime_profile.h /^ RuntimeProfile* const _profile;$/;" m class:doris::DummyProfile +_profile test/exec/broker_scanner_test.cpp /^ RuntimeProfile* _profile;$/;" m class:doris::BrokerScannerTest file: +_profile test/exec/orc_scanner_test.cpp /^ RuntimeProfile *_profile;$/;" m class:doris::OrcScannerTest file: +_profile test/exec/plain_text_line_reader_bzip_test.cpp /^ RuntimeProfile _profile;$/;" m class:doris::PlainTextLineReaderTest file: +_profile test/exec/plain_text_line_reader_gzip_test.cpp /^ RuntimeProfile _profile;$/;" m class:doris::PlainTextLineReaderTest file: +_profile test/exec/plain_text_line_reader_lz4frame_test.cpp /^ RuntimeProfile _profile;$/;" m class:doris::PlainTextLineReaderTest file: +_profile test/exec/plain_text_line_reader_lzop_test.cpp /^ RuntimeProfile _profile;$/;" m class:doris::PlainTextLineReaderTest file: +_profile test/exec/plain_text_line_reader_uncompressed_test.cpp /^ RuntimeProfile _profile;$/;" m class:doris::PlainTextLineReaderTest file: +_profile test/olap/olap_reader_test.cpp /^ RuntimeProfile* _profile;$/;" m class:doris::TestOLAPReaderColumn file: +_profile test/olap/olap_reader_test.cpp /^ RuntimeProfile* _profile;$/;" m class:doris::TestOLAPReaderColumnDeleteCondition file: +_profile test/olap/olap_reader_test.cpp /^ RuntimeProfile* _profile;$/;" m class:doris::TestOLAPReaderRow file: +_profile test/olap/vectorized_olap_reader_test.cpp /^ RuntimeProfile* _profile;$/;" m class:doris::TestVectorizedOLAPReader file: +_profile test/runtime/buffered_tuple_stream_test.cpp /^ RuntimeProfile *_profile;$/;" m class:doris::BufferedTupleStreamTest file: +_progress src/exec/olap_scan_node.h /^ ProgressUpdater _progress;$/;" m class:doris::OlapScanNode +_properties src/exec/broker_reader.h /^ const std::map& _properties;$/;" m class:doris::BrokerReader +_properties src/exec/broker_writer.h /^ const std::map& _properties;$/;" m class:doris::BrokerWriter +_properties src/exec/es_http_scan_node.h /^ std::map _properties;$/;" m class:doris::EsHttpScanNode +_properties src/exec/es_http_scanner.h /^ const std::map& _properties;$/;" m class:doris::EsHttpScanner +_properties src/exec/es_scan_node.h /^ std::map _properties;$/;" m class:doris::EsScanNode +_properties src/exec/parquet_reader.h /^ parquet::ReaderProperties _properties;$/;" m class:doris::ParquetReaderWrap +_properties test/exec/broker_reader_test.cpp /^ std::map _properties;$/;" m class:doris::BrokerReaderTest file: +_proto src/olap/file_helper.h /^ MessageType _proto;$/;" m class:doris::FileHeader +_proto_schema src/exec/tablet_info.h /^ mutable POlapTableSchemaParam* _proto_schema = nullptr;$/;" m class:doris::OlapTableSchemaParam +_proto_string src/olap/file_helper.h /^ std::string _proto_string;$/;" m class:doris::FileHeader +_protocol src/util/thrift_client.h /^ boost::shared_ptr _protocol;$/;" m class:doris::ThriftClientImpl +_protocol src/util/thrift_util.h /^ boost::shared_ptr _protocol;$/;" m class:doris::ThriftSerializer +_ptr src/common/atomic.h /^ AtomicInt _ptr;$/;" m class:doris::AtomicPtr +_ptr src/olap/row_cursor_cell.h /^ void* _ptr;$/;" m struct:doris::RowCursorCell +_publish_version_req src/olap/task/engine_publish_version_task.h /^ const TPublishVersionRequest& _publish_version_req;$/;" m class:doris::EnginePublishVersionTask +_publish_version_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_publish_version_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_publish_version_workers src/agent/agent_server.h /^ std::unique_ptr _publish_version_workers;$/;" m class:doris::AgentServer +_pull_first_block src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::_pull_first_block(AlphaMergeContext* merge_ctx) {$/;" f class:doris::AlphaRowsetReader +_pull_next_block src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::_pull_next_block(AlphaMergeContext* merge_ctx) {$/;" f class:doris::AlphaRowsetReader +_pull_next_row_for_merge_rowset src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::_pull_next_row_for_merge_rowset(RowCursor** row) {$/;" f class:doris::AlphaRowsetReader +_pull_next_row_for_merge_rowset_v2 src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::_pull_next_row_for_merge_rowset_v2(RowCursor** row) {$/;" f class:doris::AlphaRowsetReader +_push src/olap/task/engine_batch_load_task.cpp /^OLAPStatus EngineBatchLoadTask::_push(const TPushReq& request,$/;" f class:doris::EngineBatchLoadTask +_push_back_page src/olap/rowset/segment_v2/column_writer.h /^ void _push_back_page(Page* page) {$/;" f class:doris::segment_v2::ColumnWriter +_push_compute_timer src/exec/hash_join_node.h /^ RuntimeProfile::Counter* _push_compute_timer;$/;" m class:doris::HashJoinNode +_push_down_expr_ctxs src/exec/hash_join_node.h /^ std::list _push_down_expr_ctxs;$/;" m class:doris::HashJoinNode +_push_down_timer src/exec/hash_join_node.h /^ RuntimeProfile::Counter* _push_down_timer; \/\/ time to build hash table$/;" m class:doris::HashJoinNode +_push_req src/olap/task/engine_batch_load_task.h /^ TPushReq& _push_req;$/;" m class:doris::EngineBatchLoadTask +_push_req test/olap/olap_reader_test.cpp /^ TPushReq _push_req;$/;" m class:doris::TestOLAPReaderColumn file: +_push_req test/olap/olap_reader_test.cpp /^ TPushReq _push_req;$/;" m class:doris::TestOLAPReaderColumnDeleteCondition file: +_push_req test/olap/olap_reader_test.cpp /^ TPushReq _push_req;$/;" m class:doris::TestOLAPReaderRow file: +_push_req test/olap/vectorized_olap_reader_test.cpp /^ TPushReq _push_req;$/;" m class:doris::TestVectorizedOLAPReader file: +_push_rowset_meta_thread src/olap/tablet_sync_service.cpp /^void TabletSyncService::_push_rowset_meta_thread(std::vector tasks) {$/;" f class:doris::TabletSyncService +_push_rowset_pool src/olap/tablet_sync_service.h /^ BatchProcessThreadPool* _push_rowset_pool = nullptr;$/;" m class:doris::TabletSyncService +_push_tablet_meta_thread src/olap/tablet_sync_service.cpp /^void TabletSyncService::_push_tablet_meta_thread(std::vector tasks) {$/;" f class:doris::TabletSyncService +_push_tablet_pool src/olap/tablet_sync_service.h /^ BatchProcessThreadPool* _push_tablet_pool = nullptr;$/;" m class:doris::TabletSyncService +_push_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_push_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_push_workers src/agent/agent_server.h /^ std::unique_ptr _push_workers;$/;" m class:doris::AgentServer +_pushdown_conjunct_ctxs src/exec/es_scan_node.h /^ std::vector _pushdown_conjunct_ctxs;$/;" m class:doris::EsScanNode +_put_cond src/runtime/stream_load/stream_load_pipe.h /^ std::condition_variable _put_cond;$/;" m class:doris::StreamLoadPipe +_put_cv src/util/blocking_priority_queue.hpp /^ boost::condition_variable _put_cv; \/\/ 'put' callers wait on this$/;" m class:doris::BlockingPriorityQueue +_put_cv src/util/blocking_queue.hpp /^ std::condition_variable _put_cv; \/\/ 'put' callers wait on this$/;" m class:doris::BlockingQueue +_put_handlers src/http/ev_http_server.h /^ PathTrie _put_handlers;$/;" m class:doris::EvHttpServer +_quadratic_probing src/exec/partitioned_hash_table.h /^ const bool _quadratic_probing;$/;" m class:doris::PartitionedHashTable +_query src/exec/es/es_scan_reader.h /^ std::string _query;$/;" m class:doris::ESScanReader +_query_id src/runtime/buffered_block_mgr2.h /^ const TUniqueId _query_id;$/;" m class:doris::BufferedBlockMgr2 +_query_id src/runtime/fragment_mgr.cpp /^ TUniqueId _query_id;$/;" m class:doris::FragmentExecState file: +_query_id src/runtime/mem_tracker.h /^ TUniqueId _query_id;$/;" m class:doris::MemTracker +_query_id src/runtime/plan_fragment_executor.h /^ TUniqueId _query_id;$/;" m class:doris::PlanFragmentExecutor +_query_id src/runtime/runtime_state.h /^ TUniqueId _query_id;$/;" m class:doris::RuntimeState +_query_mem_tracker src/runtime/runtime_state.h /^ boost::shared_ptr _query_mem_tracker;$/;" m class:doris::RuntimeState +_query_options src/runtime/runtime_state.h /^ TQueryOptions _query_options;$/;" m class:doris::RuntimeState +_query_params src/http/http_request.h /^ std::map _query_params;$/;" m class:doris::HttpRequest +_query_slots src/exec/olap_scanner.h /^ std::vector _query_slots;$/;" m class:doris::OlapScanner +_query_states src/runtime/test_env.h /^ std::vector > _query_states;$/;" m class:doris::TestEnv +_query_statistics src/exec/data_sink.h /^ std::shared_ptr _query_statistics;$/;" m class:doris::DataSink +_query_statistics src/runtime/buffer_control_block.h /^ std::shared_ptr _query_statistics;$/;" m class:doris::BufferControlBlock +_query_statistics src/runtime/plan_fragment_executor.h /^ std::shared_ptr _query_statistics;$/;" m class:doris::PlanFragmentExecutor +_query_statistics src/runtime/query_statistics.h /^ std::map _query_statistics;$/;" m class:doris::QueryStatisticsRecvr +_query_tracker src/runtime/buffered_block_mgr2.cc /^ MemTracker* _query_tracker;$/;" m class:doris::BufferedBlockMgr2::Client file: +_queue src/runtime/memory_scratch_sink.h /^ BlockQueueSharedPtr _queue;$/;" m class:doris::MemoryScratchSink +_queue src/runtime/record_batch_queue.h /^ BlockingQueue> _queue;$/;" m class:doris::RecordBatchQueue +_queue src/runtime/routine_load/data_consumer_group.h /^ BlockingQueue _queue; $/;" m class:doris::KafkaDataConsumerGroup +_queue src/util/blocking_priority_queue.hpp /^ std::priority_queue _queue;$/;" m class:doris::BlockingPriorityQueue +_queue src/util/threadpool.h /^ std::deque _queue;$/;" m class:doris::ThreadPool +_queue test/util/blocking_queue_test.cpp /^ BlockingQueue _queue;$/;" m class:doris::MultiThreadTest file: +_queue_reader_cond src/exec/broker_scan_node.h /^ std::condition_variable _queue_reader_cond;$/;" m class:doris::BrokerScanNode +_queue_reader_cond src/exec/es_http_scan_node.h /^ std::condition_variable _queue_reader_cond;$/;" m class:doris::EsHttpScanNode +_queue_writer_cond src/exec/broker_scan_node.h /^ std::condition_variable _queue_writer_cond;$/;" m class:doris::BrokerScanNode +_queue_writer_cond src/exec/es_http_scan_node.h /^ std::condition_variable _queue_writer_cond;$/;" m class:doris::EsHttpScanNode +_quit_flag test/olap/skiplist_test.cpp /^ std::atomic _quit_flag;$/;" m class:doris::TestState file: +_rand_seed src/olap/data_dir.h /^ uint32_t _rand_seed;$/;" m class:doris::DataDir +_random_sink test/runtime/data_stream_test.cpp /^ TDataStreamSink _random_sink;$/;" m class:doris::DataStreamTest file: +_range src/runtime/dpp_sink_internal.h /^ PartRange _range;$/;" m class:doris::PartitionInfo +_range_adjust_tbl src/util/utf8_check.cpp /^static const uint8_t _range_adjust_tbl[] = {$/;" m namespace:doris file: +_range_iter src/olap/rowset/segment_v2/segment_iterator.h /^ std::unique_ptr _range_iter;$/;" m class:doris::segment_v2::SegmentIterator +_range_max_tbl src/util/utf8_check.cpp /^static const uint8_t _range_max_tbl[] = {$/;" m namespace:doris file: +_range_min_tbl src/util/utf8_check.cpp /^static const uint8_t _range_min_tbl[] = {$/;" m namespace:doris file: +_ranges src/exec/broker_scanner.h /^ const std::vector& _ranges;$/;" m class:doris::BrokerScanner +_ranges src/exec/orc_scanner.h /^ const std::vector& _ranges;$/;" m class:doris::ORCScanner +_ranges src/exec/parquet_scanner.h /^ const std::vector& _ranges;$/;" m class:doris::ParquetScanner +_ranges src/olap/rowset/segment_v2/row_ranges.h /^ std::vector _ranges;$/;" m class:doris::segment_v2::RowRanges +_raw_path src/http/http_request.h /^ std::string _raw_path;$/;" m class:doris::HttpRequest +_raw_rows_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _raw_rows_counter = nullptr;$/;" m class:doris::OlapScanNode +_raw_rows_read src/exec/olap_scanner.h /^ int64_t _raw_rows_read = 0;$/;" m class:doris::OlapScanner +_rb_batch src/exec/parquet_reader.h /^ std::shared_ptr<::arrow::RecordBatchReader> _rb_batch;$/;" m class:doris::ParquetReaderWrap +_read_all_data_streams src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_read_all_data_streams(size_t* buffer_size) {$/;" f class:doris::SegmentReader +_read_and_write_test_file src/olap/data_dir.cpp /^OLAPStatus DataDir::_read_and_write_test_file() {$/;" f class:doris::DataDir +_read_block src/olap/rowset/alpha_rowset_reader.h /^ std::unique_ptr _read_block;$/;" m class:doris::AlphaRowsetReader +_read_block src/olap/rowset/column_data.h /^ std::unique_ptr _read_block = nullptr;$/;" m class:doris::ColumnData +_read_block src/runtime/buffered_tuple_stream.h /^ std::list::iterator _read_block;$/;" m class:doris::BufferedTupleStream +_read_block src/runtime/buffered_tuple_stream2.h /^ std::list::iterator _read_block;$/;" m class:doris::BufferedTupleStream2 +_read_block_idx src/runtime/buffered_tuple_stream.h /^ int _read_block_idx;$/;" m class:doris::BufferedTupleStream +_read_block_idx src/runtime/buffered_tuple_stream2.h /^ int _read_block_idx;$/;" m class:doris::BufferedTupleStream2 +_read_buffer src/olap/rowset/column_reader.h /^ char* _read_buffer;$/;" m class:doris::StringColumnDictionaryReader +_read_byte src/olap/rowset/bit_field_reader.cpp /^OLAPStatus BitFieldReader::_read_byte() {$/;" f class:doris::BitFieldReader +_read_bytes src/runtime/buffered_tuple_stream.h /^ int64_t _read_bytes;$/;" m class:doris::BufferedTupleStream +_read_bytes src/runtime/buffered_tuple_stream2.h /^ int64_t _read_bytes;$/;" m class:doris::BufferedTupleStream2 +_read_cluster_id src/olap/data_dir.cpp /^Status DataDir::_read_cluster_id(const std::string& path, int32_t* cluster_id) {$/;" f class:doris::DataDir +_read_columns src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_read_columns(const std::vector& column_ids,$/;" f class:doris::segment_v2::SegmentIterator +_read_compressed_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _read_compressed_counter = nullptr;$/;" m class:doris::OlapScanNode +_read_data_page src/olap/rowset/segment_v2/column_reader.cpp /^Status FileColumnIterator::_read_data_page(const OrdinalPageIndexIterator& iter) {$/;" f class:doris::segment_v2::FileColumnIterator +_read_data_page src/olap/rowset/segment_v2/indexed_column_reader.cpp /^Status IndexedColumnIterator::_read_data_page(const PagePointer& pp) {$/;" f class:doris::segment_v2::IndexedColumnIterator +_read_delta_values src/olap/rowset/run_length_integer_reader.cpp /^OLAPStatus RunLengthIntegerReader::_read_delta_values(uint8_t first_byte) {$/;" f class:doris::RunLengthIntegerReader +_read_direct_values src/olap/rowset/run_length_integer_reader.cpp /^OLAPStatus RunLengthIntegerReader::_read_direct_values(uint8_t first_byte) {$/;" f class:doris::RunLengthIntegerReader +_read_next_batch src/olap/rowset/segment_v2/segment_iterator.cpp /^ void _read_next_batch() {$/;" f class:doris::segment_v2::SegmentIterator::BitmapRangeIterator file: +_read_next_value src/olap/rowset/segment_v2/binary_prefix_page.cpp /^Status BinaryPrefixPageDecoder::_read_next_value() {$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +_read_next_value_to_output src/olap/rowset/segment_v2/binary_prefix_page.cpp /^Status BinaryPrefixPageDecoder::_read_next_value_to_output(Slice prev, MemPool* mem_pool, Slice* output) {$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +_read_patched_base_values src/olap/rowset/run_length_integer_reader.cpp /^OLAPStatus RunLengthIntegerReader::_read_patched_base_values(uint8_t first_byte) {$/;" f class:doris::RunLengthIntegerReader +_read_ptr src/runtime/buffered_tuple_stream.h /^ uint8_t* _read_ptr;$/;" m class:doris::BufferedTupleStream +_read_ptr src/runtime/buffered_tuple_stream2.h /^ uint8_t* _read_ptr;$/;" m class:doris::BufferedTupleStream2 +_read_row_cursor src/exec/olap_scanner.h /^ RowCursor _read_row_cursor;$/;" m class:doris::OlapScanner +_read_short_repeat_values src/olap/rowset/run_length_integer_reader.cpp /^OLAPStatus RunLengthIntegerReader::_read_short_repeat_values(uint8_t first_byte) {$/;" f class:doris::RunLengthIntegerReader +_read_timer src/exec/base_scanner.h /^ RuntimeProfile::Counter* _read_timer;$/;" m class:doris::BaseScanner +_read_timer src/exec/es_http_scanner.h /^ RuntimeProfile::Counter* _read_timer;$/;" m class:doris::EsHttpScanner +_read_timer src/exec/plain_text_line_reader.h /^ RuntimeProfile::Counter* _read_timer;$/;" m class:doris::PlainTextLineReader +_read_timer src/exec/scan_node.h /^ RuntimeProfile::Counter* _read_timer; \/\/ total read time$/;" m class:doris::ScanNode +_read_timer src/runtime/disk_io_mgr.h /^ RuntimeProfile::Counter _read_timer;$/;" m class:doris::DiskIoMgr +_read_timer src/runtime/disk_io_mgr_internal.h /^ RuntimeProfile::Counter* _read_timer;$/;" m class:doris::DiskIoMgr::RequestContext +_read_tuple_idx src/runtime/buffered_tuple_stream.h /^ uint32_t _read_tuple_idx;$/;" m class:doris::BufferedTupleStream +_read_tuple_idx src/runtime/buffered_tuple_stream2.h /^ uint32_t _read_tuple_idx;$/;" m class:doris::BufferedTupleStream2 +_read_uncompressed_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _read_uncompressed_counter = nullptr;$/;" m class:doris::OlapScanNode +_read_values src/olap/rowset/run_length_byte_reader.cpp /^OLAPStatus RunLengthByteReader::_read_values() {$/;" f class:doris::RunLengthByteReader +_read_values src/olap/rowset/run_length_integer_reader.cpp /^OLAPStatus RunLengthIntegerReader::_read_values() {$/;" f class:doris::RunLengthIntegerReader +_read_vector_batch src/olap/rowset/column_data.h /^ std::unique_ptr _read_vector_batch;$/;" m class:doris::ColumnData +_read_write src/runtime/buffered_tuple_stream.h /^ const bool _read_write;$/;" m class:doris::BufferedTupleStream +_read_write src/runtime/buffered_tuple_stream2.h /^ const bool _read_write;$/;" m class:doris::BufferedTupleStream2 +_reader src/exec/olap_scanner.h /^ std::unique_ptr _reader;$/;" m class:doris::OlapScanner +_reader src/exec/orc_scanner.h /^ std::unique_ptr _reader;$/;" m class:doris::ORCScanner +_reader src/exec/parquet_reader.h /^ std::unique_ptr _reader;$/;" m class:doris::ParquetReaderWrap +_reader src/olap/reader.cpp /^ Reader* _reader;$/;" m class:doris::CollectIterator::ChildCtx file: +_reader src/olap/reader.cpp /^ Reader* _reader = nullptr;$/;" m class:doris::CollectIterator file: +_reader src/olap/rowset/column_reader.h /^ IntegerColumnReader _reader; \/\/ 被包裹的真实读取器$/;" m class:doris::IntegerColumnReaderWrapper +_reader src/olap/rowset/column_reader.h /^ ReaderClass _reader;$/;" m class:doris::FixLengthStringColumnReader +_reader src/olap/rowset/column_reader.h /^ ReaderClass _reader;$/;" m class:doris::VarStringColumnReader +_reader src/olap/rowset/segment_v2/bitmap_index_reader.h /^ BitmapIndexReader* _reader;$/;" m class:doris::segment_v2::BitmapIndexIterator +_reader src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ BloomFilterIndexReader* _reader;$/;" m class:doris::segment_v2::BloomFilterIndexIterator +_reader src/olap/rowset/segment_v2/column_reader.h /^ ColumnReader* _reader;$/;" m class:doris::segment_v2::FileColumnIterator +_reader src/olap/rowset/segment_v2/index_page.h /^ const IndexPageReader* _reader;$/;" m class:doris::segment_v2::IndexPageIterator +_reader src/olap/rowset/segment_v2/indexed_column_reader.h /^ const IndexedColumnReader* _reader;$/;" m class:doris::segment_v2::IndexedColumnIterator +_reader src/runtime/disk_io_mgr.h /^ RequestContext* _reader;$/;" m class:doris::DiskIoMgr::BufferDescriptor +_reader src/runtime/disk_io_mgr.h /^ RequestContext* _reader;$/;" m class:doris::DiskIoMgr::ScanRange +_reader test/olap/bit_field_test.cpp /^ BitFieldReader* _reader;$/;" m class:doris::TestBitField file: +_reader test/olap/run_length_byte_test.cpp /^ RunLengthByteReader* _reader;$/;" m class:doris::TestRunLengthByte file: +_reader test/olap/run_length_integer_test.cpp /^ RunLengthIntegerReader* _reader;$/;" m class:doris::TestRunLengthSignInteger file: +_reader test/olap/run_length_integer_test.cpp /^ RunLengthIntegerReader* _reader;$/;" m class:doris::TestRunLengthUnsignInteger file: +_reader_context src/olap/reader.h /^ RowsetReaderContext _reader_context;$/;" m class:doris::Reader +_reader_context src/olap/schema_change.h /^ RowsetReaderContext _reader_context;$/;" m class:doris::SchemaChangeHandler +_reader_init_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _reader_init_timer = nullptr;$/;" m class:doris::OlapScanNode +_reader_type src/olap/reader.h /^ ReaderType _reader_type;$/;" m class:doris::Reader +_ready src/olap/push_handler.h /^ bool _ready;$/;" m class:doris::IBinaryReader +_ready_buffers src/runtime/disk_io_mgr.h /^ std::list _ready_buffers;$/;" m class:doris::DiskIoMgr::ScanRange +_ready_buffers_capacity src/runtime/disk_io_mgr.h /^ int _ready_buffers_capacity;$/;" m class:doris::DiskIoMgr::ScanRange +_ready_to_start_ranges src/runtime/disk_io_mgr_internal.h /^ InternalQueue _ready_to_start_ranges;$/;" m class:doris::DiskIoMgr::RequestContext +_ready_to_start_ranges_cv src/runtime/disk_io_mgr_internal.h /^ boost::condition_variable _ready_to_start_ranges_cv; \/\/ used with _lock$/;" m class:doris::DiskIoMgr::RequestContext +_real_port src/http/ev_http_server.h /^ int _real_port;$/;" m class:doris::EvHttpServer +_received_first_batch src/runtime/data_stream_recvr.cc /^ bool _received_first_batch;$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_receiver_info test/runtime/data_stream_test.cpp /^ vector _receiver_info;$/;" m class:doris::DataStreamTest file: +_receiver_map src/runtime/data_stream_mgr.h /^ StreamMap _receiver_map;$/;" m class:doris::DataStreamMgr +_recover_tablet_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_recover_tablet_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_recover_tablet_workers src/agent/agent_server.h /^ std::unique_ptr _recover_tablet_workers;$/;" m class:doris::AgentServer +_recvr src/runtime/data_stream_recvr.cc /^ DataStreamRecvr* _recvr;$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_recycled_blocks_counter src/runtime/buffered_block_mgr2.h /^ RuntimeProfile::Counter* _recycled_blocks_counter;$/;" m class:doris::BufferedBlockMgr2 +_ref_count src/olap/rowset/segment_group.h /^ std::atomic _ref_count; \/\/ reference count$/;" m class:doris::SegmentGroup +_referenced_column src/olap/tablet_schema.h /^ std::string _referenced_column;$/;" m class:doris::TabletColumn +_referenced_column_id src/olap/tablet_schema.h /^ int32_t _referenced_column_id;$/;" m class:doris::TabletColumn +_refresh_current_row src/olap/reader.cpp /^ OLAPStatus _refresh_current_row() {$/;" f class:doris::CollectIterator::ChildCtx file: +_refs src/runtime/stream_load/stream_load_context.h /^ std::atomic _refs;$/;" m class:doris::StreamLoadContext +_refs src/util/ref_count_closure.h /^ std::atomic _refs;$/;" m class:doris::RefCountClosure +_refs_by_reader src/olap/rowset/rowset.h /^ std::atomic _refs_by_reader;$/;" m class:doris::Rowset +_registed_listeners src/agent/topic_subscriber.h /^ std::map> _registed_listeners;$/;" m class:doris::TopicSubscriber +_register_task_info src/agent/task_worker_pool.cpp /^bool TaskWorkerPool::_register_task_info(const TTaskType::type task_type, int64_t signature) {$/;" f class:doris::TaskWorkerPool +_registers src/olap/hll.h /^ uint8_t* _registers = nullptr;$/;" m class:doris::HyperLogLog +_registry src/util/metrics.h /^ MetricRegistry* _registry;$/;" m class:doris::Metric +_registry src/util/system_metrics.h /^ MetricRegistry* _registry = nullptr;$/;" m class:doris::SystemMetrics +_related_schema_hash src/olap/tablet_meta.h /^ int32_t _related_schema_hash;$/;" m class:doris::AlterTabletTask +_related_tablet_id src/olap/tablet_meta.h /^ int64_t _related_tablet_id;$/;" m class:doris::AlterTabletTask +_release_snapshot src/olap/task/engine_clone_task.cpp /^Status EngineCloneTask::_release_snapshot($/;" f class:doris::EngineCloneTask +_release_snapshot_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_release_snapshot_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_release_snapshot_workers src/agent/agent_server.h /^ std::unique_ptr _release_snapshot_workers;$/;" m class:doris::AgentServer +_reload_tablet src/http/action/restore_tablet_action.cpp /^Status RestoreTabletAction::_reload_tablet($/;" f class:doris::RestoreTabletAction +_remain_block src/olap/rowset/segment_reader.h /^ uint32_t _remain_block;$/;" m class:doris::SegmentReader +_remain_element_capacity src/olap/rowset/segment_v2/bitshuffle_page.h /^ int _remain_element_capacity;$/;" m class:doris::segment_v2::BitshufflePageBuilder +_remote_file_path src/olap/task/engine_batch_load_task.h /^ std::string _remote_file_path;$/;" m class:doris::EngineBatchLoadTask +_remove_check_paths_no_lock src/olap/data_dir.cpp /^void DataDir::_remove_check_paths_no_lock(const std::set& paths) {$/;" f class:doris::DataDir +_remove_fn src/exprs/agg_fn_evaluator.h /^ void* _remove_fn;$/;" m class:doris::AggFnEvaluator +_remove_is_present_positions src/olap/rowset/column_writer.cpp /^void ColumnWriter::_remove_is_present_positions() {$/;" f class:doris::ColumnWriter +_remove_tablet_from_partition src/olap/tablet_manager.cpp /^void TabletManager::_remove_tablet_from_partition(const Tablet& tablet) {$/;" f class:doris::TabletManager +_remove_task_info src/agent/task_worker_pool.cpp /^void TaskWorkerPool::_remove_task_info(const TTaskType::type task_type, int64_t signature) {$/;" f class:doris::TaskWorkerPool +_rename_remote_file src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::_rename_remote_file($/;" f class:doris::SnapshotLoader +_rename_rowset_id src/olap/snapshot_manager.cpp /^OLAPStatus SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb, const string& new_path,$/;" f class:doris::SnapshotManager +_rep src/olap/wrapper_field.h /^ Field* _rep = nullptr;$/;" m class:doris::WrapperField +_repeat src/olap/rowset/run_length_byte_reader.h /^ bool _repeat;$/;" m class:doris::RunLengthByteReader +_repeat src/olap/rowset/run_length_byte_writer.h /^ bool _repeat;$/;" m class:doris::RunLengthByteWriter +_repeat_id_idx src/exec/repeat_node.h /^ int _repeat_id_idx;$/;" m class:doris::RepeatNode +_repeat_id_list src/exec/repeat_node.h /^ std::vector _repeat_id_list;$/;" m class:doris::RepeatNode +_replace_tablet_id src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::_replace_tablet_id($/;" f class:doris::SnapshotLoader +_report_cv src/olap/storage_engine.h /^ std::condition_variable _report_cv;$/;" m class:doris::StorageEngine +_report_disk_state_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_report_disk_state_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_report_disk_state_workers src/agent/agent_server.h /^ std::unique_ptr _report_disk_state_workers;$/;" m class:doris::AgentServer +_report_every src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::_report_every($/;" f class:doris::SnapshotLoader +_report_mtx src/olap/storage_engine.h /^ std::mutex _report_mtx;$/;" m class:doris::StorageEngine +_report_status_cb src/runtime/plan_fragment_executor.h /^ report_status_callback _report_status_cb;$/;" m class:doris::PlanFragmentExecutor +_report_tablet_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_report_tablet_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_report_tablet_workers src/agent/agent_server.h /^ std::unique_ptr _report_tablet_workers;$/;" m class:doris::AgentServer +_report_task_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_report_task_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_report_task_workers src/agent/agent_server.h /^ std::unique_ptr _report_task_workers;$/;" m class:doris::AgentServer +_report_thread src/runtime/plan_fragment_executor.h /^ boost::thread _report_thread;$/;" m class:doris::PlanFragmentExecutor +_report_thread_active src/runtime/plan_fragment_executor.h /^ bool _report_thread_active; \/\/ true if we started the thread$/;" m class:doris::PlanFragmentExecutor +_report_thread_lock src/runtime/plan_fragment_executor.h /^ boost::mutex _report_thread_lock;$/;" m class:doris::PlanFragmentExecutor +_report_thread_started_cv src/runtime/plan_fragment_executor.h /^ boost::condition_variable _report_thread_started_cv;$/;" m class:doris::PlanFragmentExecutor +_req src/olap/delta_writer.h /^ WriteRequest _req;$/;" m class:doris::DeltaWriter +_request src/olap/push_handler.h /^ TPushReq _request;$/;" m class:doris::PushHandler +_request_body src/http/http_request.h /^ std::string _request_body;$/;" m class:doris::HttpRequest +_request_context_cache src/runtime/disk_io_mgr.h /^ boost::scoped_ptr _request_context_cache;$/;" m class:doris::DiskIoMgr +_request_type src/runtime/disk_io_mgr.h /^ RequestType::type _request_type;$/;" m class:doris::DiskIoMgr::RequestRange +_res_status src/olap/task/engine_batch_load_task.h /^ AgentStatus* _res_status;$/;" m class:doris::EngineBatchLoadTask +_res_status src/olap/task/engine_clone_task.h /^ AgentStatus* _res_status;$/;" m class:doris::EngineCloneTask +_reservation_counters src/runtime/mem_tracker.h /^ AtomicPtr _reservation_counters;$/;" m class:doris::MemTracker +_reserve_bytes_limit src/runtime/memory/chunk_allocator.h /^ size_t _reserve_bytes_limit;$/;" m class:doris::ChunkAllocator +_reserved_bytes src/runtime/memory/chunk_allocator.h /^ std::atomic _reserved_bytes;$/;" m class:doris::ChunkAllocator +_reserved_hours src/runtime/load_path_mgr.h /^ int _reserved_hours;$/;" m class:doris::LoadPathMgr +_reset_mem_table src/olap/delta_writer.cpp /^void DeltaWriter::_reset_mem_table() {$/;" f class:doris::DeltaWriter +_reset_readers src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_reset_readers() {$/;" f class:doris::SegmentReader +_reset_zone_map src/olap/rowset/segment_v2/zone_map_index.h /^ void _reset_zone_map(ZoneMap* zone_map) {$/;" f class:doris::segment_v2::ZoneMapIndexWriter +_resize src/olap/lru_cache.cpp /^bool HandleTable::_resize() {$/;" f class:doris::HandleTable +_resource_info src/exec/olap_scan_node.h /^ TResourceInfo* _resource_info;$/;" m class:doris::OlapScanNode +_resource_pool src/runtime/runtime_state.h /^ ThreadResourceMgr::ResourcePool* _resource_pool;$/;" m class:doris::RuntimeState +_resource_profile src/exec/exec_node.h /^ const TBackendResourceProfile _resource_profile;$/;" m class:doris::ExecNode +_restart_point_internal src/olap/rowset/segment_v2/binary_prefix_page.h /^ uint8_t _restart_point_internal = 0;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +_restart_points_offset src/olap/rowset/segment_v2/binary_prefix_page.h /^ std::vector _restart_points_offset;$/;" m class:doris::segment_v2::BinaryPrefixPageBuilder +_restarts_ptr src/olap/rowset/segment_v2/binary_prefix_page.h /^ const uint8_t* _restarts_ptr = nullptr;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +_restore src/http/action/restore_tablet_action.cpp /^Status RestoreTabletAction::_restore(const std::string& key, int64_t tablet_id, int32_t schema_hash) {$/;" f class:doris::RestoreTabletAction +_result src/exprs/expr_context.h /^ ExprValue _result;$/;" m class:doris::ExprContext +_result_comparator_fn output/udf/include/uda_test_harness.h /^ ResultComparator _result_comparator_fn;$/;" m class:doris_udf::UdaTestHarnessBase +_result_comparator_fn src/udf/uda_test_harness.h /^ ResultComparator _result_comparator_fn;$/;" m class:doris_udf::UdaTestHarnessBase +_result_expr_ctx_lists src/exec/merge_node.h /^ std::vector > _result_expr_ctx_lists;$/;" m class:doris::MergeNode +_result_mgr src/runtime/exec_env.h /^ ResultBufferMgr* _result_mgr = nullptr;$/;" m class:doris::ExecEnv +_result_queue_mgr src/runtime/exec_env.h /^ ResultQueueMgr* _result_queue_mgr = nullptr;$/;" m class:doris::ExecEnv +_result_tuple_desc src/exec/analytic_eval_node.h /^ const TupleDescriptor* _result_tuple_desc;$/;" m class:doris::AnalyticEvalNode +_result_tuple_row_size src/exec/blocking_join_node.h /^ int _result_tuple_row_size;$/;" m class:doris::BlockingJoinNode +_result_tuple_row_size src/exec/hash_join_node.h /^ int _result_tuple_row_size;$/;" m class:doris::HashJoinNode +_result_tuple_row_size src/exec/merge_join_node.h /^ int _result_tuple_row_size;$/;" m class:doris::MergeJoinNode +_result_tuples src/exec/analytic_eval_node.h /^ std::list > _result_tuples;$/;" m class:doris::AnalyticEvalNode +_results_buffer_size src/exec/hash_table.h /^ int _results_buffer_size;$/;" m class:doris::HashTable +_results_buffer_size src/exec/partitioned_hash_table.h /^ int _results_buffer_size;$/;" m class:doris::PartitionedHashTableCtx +_return_columns src/exec/olap_scanner.h /^ std::vector _return_columns;$/;" m class:doris::OlapScanner +_return_columns src/olap/reader.h /^ std::vector _return_columns;$/;" m class:doris::Reader +_return_columns src/olap/rowset/column_data.h /^ std::vector _return_columns;$/;" m class:doris::ColumnData +_return_type src/exprs/agg_fn_evaluator.h /^ const TypeDescriptor _return_type;$/;" m class:doris::AggFnEvaluator +_return_type src/udf/udf_internal.h /^ doris_udf::FunctionContext::TypeDesc _return_type;$/;" m class:doris::FunctionContextImpl +_reverted_index_size src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^ uint64_t _reverted_index_size;$/;" m class:doris::segment_v2::__anon54::BitmapIndexWriterImpl file: +_rhs_expr_ctxs src/runtime/qsorter.cpp /^ std::vector& _rhs_expr_ctxs;$/;" m class:doris::TupleRowLessThan file: +_rhs_expr_ctxs src/runtime/qsorter.h /^ std::vector _rhs_expr_ctxs;$/;" m class:doris::QSorter +_rhs_ordering_expr_ctxs src/exec/sort_exec_exprs.h /^ std::vector _rhs_ordering_expr_ctxs;$/;" m class:doris::SortExecExprs +_rhs_slot_ctx test/runtime/data_stream_test.cpp /^ ExprContext* _rhs_slot_ctx;$/;" m class:doris::DataStreamTest file: +_rid src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^ rowid_t _rid = 0;$/;" m class:doris::segment_v2::__anon54::BitmapIndexWriterImpl file: +_right_child_ctx src/exec/merge_join_node.h /^ boost::scoped_ptr _right_child_ctx;$/;" m class:doris::MergeJoinNode +_right_expr_ctxs src/exec/merge_join_node.h /^ std::vector _right_expr_ctxs;$/;" m class:doris::MergeJoinNode +_right_tuple_idx src/exec/merge_join_node.h /^ std::vector _right_tuple_idx;$/;" m class:doris::MergeJoinNode +_right_tuple_size src/exec/merge_join_node.h /^ int _right_tuple_size;$/;" m class:doris::MergeJoinNode +_rle_decoder src/olap/rowset/segment_v2/rle_page.h /^ RleDecoder _rle_decoder;$/;" m class:doris::segment_v2::RlePageDecoder +_rle_encoder src/olap/rowset/segment_v2/column_writer.cpp /^ RleEncoder _rle_encoder;$/;" m class:doris::segment_v2::NullBitmapBuilder file: +_rle_encoder src/olap/rowset/segment_v2/rle_page.h /^ RleEncoder* _rle_encoder;$/;" m class:doris::segment_v2::RlePageBuilder +_rollup_map src/runtime/data_spliter.h /^ std::map _rollup_map;$/;" m class:doris::DataSpliter +_rollup_map src/runtime/dpp_sink.h /^ const std::map& _rollup_map;$/;" m class:doris::DppSink +_rollup_name src/runtime/dpp_sink.cpp /^ const std::string& _rollup_name;$/;" m class:doris::Translator file: +_rollup_schema src/runtime/dpp_sink.cpp /^ const RollupSchema& _rollup_schema;$/;" m class:doris::Translator file: +_rollups test/runtime/dpp_sink_test.cpp /^ std::map _rollups;$/;" m class:doris::DppSinkTest file: +_root src/exprs/expr_context.h /^ Expr* _root;$/;" m class:doris::ExprContext +_root src/util/path_trie.hpp /^ TrieNode _root;$/;" m class:doris::PathTrie +_root_cgroups_path src/agent/cgroups_mgr.h /^ std::string _root_cgroups_path;$/;" m class:doris::CgroupsMgr +_root_node_id src/runtime/runtime_state.h /^ PlanNodeId _root_node_id;$/;" m class:doris::RuntimeState +_root_path src/olap/olap_meta.h /^ std::string _root_path;$/;" m class:doris::OlapMeta +_root_path test/olap/olap_meta_test.cpp /^ std::string _root_path;$/;" m class:doris::OlapMetaTest file: +_root_value src/util/path_trie.hpp /^ T* _root_value;$/;" m class:doris::PathTrie +_root_writers src/olap/rowset/segment_writer.h /^ std::vector _root_writers;$/;" m class:doris::SegmentWriter +_routine_load_task_executor src/runtime/exec_env.h /^ RoutineLoadTaskExecutor* _routine_load_task_executor = nullptr;$/;" m class:doris::ExecEnv +_row src/exec/partitioned_hash_table.h /^ TupleRow* _row;$/;" m class:doris::PartitionedHashTable::Iterator +_row src/exec/partitioned_hash_table.h /^ TupleRow* _row;$/;" m class:doris::PartitionedHashTableCtx +_row src/olap/row.h /^ void* _row;$/;" m struct:doris::ContiguousRow +_row src/olap/rowset/beta_rowset_reader.h /^ std::unique_ptr _row;$/;" m class:doris::BetaRowsetReader +_row src/runtime/row_batch.h /^ Tuple** _row;$/;" m class:doris::RowBatch::Iterator +_row_batch src/runtime/plan_fragment_executor.h /^ boost::scoped_ptr _row_batch;$/;" m class:doris::PlanFragmentExecutor +_row_batch test/runtime/dpp_writer_test.cpp /^ RowBatch _row_batch;$/;" m class:doris::DppWriterTest file: +_row_batch test/runtime/mysql_table_writer_test.cpp /^ RowBatch _row_batch;$/;" m class:doris::MysqlTableWriterTest file: +_row_batch_added_cv src/exec/olap_scan_node.h /^ boost::condition_variable _row_batch_added_cv;$/;" m class:doris::OlapScanNode +_row_batch_consumed_cv src/exec/olap_scan_node.h /^ boost::condition_variable _row_batch_consumed_cv;$/;" m class:doris::OlapScanNode +_row_batch_end src/runtime/row_batch.h /^ Tuple** const _row_batch_end;$/;" m class:doris::RowBatch::Iterator +_row_batches src/exec/row_batch_list.h /^ std::vector _row_batches;$/;" m class:doris::RowBatchList +_row_batches_lock src/exec/olap_scan_node.h /^ boost::mutex _row_batches_lock;$/;" m class:doris::OlapScanNode +_row_bitmap src/olap/rowset/segment_v2/segment_iterator.h /^ Roaring _row_bitmap;$/;" m class:doris::segment_v2::SegmentIterator +_row_block src/olap/reader.cpp /^ RowBlock* _row_block = nullptr;$/;" m class:doris::CollectIterator::ChildCtx file: +_row_block src/olap/rowset/column_data_writer.h /^ RowBlock* _row_block; \/\/ 使用RowBlcok缓存要写入的数据$/;" m class:doris::ColumnDataWriter +_row_block_allocator src/olap/schema_change.cpp /^ RowBlockAllocator* _row_block_allocator;$/;" m class:doris::RowBlockSorter file: +_row_block_allocator src/olap/schema_change.h /^ RowBlockAllocator* _row_block_allocator;$/;" m class:doris::SchemaChangeDirectly +_row_block_allocator src/olap/schema_change.h /^ RowBlockAllocator* _row_block_allocator;$/;" m class:doris::SchemaChangeWithSorting +_row_block_changer src/olap/schema_change.h /^ const RowBlockChanger& _row_block_changer;$/;" m class:doris::LinkedSchemaChange +_row_block_changer src/olap/schema_change.h /^ const RowBlockChanger& _row_block_changer;$/;" m class:doris::SchemaChangeDirectly +_row_block_changer src/olap/schema_change.h /^ const RowBlockChanger& _row_block_changer;$/;" m class:doris::SchemaChangeWithSorting +_row_buf src/olap/push_handler.h /^ char* _row_buf;$/;" m class:doris::BinaryReader +_row_buf src/olap/push_handler.h /^ char* _row_buf;$/;" m class:doris::LzoBinaryReader +_row_buf_size src/olap/push_handler.h /^ size_t _row_buf_size;$/;" m class:doris::BinaryReader +_row_buffer src/runtime/result_writer.h /^ MysqlRowBuffer* _row_buffer;$/;" m class:doris::ResultWriter +_row_comparator src/olap/memtable.h /^ RowCursorComparator _row_comparator;$/;" m class:doris::MemTable +_row_compressed_buf src/olap/push_handler.h /^ char* _row_compressed_buf;$/;" m class:doris::LzoBinaryReader +_row_count src/olap/rowset/segment_v2/segment_writer.h /^ uint32_t _row_count = 0;$/;" m class:doris::segment_v2::SegmentWriter +_row_count src/olap/rowset/segment_writer.h /^ uint64_t _row_count; \/\/ 已经写入的行总数$/;" m class:doris::SegmentWriter +_row_cursor src/olap/reader.cpp /^ RowCursor _row_cursor; \/\/ point to rows inside `_row_block`$/;" m class:doris::CollectIterator::ChildCtx file: +_row_cursor_comparator src/olap/schema_change.cpp /^ static bool _row_cursor_comparator(const RowCursor* a, const RowCursor* b) {$/;" f class:doris::RowBlockSorter file: +_row_desc src/exec/base_scanner.h /^ std::unique_ptr _row_desc;$/;" m class:doris::BaseScanner +_row_desc src/exec/es_http_scanner.h /^ std::unique_ptr _row_desc;$/;" m class:doris::EsHttpScanner +_row_desc src/runtime/data_spliter.h /^ const RowDescriptor& _row_desc;$/;" m class:doris::DataSpliter +_row_desc src/runtime/data_stream_recvr.h /^ RowDescriptor _row_desc;$/;" m class:doris::DataStreamRecvr +_row_desc src/runtime/data_stream_sender.cpp /^ const RowDescriptor& _row_desc;$/;" m class:doris::DataStreamSender::Channel file: +_row_desc src/runtime/data_stream_sender.h /^ const RowDescriptor& _row_desc;$/;" m class:doris::DataStreamSender +_row_desc src/runtime/dpp_sink.cpp /^ const RowDescriptor& _row_desc;$/;" m class:doris::Translator file: +_row_desc src/runtime/dpp_sink.h /^ const RowDescriptor& _row_desc;$/;" m class:doris::DppSink +_row_desc src/runtime/export_sink.h /^ const RowDescriptor& _row_desc;$/;" m class:doris::ExportSink +_row_desc src/runtime/memory_scratch_sink.h /^ const RowDescriptor& _row_desc;$/;" m class:doris::MemoryScratchSink +_row_desc src/runtime/mysql_table_sink.h /^ const RowDescriptor& _row_desc;$/;" m class:doris::MysqlTableSink +_row_desc src/runtime/qsorter.h /^ const RowDescriptor& _row_desc;$/;" m class:doris::QSorter +_row_desc src/runtime/result_sink.h /^ const RowDescriptor& _row_desc;$/;" m class:doris::ResultSink +_row_desc src/runtime/row_batch.h /^ RowDescriptor _row_desc;$/;" m class:doris::RowBatch +_row_desc src/runtime/tablets_channel.h /^ RowDescriptor* _row_desc = nullptr;$/;" m class:doris::TabletsChannel +_row_desc src/util/arrow/row_batch.cpp /^ const RowDescriptor& _row_desc;$/;" m class:doris::ToRowBatchConverter file: +_row_desc test/exec/tablet_sink_test.cpp /^ RowDescriptor* _row_desc = nullptr;$/;" m class:doris::stream_load::TestInternalService file: +_row_desc test/exprs/binary_predicate_test.cpp /^ RowDescriptor* _row_desc;$/;" m class:doris::BinaryOpTest file: +_row_desc test/exprs/in_op_test.cpp /^ RowDescriptor* _row_desc;$/;" m class:doris::InOpTest file: +_row_desc test/exprs/in_predicate_test.cpp /^ RowDescriptor _row_desc;$/;" m class:doris::InPredicateTest file: +_row_desc test/olap/vectorized_olap_reader_test.cpp /^ RowDescriptor* _row_desc;$/;" m class:doris::TestVectorizedOLAPReader file: +_row_desc test/runtime/data_spliter_test.cpp /^ RowDescriptor* _row_desc;$/;" m class:doris::DataSplitTest file: +_row_desc test/runtime/data_stream_test.cpp /^ const RowDescriptor* _row_desc;$/;" m class:doris::DataStreamTest file: +_row_desc test/runtime/dpp_sink_test.cpp /^ RowDescriptor* _row_desc;$/;" m class:doris::DppSinkTest file: +_row_desc test/runtime/dpp_writer_test.cpp /^ RowDescriptor _row_desc;$/;" m class:doris::DppWriterTest file: +_row_desc test/runtime/memory_scratch_sink_test.cpp /^ RowDescriptor* _row_desc;$/;" m class:doris::MemoryScratchSinkTest file: +_row_desc test/runtime/mysql_table_writer_test.cpp /^ RowDescriptor _row_desc;$/;" m class:doris::MysqlTableWriterTest file: +_row_desc test/runtime/qsorter_test.cpp /^ RowDescriptor* _row_desc;$/;" m class:doris::QSorterTest file: +_row_desc test/runtime/result_sink_test.cpp /^ RowDescriptor _row_desc;$/;" m class:doris::ResultSinkTest file: +_row_desc test/util/arrow/arrow_work_flow_test.cpp /^ RowDescriptor* _row_desc;$/;" m class:doris::ArrowWorkFlowTest file: +_row_descriptor src/exec/exec_node.h /^ RowDescriptor _row_descriptor;$/;" m class:doris::ExecNode +_row_idx src/exec/row_batch_list.h /^ int64_t _row_idx;$/;" m class:doris::RowBatchList::TupleRowIterator +_row_index src/olap/row_block2.h /^ size_t _row_index;$/;" m class:doris::RowBlockRow +_row_index src/olap/rowset/column_data_writer.h /^ uint32_t _row_index;$/;" m class:doris::ColumnDataWriter +_row_info_buf src/olap/push_handler.h /^ char* _row_info_buf;$/;" m class:doris::LzoBinaryReader +_row_len src/olap/schema_change.h /^ size_t _row_len;$/;" m class:doris::RowBlockAllocator +_row_num src/olap/push_handler.h /^ size_t _row_num;$/;" m class:doris::LzoBinaryReader +_row_offset src/olap/column_block.h /^ size_t _row_offset;$/;" m class:doris::ColumnBlockView +_row_offset src/olap/selection_vector.h /^ size_t _row_offset;$/;" m class:doris::SelectionVectorView +_row_reader src/exec/orc_scanner.h /^ std::unique_ptr _row_reader;$/;" m class:doris::ORCScanner +_row_reader_options src/exec/orc_scanner.h /^ orc::RowReaderOptions _row_reader_options;$/;" m class:doris::ORCScanner +_rows_end_offset src/exec/analytic_eval_node.h /^ int64_t _rows_end_offset;$/;" m class:doris::AnalyticEvalNode +_rows_of_group src/exec/orc_scanner.h /^ int64_t _rows_of_group; \/\/ rows in a group.$/;" m class:doris::ORCScanner +_rows_of_group src/exec/parquet_reader.h /^ int _rows_of_group; \/\/ rows in a group.$/;" m class:doris::ParquetReaderWrap +_rows_produced_counter src/runtime/plan_fragment_executor.h /^ RuntimeProfile::Counter* _rows_produced_counter;$/;" m class:doris::PlanFragmentExecutor +_rows_pushed_cond_filtered_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _rows_pushed_cond_filtered_counter = nullptr;$/;" m class:doris::OlapScanNode +_rows_pushed_cond_filtered_counter src/exec/olap_scanner.h /^ RuntimeProfile::Counter* _rows_pushed_cond_filtered_counter = nullptr;$/;" m class:doris::OlapScanner +_rows_read_counter src/exec/base_scanner.h /^ RuntimeProfile::Counter* _rows_read_counter;$/;" m class:doris::BaseScanner +_rows_read_counter src/exec/es_http_scanner.h /^ RuntimeProfile::Counter* _rows_read_counter;$/;" m class:doris::EsHttpScanner +_rows_read_counter src/exec/olap_scanner.h /^ RuntimeProfile::Counter* _rows_read_counter = nullptr;$/;" m class:doris::OlapScanner +_rows_read_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* _rows_read_counter;$/;" m class:doris::ScanNode +_rows_returned src/olap/generic_iterators.cpp /^ size_t _rows_returned;$/;" m class:doris::AutoIncrementIterator file: +_rows_returned src/runtime/buffered_tuple_stream.h /^ int64_t _rows_returned;$/;" m class:doris::BufferedTupleStream +_rows_returned src/runtime/buffered_tuple_stream2.h /^ int64_t _rows_returned;$/;" m class:doris::BufferedTupleStream2 +_rows_returned_counter src/exec/exec_node.h /^ RuntimeProfile::Counter* _rows_returned_counter;$/;" m class:doris::ExecNode +_rows_returned_rate src/exec/exec_node.h /^ RuntimeProfile::Counter* _rows_returned_rate;$/;" m class:doris::ExecNode +_rows_start_offset src/exec/analytic_eval_node.h /^ int64_t _rows_start_offset;$/;" m class:doris::AnalyticEvalNode +_rows_vec_cond_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _rows_vec_cond_counter = nullptr;$/;" m class:doris::OlapScanNode +_rows_written_counter src/runtime/export_sink.h /^ RuntimeProfile::Counter* _rows_written_counter;$/;" m class:doris::ExportSink +_rowset src/olap/rowset/alpha_rowset_reader.h /^ AlphaRowsetSharedPtr _rowset;$/;" m class:doris::AlphaRowsetReader +_rowset src/olap/rowset/beta_rowset_reader.h /^ BetaRowsetSharedPtr _rowset;$/;" m class:doris::BetaRowsetReader +_rowset_build src/olap/rowset/alpha_rowset_writer.h /^ bool _rowset_build;$/;" m class:doris::AlphaRowsetWriter +_rowset_id src/olap/rowset/rowset_meta.h /^ RowsetId _rowset_id;$/;" m class:doris::RowsetMeta +_rowset_id src/olap/rowset/segment_group.h /^ RowsetId _rowset_id;$/;" m class:doris::SegmentGroup +_rowset_id_generator src/olap/storage_engine.h /^ std::unique_ptr _rowset_id_generator;$/;" m class:doris::StorageEngine +_rowset_meta src/olap/rowset/beta_rowset_writer.h /^ std::shared_ptr _rowset_meta;$/;" m class:doris::BetaRowsetWriter +_rowset_meta src/olap/rowset/rowset.h /^ RowsetMetaSharedPtr _rowset_meta;$/;" m class:doris::Rowset +_rowset_meta_pb src/olap/rowset/rowset_meta.h /^ RowsetMetaPB _rowset_meta_pb;$/;" m class:doris::RowsetMeta +_rowset_path src/olap/rowset/alpha_rowset_reader.h /^ std::string _rowset_path;$/;" m class:doris::AlphaRowsetReader +_rowset_path src/olap/rowset/rowset.h /^ std::string _rowset_path;$/;" m class:doris::Rowset +_rowset_path_prefix src/olap/rowset/segment_group.h /^ std::string _rowset_path_prefix; \/\/ path of rowset$/;" m class:doris::SegmentGroup +_rowset_state src/olap/rowset/rowset.h /^ RowsetState _rowset_state;$/;" m class:doris::RowsetStateMachine +_rowset_state_machine src/olap/rowset/rowset.h /^ RowsetStateMachine _rowset_state_machine;$/;" m class:doris::Rowset +_rowset_with_largest_size src/olap/tablet.cpp /^RowsetSharedPtr Tablet::_rowset_with_largest_size() {$/;" f class:doris::Tablet +_rowset_writer src/olap/delta_writer.h /^ std::unique_ptr _rowset_writer;$/;" m class:doris::DeltaWriter +_rowset_writer src/olap/memtable.h /^ RowsetWriter* _rowset_writer;$/;" m class:doris::MemTable +_rowset_writer_context src/olap/rowset/alpha_rowset_writer.h /^ RowsetWriterContext _rowset_writer_context;$/;" m class:doris::AlphaRowsetWriter +_rpc_timeout_ms src/exec/tablet_sink.h /^ int _rpc_timeout_ms = 60000;$/;" m class:doris::stream_load::NodeChannel +_rs_graph src/olap/tablet.h /^ RowsetGraph _rs_graph;$/;" m class:doris::Tablet +_rs_metas src/olap/tablet_meta.h /^ std::vector _rs_metas;$/;" m class:doris::TabletMeta +_rs_reader src/olap/reader.cpp /^ RowsetReaderSharedPtr _rs_reader;$/;" m class:doris::CollectIterator::ChildCtx file: +_rs_readers src/olap/reader.h /^ std::vector _rs_readers;$/;" m class:doris::Reader +_rs_version_map src/olap/tablet.h /^ std::unordered_map _rs_version_map;$/;" m class:doris::Tablet +_run src/runtime/merge_sorter.cpp /^ Run* _run;$/;" m class:doris::MergeSorter::TupleSorter file: +_run src/runtime/spill_sorter.cc /^ Run* _run;$/;" m class:doris::SpillSorter::TupleSorter file: +_running src/util/stopwatch.hpp /^ bool _running;$/;" m class:doris::MonotonicStopWatch +_running_jobs src/runtime/etl_job_mgr.h /^ std::unordered_set _running_jobs;$/;" m class:doris::EtlJobMgr +_running_tasks src/runtime/export_task_mgr.h /^ std::unordered_set _running_tasks;$/;" m class:doris::ExportTaskMgr +_running_thread src/exec/olap_scan_node.h /^ int64_t _running_thread;$/;" m class:doris::OlapScanNode +_runtim_state test/exec/mysql_scan_node_test.cpp /^ RuntimeState _runtim_state;$/;" m class:doris::MysqlScanNodeTest file: +_runtim_state test/exec/schema_scan_node_test.cpp /^ RuntimeState _runtim_state;$/;" m class:doris::SchemaScanNodeTest file: +_runtim_state test/exec/set_executor_test.cpp /^ RuntimeState _runtim_state;$/;" m class:doris::SetExecutorTest file: +_runtime_exec_options src/exec/exec_node.h /^ std::string _runtime_exec_options;$/;" m class:doris::ExecNode +_runtime_profile src/exec/exec_node.h /^ boost::scoped_ptr _runtime_profile;$/;" m class:doris::ExecNode +_runtime_profile test/runtime/sorter_test.cpp /^ RuntimeProfile *_runtime_profile;$/;" m class:doris::SorterTest file: +_runtime_stat test/exec/new_olap_scan_node_test.cpp /^ RuntimeState _runtime_stat;$/;" m class:doris::TestOlapScanNode file: +_runtime_stat test/exec/olap_scan_node_test.cpp /^ RuntimeState _runtime_stat;$/;" m class:doris::OlapScanNodeTest file: +_runtime_stat test/exprs/in_predicate_test.cpp /^ RuntimeState _runtime_stat;$/;" m class:doris::InPredicateTest file: +_runtime_stat test/olap/olap_reader_test.cpp /^ RuntimeState _runtime_stat;$/;" m class:doris::TestOLAPReaderColumn file: +_runtime_stat test/olap/olap_reader_test.cpp /^ RuntimeState _runtime_stat;$/;" m class:doris::TestOLAPReaderColumnDeleteCondition file: +_runtime_stat test/olap/olap_reader_test.cpp /^ RuntimeState _runtime_stat;$/;" m class:doris::TestOLAPReaderRow file: +_runtime_stat test/olap/vectorized_olap_reader_test.cpp /^ RuntimeState _runtime_stat;$/;" m class:doris::TestVectorizedOLAPReader file: +_runtime_state src/exec/broker_scan_node.h /^ RuntimeState* _runtime_state;$/;" m class:doris::BrokerScanNode +_runtime_state src/exec/csv_scan_node.h /^ RuntimeState* _runtime_state;$/;" m class:doris::CsvScanNode +_runtime_state src/exec/es_http_scan_node.h /^ RuntimeState* _runtime_state;$/;" m class:doris::EsHttpScanNode +_runtime_state src/exec/kudu_scan_node.h /^ RuntimeState* _runtime_state;$/;" m class:doris::KuduScanNode +_runtime_state src/exec/olap_scan_node.h /^ RuntimeState* _runtime_state;$/;" m class:doris::OlapScanNode +_runtime_state src/exec/olap_scanner.h /^ RuntimeState* _runtime_state;$/;" m class:doris::OlapScanner +_runtime_state src/exec/repeat_node.h /^ RuntimeState* _runtime_state;$/;" m class:doris::RepeatNode +_runtime_state src/olap/rowset/column_data.h /^ RuntimeState* _runtime_state;$/;" m class:doris::ColumnData +_runtime_state src/olap/rowset/segment_reader.h /^ RuntimeState* _runtime_state; \/\/ 用于统计内存消耗等运行时信息$/;" m class:doris::SegmentReader +_runtime_state src/runtime/plan_fragment_executor.h /^ boost::scoped_ptr _runtime_state;$/;" m class:doris::PlanFragmentExecutor +_runtime_state test/exec/broker_scan_node_test.cpp /^ RuntimeState _runtime_state;$/;" m class:doris::BrokerScanNodeTest file: +_runtime_state test/exec/broker_scanner_test.cpp /^ RuntimeState _runtime_state;$/;" m class:doris::BrokerScannerTest file: +_runtime_state test/exec/es_http_scan_node_test.cpp /^ RuntimeState _runtime_state;$/;" m class:doris::EsHttpScanNodeTest file: +_runtime_state test/exec/es_predicate_test.cpp /^ RuntimeState _runtime_state;$/;" m class:doris::EsPredicateTest file: +_runtime_state test/exec/es_scan_node_test.cpp /^ RuntimeState _runtime_state;$/;" m class:doris::EsScanNodeTest file: +_runtime_state test/exec/orc_scanner_test.cpp /^ RuntimeState _runtime_state;$/;" m class:doris::OrcScannerTest file: +_runtime_state test/exec/parquet_scanner_test.cpp /^ RuntimeState _runtime_state;$/;" m class:doris::ParquetSannerTest file: +_runtime_state test/exec/partitioned_hash_table_test.cpp /^ RuntimeState* _runtime_state;$/;" m class:doris::PartitionedHashTableTest file: +_runtime_state test/exprs/binary_predicate_test.cpp /^ RuntimeState* _runtime_state;$/;" m class:doris::BinaryOpTest file: +_runtime_state test/exprs/in_op_test.cpp /^ RuntimeState* _runtime_state;$/;" m class:doris::InOpTest file: +_runtime_state test/runtime/buffered_tuple_stream2_test.cpp /^ RuntimeState* _runtime_state;$/;" m class:doris::SimpleTupleStreamTest file: +_runtime_state test/runtime/buffered_tuple_stream_test.cpp /^ RuntimeState *_runtime_state;$/;" m class:doris::BufferedTupleStreamTest file: +_runtime_state test/runtime/data_stream_test.cpp /^ RuntimeState _runtime_state;$/;" m class:doris::DataStreamTest file: +_runtime_state test/runtime/result_sink_test.cpp /^ RuntimeState* _runtime_state;$/;" m class:doris::ResultSinkTest file: +_runtime_state test/runtime/result_writer_test.cpp /^ RuntimeState* _runtime_state;$/;" m class:doris::ResultWriterTest file: +_runtime_state test/runtime/sorter_test.cpp /^ RuntimeState *_runtime_state;$/;" m class:doris::SorterTest file: +_rw_lock src/http/ev_http_server.h /^ pthread_rwlock_t _rw_lock;$/;" m class:doris::EvHttpServer +_s_all_range src/runtime/dpp_sink_internal.cpp /^PartRange PartRange::_s_all_range = {$/;" m class:doris::PartRange file: +_s_all_range src/runtime/dpp_sink_internal.h /^ static PartRange _s_all_range;$/;" m class:doris::PartRange +_s_alphanumeric_chars src/exprs/math_functions.cpp /^const char* MathFunctions::_s_alphanumeric_chars = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ";$/;" m class:doris::MathFunctions file: +_s_alphanumeric_chars src/exprs/math_functions.h /^ static const char* _s_alphanumeric_chars;$/;" m class:doris::MathFunctions +_s_at src/util/url_parser.cpp /^const StringSearch UrlParser::_s_at_search(&_s_at);$/;" m class:doris::UrlParser file: +_s_at src/util/url_parser.h /^ static const StringValue _s_at;$/;" m class:doris::UrlParser +_s_at_search src/util/url_parser.h /^ static const StringSearch _s_at_search;$/;" m class:doris::UrlParser +_s_available_space_threshold_mb src/runtime/tmp_file_mgr.cc /^const uint64_t _s_available_space_threshold_mb = 1024;$/;" m namespace:doris file: +_s_available_space_threshold_mb src/runtime/tmp_file_mgr.h /^ const static uint64_t _s_available_space_threshold_mb;$/;" m class:doris::TmpFileMgr::File +_s_average_io_mgr_queue_capacity src/exec/scan_node.h /^ static const std::string _s_average_io_mgr_queue_capacity;$/;" m class:doris::ScanNode +_s_block_mgrs_lock src/runtime/buffered_block_mgr2.cc /^SpinLock BufferedBlockMgr2::_s_block_mgrs_lock;$/;" m class:doris::BufferedBlockMgr2 file: +_s_block_mgrs_lock src/runtime/buffered_block_mgr2.h /^ static SpinLock _s_block_mgrs_lock;$/;" m class:doris::BufferedBlockMgr2 +_s_bytes_read_counter src/exec/scan_node.cpp /^const string ScanNode::_s_bytes_read_counter = "BytesRead";$/;" m class:doris::ScanNode file: +_s_bytes_read_counter src/exec/scan_node.h /^ static const std::string _s_bytes_read_counter;$/;" m class:doris::ScanNode +_s_cgroup_path test/agent/cgroups_mgr_test.cpp /^ static std::string _s_cgroup_path;$/;" m class:doris::CgroupsMgrTest file: +_s_cgroup_path test/agent/cgroups_mgr_test.cpp /^std::string CgroupsMgrTest::_s_cgroup_path = ".\/doris_cgroup_testxxxx123";$/;" m class:doris::CgroupsMgrTest file: +_s_cgroups_mgr test/agent/cgroups_mgr_test.cpp /^ static CgroupsMgr _s_cgroups_mgr;$/;" m class:doris::CgroupsMgrTest file: +_s_charsets src/exec/schema_scanner/schema_charsets_scanner.cpp /^SchemaCharsetsScanner::CharsetStruct SchemaCharsetsScanner::_s_charsets[] = {$/;" m class:doris::SchemaCharsetsScanner file: +_s_charsets src/exec/schema_scanner/schema_charsets_scanner.h /^ static CharsetStruct _s_charsets[];$/;" m class:doris::SchemaCharsetsScanner +_s_col_columns src/exec/schema_scanner/schema_columns_scanner.cpp /^SchemaScanner::ColumnDesc SchemaColumnsScanner::_s_col_columns[] = {$/;" m class:doris::SchemaColumnsScanner file: +_s_col_columns src/exec/schema_scanner/schema_columns_scanner.h /^ static SchemaScanner::ColumnDesc _s_col_columns[];$/;" m class:doris::SchemaColumnsScanner +_s_collations src/exec/schema_scanner/schema_collations_scanner.cpp /^SchemaCollationsScanner::CollationStruct SchemaCollationsScanner::_s_collations[] = {$/;" m class:doris::SchemaCollationsScanner file: +_s_collations src/exec/schema_scanner/schema_collations_scanner.h /^ static CollationStruct _s_collations[];$/;" m class:doris::SchemaCollationsScanner +_s_colon src/util/url_parser.cpp /^const StringSearch UrlParser::_s_colon_search(&_s_colon);$/;" m class:doris::UrlParser file: +_s_colon src/util/url_parser.h /^ static const StringValue _s_colon;$/;" m class:doris::UrlParser +_s_colon_search src/util/url_parser.h /^ static const StringSearch _s_colon_search;$/;" m class:doris::UrlParser +_s_cols_columns src/exec/schema_scanner/schema_collations_scanner.cpp /^SchemaScanner::ColumnDesc SchemaCollationsScanner::_s_cols_columns[] = {$/;" m class:doris::SchemaCollationsScanner file: +_s_cols_columns src/exec/schema_scanner/schema_collations_scanner.h /^ static SchemaScanner::ColumnDesc _s_cols_columns[];$/;" m class:doris::SchemaCollationsScanner +_s_columns src/exec/schema_scanner/schema_schemata_scanner.cpp /^SchemaScanner::ColumnDesc SchemaSchemataScanner::_s_columns[] = {$/;" m class:doris::SchemaSchemataScanner file: +_s_columns src/exec/schema_scanner/schema_schemata_scanner.h /^ static SchemaScanner::ColumnDesc _s_columns[];$/;" m class:doris::SchemaSchemataScanner +_s_css_columns src/exec/schema_scanner/schema_charsets_scanner.cpp /^SchemaScanner::ColumnDesc SchemaCharsetsScanner::_s_css_columns[] = {$/;" m class:doris::SchemaCharsetsScanner file: +_s_css_columns src/exec/schema_scanner/schema_charsets_scanner.h /^ static SchemaScanner::ColumnDesc _s_css_columns[];$/;" m class:doris::SchemaCharsetsScanner +_s_current test/olap/lru_cache_test.cpp /^ static CacheTest* _s_current;$/;" m class:doris::CacheTest file: +_s_current test/olap/lru_cache_test.cpp /^CacheTest* CacheTest::_s_current;$/;" m class:doris::CacheTest file: +_s_device_id_to_disk_id src/util/disk_info.cpp /^std::map DiskInfo::_s_device_id_to_disk_id;$/;" m class:doris::DiskInfo file: +_s_device_id_to_disk_id src/util/disk_info.h /^ static std::map _s_device_id_to_disk_id;$/;" m class:doris::DiskInfo +_s_disk_name_to_disk_id src/util/disk_info.cpp /^std::map DiskInfo::_s_disk_name_to_disk_id;$/;" m class:doris::DiskInfo file: +_s_disk_name_to_disk_id src/util/disk_info.h /^ static std::map _s_disk_name_to_disk_id;$/;" m class:doris::DiskInfo +_s_disks src/util/disk_info.cpp /^std::vector DiskInfo::_s_disks;$/;" m class:doris::DiskInfo file: +_s_disks src/util/disk_info.h /^ static std::vector _s_disks;$/;" m class:doris::DiskInfo +_s_doris_home src/util/path_builder.cpp /^const char* PathBuilder::_s_doris_home;$/;" m class:doris::PathBuilder file: +_s_doris_home src/util/path_builder.h /^ static const char* _s_doris_home;$/;" m class:doris::PathBuilder +_s_doris_metrics src/util/doris_metrics.cpp /^DorisMetrics DorisMetrics::_s_doris_metrics;$/;" m class:doris::DorisMetrics file: +_s_doris_metrics src/util/doris_metrics.h /^ static DorisMetrics _s_doris_metrics;$/;" m class:doris::DorisMetrics +_s_doris_server src/exec/schema_scanner.cpp /^DorisServer* SchemaScanner::_s_doris_server;$/;" m class:doris::SchemaScanner file: +_s_doris_server src/exec/schema_scanner.h /^ static DorisServer* _s_doris_server;$/;" m class:doris::SchemaScanner +_s_dummy_columns src/exec/schema_scanner/schema_dummy_scanner.cpp /^SchemaScanner::ColumnDesc SchemaDummyScanner::_s_dummy_columns[] = {$/;" m class:doris::SchemaDummyScanner file: +_s_dummy_columns src/exec/schema_scanner/schema_dummy_scanner.h /^ static SchemaScanner::ColumnDesc _s_dummy_columns[];$/;" m class:doris::SchemaDummyScanner +_s_exec_env src/util/thrift_rpc_helper.h /^ static ExecEnv* _s_exec_env;$/;" m class:doris::ThriftRpcHelper +_s_fd_cache src/olap/file_helper.cpp /^Cache* FileHandler::_s_fd_cache;$/;" m class:doris::FileHandler file: +_s_fd_cache src/olap/file_helper.h /^ static Cache* _s_fd_cache;$/;" m class:doris::FileHandler +_s_fieldlist src/common/configbase.cpp /^std::list* Register::_s_fieldlist = NULL;$/;" m class:doris::config::Register file: +_s_fieldlist src/common/configbase.h /^ static std::list* _s_fieldlist;$/;" m class:doris::config::Register +_s_get_constant_symbol_prefix src/exprs/expr.cpp /^const char* Expr::_s_get_constant_symbol_prefix = "_ZN4doris4Expr12get_constant";$/;" m class:doris::Expr file: +_s_get_constant_symbol_prefix src/exprs/expr.h /^ static const char* _s_get_constant_symbol_prefix;$/;" m class:doris::Expr +_s_hash src/util/url_parser.cpp /^const StringSearch UrlParser::_s_hash_search(&_s_hash);$/;" m class:doris::UrlParser file: +_s_hash src/util/url_parser.h /^ static const StringValue _s_hash;$/;" m class:doris::UrlParser +_s_hash_search src/util/url_parser.h /^ static const StringSearch _s_hash_search;$/;" m class:doris::UrlParser +_s_hook_name src/util/doris_metrics.cpp /^const char* DorisMetrics::_s_hook_name = "doris_metrics";$/;" m class:doris::DorisMetrics file: +_s_hook_name src/util/doris_metrics.h /^ static const char* _s_hook_name;$/;" m class:doris::DorisMetrics +_s_hook_name src/util/system_metrics.cpp /^const char* SystemMetrics::_s_hook_name = "system_metrics";$/;" m class:doris::SystemMetrics file: +_s_hook_name src/util/system_metrics.h /^ static const char* _s_hook_name;$/;" m class:doris::SystemMetrics +_s_initialized src/util/disk_info.cpp /^bool DiskInfo::_s_initialized;$/;" m class:doris::DiskInfo file: +_s_initialized src/util/disk_info.h /^ static bool _s_initialized;$/;" m class:doris::DiskInfo +_s_initialized src/util/mem_info.cpp /^bool MemInfo::_s_initialized = false;$/;" m class:doris::MemInfo file: +_s_initialized src/util/mem_info.h /^ static bool _s_initialized;$/;" m class:doris::MemInfo +_s_instance src/olap/page_cache.cpp /^StoragePageCache* StoragePageCache::_s_instance = &s_ut_cache;$/;" m class:doris::StoragePageCache file: +_s_instance src/olap/page_cache.h /^ static StoragePageCache* _s_instance;$/;" m class:doris::StoragePageCache +_s_instance src/olap/snapshot_manager.cpp /^SnapshotManager* SnapshotManager::_s_instance = nullptr;$/;" m class:doris::SnapshotManager file: +_s_instance src/olap/snapshot_manager.h /^ static SnapshotManager* _s_instance;$/;" m class:doris::SnapshotManager +_s_instance src/olap/storage_engine.cpp /^StorageEngine* StorageEngine::_s_instance = nullptr;$/;" m class:doris::StorageEngine file: +_s_instance src/olap/storage_engine.h /^ static StorageEngine* _s_instance;$/;" m class:doris::StorageEngine +_s_instance src/runtime/memory/chunk_allocator.cpp /^ChunkAllocator* ChunkAllocator::_s_instance = nullptr;$/;" m class:doris::ChunkAllocator file: +_s_instance src/runtime/memory/chunk_allocator.h /^ static ChunkAllocator* _s_instance;$/;" m class:doris::ChunkAllocator +_s_localhost src/service/backend_options.cpp /^std::string BackendOptions::_s_localhost;$/;" m class:doris::BackendOptions file: +_s_localhost src/service/backend_options.h /^ static std::string _s_localhost;$/;" m class:doris::BackendOptions +_s_materialize_tuple_timer src/exec/scan_node.cpp /^const string ScanNode::_s_materialize_tuple_timer = "MaterializeTupleTime(*)";$/;" m class:doris::ScanNode file: +_s_materialize_tuple_timer src/exec/scan_node.h /^ static const std::string _s_materialize_tuple_timer;$/;" m class:doris::ScanNode +_s_max_datetime_value src/runtime/datetime_value.h /^ static DateTimeValue _s_max_datetime_value;$/;" m class:doris::DateTimeValue +_s_mem_trackers_lock src/runtime/mem_tracker.h /^ static std::mutex _s_mem_trackers_lock;$/;" m class:doris::MemTracker +_s_min_datetime_value src/runtime/datetime_value.h /^ static DateTimeValue _s_min_datetime_value;$/;" m class:doris::DateTimeValue +_s_neg_infinite src/runtime/dpp_sink_internal.h /^ static PartRangeKey _s_neg_infinite;$/;" m class:doris::PartRangeKey +_s_num_datanode_dirs src/util/disk_info.cpp /^int DiskInfo::_s_num_datanode_dirs;$/;" m class:doris::DiskInfo file: +_s_num_datanode_dirs src/util/disk_info.h /^ static int _s_num_datanode_dirs;$/;" m class:doris::DiskInfo +_s_num_disks_accessed_counter src/exec/scan_node.cpp /^const string ScanNode::_s_num_disks_accessed_counter = "NumThread";$/;" m class:doris::ScanNode file: +_s_num_disks_accessed_counter src/exec/scan_node.h /^ static const std::string _s_num_disks_accessed_counter;$/;" m class:doris::ScanNode +_s_num_scanner_threads_started src/exec/scan_node.cpp /^const string ScanNode::_s_num_scanner_threads_started ="NumScannerThreadsStarted";$/;" m class:doris::ScanNode file: +_s_num_scanner_threads_started src/exec/scan_node.h /^ static const std::string _s_num_scanner_threads_started;$/;" m class:doris::ScanNode +_s_per_read_thread_throughput_counter src/exec/scan_node.cpp /^const string ScanNode::_s_per_read_thread_throughput_counter =$/;" m class:doris::ScanNode file: +_s_per_read_thread_throughput_counter src/exec/scan_node.h /^ static const std::string _s_per_read_thread_throughput_counter;$/;" m class:doris::ScanNode +_s_periodic_counter_update_state src/util/runtime_profile.cpp /^RuntimeProfile::PeriodicCounterUpdateState RuntimeProfile::_s_periodic_counter_update_state;$/;" m class:doris::RuntimeProfile file: +_s_periodic_counter_update_state src/util/runtime_profile.h /^ static PeriodicCounterUpdateState _s_periodic_counter_update_state;$/;" m class:doris::RuntimeProfile +_s_physical_mem src/util/mem_info.cpp /^int64_t MemInfo::_s_physical_mem = -1;$/;" m class:doris::MemInfo file: +_s_physical_mem src/util/mem_info.h /^ static int64_t _s_physical_mem;$/;" m class:doris::MemInfo +_s_pos_infinite src/runtime/dpp_sink_internal.h /^ static PartRangeKey _s_pos_infinite;$/;" m class:doris::PartRangeKey +_s_preferences src/util/block_compression.cpp /^ static LZ4F_preferences_t _s_preferences;$/;" m class:doris::Lz4fBlockCompression file: +_s_preferences src/util/block_compression.cpp /^LZ4F_preferences_t Lz4fBlockCompression::_s_preferences = {$/;" m class:doris::Lz4fBlockCompression file: +_s_priority_cidrs src/service/backend_options.cpp /^std::vector BackendOptions::_s_priority_cidrs;$/;" m class:doris::BackendOptions file: +_s_priority_cidrs src/service/backend_options.h /^ static std::vector _s_priority_cidrs;$/;" m class:doris::BackendOptions +_s_protocol src/util/url_parser.cpp /^const StringSearch UrlParser::_s_protocol_search(&_s_protocol);$/;" m class:doris::UrlParser file: +_s_protocol src/util/url_parser.h /^ static const StringValue _s_protocol;$/;" m class:doris::UrlParser +_s_protocol_search src/util/url_parser.h /^ static const StringSearch _s_protocol_search;$/;" m class:doris::UrlParser +_s_query_to_block_mgrs src/runtime/buffered_block_mgr2.cc /^BufferedBlockMgr2::BlockMgrsMap BufferedBlockMgr2::_s_query_to_block_mgrs;$/;" m class:doris::BufferedBlockMgr2 file: +_s_query_to_block_mgrs src/runtime/buffered_block_mgr2.h /^ static BlockMgrsMap _s_query_to_block_mgrs;$/;" m class:doris::BufferedBlockMgr2 +_s_question src/util/url_parser.cpp /^const StringSearch UrlParser::_s_question_search(&_s_question);$/;" m class:doris::UrlParser file: +_s_question src/util/url_parser.h /^ static const StringValue _s_question;$/;" m class:doris::UrlParser +_s_question_search src/util/url_parser.h /^ static const StringSearch _s_question_search;$/;" m class:doris::UrlParser +_s_report_version src/agent/task_worker_pool.h /^ static std::atomic_ulong _s_report_version;$/;" m class:doris::TaskWorkerPool +_s_request_to_mem_trackers src/runtime/mem_tracker.h /^ static RequestTrackersMap _s_request_to_mem_trackers;$/;" m class:doris::MemTracker +_s_resource_cgroups src/agent/cgroups_mgr.cpp /^std::map CgroupsMgr::_s_resource_cgroups =$/;" m class:doris::CgroupsMgr file: +_s_resource_cgroups src/agent/cgroups_mgr.h /^ static std::map _s_resource_cgroups; $/;" m class:doris::CgroupsMgr +_s_rows_read_counter src/exec/scan_node.cpp /^const string ScanNode::_s_rows_read_counter = "RowsRead";$/;" m class:doris::ScanNode file: +_s_rows_read_counter src/exec/scan_node.h /^ static const std::string _s_rows_read_counter;$/;" m class:doris::ScanNode +_s_scan_ranges_complete_counter src/exec/scan_node.cpp /^const string ScanNode::_s_scan_ranges_complete_counter = "ScanRangesComplete";$/;" m class:doris::ScanNode file: +_s_scan_ranges_complete_counter src/exec/scan_node.h /^ static const std::string _s_scan_ranges_complete_counter;$/;" m class:doris::ScanNode +_s_scanner_thread_counters_prefix src/exec/scan_node.cpp /^const string ScanNode::_s_scanner_thread_counters_prefix = "ScannerThreads";$/;" m class:doris::ScanNode file: +_s_scanner_thread_counters_prefix src/exec/scan_node.h /^ static const std::string _s_scanner_thread_counters_prefix;$/;" m class:doris::ScanNode +_s_scanner_thread_total_wallclock_time src/exec/scan_node.cpp /^const string ScanNode::_s_scanner_thread_total_wallclock_time =$/;" m class:doris::ScanNode file: +_s_scanner_thread_total_wallclock_time src/exec/scan_node.h /^ static const std::string _s_scanner_thread_total_wallclock_time;$/;" m class:doris::ScanNode +_s_slash src/util/url_parser.cpp /^const StringSearch UrlParser::_s_slash_search(&_s_slash);$/;" m class:doris::UrlParser file: +_s_slash src/util/url_parser.h /^ static const StringValue _s_slash;$/;" m class:doris::UrlParser +_s_slash_search src/util/url_parser.h /^ static const StringSearch _s_slash_search;$/;" m class:doris::UrlParser +_s_static_metrics src/runtime/test_env.cc /^boost::scoped_ptr TestEnv::_s_static_metrics;$/;" m class:doris::TestEnv file: +_s_static_metrics src/runtime/test_env.h /^ static boost::scoped_ptr _s_static_metrics;$/;" m class:doris::TestEnv +_s_system_group src/agent/cgroups_mgr.cpp /^const std::string CgroupsMgr::_s_system_group = "normal";$/;" m class:doris::CgroupsMgr file: +_s_system_group src/agent/cgroups_mgr.h /^ const static std::string _s_system_group;$/;" m class:doris::CgroupsMgr +_s_system_user src/agent/cgroups_mgr.cpp /^const std::string CgroupsMgr::_s_system_user = "system";$/;" m class:doris::CgroupsMgr file: +_s_system_user src/agent/cgroups_mgr.h /^ const static std::string _s_system_user;$/;" m class:doris::CgroupsMgr +_s_task_signatures src/agent/task_worker_pool.cpp /^map> TaskWorkerPool::_s_task_signatures;$/;" m class:doris::TaskWorkerPool file: +_s_task_signatures src/agent/task_worker_pool.h /^ static std::map> _s_task_signatures;$/;" m class:doris::TaskWorkerPool +_s_task_signatures_lock src/agent/task_worker_pool.cpp /^Mutex TaskWorkerPool::_s_task_signatures_lock;$/;" m class:doris::TaskWorkerPool file: +_s_task_signatures_lock src/agent/task_worker_pool.h /^ static Mutex _s_task_signatures_lock;$/;" m class:doris::TaskWorkerPool +_s_tbls_columns src/exec/schema_scanner/schema_tables_scanner.cpp /^SchemaScanner::ColumnDesc SchemaTablesScanner::_s_tbls_columns[] = {$/;" m class:doris::SchemaTablesScanner file: +_s_tbls_columns src/exec/schema_scanner/schema_tables_scanner.h /^ static SchemaScanner::ColumnDesc _s_tbls_columns[];$/;" m class:doris::SchemaTablesScanner +_s_test_data_path test/olap/file_helper_test.cpp /^ static std::string _s_test_data_path;$/;" m class:doris::FileHandlerTest file: +_s_test_data_path test/olap/file_helper_test.cpp /^std::string FileHandlerTest::_s_test_data_path = ".\/log\/file_handler_testxxxx123";$/;" m class:doris::FileHandlerTest file: +_s_test_data_path test/olap/file_utils_test.cpp /^ static std::string _s_test_data_path;$/;" m class:doris::FileUtilsTest file: +_s_test_data_path test/olap/file_utils_test.cpp /^std::string FileUtilsTest::_s_test_data_path = ".\/file_utils_testxxxx123";$/;" m class:doris::FileUtilsTest file: +_s_timezone_database_str src/exprs/timezone_db.cpp /^const char* TimezoneDatabase::_s_timezone_database_str =$/;" m class:doris::TimezoneDatabase file: +_s_timezone_database_str src/exprs/timezone_db.h /^ static const char *_s_timezone_database_str;$/;" m class:doris::TimezoneDatabase +_s_tmp_sub_dir_name src/runtime/tmp_file_mgr.cc /^const std::string _s_tmp_sub_dir_name = "doris-scratch";$/;" m namespace:doris file: +_s_tmp_sub_dir_name src/runtime/tmp_file_mgr.h /^ const static std::string _s_tmp_sub_dir_name;$/;" m class:doris::TmpFileMgr::File +_s_total_read_timer src/exec/scan_node.cpp /^const string ScanNode::_s_total_read_timer = "TotalRawReadTime(*)";$/;" m class:doris::ScanNode file: +_s_total_read_timer src/exec/scan_node.h /^ static const std::string _s_total_read_timer;$/;" m class:doris::ScanNode +_s_total_throughput_counter src/exec/scan_node.cpp /^const string ScanNode::_s_total_throughput_counter = "TotalReadThroughput";$/;" m class:doris::ScanNode file: +_s_total_throughput_counter src/exec/scan_node.h /^ static const std::string _s_total_throughput_counter;$/;" m class:doris::ScanNode +_s_tz_database src/exprs/timezone_db.cpp /^boost::local_time::tz_database TimezoneDatabase::_s_tz_database;$/;" m class:doris::TimezoneDatabase file: +_s_tz_database src/exprs/timezone_db.h /^ static boost::local_time::tz_database _s_tz_database;$/;" m class:doris::TimezoneDatabase +_s_url_authority src/util/url_parser.h /^ static const StringValue _s_url_authority;$/;" m class:doris::UrlParser +_s_url_file src/util/url_parser.h /^ static const StringValue _s_url_file;$/;" m class:doris::UrlParser +_s_url_host src/util/url_parser.h /^ static const StringValue _s_url_host;$/;" m class:doris::UrlParser +_s_url_path src/util/url_parser.h /^ static const StringValue _s_url_path;$/;" m class:doris::UrlParser +_s_url_protocol src/util/url_parser.h /^ static const StringValue _s_url_protocol;$/;" m class:doris::UrlParser +_s_url_query src/util/url_parser.h /^ static const StringValue _s_url_query;$/;" m class:doris::UrlParser +_s_url_ref src/util/url_parser.h /^ static const StringValue _s_url_ref;$/;" m class:doris::UrlParser +_s_url_userinfo src/util/url_parser.h /^ static const StringValue _s_url_userinfo;$/;" m class:doris::UrlParser +_s_vars_columns src/exec/schema_scanner/schema_variables_scanner.cpp /^SchemaScanner::ColumnDesc SchemaVariablesScanner::_s_vars_columns[] = {$/;" m class:doris::SchemaVariablesScanner file: +_s_vars_columns src/exec/schema_scanner/schema_variables_scanner.h /^ static SchemaScanner::ColumnDesc _s_vars_columns[];$/;" m class:doris::SchemaVariablesScanner +_save_alter_state src/olap/schema_change.cpp /^OLAPStatus SchemaChangeHandler::_save_alter_state($/;" f class:doris::SchemaChangeHandler +_save_meta src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::_save_meta(DataDir* data_dir) {$/;" f class:doris::TabletMeta +_scalar_fn src/exprs/scalar_fn_call.h /^ void* _scalar_fn;$/;" m class:doris::ScalarFnCall +_scalar_fn_wrapper src/exprs/scalar_fn_call.h /^ void* _scalar_fn_wrapper;$/;" m class:doris::ScalarFnCall +_scan_batch_added_cv src/exec/olap_scan_node.h /^ boost::condition_variable _scan_batch_added_cv;$/;" m class:doris::OlapScanNode +_scan_batches_lock src/exec/olap_scan_node.h /^ boost::mutex _scan_batches_lock;$/;" m class:doris::OlapScanNode +_scan_context_gc_interval_min src/runtime/external_scan_context_mgr.h /^ u_int32_t _scan_context_gc_interval_min;$/;" m class:doris::ExternalScanContextMgr +_scan_finished src/exec/broker_scan_node.h /^ std::atomic _scan_finished;$/;" m class:doris::BrokerScanNode +_scan_finished src/exec/es_http_scan_node.h /^ std::atomic _scan_finished;$/;" m class:doris::EsHttpScanNode +_scan_handles src/exec/es_scan_node.h /^ std::vector _scan_handles;$/;" m class:doris::EsScanNode +_scan_hash src/exec/hash_table.h /^ uint32_t _scan_hash;$/;" m class:doris::HashTable::Iterator +_scan_keys src/exec/olap_scan_node.h /^ OlapScanKeys& _scan_keys;$/;" m class:doris::OlapScanNode::ExtendScanKeyVisitor +_scan_keys src/exec/olap_scan_node.h /^ OlapScanKeys _scan_keys;$/;" m class:doris::OlapScanNode +_scan_node src/exec/kudu_scanner.h /^ KuduScanNode* _scan_node;$/;" m class:doris::KuduScanner +_scan_range src/runtime/disk_io_mgr.h /^ ScanRange* _scan_range;$/;" m class:doris::DiskIoMgr::BufferDescriptor +_scan_range_idx src/exec/es_scan_node.h /^ int _scan_range_idx;$/;" m class:doris::EsScanNode +_scan_range_offset src/runtime/disk_io_mgr.h /^ int64_t _scan_range_offset;$/;" m class:doris::DiskIoMgr::BufferDescriptor +_scan_ranges src/exec/broker_scan_node.h /^ std::vector _scan_ranges;$/;" m class:doris::BrokerScanNode +_scan_ranges src/exec/es_http_scan_node.h /^ std::vector _scan_ranges;$/;" m class:doris::EsHttpScanNode +_scan_ranges src/exec/es_scan_node.h /^ std::vector _scan_ranges;$/;" m class:doris::EsScanNode +_scan_ranges src/exec/olap_scan_node.h /^ std::vector > _scan_ranges;$/;" m class:doris::OlapScanNode +_scan_ranges test/exec/new_olap_scan_node_test.cpp /^ vector _scan_ranges;$/;" m class:doris::TestOlapScanNode file: +_scan_ranges test/exec/olap_scan_node_test.cpp /^ std::vector _scan_ranges;$/;" m class:doris::OlapScanNodeTest file: +_scan_ranges test/olap/olap_reader_test.cpp /^ vector _scan_ranges;$/;" m class:doris::TestOLAPReaderColumn file: +_scan_ranges test/olap/olap_reader_test.cpp /^ vector _scan_ranges;$/;" m class:doris::TestOLAPReaderColumnDeleteCondition file: +_scan_ranges test/olap/olap_reader_test.cpp /^ vector _scan_ranges;$/;" m class:doris::TestOLAPReaderRow file: +_scan_ranges test/olap/vectorized_olap_reader_test.cpp /^ vector _scan_ranges;$/;" m class:doris::TestVectorizedOLAPReader file: +_scan_ranges_complete_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* _scan_ranges_complete_counter;$/;" m class:doris::ScanNode +_scan_row_batches src/exec/olap_scan_node.h /^ std::list _scan_row_batches;$/;" m class:doris::OlapScanNode +_scan_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _scan_timer;$/;" m class:doris::OlapScanNode +_scan_tokens src/exec/kudu_scan_node.h /^ std::vector _scan_tokens;$/;" m class:doris::KuduScanNode +_scanner src/exec/kudu_scanner.h /^ boost::scoped_ptr _scanner;$/;" m class:doris::KuduScanner +_scanner_done src/exec/olap_scan_node.h /^ bool _scanner_done;$/;" m class:doris::OlapScanNode +_scanner_eof src/exec/broker_scanner.h /^ bool _scanner_eof;$/;" m class:doris::BrokerScanner +_scanner_eof src/exec/orc_scanner.h /^ bool _scanner_eof;$/;" m class:doris::ORCScanner +_scanner_eof src/exec/parquet_scanner.h /^ bool _scanner_eof;$/;" m class:doris::ParquetScanner +_scanner_id src/runtime/row_batch.h /^ int _scanner_id;$/;" m class:doris::RowBatch +_scanner_param src/exec/schema_scan_node.h /^ SchemaScannerParam _scanner_param;$/;" m class:doris::SchemaScanNode +_scanner_pool src/exec/olap_scan_node.h /^ boost::scoped_ptr _scanner_pool;$/;" m class:doris::OlapScanNode +_scanner_task_finish_count src/exec/olap_scan_node.h /^ int32_t _scanner_task_finish_count;$/;" m class:doris::OlapScanNode +_scanner_thread_counters src/exec/scan_node.h /^ RuntimeProfile::ThreadCounters* _scanner_thread_counters;$/;" m class:doris::ScanNode +_scanner_threads src/exec/broker_scan_node.h /^ std::vector _scanner_threads;$/;" m class:doris::BrokerScanNode +_scanner_threads src/exec/es_http_scan_node.h /^ std::vector _scanner_threads;$/;" m class:doris::EsHttpScanNode +_scanner_threads src/exec/kudu_scan_node.h /^ boost::thread_group _scanner_threads;$/;" m class:doris::KuduScanNode +_scanners_status src/exec/es_http_scan_node.h /^ std::vector> _scanners_status;$/;" m class:doris::EsHttpScanNode +_schema src/exec/tablet_info.h /^ std::shared_ptr _schema;$/;" m class:doris::OlapTablePartitionParam +_schema src/exec/tablet_sink.h /^ std::shared_ptr _schema;$/;" m class:doris::stream_load::OlapTableSink +_schema src/olap/delta_writer.h /^ std::unique_ptr _schema;$/;" m class:doris::DeltaWriter +_schema src/olap/generic_iterators.cpp /^ Schema _schema;$/;" m class:doris::AutoIncrementIterator file: +_schema src/olap/generic_iterators.cpp /^ std::unique_ptr _schema;$/;" m class:doris::MergeIterator file: +_schema src/olap/memtable.h /^ const Schema* _schema;$/;" m class:doris::MemTable::RowCursorComparator +_schema src/olap/memtable.h /^ Schema* _schema;$/;" m class:doris::MemTable +_schema src/olap/olap_cond.h /^ const TabletSchema* _schema;$/;" m class:doris::Conditions +_schema src/olap/row.h /^ const Schema* _schema;$/;" m struct:doris::ContiguousRow +_schema src/olap/row_block.h /^ const TabletSchema* _schema; \/\/ 内部保存的schema句柄$/;" m class:doris::RowBlock +_schema src/olap/row_block2.h /^ Schema _schema;$/;" m class:doris::RowBlockV2 +_schema src/olap/row_cursor.h /^ std::unique_ptr _schema;$/;" m class:doris::RowCursor +_schema src/olap/rowset/rowset.h /^ const TabletSchema* _schema;$/;" m class:doris::Rowset +_schema src/olap/rowset/segment_group.h /^ const TabletSchema* _schema;$/;" m class:doris::SegmentGroup +_schema src/olap/rowset/segment_v2/empty_segment_iterator.h /^ Schema _schema;$/;" m class:doris::segment_v2::EmptySegmentIterator +_schema src/olap/rowset/segment_v2/segment_iterator.h /^ Schema _schema;$/;" m class:doris::segment_v2::SegmentIterator +_schema src/olap/tablet.h /^ TabletSchema _schema;$/;" m class:doris::Tablet +_schema src/olap/tablet_meta.h /^ TabletSchema _schema;$/;" m class:doris::TabletMeta +_schema src/runtime/tablets_channel.h /^ OlapTableSchemaParam* _schema = nullptr;$/;" m class:doris::TabletsChannel +_schema src/runtime/vectorized_row_batch.h /^ const TabletSchema* _schema;$/;" m class:doris::VectorizedRowBatch +_schema src/util/arrow/row_batch.cpp /^ const std::shared_ptr& _schema;$/;" m class:doris::FromRowBatchConverter file: +_schema src/util/arrow/row_block.cpp /^ const Schema& _schema;$/;" m class:doris::ToRowBlockConverter file: +_schema src/util/arrow/row_block.cpp /^ std::shared_ptr _schema;$/;" m class:doris::FromRowBlockConverter file: +_schema test/exprs/binary_predicate_test.cpp /^ std::vector _schema;$/;" m class:doris::BinaryOpTest file: +_schema test/olap/txn_manager_test.cpp /^ std::unique_ptr _schema;$/;" m class:doris::TxnManagerTest file: +_schema_hash src/exec/tablet_sink.h /^ int32_t _schema_hash = 0;$/;" m class:doris::stream_load::NodeChannel +_schema_hash src/exec/tablet_sink.h /^ int32_t _schema_hash;$/;" m class:doris::stream_load::IndexChannel +_schema_hash src/olap/tablet_meta.h /^ int32_t _schema_hash = 0;$/;" m class:doris::TabletMeta +_schema_hash src/olap/task/engine_checksum_task.h /^ TSchemaHash _schema_hash; $/;" m class:doris::EngineChecksumTask +_schema_hash src/runtime/dpp_writer.h /^ int32_t _schema_hash;$/;" m class:doris::DppWriter +_schema_hash test/olap/olap_snapshot_converter_test.cpp /^ int32_t _schema_hash;$/;" m class:doris::OlapSnapshotConverterTest file: +_schema_hash test/olap/tablet_mgr_test.cpp /^ int32_t _schema_hash;$/;" m class:doris::TabletMgrTest file: +_schema_hash_path test/olap/rowset/rowset_converter_test.cpp /^ std::string _schema_hash_path;$/;" m class:doris::RowsetConverterTest file: +_schema_length src/olap/rowset/segment_v2/column_reader.h /^ size_t _schema_length;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_schema_mapping src/olap/schema_change.h /^ SchemaMapping _schema_mapping;$/;" m class:doris::RowBlockChanger +_schema_scanner src/exec/schema_scan_node.h /^ boost::scoped_ptr _schema_scanner;$/;" m class:doris::SchemaScanNode +_schema_size src/olap/memtable.h /^ size_t _schema_size;$/;" m class:doris::MemTable +_schema_size src/olap/schema.h /^ size_t _schema_size;$/;" m class:doris::Schema +_schema_table_type src/runtime/descriptors.h /^ TSchemaTableType::type _schema_table_type;$/;" m class:doris::SchemaTableDescriptor +_scroll_id src/exec/es/es_scan_reader.h /^ std::string _scroll_id;$/;" m class:doris::ESScanReader +_scroll_id src/exec/es/es_scroll_parser.h /^ std::string _scroll_id;$/;" m class:doris::ScrollParser +_scroll_keep_alive src/exec/es/es_scan_reader.h /^ std::string _scroll_keep_alive;$/;" m class:doris::ESScanReader +_search_url src/exec/es/es_scan_reader.h /^ std::string _search_url;$/;" m class:doris::ESScanReader +_second src/runtime/datetime_value.h /^ uint8_t _second;$/;" m class:doris::DateTimeValue +_second_buffers test/olap/column_reader_test.cpp /^ std::vector _second_buffers;$/;" m class:doris::TestColumn file: +_second_buffers test/olap/schema_change_test.cpp /^ std::vector _second_buffers;$/;" m class:doris::TestColumn file: +_seed test/olap/skiplist_test.cpp /^ int _seed;$/;" m class:doris::TestState file: +_seeds src/exec/partitioned_hash_table.h /^ std::vector _seeds;$/;" m class:doris::PartitionedHashTableCtx +_seek src/olap/in_stream.cpp /^OLAPStatus InStream::_seek(uint64_t position) {$/;" f class:doris::InStream +_seek_and_peek src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_seek_and_peek(rowid_t rowid) {$/;" f class:doris::segment_v2::SegmentIterator +_seek_block src/olap/rowset/segment_v2/segment_iterator.h /^ std::unique_ptr _seek_block;$/;" m class:doris::segment_v2::SegmentIterator +_seek_columns src/olap/reader.h /^ std::vector _seek_columns;$/;" m class:doris::Reader +_seek_columns src/olap/rowset/column_data.h /^ std::vector _seek_columns;$/;" m class:doris::ColumnData +_seek_columns src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::_seek_columns(const std::vector& column_ids, rowid_t pos) {$/;" f class:doris::segment_v2::SegmentIterator +_seek_schema src/olap/rowset/segment_v2/segment_iterator.h /^ std::unique_ptr _seek_schema;$/;" m class:doris::segment_v2::SegmentIterator +_seek_to_block src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::_seek_to_block(const RowBlockPosition& block_pos, bool without_filter) {$/;" f class:doris::ColumnData +_seek_to_block src/olap/rowset/segment_reader.cpp /^void SegmentReader::_seek_to_block(int64_t block_id, bool without_filter) {$/;" f class:doris::SegmentReader +_seek_to_block_directly src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_seek_to_block_directly($/;" f class:doris::SegmentReader +_seek_to_pos_in_page src/olap/rowset/segment_v2/column_reader.cpp /^void FileColumnIterator::_seek_to_pos_in_page(ParsedPage* page, ordinal_t offset_in_page) {$/;" f class:doris::segment_v2::FileColumnIterator +_seek_to_restart_point src/olap/rowset/segment_v2/binary_prefix_page.cpp /^Status BinaryPrefixPageDecoder::_seek_to_restart_point(size_t restart_point_index) {$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +_seek_to_row src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::_seek_to_row(const RowCursor& key, bool find_last_key, bool is_end_key) {$/;" f class:doris::ColumnData +_seek_vector_batch src/olap/rowset/column_data.h /^ std::unique_ptr _seek_vector_batch;$/;" m class:doris::ColumnData +_seeked src/olap/rowset/segment_v2/indexed_column_reader.h /^ bool _seeked = false;$/;" m class:doris::segment_v2::IndexedColumnIterator +_seg_pb_map src/olap/rowset/segment_group.h /^ std::unordered_map > _seg_pb_map;$/;" m class:doris::SegmentGroup +_segment src/olap/rowset/column_data_writer.h /^ uint32_t _segment;$/;" m class:doris::ColumnDataWriter +_segment src/olap/rowset/segment_v2/segment_iterator.h /^ std::shared_ptr _segment;$/;" m class:doris::segment_v2::SegmentIterator +_segment_eof src/olap/rowset/column_data.h /^ bool _segment_eof = false;$/;" m class:doris::ColumnData +_segment_group src/olap/rowset/column_data.h /^ SegmentGroup* _segment_group;$/;" m class:doris::ColumnData +_segment_group src/olap/rowset/column_data.h /^ const SegmentGroup* _segment_group;$/;" m class:doris::ColumnDataComparator +_segment_group src/olap/rowset/column_data_writer.h /^ SegmentGroup* _segment_group;$/;" m class:doris::ColumnDataWriter +_segment_group src/olap/rowset/segment_reader.h /^ SegmentGroup* _segment_group;$/;" m class:doris::SegmentReader +_segment_group src/olap/rowset/segment_writer.h /^ SegmentGroup* _segment_group;$/;" m class:doris::SegmentWriter +_segment_group_id src/olap/rowset/alpha_rowset_writer.h /^ int32_t _segment_group_id;$/;" m class:doris::AlphaRowsetWriter +_segment_group_id src/olap/rowset/segment_group.h /^ int32_t _segment_group_id; \/\/ segmentgroup id of segmentgroup$/;" m class:doris::SegmentGroup +_segment_group_with_largest_size src/olap/rowset/alpha_rowset.cpp /^std::shared_ptr AlphaRowset::_segment_group_with_largest_size() {$/;" f class:doris::AlphaRowset +_segment_groups src/olap/rowset/alpha_rowset.h /^ std::vector> _segment_groups;$/;" m class:doris::AlphaRowset +_segment_groups src/olap/rowset/alpha_rowset_reader.h /^ const std::vector>& _segment_groups;$/;" m class:doris::AlphaRowsetReader +_segment_groups src/olap/rowset/alpha_rowset_writer.h /^ std::vector _segment_groups;$/;" m class:doris::AlphaRowsetWriter +_segment_id src/olap/rowset/segment_reader.h /^ uint32_t _segment_id;$/;" m class:doris::SegmentReader +_segment_id src/olap/rowset/segment_v2/segment.h /^ uint32_t _segment_id;$/;" m class:doris::segment_v2::Segment +_segment_id src/olap/rowset/segment_v2/segment_writer.h /^ uint32_t _segment_id;$/;" m class:doris::segment_v2::SegmentWriter +_segment_id src/olap/short_key_index.h /^ uint32_t _segment_id;$/;" m class:doris::ShortKeyIndexBuilder +_segment_reader src/olap/rowset/column_data.h /^ SegmentReader* _segment_reader;$/;" m class:doris::ColumnData +_segment_statistics src/olap/rowset/column_writer.h /^ ColumnStatistics _segment_statistics;$/;" m class:doris::ColumnWriter +_segment_writer src/olap/rowset/beta_rowset_writer.h /^ std::unique_ptr _segment_writer;$/;" m class:doris::BetaRowsetWriter +_segment_writer src/olap/rowset/column_data_writer.h /^ SegmentWriter* _segment_writer;$/;" m class:doris::ColumnDataWriter +_segment_zone_map src/olap/rowset/segment_v2/zone_map_index.h /^ ZoneMap _segment_zone_map;$/;" m class:doris::segment_v2::ZoneMapIndexWriter +_segments src/olap/rowset/beta_rowset.h /^ std::vector _segments;$/;" m class:doris::BetaRowset +_sel_vec src/olap/selection_vector.h /^ SelectionVector* _sel_vec;$/;" m class:doris::SelectionVectorView +_selected src/runtime/vectorized_row_batch.h /^ uint16_t* _selected = nullptr;$/;" m class:doris::VectorizedRowBatch +_selected_in_use src/runtime/vectorized_row_batch.h /^ bool _selected_in_use = false;$/;" m class:doris::VectorizedRowBatch +_selected_size src/olap/row_block2.h /^ uint16_t _selected_size;$/;" m class:doris::RowBlockV2 +_selection_vector src/olap/row_block2.h /^ uint16_t* _selection_vector;$/;" m class:doris::RowBlockV2 +_send_cur_batch src/exec/tablet_sink.cpp /^Status NodeChannel::_send_cur_batch(bool eos) {$/;" f class:doris::stream_load::NodeChannel +_send_data_ns src/exec/tablet_sink.h /^ int64_t _send_data_ns = 0;$/;" m class:doris::stream_load::OlapTableSink +_send_data_timer src/exec/tablet_sink.h /^ RuntimeProfile::Counter* _send_data_timer = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_send_query_statistics_with_every_batch src/runtime/data_stream_sender.cpp /^ bool _send_query_statistics_with_every_batch;$/;" m class:doris::DataStreamSender::Channel file: +_sender src/runtime/result_sink.h /^ boost::shared_ptr _sender;$/;" m class:doris::ResultSink +_sender_eos_set src/runtime/data_stream_recvr.cc /^ std::unordered_set _sender_eos_set; \/\/ sender_id$/;" m class:doris::DataStreamRecvr::SenderQueue file: +_sender_id src/exec/tablet_sink.h /^ int _sender_id = -1;$/;" m class:doris::stream_load::OlapTableSink +_sender_id src/runtime/data_stream_sender.h /^ int _sender_id;$/;" m class:doris::DataStreamSender +_sender_info test/runtime/data_stream_test.cpp /^ vector _sender_info;$/;" m class:doris::DataStreamTest file: +_sender_queue_pool src/runtime/data_stream_recvr.h /^ ObjectPool _sender_queue_pool;$/;" m class:doris::DataStreamRecvr +_sender_queues src/runtime/data_stream_recvr.h /^ std::vector _sender_queues;$/;" m class:doris::DataStreamRecvr +_separator src/util/path_trie.hpp /^ char _separator;$/;" m class:doris::PathTrie +_serialize_batch_ns src/exec/tablet_sink.h /^ int64_t _serialize_batch_ns = 0;$/;" m class:doris::stream_load::OlapTableSink +_serialize_batch_timer src/exec/tablet_sink.h /^ RuntimeProfile::Counter* _serialize_batch_timer = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_serialize_batch_timer src/runtime/data_stream_sender.h /^ RuntimeProfile::Counter* _serialize_batch_timer;$/;" m class:doris::DataStreamSender +_serialize_fn output/udf/include/uda_test_harness.h /^ SerializeFn _serialize_fn;$/;" m class:doris_udf::UdaTestHarnessBase +_serialize_fn src/exprs/agg_fn_evaluator.h /^ void* _serialize_fn;$/;" m class:doris::AggFnEvaluator +_serialize_fn src/udf/uda_test_harness.h /^ SerializeFn _serialize_fn;$/;" m class:doris_udf::UdaTestHarnessBase +_serialize_fn_symbol src/exprs/agg_fn_evaluator.h /^ std::string _serialize_fn_symbol;$/;" m class:doris::AggFnEvaluator +_serialize_stream src/exec/partitioned_aggregation_node.h /^ boost::scoped_ptr _serialize_stream;$/;" m class:doris::PartitionedAggregationNode +_serialize_to_pb src/olap/rowset/rowset_meta.h /^ bool _serialize_to_pb(std::string* value) {$/;" f class:doris::RowsetMeta +_server src/service/brpc_service.h /^ std::unique_ptr _server;$/;" m class:doris::BRpcService +_server src/util/thrift_server.h /^ boost::scoped_ptr _server;$/;" m class:doris::ThriftServer +_server test/runtime/data_stream_test.cpp /^ ThriftServer* _server;$/;" m class:doris::DataStreamTest file: +_server_fd src/http/ev_http_server.h /^ int _server_fd = -1;$/;" m class:doris::EvHttpServer +_server_thread src/util/thrift_server.h /^ boost::scoped_ptr _server_thread;$/;" m class:doris::ThriftServer +_server_type src/util/thrift_server.h /^ ServerType _server_type;$/;" m class:doris::ThriftServer +_session_handler src/util/thrift_server.h /^ SessionHandlerIf* _session_handler;$/;" m class:doris::ThriftServer +_session_key src/util/thrift_server.cpp /^__thread ThriftServer::SessionKey* _session_key;$/;" m namespace:doris file: +_session_keys src/util/thrift_server.h /^ SessionKeySet _session_keys;$/;" m class:doris::ThriftServer +_session_keys_lock src/util/thrift_server.h /^ boost::mutex _session_keys_lock;$/;" m class:doris::ThriftServer +_set src/exprs/aggregate_functions.cpp /^ StringValueSet _set;$/;" m class:doris::MultiDistinctStringCountState file: +_set src/exprs/aggregate_functions.cpp /^ std::unordered_set _set;$/;" m class:doris::MultiDistinctCountDateState file: +_set src/exprs/aggregate_functions.cpp /^ std::unordered_set _set;$/;" m class:doris::MultiDistinctDecimalV2State file: +_set src/exprs/aggregate_functions.cpp /^ std::unordered_set _set;$/;" m class:doris::MultiDistinctDecimalState file: +_set src/exprs/aggregate_functions.cpp /^ std::unordered_set _set;$/;" m class:doris::MultiDistinctNumericState file: +_set src/exprs/hybird_set.h /^ std::unordered_set _set;$/;" m class:doris::HybirdSet +_set src/exprs/hybird_set.h /^ std::unordered_set _set;$/;" m class:doris::StringValueSet +_set_column_map src/olap/rowset/segment_reader.cpp /^void SegmentReader::_set_column_map() {$/;" f class:doris::SegmentReader +_set_decompressor src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_set_decompressor() {$/;" f class:doris::SegmentReader +_set_masks src/olap/rowset/segment_v2/block_split_bloom_filter.h /^ void _set_masks(uint32_t key, uint32_t* masks) const {$/;" f class:doris::segment_v2::BlockSplitBloomFilter +_set_rsc_info src/runtime/fragment_mgr.cpp /^ bool _set_rsc_info;$/;" m class:doris::FragmentExecState file: +_set_segment_info src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::_set_segment_info() {$/;" f class:doris::SegmentReader +_set_tablet_info src/olap/task/engine_clone_task.cpp /^void EngineCloneTask::_set_tablet_info(AgentStatus status, bool is_new_tablet) {$/;" f class:doris::EngineCloneTask +_set_to_max src/olap/types.h /^ void (*_set_to_max)(void* buf);$/;" m class:doris::TypeInfo +_set_to_min src/olap/types.h /^ void (*_set_to_min)(void* buf);$/;" m class:doris::TypeInfo +_set_type src/olap/hll.h /^ HllDataType _set_type; \/\/set type$/;" m class:doris::HllSetResolver +_shallow_copy src/olap/types.h /^ void (*_shallow_copy)(void* dest, const void* src);$/;" m class:doris::TypeInfo +_shard src/olap/lru_cache.cpp /^uint32_t ShardedLRUCache::_shard(uint32_t hash) {$/;" f class:doris::ShardedLRUCache +_shard_id src/olap/tablet_meta.h /^ int32_t _shard_id = 0;$/;" m class:doris::TabletMeta +_shards src/exec/es/es_scan_reader.h /^ std::string _shards;$/;" m class:doris::ESScanReader +_shards src/olap/lru_cache.h /^ LRUCache _shards[kNumShards];$/;" m class:doris::CachePriority::ShardedLRUCache +_shared_buffer src/olap/file_stream.h /^ StorageByteBuffer** _shared_buffer;$/;" m class:doris::ReadOnlyFileStream +_shared_buffer src/olap/rowset/segment_reader.h /^ StorageByteBuffer* _shared_buffer;$/;" m class:doris::SegmentReader +_shared_buffer test/olap/bit_field_test.cpp /^ StorageByteBuffer* _shared_buffer;$/;" m class:doris::TestBitField file: +_shared_buffer test/olap/column_reader_test.cpp /^ StorageByteBuffer* _shared_buffer;$/;" m class:doris::TestColumn file: +_shared_buffer test/olap/run_length_byte_test.cpp /^ StorageByteBuffer* _shared_buffer;$/;" m class:doris::TestRunLengthByte file: +_shared_buffer test/olap/run_length_integer_test.cpp /^ StorageByteBuffer* _shared_buffer;$/;" m class:doris::TestRunLengthSignInteger file: +_shared_buffer test/olap/run_length_integer_test.cpp /^ StorageByteBuffer* _shared_buffer;$/;" m class:doris::TestRunLengthUnsignInteger file: +_shared_buffer test/olap/schema_change_test.cpp /^ StorageByteBuffer* _shared_buffer;$/;" m class:doris::TestColumn file: +_short_key_buf src/olap/rowset/segment_group.h /^ char* _short_key_buf;$/;" m class:doris::SegmentGroup +_short_key_columns src/olap/olap_index.h /^ std::vector* _short_key_columns;$/;" m class:doris::MemIndex +_short_key_columns src/olap/rowset/segment_group.h /^ std::vector _short_key_columns;$/;" m class:doris::SegmentGroup +_short_key_cursor src/olap/rowset/column_data.h /^ RowCursor _short_key_cursor;$/;" m class:doris::ColumnData +_short_key_length src/olap/rowset/segment_group.h /^ size_t _short_key_length;$/;" m class:doris::SegmentGroup +_should_clauses src/exec/es/es_query_builder.h /^ std::vector _should_clauses;$/;" m class:doris::BooleanQueryBuilder +_shut_down src/runtime/disk_io_mgr.h /^ volatile bool _shut_down;$/;" m class:doris::DiskIoMgr +_shutdown src/util/batch_process_thread_pool.hpp /^ bool _shutdown;$/;" m class:doris::BatchProcessThreadPool +_shutdown src/util/blocking_priority_queue.hpp /^ bool _shutdown;$/;" m class:doris::BlockingPriorityQueue +_shutdown src/util/blocking_queue.hpp /^ bool _shutdown;$/;" m class:doris::BlockingQueue +_shutdown src/util/priority_thread_pool.hpp /^ bool _shutdown;$/;" m class:doris::PriorityThreadPool +_shutdown_tablets src/olap/tablet_manager.h /^ std::vector _shutdown_tablets;$/;" m class:doris::TabletManager +_shutdown_tablets_lock src/olap/tablet_manager.h /^ RWMutex _shutdown_tablets_lock;$/;" m class:doris::TabletManager +_sign src/runtime/decimal_value.h /^ bool _sign;$/;" m class:doris::DecimalValue +_sign src/runtime/dpp_sink_internal.h /^ int _sign;$/;" m class:doris::PartRangeKey +_signal_cond src/util/thrift_server.cpp /^ boost::condition_variable _signal_cond;$/;" m class:doris::ThriftServer::ThriftServerEventProcessor file: +_signal_fired src/util/thrift_server.cpp /^ bool _signal_fired;$/;" m class:doris::ThriftServer::ThriftServerEventProcessor file: +_signal_lock src/util/thrift_server.cpp /^ boost::mutex _signal_lock;$/;" m class:doris::ThriftServer::ThriftServerEventProcessor file: +_signature src/olap/task/engine_alter_tablet_task.h /^ int64_t _signature;$/;" m class:doris::EngineAlterTabletTask +_signature src/olap/task/engine_batch_load_task.h /^ int64_t _signature;$/;" m class:doris::EngineBatchLoadTask +_signature src/olap/task/engine_clone_task.h /^ int64_t _signature;$/;" m class:doris::EngineCloneTask +_signed src/olap/rowset/run_length_integer_reader.h /^ bool _signed;$/;" m class:doris::RunLengthIntegerReader +_singleton_agg_row src/exec/pre_aggregation_node.h /^ TupleRow* _singleton_agg_row;$/;" m class:doris::PreAggregationNode +_singleton_output_tuple src/exec/aggregation_node.h /^ Tuple* _singleton_output_tuple; \/\/ result of aggregation w\/o GROUP BY$/;" m class:doris::AggregationNode +_singleton_output_tuple src/exec/partitioned_aggregation_node.h /^ Tuple* _singleton_output_tuple;$/;" m class:doris::PartitionedAggregationNode +_singleton_output_tuple_returned src/exec/partitioned_aggregation_node.h /^ bool _singleton_output_tuple_returned;$/;" m class:doris::PartitionedAggregationNode +_sink src/runtime/plan_fragment_executor.h /^ boost::scoped_ptr _sink;$/;" m class:doris::PlanFragmentExecutor +_sink test/runtime/result_writer_test.cpp /^ BufferControlBlock* _sink;$/;" m class:doris::ResultWriterTest file: +_sink_map src/runtime/data_spliter.h /^ std::unordered_map _sink_map;$/;" m class:doris::DataSpliter +_sinker src/runtime/result_writer.h /^ BufferControlBlock* _sinker;$/;" m class:doris::ResultWriter +_size src/exec/csv_scan_node.cpp /^ int _size;$/;" m class:doris::StringRef file: +_size src/exec/es/es_scroll_parser.h /^ int _size;$/;" m class:doris::ScrollParser +_size src/olap/lru_cache.h /^ size_t _size;$/;" m class:doris::CacheKey +_size src/olap/rowset/segment_v2/bloom_filter.h /^ uint32_t _size;$/;" m class:doris::segment_v2::BloomFilter +_size src/olap/types.h /^ const size_t _size;$/;" m class:doris::TypeInfo +_size src/runtime/vectorized_row_batch.h /^ uint16_t _size = 0;$/;" m class:doris::VectorizedRowBatch +_size src/util/buffer_builder.hpp /^ int _size;$/;" m class:doris::BufferBuilder +_size src/util/core_local.h /^ size_t _size = 0;$/;" m class:doris::CoreLocalValue +_size src/util/core_local.h /^ size_t _size;$/;" m class:doris::CoreLocalValueController +_size_estimate src/olap/rowset/segment_v2/binary_plain_page.h /^ size_t _size_estimate;$/;" m class:doris::segment_v2::BinaryPlainPageBuilder +_size_of_element src/olap/rowset/segment_v2/bitshuffle_page.h /^ int _size_of_element;$/;" m class:doris::segment_v2::BitShufflePageDecoder +_sk_index_decoder src/olap/rowset/segment_v2/segment.h /^ std::unique_ptr _sk_index_decoder;$/;" m class:doris::segment_v2::Segment +_sk_index_handle src/olap/rowset/segment_v2/segment.h /^ PageHandle _sk_index_handle;$/;" m class:doris::segment_v2::Segment +_skip src/runtime/string_search.hpp /^ int64_t _skip;$/;" m class:doris::StringSearch +_skip_list src/olap/memtable.h /^ Table* _skip_list;$/;" m class:doris::MemTable +_skip_next_line src/exec/broker_scanner.h /^ bool _skip_next_line;$/;" m class:doris::BrokerScanner +_skip_size src/olap/in_stream.h /^ size_t _skip_size;$/;" m class:doris::InStreamBufferWrapper +_slice src/olap/in_stream.cpp /^OLAPStatus InStream::_slice(uint64_t chunk_size, StorageByteBuffer** out_slice) {$/;" f class:doris::InStream +_slice src/util/slice.h /^ Slice _slice;$/;" m class:doris::OwnedSlice +_slice_off src/util/block_compression.cpp /^ size_t _slice_off;$/;" m class:doris::SnappySlicesSource file: +_slices src/util/block_compression.cpp /^ std::vector _slices;$/;" m class:doris::SnappySlicesSource file: +_slot_desc src/runtime/descriptor_helper.h /^ TSlotDescriptor _slot_desc;$/;" m class:doris::TSlotDescriptorBuilder +_slot_desc_map src/runtime/descriptors.h /^ SlotDescriptorMap _slot_desc_map;$/;" m class:doris::DescriptorTbl +_slot_descs src/exec/es_http_scanner.h /^ std::vector _slot_descs;$/;" m class:doris::EsHttpScanner +_slot_descs src/exec/tablet_info.h /^ std::vector _slot_descs;$/;" m class:doris::OlapTablePartKeyComparator +_slot_descs src/olap/memtable.h /^ const std::vector* _slot_descs;$/;" m class:doris::MemTable +_slot_descs src/runtime/descriptor_helper.h /^ std::vector _slot_descs;$/;" m class:doris::TTupleDescriptorBuilder +_slot_id src/exprs/slot_ref.h /^ const SlotId _slot_id;$/;" m class:doris::SlotRef +_slot_id_set_list src/exec/repeat_node.h /^ std::vector> _slot_id_set_list;$/;" m class:doris::RepeatNode +_slot_idx src/runtime/descriptors.h /^ const int _slot_idx;$/;" m class:doris::SlotDescriptor +_slot_num src/exec/csv_scan_node.h /^ int _slot_num;$/;" m class:doris::CsvScanNode +_slot_num src/exec/mysql_scan_node.h /^ int _slot_num;$/;" m class:doris::MysqlScanNode +_slot_num src/exec/schema_scan_node.h /^ int _slot_num;$/;" m class:doris::SchemaScanNode +_slot_offset src/exprs/slot_ref.h /^ int _slot_offset; \/\/ within tuple$/;" m class:doris::SlotRef +_slot_size src/runtime/descriptors.h /^ const int _slot_size;$/;" m class:doris::SlotDescriptor +_slot_types src/testutil/desc_tbl_builder.h /^ std::vector _slot_types;$/;" m class:doris::TupleDescBuilder +_slots src/runtime/descriptors.h /^ std::vector _slots; \/\/ contains all slots$/;" m class:doris::TupleDescriptor +_slots_map src/exec/broker_scan_node.h /^ std::map _slots_map;$/;" m class:doris::BrokerScanNode +_slots_map test/exec/broker_scan_node_test.cpp /^ std::map _slots_map;$/;" m class:doris::BrokerScanNodeTest file: +_slots_map test/exec/broker_scanner_test.cpp /^ std::map _slots_map;$/;" m class:doris::BrokerScannerTest file: +_slots_map test/exec/parquet_scanner_test.cpp /^ std::map _slots_map;$/;" m class:doris::ParquetSannerTest file: +_small_file_mgr src/runtime/exec_env.h /^ SmallFileMgr* _small_file_mgr = nullptr;$/;" m class:doris::ExecEnv +_snapshot_base_id src/olap/snapshot_manager.h /^ uint64_t _snapshot_base_id;$/;" m class:doris::SnapshotManager +_snapshot_mutex src/olap/snapshot_manager.h /^ Mutex _snapshot_mutex;$/;" m class:doris::SnapshotManager +_snapshot_names src/util/perf_counters.h /^ std::vector _snapshot_names;$/;" m class:doris::PerfCounters +_snapshots src/util/perf_counters.h /^ std::vector > _snapshots;$/;" m class:doris::PerfCounters +_socket src/util/thrift_client.h /^ boost::shared_ptr _socket;$/;" m class:doris::ThriftClientImpl +_sole_data_page src/olap/rowset/segment_v2/indexed_column_reader.h /^ PagePointer _sole_data_page;$/;" m class:doris::segment_v2::IndexedColumnReader +_sort_exec_exprs src/exec/exchange_node.h /^ SortExecExprs _sort_exec_exprs;$/;" m class:doris::ExchangeNode +_sort_exec_exprs src/exec/sort_node.h /^ SortExecExprs _sort_exec_exprs;$/;" m class:doris::SortNode +_sort_exec_exprs src/exec/spill_sort_node.h /^ SortExecExprs _sort_exec_exprs;$/;" m class:doris::SpillSortNode +_sort_exec_exprs src/exec/topn_node.h /^ SortExecExprs _sort_exec_exprs;$/;" m class:doris::TopNNode +_sort_timer src/runtime/dpp_sink.cpp /^ RuntimeProfile::Counter* _sort_timer;$/;" m class:doris::Translator file: +_sort_tuple_desc src/runtime/merge_sorter.cpp /^ const TupleDescriptor* _sort_tuple_desc;$/;" m class:doris::MergeSorter::Run file: +_sort_tuple_desc src/runtime/spill_sorter.cc /^ const TupleDescriptor* _sort_tuple_desc;$/;" m class:doris::SpillSorter::Run file: +_sort_tuple_size src/runtime/merge_sorter.cpp /^ const int _sort_tuple_size;$/;" m class:doris::MergeSorter::Run file: +_sort_tuple_size src/runtime/spill_sorter.cc /^ const int _sort_tuple_size;$/;" m class:doris::SpillSorter::Run file: +_sort_tuple_slot_expr test/runtime/buffered_tuple_stream_test.cpp /^ std::vector _sort_tuple_slot_expr;$/;" m class:doris::BufferedTupleStreamTest file: +_sort_tuple_slot_expr test/runtime/sorter_test.cpp /^ std::vector _sort_tuple_slot_expr;$/;" m class:doris::SorterTest file: +_sort_tuple_slot_expr_ctxs src/exec/sort_exec_exprs.h /^ std::vector _sort_tuple_slot_expr_ctxs;$/;" m class:doris::SortExecExprs +_sort_tuple_slot_expr_ctxs src/runtime/merge_sorter.h /^ std::vector _sort_tuple_slot_expr_ctxs;$/;" m class:doris::MergeSorter +_sort_tuple_slot_expr_ctxs src/runtime/spill_sorter.h /^ std::vector _sort_tuple_slot_expr_ctxs;$/;" m class:doris::SpillSorter +_sorted_data_size src/runtime/merge_sorter.h /^ RuntimeProfile::Counter* _sorted_data_size;$/;" m class:doris::MergeSorter +_sorted_data_size src/runtime/spill_sorter.h /^ RuntimeProfile::Counter* _sorted_data_size;$/;" m class:doris::SpillSorter +_sorted_rows src/runtime/qsorter.h /^ std::vector _sorted_rows;$/;" m class:doris::QSorter +_sorted_run src/runtime/sorted_run_merger.cc /^ RunBatchSupplier _sorted_run;$/;" m class:doris::SortedRunMerger::BatchedRowSupplier file: +_sorted_runs src/runtime/merge_sorter.h /^ std::list _sorted_runs;$/;" m class:doris::MergeSorter +_sorted_runs src/runtime/spill_sorter.h /^ std::deque _sorted_runs;$/;" m class:doris::SpillSorter +_sorted_top_n src/exec/topn_node.h /^ std::vector _sorted_top_n;$/;" m class:doris::TopNNode +_sorter src/exec/sort_node.h /^ boost::scoped_ptr _sorter;$/;" m class:doris::SortNode +_sorter src/exec/spill_sort_node.h /^ boost::scoped_ptr _sorter;$/;" m class:doris::SpillSortNode +_sorter src/runtime/dpp_sink.cpp /^ Sorter* _sorter;$/;" m class:doris::Translator file: +_sorter src/runtime/merge_sorter.cpp /^ const MergeSorter* _sorter;$/;" m class:doris::MergeSorter::Run file: +_sorter src/runtime/spill_sorter.cc /^ const SpillSorter* _sorter;$/;" m class:doris::SpillSorter::Run file: +_sparse_count src/olap/hll.h /^ SparseLengthValueType* _sparse_count;$/;" m class:doris::HllSetResolver +_sparse_map src/olap/hll.h /^ std::map _sparse_map;$/;" m class:doris::HllSetResolver +_spawn_callback_worker_thread src/agent/task_worker_pool.cpp /^void TaskWorkerPool::_spawn_callback_worker_thread(CALLBACK_FUNCTION callback_func) {$/;" f class:doris::TaskWorkerPool +_spill src/olap/out_stream.cpp /^OLAPStatus OutStream::_spill() {$/;" f class:doris::OutStream +_spilled_bytes src/olap/out_stream.h /^ uint64_t _spilled_bytes; \/\/ 已经输出到output的字节数$/;" m class:doris::OutStream +_spilled_partitions src/exec/partitioned_aggregation_node.h /^ std::list _spilled_partitions;$/;" m class:doris::PartitionedAggregationNode +_split_check_timer src/exec/csv_scan_node.h /^ RuntimeProfile::Counter* _split_check_timer;$/;" m class:doris::CsvScanNode +_split_line_timer src/exec/csv_scan_node.h /^ RuntimeProfile::Counter* _split_line_timer;$/;" m class:doris::CsvScanNode +_split_timer src/runtime/data_spliter.h /^ RuntimeProfile::Counter* _split_timer;$/;" m class:doris::DataSpliter +_sql_str src/exec/mysql_scanner.h /^ std::string _sql_str;$/;" m class:doris::MysqlScanner +_src_slot_descs src/exec/base_scanner.h /^ std::vector _src_slot_descs;$/;" m class:doris::BaseScanner +_src_slot_descs_order_by_dest src/exec/base_scanner.h /^ std::vector _src_slot_descs_order_by_dest;$/;" m class:doris::BaseScanner +_src_tuple src/exec/base_scanner.h /^ Tuple* _src_tuple;$/;" m class:doris::BaseScanner +_src_tuple src/exec/schema_scan_node.h /^ Tuple* _src_tuple;$/;" m class:doris::SchemaScanNode +_src_tuple_desc src/exec/schema_scan_node.h /^ const TupleDescriptor* _src_tuple_desc;$/;" m class:doris::SchemaScanNode +_src_tuple_row src/exec/base_scanner.h /^ TupleRow* _src_tuple_row;$/;" m class:doris::BaseScanner +_ss src/http/action/metrics_action.cpp /^ std::stringstream _ss;$/;" m class:doris::PrometheusMetricsVisitor file: +_ss src/http/action/metrics_action.cpp /^ std::stringstream _ss;$/;" m class:doris::SimpleCoreMetricsVisitor file: +_ss test/util/doris_metrics_test.cpp /^ std::stringstream _ss;$/;" m class:doris::TestMetricsVisitor file: +_ss test/util/new_metrics_test.cpp /^ std::stringstream _ss;$/;" m class:doris::TestMetricsVisitor file: +_ss test/util/system_metrics_test.cpp /^ std::stringstream _ss;$/;" m class:doris::TestMetricsVisitor file: +_staging_input_vals src/exprs/agg_fn_evaluator.h /^ std::vector _staging_input_vals;$/;" m class:doris::AggFnEvaluator +_staging_input_vals src/udf/udf_internal.h /^ std::vector _staging_input_vals;$/;" m class:doris::FunctionContextImpl +_staging_intermediate_val src/exprs/agg_fn_evaluator.h /^ doris_udf::AnyVal* _staging_intermediate_val;$/;" m class:doris::AggFnEvaluator +_staging_merge_input_val src/exprs/agg_fn_evaluator.h /^ doris_udf::AnyVal* _staging_merge_input_val;$/;" m class:doris::AggFnEvaluator +_start src/exec/olap_scan_node.h /^ bool _start;$/;" m class:doris::OlapScanNode +_start src/util/stopwatch.hpp /^ timespec _start;$/;" m class:doris::MonotonicStopWatch +_start_bg_worker src/olap/olap_server.cpp /^OLAPStatus StorageEngine::_start_bg_worker() {$/;" f class:doris::StorageEngine +_start_bg_worker src/runtime/load_channel_mgr.cpp /^Status LoadChannelMgr::_start_bg_worker() {$/;" f class:doris::LoadChannelMgr +_start_block_position src/olap/rowset/column_data.h /^ const RowBlockPosition _start_block_position;$/;" m class:doris::ColumnDataComparator +_start_clean_fd_cache src/olap/storage_engine.cpp /^void StorageEngine::_start_clean_fd_cache() {$/;" f class:doris::StorageEngine +_start_disk_stat_monitor src/olap/storage_engine.cpp /^void StorageEngine::_start_disk_stat_monitor() {$/;" f class:doris::StorageEngine +_start_key src/runtime/dpp_sink_internal.h /^ PartRangeKey _start_key;$/;" m class:doris::PartRange +_start_load_channels_clean src/runtime/load_channel_mgr.cpp /^Status LoadChannelMgr::_start_load_channels_clean() {$/;" f class:doris::LoadChannelMgr +_start_offset src/exec/local_file_writer.h /^ int64_t _start_offset;$/;" m class:doris::LocalFileWriter +_start_offset src/olap/bloom_filter_reader.h /^ size_t _start_offset;$/;" m class:doris::BloomFilterIndexReader +_start_offset src/olap/stream_index_reader.h /^ size_t _start_offset;$/;" m class:doris::StreamIndexReader +_start_time src/runtime/fragment_mgr.cpp /^ DateTimeValue _start_time;$/;" m class:doris::FragmentExecState file: +_start_trash_sweep src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::_start_trash_sweep(double* max_usage) {$/;" f class:doris::StorageEngine +_started src/util/thrift_server.h /^ bool _started;$/;" m class:doris::ThriftServer +_state src/common/status.h /^ Status(Status&& s) noexcept : _state(s._state) {$/;" f class:doris::Status +_state src/common/status.h /^ const char* _state;$/;" m class:doris::Status +_state src/exec/base_scanner.h /^ RuntimeState* _state;$/;" m class:doris::BaseScanner +_state src/exec/es_http_scanner.h /^ RuntimeState* _state;$/;" m class:doris::EsHttpScanner +_state src/exec/kudu_scanner.h /^ RuntimeState* _state;$/;" m class:doris::KuduScanner +_state src/exec/partitioned_aggregation_node.h /^ RuntimeState* _state;$/;" m class:doris::PartitionedAggregationNode +_state src/exec/partitioned_hash_table.h /^ RuntimeState* _state;$/;" m class:doris::PartitionedHashTable +_state src/olap/compaction.h /^ CompactionState _state;$/;" m class:doris::Compaction +_state src/olap/fs/file_block_manager.cpp /^ State _state;$/;" m class:doris::fs::internal::FileWritableBlock file: +_state src/olap/tablet.h /^ TabletState _state;$/;" m class:doris::Tablet +_state src/runtime/buffered_block_mgr.h /^ RuntimeState* _state;$/;" m class:doris::BufferedBlockMgr +_state src/runtime/buffered_block_mgr2.cc /^ RuntimeState* _state;$/;" m class:doris::BufferedBlockMgr2::Client file: +_state src/runtime/buffered_tuple_stream.h /^ RuntimeState* const _state;$/;" m class:doris::BufferedTupleStream +_state src/runtime/buffered_tuple_stream2.h /^ RuntimeState* const _state;$/;" m class:doris::BufferedTupleStream2 +_state src/runtime/data_stream_sender.h /^ RuntimeState* _state;$/;" m class:doris::DataStreamSender +_state src/runtime/disk_io_mgr_internal.h /^ State _state;$/;" m class:doris::DiskIoMgr::RequestContext +_state src/runtime/export_sink.h /^ RuntimeState* _state;$/;" m class:doris::ExportSink +_state src/runtime/merge_sorter.cpp /^ RuntimeState* const _state;$/;" m class:doris::MergeSorter::TupleSorter file: +_state src/runtime/merge_sorter.h /^ RuntimeState* const _state;$/;" m class:doris::MergeSorter +_state src/runtime/spill_sorter.cc /^ RuntimeState* const _state;$/;" m class:doris::SpillSorter::TupleSorter file: +_state src/runtime/spill_sorter.h /^ RuntimeState* const _state;$/;" m class:doris::SpillSorter +_state src/runtime/tablets_channel.h /^ State _state;$/;" m class:doris::TabletsChannel +_state src/testutil/function_utils.h /^ RuntimeState* _state = nullptr;$/;" m class:doris::FunctionUtils +_state src/udf/udf_internal.h /^ RuntimeState* _state;$/;" m class:doris::FunctionContextImpl +_state src/util/threadpool.h /^ State _state;$/;" m class:doris::ThreadPoolToken +_state test/exec/csv_scan_bench_test.cpp /^ RuntimeState* _state;$/;" m class:doris::CsvScanNodeBenchTest file: +_state test/exec/csv_scan_node_test.cpp /^ RuntimeState* _state;$/;" m class:doris::CsvScanNodeTest file: +_state test/olap/skiplist_test.cpp /^ ReaderState _state;$/;" m class:doris::TestState file: +_state test/runtime/data_spliter_test.cpp /^ RuntimeState* _state;$/;" m class:doris::DataSplitTest file: +_state test/runtime/dpp_sink_test.cpp /^ RuntimeState* _state;$/;" m class:doris::DppSinkTest file: +_state test/runtime/memory_scratch_sink_test.cpp /^ RuntimeState* _state;$/;" m class:doris::MemoryScratchSinkTest file: +_state test/runtime/qsorter_test.cpp /^ RuntimeState* _state;$/;" m class:doris::QSorterTest file: +_state test/util/arrow/arrow_work_flow_test.cpp /^ RuntimeState* _state;$/;" m class:doris::ArrowWorkFlowTest file: +_statistics src/olap/stream_index_reader.h /^ ColumnStatistics _statistics;$/;" m class:doris::PositionEntryReader +_statistics_buffer src/olap/stream_index_writer.h /^ char _statistics_buffer[MAX_STATISTIC_LENGTH];$/;" m class:doris::PositionEntryWriter +_statistics_offset src/olap/stream_index_reader.h /^ size_t _statistics_offset;$/;" m class:doris::PositionEntryReader +_statistics_size src/olap/stream_index_writer.h /^ size_t _statistics_size;$/;" m class:doris::PositionEntryWriter +_stats src/olap/file_stream.h /^ OlapReaderStatistics* _stats;$/;" m class:doris::ReadOnlyFileStream +_stats src/olap/memtable_flush_executor.h /^ FlushStatistic _stats;$/;" m class:doris::FlushToken +_stats src/olap/reader.h /^ OlapReaderStatistics _stats;$/;" m class:doris::Reader +_stats src/olap/rowset/alpha_rowset_reader.h /^ OlapReaderStatistics* _stats = &_owned_stats;$/;" m class:doris::AlphaRowsetReader +_stats src/olap/rowset/beta_rowset_reader.h /^ OlapReaderStatistics* _stats;$/;" m class:doris::BetaRowsetReader +_stats src/olap/rowset/column_data.h /^ OlapReaderStatistics* _stats;$/;" m class:doris::ColumnData +_stats src/olap/rowset/column_reader.h /^ OlapReaderStatistics* _stats = nullptr;$/;" m class:doris::ColumnReader +_stats src/olap/rowset/segment_reader.h /^ OlapReaderStatistics* _stats;$/;" m class:doris::SegmentReader +_stats test/olap/bit_field_test.cpp /^ OlapReaderStatistics _stats;$/;" m class:doris::TestBitField file: +_stats test/olap/column_reader_test.cpp /^ OlapReaderStatistics _stats;$/;" m class:doris::TestColumn file: +_stats test/olap/rowset/beta_rowset_test.cpp /^ OlapReaderStatistics _stats;$/;" m class:doris::BetaRowsetTest file: +_stats test/olap/run_length_byte_test.cpp /^ OlapReaderStatistics _stats;$/;" m class:doris::TestRunLengthByte file: +_stats test/olap/run_length_integer_test.cpp /^ OlapReaderStatistics _stats;$/;" m class:doris::TestRunLengthSignInteger file: +_stats test/olap/run_length_integer_test.cpp /^ OlapReaderStatistics _stats;$/;" m class:doris::TestRunLengthUnsignInteger file: +_stats test/olap/schema_change_test.cpp /^ OlapReaderStatistics _stats;$/;" m class:doris::TestColumn file: +_stats_filtered_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _stats_filtered_counter = nullptr;$/;" m class:doris::OlapScanNode +_status src/exec/kudu_scan_node.h /^ Status _status;$/;" m class:doris::KuduScanNode +_status src/exec/olap_scan_node.h /^ Status _status;$/;" m class:doris::OlapScanNode +_status src/http/http_response.h /^ HttpStatus _status;$/;" m class:doris::HttpResponse +_status src/runtime/buffer_control_block.h /^ Status _status;$/;" m class:doris::BufferControlBlock +_status src/runtime/buffered_tuple_stream.h /^ Status _status;$/;" m class:doris::BufferedTupleStream +_status src/runtime/disk_io_mgr.h /^ Status _status;$/;" m class:doris::DiskIoMgr::BufferDescriptor +_status src/runtime/disk_io_mgr.h /^ Status _status;$/;" m class:doris::DiskIoMgr::ScanRange +_status src/runtime/disk_io_mgr_internal.h /^ Status _status;$/;" m class:doris::DiskIoMgr::RequestContext +_status src/runtime/plan_fragment_executor.h /^ Status _status;$/;" m class:doris::PlanFragmentExecutor +_status src/runtime/record_batch_queue.h /^ Status _status;$/;" m class:doris::RecordBatchQueue +_status src/util/once.h /^ ReturnType _status;$/;" m class:doris::DorisCallOnce +_status_lock src/runtime/fragment_mgr.cpp /^ std::mutex _status_lock;$/;" m class:doris::FragmentExecState file: +_status_lock src/runtime/plan_fragment_executor.h /^ boost::mutex _status_lock;$/;" m class:doris::PlanFragmentExecutor +_status_lock src/runtime/record_batch_queue.h /^ SpinLock _status_lock;$/;" m class:doris::RecordBatchQueue +_status_mutex src/exec/olap_scan_node.h /^ SpinLock _status_mutex;$/;" m class:doris::OlapScanNode +_step_size src/olap/bloom_filter_reader.h /^ size_t _step_size;$/;" m class:doris::BloomFilterIndexReader +_step_size src/olap/stream_index_reader.h /^ size_t _step_size;$/;" m class:doris::StreamIndexReader +_stmt test/runtime/data_stream_test.cpp /^ string _stmt;$/;" m class:doris::DataStreamTest file: +_stop src/runtime/fragment_mgr.h /^ bool _stop;$/;" m class:doris::FragmentMgr +_stop_report_thread_cv src/runtime/plan_fragment_executor.h /^ boost::condition_variable _stop_report_thread_cv;$/;" m class:doris::PlanFragmentExecutor +_storage_engine src/olap/delta_writer.h /^ StorageEngine* _storage_engine;$/;" m class:doris::DeltaWriter +_storage_engine src/runtime/exec_env.h /^ StorageEngine* _storage_engine = nullptr;$/;" m class:doris::ExecEnv +_storage_formats src/util/frame_of_reference_coding.h /^ std::vector _storage_formats;$/;" m class:doris::ForDecoder +_storage_formats src/util/frame_of_reference_coding.h /^ std::vector _storage_formats;$/;" m class:doris::ForEncoder +_storage_medium src/olap/data_dir.h /^ TStorageMedium::type _storage_medium;$/;" m class:doris::DataDir +_storage_medium_migrate src/olap/task/engine_storage_migration_task.cpp /^OLAPStatus EngineStorageMigrationTask::_storage_medium_migrate($/;" f class:doris::EngineStorageMigrationTask +_storage_medium_migrate_req src/olap/task/engine_storage_migration_task.h /^ const TStorageMediumMigrateReq& _storage_medium_migrate_req;$/;" m class:doris::EngineStorageMigrationTask +_storage_medium_migrate_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_storage_medium_migrate_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_storage_medium_migrate_workers src/agent/agent_server.h /^ std::unique_ptr _storage_medium_migrate_workers;$/;" m class:doris::AgentServer +_store_lock src/olap/storage_engine.h /^ std::mutex _store_lock;$/;" m class:doris::StorageEngine +_store_map src/olap/storage_engine.h /^ std::map _store_map;$/;" m class:doris::StorageEngine +_store_paths src/runtime/exec_env.h /^ std::vector _store_paths;$/;" m class:doris::ExecEnv +_stores_nulls src/exec/hash_table.h /^ const bool _stores_nulls;$/;" m class:doris::HashTable +_stores_nulls src/exec/partitioned_hash_table.h /^ const bool _stores_nulls;$/;" m class:doris::PartitionedHashTableCtx +_stores_tuples src/exec/partitioned_hash_table.h /^ const bool _stores_tuples;$/;" m class:doris::PartitionedHashTable +_str src/exec/es/es_predicate.h /^ std::string _str;$/;" m class:doris::ExtLiteral +_str_key src/runtime/dpp_sink_internal.h /^ std::string _str_key;$/;" m class:doris::PartRangeKey +_str_ref src/olap/rowset/column_writer.h /^ const std::string _str_ref;$/;" m class:doris::VarStringColumnWriter::DictKey +_str_value src/exprs/info_func.h /^ std::string _str_value;$/;" m class:doris::InfoFunc +_stream src/olap/in_stream.h /^ InStream* _stream;$/;" m class:doris::InStreamBufferWrapper +_stream src/olap/rowset/column_writer.h /^ OutStream* _stream;$/;" m class:doris::DoubleColumnWriterBase +_stream test/olap/bit_field_test.cpp /^ ReadOnlyFileStream* _stream;$/;" m class:doris::TestBitField file: +_stream test/olap/run_length_byte_test.cpp /^ ReadOnlyFileStream* _stream;$/;" m class:doris::TestRunLengthByte file: +_stream test/olap/run_length_integer_test.cpp /^ ReadOnlyFileStream* _stream;$/;" m class:doris::TestRunLengthSignInteger file: +_stream test/olap/run_length_integer_test.cpp /^ ReadOnlyFileStream* _stream;$/;" m class:doris::TestRunLengthUnsignInteger file: +_stream_buffer_size src/olap/out_stream.h /^ uint32_t _stream_buffer_size;$/;" m class:doris::OutStreamFactory +_stream_buffer_size src/olap/rowset/segment_writer.h /^ uint32_t _stream_buffer_size; \/\/ 输出缓冲区大小$/;" m class:doris::SegmentWriter +_stream_end src/exec/plain_text_line_reader.h /^ bool _stream_end;$/;" m class:doris::PlainTextLineReader +_stream_factory src/olap/rowset/column_writer.h /^ OutStreamFactory* _stream_factory; \/\/ 该对象由外部调用者所有$/;" m class:doris::ColumnWriter +_stream_factory src/olap/rowset/column_writer.h /^ OutStreamFactory* _stream_factory;$/;" m class:doris::IntegerColumnWriter +_stream_factory src/olap/rowset/segment_writer.h /^ OutStreamFactory* _stream_factory;$/;" m class:doris::SegmentWriter +_stream_factory test/olap/column_reader_test.cpp /^ OutStreamFactory *_stream_factory;$/;" m class:doris::TestColumn file: +_stream_factory test/olap/schema_change_test.cpp /^ OutStreamFactory *_stream_factory;$/;" m class:doris::TestColumn file: +_stream_load_executor src/runtime/exec_env.h /^ StreamLoadExecutor* _stream_load_executor = nullptr;$/;" m class:doris::ExecEnv +_stream_load_pipe src/exec/broker_scanner.h /^ std::shared_ptr _stream_load_pipe;$/;" m class:doris::BrokerScanner +_stream_load_pipe src/exec/parquet_scanner.h /^ std::shared_ptr _stream_load_pipe;$/;" m class:doris::ParquetScanner +_stream_map src/runtime/stream_load/load_stream_mgr.h /^ std::unordered_map> _stream_map;$/;" m class:doris::LoadStreamMgr +_stream_mgr src/runtime/exec_env.h /^ DataStreamMgr* _stream_mgr = nullptr;$/;" m class:doris::ExecEnv +_stream_mgr test/runtime/data_stream_test.cpp /^ DataStreamMgr* _stream_mgr;$/;" m class:doris::DataStreamTest file: +_stream_recvr src/exec/exchange_node.h /^ boost::shared_ptr _stream_recvr;$/;" m class:doris::ExchangeNode +_streaming_function_name src/http/action/mini_load.h /^ const std::string _streaming_function_name = "STREAMING_MINI_LOAD";$/;" m class:doris::MiniLoadAction +_streams src/olap/out_stream.h /^ std::map _streams; \/\/ 所有创建过的流$/;" m class:doris::OutStreamFactory +_streams src/olap/rowset/segment_reader.h /^ std::map _streams; \/\/需要读取的流$/;" m class:doris::SegmentReader +_strict_mode src/exec/base_scanner.h /^ bool _strict_mode;$/;" m class:doris::BaseScanner +_string_buffer src/exprs/agg_fn_evaluator.h /^ boost::scoped_array _string_buffer; \/\/for count distinct$/;" m class:doris::AggFnEvaluator +_string_buffer_len src/exprs/agg_fn_evaluator.h /^ int _string_buffer_len; \/\/for count distinct$/;" m class:doris::AggFnEvaluator +_string_content src/olap/wrapper_field.h /^ std::unique_ptr _string_content;$/;" m class:doris::WrapperField +_string_desc test/runtime/buffered_tuple_stream2_test.cpp /^ RowDescriptor* _string_desc;$/;" m class:doris::SimpleTupleStreamTest file: +_string_dict src/olap/rowset/column_writer.h /^ StringDict _string_dict;$/;" m class:doris::VarStringColumnWriter +_string_id src/olap/rowset/column_writer.h /^ std::vector _string_id;$/;" m class:doris::VarStringColumnWriter +_string_keys src/olap/rowset/column_writer.h /^ std::vector _string_keys;$/;" m class:doris::VarStringColumnWriter +_string_length src/olap/rowset/column_reader.h /^ uint32_t _string_length;$/;" m class:doris::FixLengthStringColumnReader +_string_result src/udf/udf_internal.h /^ std::string _string_result;$/;" m class:doris::FunctionContextImpl +_string_slots src/exec/olap_scan_node.h /^ std::vector _string_slots;$/;" m class:doris::OlapScanNode +_string_slots src/exec/olap_scanner.h /^ const std::vector& _string_slots;$/;" m class:doris::OlapScanner +_string_slots src/runtime/buffered_tuple_stream.h /^ std::vector > > _string_slots;$/;" m class:doris::BufferedTupleStream +_string_slots src/runtime/buffered_tuple_stream2.h /^ std::vector > > _string_slots;$/;" m class:doris::BufferedTupleStream2 +_string_slots src/runtime/descriptors.h /^ std::vector _string_slots; \/\/ contains only materialized string slots$/;" m class:doris::TupleDescriptor +_string_value src/runtime/string_buffer.hpp /^ StringValue _string_value;$/;" m class:doris::StringBuffer +_stub src/exec/tablet_sink.h /^ palo::PInternalService_Stub* _stub = nullptr;$/;" m class:doris::stream_load::NodeChannel +_stub_map src/util/brpc_stub_cache.h /^ butil::FlatMap _stub_map;$/;" m class:doris::BrpcStubCache +_sub_plan_query_statistics_recvr src/exec/exchange_node.h /^ std::shared_ptr _sub_plan_query_statistics_recvr;$/;" m class:doris::ExchangeNode +_sub_plan_query_statistics_recvr src/runtime/data_stream_recvr.h /^ std::shared_ptr _sub_plan_query_statistics_recvr;$/;" m class:doris::DataStreamRecvr +_sub_readers src/olap/rowset/column_reader.h /^ std::vector _sub_readers;$/;" m class:doris::ColumnReader +_sub_writers src/olap/rowset/column_writer.h /^ std::vector _sub_writers; \/\/ 保存子列的writer$/;" m class:doris::ColumnWriter +_subquery_string src/exec/assert_num_rows_node.h /^ const std::string _subquery_string;$/;" m class:doris::AssertNumRowsNode +_success_jobs src/runtime/etl_job_mgr.h /^ LruCache _success_jobs;$/;" m class:doris::EtlJobMgr +_success_tasks src/runtime/export_task_mgr.h /^ LruCache _success_tasks;$/;" m class:doris::ExportTaskMgr +_sv src/util/bitmap_value.h /^ uint64_t _sv = 0; \/\/ store the single value when _type == SINGLE$/;" m class:doris::BitmapValue +_sw src/util/runtime_profile.h /^ MonotonicStopWatch _sw;$/;" m class:doris::RuntimeProfile::EventSequence +_sw src/util/runtime_profile.h /^ T _sw;$/;" m class:doris::ScopedRawTimer +_sw src/util/runtime_profile.h /^ T _sw;$/;" m class:doris::ScopedTimer +_swap_buffer src/runtime/merge_sorter.cpp /^ uint8_t* _swap_buffer;$/;" m class:doris::MergeSorter::TupleSorter file: +_swap_buffer src/runtime/spill_sorter.cc /^ uint8_t* _swap_buffer;$/;" m class:doris::SpillSorter::TupleSorter file: +_swap_row_block src/olap/schema_change.cpp /^ RowBlock* _swap_row_block;$/;" m class:doris::RowBlockSorter file: +_symbol_size src/util/bfd_parser.h /^ unsigned int _symbol_size;$/;" m class:doris::BfdParser +_syms src/util/bfd_parser.h /^ bfd_symbol** _syms;$/;" m class:doris::BfdParser +_sync_metadata src/olap/fs/file_block_manager.cpp /^Status FileBlockManager::_sync_metadata(const string& path) {$/;" f class:doris::fs::FileBlockManager +_sync_on_close src/env/env_posix.cpp /^ const bool _sync_on_close = false;$/;" m class:doris::PosixRandomRWFile file: +_sync_on_close src/env/env_posix.cpp /^ const bool _sync_on_close = false;$/;" m class:doris::PosixWritableFile file: +_sys_time src/util/runtime_profile.h /^ Counter* _sys_time; \/\/ system CPU time$/;" m class:doris::RuntimeProfile::ThreadCounters +_syscall0 src/gutil/linux_syscall_support.h 1071;" d +_syscall0 src/gutil/linux_syscall_support.h 1072;" d +_syscall0 src/gutil/linux_syscall_support.h 1368;" d +_syscall0 src/gutil/linux_syscall_support.h 1369;" d +_syscall0 src/gutil/linux_syscall_support.h 1541;" d +_syscall0 src/gutil/linux_syscall_support.h 1542;" d +_syscall0 src/gutil/linux_syscall_support.h 1726;" d +_syscall0 src/gutil/linux_syscall_support.h 1727;" d +_syscall0 src/gutil/linux_syscall_support.h 2000;" d +_syscall0 src/gutil/linux_syscall_support.h 2001;" d +_syscall0 src/gutil/linux_syscall_support.h 2213;" d +_syscall0 src/gutil/linux_syscall_support.h 2214;" d +_syscall0 src/gutil/linux_syscall_support.h 2347;" d +_syscall0 src/gutil/linux_syscall_support.h 2348;" d +_syscall1 src/gutil/linux_syscall_support.h 1081;" d +_syscall1 src/gutil/linux_syscall_support.h 1082;" d +_syscall1 src/gutil/linux_syscall_support.h 1373;" d +_syscall1 src/gutil/linux_syscall_support.h 1374;" d +_syscall1 src/gutil/linux_syscall_support.h 1546;" d +_syscall1 src/gutil/linux_syscall_support.h 1547;" d +_syscall1 src/gutil/linux_syscall_support.h 1732;" d +_syscall1 src/gutil/linux_syscall_support.h 1733;" d +_syscall1 src/gutil/linux_syscall_support.h 2005;" d +_syscall1 src/gutil/linux_syscall_support.h 2006;" d +_syscall1 src/gutil/linux_syscall_support.h 2218;" d +_syscall1 src/gutil/linux_syscall_support.h 2219;" d +_syscall1 src/gutil/linux_syscall_support.h 2352;" d +_syscall1 src/gutil/linux_syscall_support.h 2353;" d +_syscall2 src/gutil/linux_syscall_support.h 1088;" d +_syscall2 src/gutil/linux_syscall_support.h 1089;" d +_syscall2 src/gutil/linux_syscall_support.h 1378;" d +_syscall2 src/gutil/linux_syscall_support.h 1379;" d +_syscall2 src/gutil/linux_syscall_support.h 1553;" d +_syscall2 src/gutil/linux_syscall_support.h 1554;" d +_syscall2 src/gutil/linux_syscall_support.h 1738;" d +_syscall2 src/gutil/linux_syscall_support.h 1739;" d +_syscall2 src/gutil/linux_syscall_support.h 2010;" d +_syscall2 src/gutil/linux_syscall_support.h 2011;" d +_syscall2 src/gutil/linux_syscall_support.h 2223;" d +_syscall2 src/gutil/linux_syscall_support.h 2229;" d +_syscall2 src/gutil/linux_syscall_support.h 2358;" d +_syscall2 src/gutil/linux_syscall_support.h 2359;" d +_syscall2_long src/gutil/linux_syscall_support.h 2224;" d +_syscall3 src/gutil/linux_syscall_support.h 1095;" d +_syscall3 src/gutil/linux_syscall_support.h 1096;" d +_syscall3 src/gutil/linux_syscall_support.h 1383;" d +_syscall3 src/gutil/linux_syscall_support.h 1384;" d +_syscall3 src/gutil/linux_syscall_support.h 1562;" d +_syscall3 src/gutil/linux_syscall_support.h 1563;" d +_syscall3 src/gutil/linux_syscall_support.h 1745;" d +_syscall3 src/gutil/linux_syscall_support.h 1746;" d +_syscall3 src/gutil/linux_syscall_support.h 2015;" d +_syscall3 src/gutil/linux_syscall_support.h 2016;" d +_syscall3 src/gutil/linux_syscall_support.h 2231;" d +_syscall3 src/gutil/linux_syscall_support.h 2238;" d +_syscall3 src/gutil/linux_syscall_support.h 2364;" d +_syscall3 src/gutil/linux_syscall_support.h 2365;" d +_syscall3_long src/gutil/linux_syscall_support.h 2232;" d +_syscall4 src/gutil/linux_syscall_support.h 1103;" d +_syscall4 src/gutil/linux_syscall_support.h 1104;" d +_syscall4 src/gutil/linux_syscall_support.h 1389;" d +_syscall4 src/gutil/linux_syscall_support.h 1390;" d +_syscall4 src/gutil/linux_syscall_support.h 1573;" d +_syscall4 src/gutil/linux_syscall_support.h 1574;" d +_syscall4 src/gutil/linux_syscall_support.h 1752;" d +_syscall4 src/gutil/linux_syscall_support.h 1753;" d +_syscall4 src/gutil/linux_syscall_support.h 2020;" d +_syscall4 src/gutil/linux_syscall_support.h 2021;" d +_syscall4 src/gutil/linux_syscall_support.h 2241;" d +_syscall4 src/gutil/linux_syscall_support.h 2242;" d +_syscall4 src/gutil/linux_syscall_support.h 2370;" d +_syscall4 src/gutil/linux_syscall_support.h 2371;" d +_syscall5 src/gutil/linux_syscall_support.h 1111;" d +_syscall5 src/gutil/linux_syscall_support.h 1112;" d +_syscall5 src/gutil/linux_syscall_support.h 1395;" d +_syscall5 src/gutil/linux_syscall_support.h 1396;" d +_syscall5 src/gutil/linux_syscall_support.h 1587;" d +_syscall5 src/gutil/linux_syscall_support.h 1588;" d +_syscall5 src/gutil/linux_syscall_support.h 1759;" d +_syscall5 src/gutil/linux_syscall_support.h 1764;" d +_syscall5 src/gutil/linux_syscall_support.h 1786;" d +_syscall5 src/gutil/linux_syscall_support.h 2026;" d +_syscall5 src/gutil/linux_syscall_support.h 2027;" d +_syscall5 src/gutil/linux_syscall_support.h 2248;" d +_syscall5 src/gutil/linux_syscall_support.h 2249;" d +_syscall5 src/gutil/linux_syscall_support.h 2380;" d +_syscall5 src/gutil/linux_syscall_support.h 2381;" d +_syscall6 src/gutil/linux_syscall_support.h 1129;" d +_syscall6 src/gutil/linux_syscall_support.h 1130;" d +_syscall6 src/gutil/linux_syscall_support.h 1404;" d +_syscall6 src/gutil/linux_syscall_support.h 1405;" d +_syscall6 src/gutil/linux_syscall_support.h 1604;" d +_syscall6 src/gutil/linux_syscall_support.h 1605;" d +_syscall6 src/gutil/linux_syscall_support.h 1796;" d +_syscall6 src/gutil/linux_syscall_support.h 1801;" d +_syscall6 src/gutil/linux_syscall_support.h 1826;" d +_syscall6 src/gutil/linux_syscall_support.h 2033;" d +_syscall6 src/gutil/linux_syscall_support.h 2034;" d +_syscall6 src/gutil/linux_syscall_support.h 2258;" d +_syscall6 src/gutil/linux_syscall_support.h 2268;" d +_syscall6 src/gutil/linux_syscall_support.h 2390;" d +_syscall6 src/gutil/linux_syscall_support.h 2391;" d +_syscall6_long src/gutil/linux_syscall_support.h 2259;" d +_system_metrics src/util/doris_metrics.h /^ SystemMetrics* _system_metrics;$/;" m class:doris::DorisMetrics +_system_threads_quota src/runtime/thread_resource_mgr.h /^ int _system_threads_quota;$/;" m class:doris::ThreadResourceMgr +_t test/olap/skiplist_test.cpp /^ ConcurrentTest _t;$/;" m class:doris::TestState file: +_t_desc_table test/exec/csv_scan_bench_test.cpp /^ TDescriptorTable _t_desc_table;$/;" m class:doris::CsvScanNodeBenchTest file: +_t_desc_table test/exec/csv_scan_node_test.cpp /^ TDescriptorTable _t_desc_table;$/;" m class:doris::CsvScanNodeTest file: +_t_desc_table test/runtime/memory_scratch_sink_test.cpp /^ TDescriptorTable _t_desc_table;$/;" m class:doris::MemoryScratchSinkTest file: +_t_desc_table test/util/arrow/arrow_work_flow_test.cpp /^ TDescriptorTable _t_desc_table;$/;" m class:doris::ArrowWorkFlowTest file: +_t_desc_tbl test/runtime/data_spliter_test.cpp /^ TDescriptorTable _t_desc_tbl;$/;" m class:doris::DataSplitTest file: +_t_desc_tbl test/runtime/dpp_sink_test.cpp /^ TDescriptorTable _t_desc_tbl;$/;" m class:doris::DppSinkTest file: +_t_desc_tbl test/runtime/qsorter_test.cpp /^ TDescriptorTable _t_desc_tbl;$/;" m class:doris::QSorterTest file: +_t_export_sink src/runtime/export_sink.h /^ TExportSink _t_export_sink;$/;" m class:doris::ExportSink +_t_output_expr src/runtime/export_sink.h /^ const std::vector& _t_output_expr;$/;" m class:doris::ExportSink +_t_output_expr src/runtime/memory_scratch_sink.h /^ const std::vector& _t_output_expr;$/;" m class:doris::MemoryScratchSink +_t_output_expr src/runtime/mysql_table_sink.h /^ const std::vector& _t_output_expr;$/;" m class:doris::MysqlTableSink +_t_output_expr src/runtime/result_sink.h /^ const std::vector& _t_output_expr;$/;" m class:doris::ResultSink +_t_param src/exec/tablet_info.h /^ TOlapTableLocationParam _t_param;$/;" m class:doris::OlapTableLocationParam +_t_param src/exec/tablet_info.h /^ TOlapTablePartitionParam _t_param;$/;" m class:doris::OlapTablePartitionParam +_table src/exec/hash_table.h /^ HashTable* _table;$/;" m class:doris::HashTable::Iterator +_table src/exec/kudu_scan_node.h /^ kudu::client::sp::shared_ptr _table;$/;" m class:doris::KuduScanNode +_table src/exec/partitioned_hash_table.h /^ PartitionedHashTable* _table;$/;" m class:doris::PartitionedHashTable::Iterator +_table src/olap/lru_cache.h /^ HandleTable _table;$/;" m class:doris::CachePriority::LRUCache +_table test/exec/schema_scanner/schema_authors_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaAuthorScannerTest file: +_table test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaCharsetsScannerTest file: +_table test/exec/schema_scanner/schema_collations_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaCollationsScannerTest file: +_table test/exec/schema_scanner/schema_columns_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaColumnsScannerTest file: +_table test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaCreateTableScannerTest file: +_table test/exec/schema_scanner/schema_engines_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaEnginesScannerTest file: +_table test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaOpenTablesScannerTest file: +_table test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaSchemataScannerTest file: +_table test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaTableNamesScannerTest file: +_table test/exec/schema_scanner/schema_tables_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaTablesScannerTest file: +_table test/exec/schema_scanner/schema_variables_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaVariablesScannerTest file: +_table test/exec/schema_scanner_test.cpp /^ std::string _table;$/;" m class:doris::SchemaScannerTest file: +_table_desc src/runtime/descriptors.h /^ TableDescriptor* _table_desc;$/;" m class:doris::TupleDescriptor +_table_id src/exec/tablet_info.h /^ int64_t _table_id;$/;" m class:doris::OlapTableSchemaParam +_table_id src/exec/tablet_sink.h /^ int64_t _table_id = -1;$/;" m class:doris::stream_load::OlapTableSink +_table_id src/olap/tablet_meta.h /^ int64_t _table_id = 0;$/;" m class:doris::TabletMeta +_table_index src/exec/schema_scanner/schema_columns_scanner.h /^ int _table_index;$/;" m class:doris::SchemaColumnsScanner +_table_index src/exec/schema_scanner/schema_tables_scanner.h /^ int _table_index;$/;" m class:doris::SchemaTablesScanner +_table_mem_pool src/olap/memtable.h /^ std::unique_ptr _table_mem_pool;$/;" m class:doris::MemTable +_table_name src/exec/mysql_scan_node.h /^ std::string _table_name;$/;" m class:doris::MysqlScanNode +_table_name src/exec/schema_scan_node.h /^ const std::string _table_name;$/;" m class:doris::SchemaScanNode +_table_name src/exec/tablet_sink.h /^ std::string _table_name;$/;" m class:doris::stream_load::OlapTableSink +_table_result src/exec/schema_scanner/schema_columns_scanner.h /^ TGetTablesResult _table_result;$/;" m class:doris::SchemaColumnsScanner +_table_result src/exec/schema_scanner/schema_tables_scanner.h /^ TListTableStatusResult _table_result;$/;" m class:doris::SchemaTablesScanner +_tablet src/exec/olap_scanner.h /^ TabletSharedPtr _tablet;$/;" m class:doris::OlapScanner +_tablet src/olap/compaction.h /^ TabletSharedPtr _tablet;$/;" m class:doris::Compaction +_tablet src/olap/delta_writer.h /^ TabletSharedPtr _tablet;$/;" m class:doris::DeltaWriter +_tablet src/olap/push_handler.h /^ TabletSharedPtr _tablet;$/;" m class:doris::IBinaryReader +_tablet src/olap/reader.h /^ TabletSharedPtr _tablet;$/;" m class:doris::Reader +_tablet src/olap/rowset/column_data.h /^ Tablet* _tablet;$/;" m class:doris::ColumnData +_tablet src/olap/schema_change.cpp /^ TabletSharedPtr _tablet;$/;" m class:doris::RowBlockMerger file: +_tablet_checkpoint_callback src/olap/storage_engine.cpp /^void* StorageEngine::_tablet_checkpoint_callback(void* arg) {$/;" f class:doris::StorageEngine +_tablet_checkpoint_threads src/olap/storage_engine.h /^ std::vector _tablet_checkpoint_threads;$/;" m class:doris::StorageEngine +_tablet_commit_infos src/runtime/runtime_state.h /^ std::vector _tablet_commit_infos;$/;" m class:doris::RuntimeState +_tablet_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _tablet_counter;$/;" m class:doris::OlapScanNode +_tablet_data_path test/olap/olap_snapshot_converter_test.cpp /^ string _tablet_data_path;$/;" m class:doris::OlapSnapshotConverterTest file: +_tablet_data_path test/olap/tablet_mgr_test.cpp /^ string _tablet_data_path;$/;" m class:doris::TabletMgrTest file: +_tablet_desc src/runtime/dpp_sink.cpp /^ TabletDesc _tablet_desc;$/;" m class:doris::Translator file: +_tablet_id src/olap/memtable.h /^ int64_t _tablet_id;$/;" m class:doris::MemTable +_tablet_id src/olap/rowset/segment_group.h /^ int64_t _tablet_id;$/;" m class:doris::SegmentGroup +_tablet_id src/olap/tablet_meta.h /^ int64_t _tablet_id = 0;$/;" m class:doris::TabletMeta +_tablet_id src/olap/task/engine_checksum_task.h /^ TTabletId _tablet_id;$/;" m class:doris::EngineChecksumTask +_tablet_id test/olap/olap_snapshot_converter_test.cpp /^ int64_t _tablet_id;$/;" m class:doris::OlapSnapshotConverterTest file: +_tablet_id test/olap/tablet_mgr_test.cpp /^ int64_t _tablet_id;$/;" m class:doris::TabletMgrTest file: +_tablet_id_to_unique_id_map src/olap/rowset/segment_reader.h /^ UniqueIdToColumnIdMap _tablet_id_to_unique_id_map; \/\/ tablet id到unique id的映射$/;" m class:doris::SegmentReader +_tablet_infos src/olap/task/engine_batch_load_task.h /^ std::vector* _tablet_infos;$/;" m class:doris::EngineBatchLoadTask +_tablet_infos src/olap/task/engine_clone_task.h /^ vector* _tablet_infos;$/;" m class:doris::EngineCloneTask +_tablet_manager src/olap/data_dir.h /^ TabletManager* _tablet_manager;$/;" m class:doris::DataDir +_tablet_manager src/olap/storage_engine.h /^ std::unique_ptr _tablet_manager;$/;" m class:doris::StorageEngine +_tablet_map_array src/olap/tablet_manager.h /^ tablet_map_t *_tablet_map_array;$/;" m class:doris::TabletManager +_tablet_map_lock_array src/olap/tablet_manager.h /^ RWMutex *_tablet_map_lock_array;$/;" m class:doris::TabletManager +_tablet_map_lock_shard_size src/olap/tablet_manager.h /^ int32_t _tablet_map_lock_shard_size;$/;" m class:doris::TabletManager +_tablet_meta src/olap/rowset/rowset_converter.h /^ TabletMetaSharedPtr _tablet_meta;$/;" m class:doris::RowsetConverter +_tablet_meta src/olap/tablet.h /^ TabletMetaSharedPtr _tablet_meta;$/;" m class:doris::Tablet +_tablet_meta test/exec/new_olap_scan_node_test.cpp /^ TabletMeta* _tablet_meta;$/;" m class:doris::TestOlapScanNode file: +_tablet_mgr test/olap/tablet_mgr_test.cpp /^ TabletManager* _tablet_mgr;$/;" m class:doris::TabletMgrTest file: +_tablet_name test/olap/vectorized_olap_reader_test.cpp /^ std::string _tablet_name;$/;" m class:doris::TestVectorizedOLAPReader file: +_tablet_path src/olap/tablet.h /^ std::string _tablet_path;$/;" m class:doris::Tablet +_tablet_path test/olap/delete_handler_test.cpp /^ std::string _tablet_path;$/;" m class:doris::TestDeleteConditionHandler file: +_tablet_path test/olap/delete_handler_test.cpp /^ std::string _tablet_path;$/;" m class:doris::TestDeleteConditionHandler2 file: +_tablet_path test/olap/delete_handler_test.cpp /^ std::string _tablet_path;$/;" m class:doris::TestDeleteHandler file: +_tablet_path test/olap/olap_reader_test.cpp /^ std::string _tablet_path;$/;" m class:doris::TestOLAPReaderColumn file: +_tablet_path test/olap/olap_reader_test.cpp /^ std::string _tablet_path;$/;" m class:doris::TestOLAPReaderColumnDeleteCondition file: +_tablet_path test/olap/olap_reader_test.cpp /^ std::string _tablet_path;$/;" m class:doris::TestOLAPReaderRow file: +_tablet_path_map src/http/action/restore_tablet_action.h /^ std::map _tablet_path_map;$/;" m class:doris::RestoreTabletAction +_tablet_restore_lock src/http/action/restore_tablet_action.h /^ std::mutex _tablet_restore_lock;$/;" m class:doris::RestoreTabletAction +_tablet_schema src/olap/delta_writer.h /^ const TabletSchema* _tablet_schema;$/;" m class:doris::DeltaWriter +_tablet_schema src/olap/memtable.h /^ const TabletSchema* _tablet_schema;$/;" m class:doris::MemTable +_tablet_schema src/olap/rowset/segment_v2/segment.h /^ const TabletSchema* _tablet_schema;$/;" m class:doris::segment_v2::Segment +_tablet_schema src/olap/rowset/segment_v2/segment_writer.h /^ const TabletSchema* _tablet_schema;$/;" m class:doris::segment_v2::SegmentWriter +_tablet_schema src/olap/schema_change.h /^ const TabletSchema& _tablet_schema;$/;" m class:doris::RowBlockAllocator +_tablet_set src/olap/data_dir.h /^ std::set _tablet_set;$/;" m class:doris::DataDir +_tablet_stat_cache src/olap/tablet_manager.h /^ std::map _tablet_stat_cache;$/;" m class:doris::TabletManager +_tablet_stat_mutex src/olap/tablet_manager.h /^ std::mutex _tablet_stat_mutex;$/;" m class:doris::TabletManager +_tablet_state src/olap/tablet_meta.h /^ TabletState _tablet_state = TABLET_NOTREADY;$/;" m class:doris::TabletMeta +_tablet_uid src/olap/tablet_meta.h /^ TabletUid _tablet_uid;$/;" m class:doris::TabletMeta +_tablet_worker_pool src/service/internal_service.h /^ PriorityThreadPool _tablet_worker_pool;$/;" m class:doris::PInternalServiceImpl +_tablet_writers src/runtime/tablets_channel.h /^ std::unordered_map _tablet_writers;$/;" m class:doris::TabletsChannel +_tablets src/exec/tablet_info.h /^ std::unordered_map _tablets;$/;" m class:doris::OlapTableLocationParam +_tablets_channels src/runtime/load_channel.h /^ std::unordered_map> _tablets_channels;$/;" m class:doris::LoadChannel +_tail_run_length src/olap/rowset/run_length_byte_writer.h /^ int32_t _tail_run_length;$/;" m class:doris::RunLengthByteWriter +_target src/exec/es/es_scan_reader.h /^ std::string _target;$/;" m class:doris::ESScanReader +_task_id src/runtime/snapshot_loader.h /^ int64_t _task_id;$/;" m class:doris::SnapshotLoader +_task_map src/runtime/routine_load/routine_load_task_executor.h /^ std::unordered_map _task_map;$/;" m class:doris::RoutineLoadTaskExecutor +_task_type src/olap/task/engine_alter_tablet_task.h /^ const TTaskType::type _task_type;$/;" m class:doris::EngineAlterTabletTask +_task_worker_type src/agent/task_worker_pool.h /^ TaskWorkerType _task_worker_type;$/;" m class:doris::TaskWorkerPool +_tasks src/agent/task_worker_pool.h /^ std::deque _tasks;$/;" m class:doris::TaskWorkerPool +_tbl_desc_map src/runtime/descriptors.h /^ TableDescriptorMap _tbl_desc_map;$/;" m class:doris::DescriptorTbl +_temp_delta_versions src/olap/schema_change.h /^ Version _temp_delta_versions;$/;" m class:doris::SchemaChangeWithSorting +_temp_tuple_buffer src/runtime/merge_sorter.cpp /^ uint8_t* _temp_tuple_buffer;$/;" m class:doris::MergeSorter::TupleSorter file: +_temp_tuple_buffer src/runtime/spill_sorter.cc /^ uint8_t* _temp_tuple_buffer;$/;" m class:doris::SpillSorter::TupleSorter file: +_temp_tuple_row src/runtime/merge_sorter.cpp /^ TupleRow* _temp_tuple_row;$/;" m class:doris::MergeSorter::TupleSorter file: +_temp_tuple_row src/runtime/spill_sorter.cc /^ TupleRow* _temp_tuple_row;$/;" m class:doris::SpillSorter::TupleSorter file: +_term src/exec/es/es_query_builder.h /^ std::string _term;$/;" m class:doris::TermQueryBuilder +_test_env test/exec/partitioned_hash_table_test.cpp /^ scoped_ptr _test_env;$/;" m class:doris::PartitionedHashTableTest file: +_test_env test/runtime/buffered_block_mgr2_test.cpp /^ scoped_ptr _test_env;$/;" m class:doris::BufferedBlockMgrTest file: +_test_env test/runtime/buffered_tuple_stream2_test.cpp /^ scoped_ptr _test_env;$/;" m class:doris::SimpleTupleStreamTest file: +_test_file_read_buf src/olap/data_dir.h /^ char* _test_file_read_buf;$/;" m class:doris::DataDir +_test_file_write_buf src/olap/data_dir.h /^ char* _test_file_write_buf;$/;" m class:doris::DataDir +_text_converter src/exec/broker_scanner.h /^ std::unique_ptr _text_converter;$/;" m class:doris::BrokerScanner +_text_converter src/exec/csv_scan_node.h /^ boost::scoped_ptr _text_converter;$/;" m class:doris::CsvScanNode +_text_converter src/exec/mysql_scan_node.h /^ std::unique_ptr _text_converter;$/;" m class:doris::MysqlScanNode +_thread src/util/thread.h /^ Thread* _thread;$/;" m class:doris::ThreadJoiner +_thread src/util/thread.h /^ pthread_t _thread;$/;" m class:doris::Thread +_thread_avail_cb_id src/exec/kudu_scan_node.h /^ int _thread_avail_cb_id;$/;" m class:doris::KuduScanNode +_thread_available_fn src/runtime/thread_resource_mgr.h /^ thread_available_cb _thread_available_fn;$/;" m class:doris::ThreadResourceMgr::ResourcePool +_thread_categories src/util/thread.cpp /^ ThreadCategoryMap _thread_categories;$/;" m class:doris::ThreadMgr file: +_thread_id src/util/thread.cpp /^ int64_t _thread_id;$/;" m class:doris::ThreadMgr::ThreadDescriptor file: +_thread_local_fn_state src/udf/udf_internal.h /^ void* _thread_local_fn_state;$/;" m class:doris::FunctionContextImpl +_thread_mgr src/runtime/exec_env.h /^ ThreadResourceMgr* _thread_mgr = nullptr;$/;" m class:doris::ExecEnv +_thread_num src/util/batch_process_thread_pool.hpp /^ uint32_t _thread_num;$/;" m class:doris::BatchProcessThreadPool +_thread_pool src/runtime/exec_env.h /^ PriorityThreadPool* _thread_pool = nullptr;$/;" m class:doris::ExecEnv +_thread_pool src/runtime/fragment_mgr.h /^ PriorityThreadPool _thread_pool;$/;" m class:doris::FragmentMgr +_thread_pool src/runtime/routine_load/data_consumer_group.h /^ PriorityThreadPool _thread_pool;$/;" m class:doris::DataConsumerGroup +_thread_pool src/runtime/routine_load/routine_load_task_executor.h /^ PriorityThreadPool _thread_pool;$/;" m class:doris::RoutineLoadTaskExecutor +_thread_stop src/runtime/broker_mgr.h /^ bool _thread_stop;$/;" m class:doris::BrokerMgr +_threads src/util/batch_process_thread_pool.hpp /^ boost::thread_group _threads;$/;" m class:doris::BatchProcessThreadPool +_threads src/util/priority_thread_pool.hpp /^ boost::thread_group _threads;$/;" m class:doris::PriorityThreadPool +_threads src/util/threadpool.h /^ std::unordered_set _threads;$/;" m class:doris::ThreadPool +_threads test/util/blocking_queue_test.cpp /^ ThreadVector _threads;$/;" m class:doris::MultiThreadTest file: +_threads_running_metric src/util/thread.cpp /^ uint64_t _threads_running_metric;$/;" m class:doris::ThreadMgr file: +_threads_started_metric src/util/thread.cpp /^ uint64_t _threads_started_metric;$/;" m class:doris::ThreadMgr file: +_thrift_server src/util/thrift_server.cpp /^ ThriftServer* _thrift_server;$/;" m class:doris::ThriftServer::ThriftServerEventProcessor file: +_thrift_transmit_timer src/runtime/data_stream_sender.h /^ RuntimeProfile::Counter* _thrift_transmit_timer;$/;" m class:doris::DataStreamSender +_tid src/util/thread.h /^ int64_t _tid;$/;" m class:doris::Thread +_time_zone src/util/arrow/row_batch.cpp /^ std::string _time_zone;$/;" m class:doris::FromRowBatchConverter file: +_timeout_lock src/runtime/result_buffer_mgr.h /^ boost::mutex _timeout_lock;$/;" m class:doris::ResultBufferMgr +_timeout_map src/runtime/result_buffer_mgr.h /^ TimeoutMap _timeout_map;$/;" m class:doris::ResultBufferMgr +_timeout_s src/runtime/load_channel.h /^ int64_t _timeout_s;$/;" m class:doris::LoadChannel +_timeout_second src/runtime/fragment_mgr.cpp /^ int _timeout_second;$/;" m class:doris::FragmentExecState file: +_timestamp_ms src/runtime/runtime_state.h /^ int64_t _timestamp_ms;$/;" m class:doris::RuntimeState +_timezone src/runtime/runtime_state.h /^ std::string _timezone;$/;" m class:doris::RuntimeState +_tls src/util/thread.cpp /^__thread Thread* Thread::_tls = NULL;$/;" m class:doris::Thread file: +_tls src/util/thread.h /^ static __thread Thread* _tls;$/;" m class:doris::Thread +_tmp_dirs src/runtime/tmp_file_mgr.h /^ std::vector _tmp_dirs;$/;" m class:doris::TmpFileMgr +_tmp_file src/runtime/buffered_block_mgr2.h /^ TmpFileMgr::File* _tmp_file;$/;" m class:doris::BufferedBlockMgr2::Block +_tmp_file_mgr src/runtime/buffered_block_mgr2.h /^ TmpFileMgr* _tmp_file_mgr;$/;" m class:doris::BufferedBlockMgr2 +_tmp_file_mgr src/runtime/exec_env.h /^ TmpFileMgr* _tmp_file_mgr = nullptr;$/;" m class:doris::ExecEnv +_tmp_file_mgr src/runtime/test_env.h /^ boost::scoped_ptr _tmp_file_mgr;$/;" m class:doris::TestEnv +_tmp_files src/runtime/buffered_block_mgr2.h /^ boost::ptr_vector _tmp_files;$/;" m class:doris::BufferedBlockMgr2 +_tmp_tuple src/exec/topn_node.h /^ Tuple* _tmp_tuple;$/;" m class:doris::TopNNode +_tnode test/exec/broker_scan_node_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::BrokerScanNodeTest file: +_tnode test/exec/csv_scan_bench_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::CsvScanNodeBenchTest file: +_tnode test/exec/csv_scan_node_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::CsvScanNodeTest file: +_tnode test/exec/es_http_scan_node_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::EsHttpScanNodeTest file: +_tnode test/exec/es_scan_node_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::EsScanNodeTest file: +_tnode test/exec/mysql_scan_node_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::MysqlScanNodeTest file: +_tnode test/exec/new_olap_scan_node_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::TestOlapScanNode file: +_tnode test/exec/olap_scan_node_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::OlapScanNodeTest file: +_tnode test/exec/parquet_scanner_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::ParquetSannerTest file: +_tnode test/exec/schema_scan_node_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::SchemaScanNodeTest file: +_tnode test/olap/olap_reader_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::TestOLAPReaderColumn file: +_tnode test/olap/olap_reader_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::TestOLAPReaderColumnDeleteCondition file: +_tnode test/olap/olap_reader_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::TestOLAPReaderRow file: +_tnode test/olap/vectorized_olap_reader_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::TestVectorizedOLAPReader file: +_tnode test/runtime/memory_scratch_sink_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::MemoryScratchSinkTest file: +_tnode test/util/arrow/arrow_work_flow_test.cpp /^ TPlanNode _tnode;$/;" m class:doris::ArrowWorkFlowTest file: +_to src/olap/rowset/segment_v2/row_ranges.h /^ int64_t _to;$/;" m class:doris::segment_v2::RowRange +_to_be_deleted src/olap/data_dir.h /^ bool _to_be_deleted;$/;" m class:doris::DataDir +_to_close_child_idx src/exec/except_node.h /^ int _to_close_child_idx;$/;" m class:doris::ExceptNode +_to_close_child_idx src/exec/union_node.h /^ int _to_close_child_idx;$/;" m class:doris::UnionNode +_to_errmsg src/http/http_client.cpp /^const char* HttpClient::_to_errmsg(CURLcode code) {$/;" f class:doris::HttpClient +_to_string src/olap/types.h /^ std::string (*_to_string)(const void* src);$/;" m class:doris::TypeInfo::std +_tokenless src/util/threadpool.h /^ std::unique_ptr _tokenless;$/;" m class:doris::ThreadPool +_tokens src/util/threadpool.h /^ std::unordered_set _tokens;$/;" m class:doris::ThreadPool +_topic src/runtime/routine_load/data_consumer.h /^ std::string _topic;$/;" m class:doris::KafkaDataConsumer +_topic_subscriber src/agent/agent_server.h /^ std::unique_ptr _topic_subscriber;$/;" m class:doris::AgentServer +_total src/exec/es/es_scroll_parser.h /^ int _total;$/;" m class:doris::ScrollParser +_total src/util/progress_updater.h /^ int64_t _total;$/;" m class:doris::ProgressUpdater +_total_assign_num src/exec/olap_scan_node.h /^ int _total_assign_num;$/;" m class:doris::OlapScanNode +_total_buffer_limit src/runtime/data_stream_recvr.h /^ int _total_buffer_limit;$/;" m class:doris::DataStreamRecvr +_total_byte_size src/runtime/buffered_tuple_stream.h /^ int64_t _total_byte_size;$/;" m class:doris::BufferedTupleStream +_total_byte_size src/runtime/buffered_tuple_stream2.h /^ int64_t _total_byte_size;$/;" m class:doris::BufferedTupleStream2 +_total_bytes_read_counter src/runtime/disk_io_mgr.h /^ RuntimeProfile::Counter _total_bytes_read_counter;$/;" m class:doris::DiskIoMgr +_total_data_page_size src/exec/partitioned_hash_table.h /^ int64_t _total_data_page_size;$/;" m class:doris::PartitionedHashTable +_total_data_size src/olap/rowset/beta_rowset_writer.h /^ int64_t _total_data_size;$/;" m class:doris::BetaRowsetWriter +_total_error_num src/util/broker_load_error_hub.h /^ int32_t _total_error_num = 0;$/;" m class:doris::BrokerLoadErrorHub +_total_error_num src/util/load_error_hub.h /^ int32_t _total_error_num = 0;$/;" m class:doris::LoadErrorHub +_total_error_num src/util/mysql_load_error_hub.h /^ int32_t _total_error_num = 0;$/;" m class:doris::MysqlLoadErrorHub +_total_get_wait_time src/util/blocking_priority_queue.hpp /^ uint64_t _total_get_wait_time;$/;" m class:doris::BlockingPriorityQueue +_total_get_wait_time src/util/blocking_queue.hpp /^ uint64_t _total_get_wait_time;$/;" m class:doris::BlockingQueue +_total_groups src/exec/orc_scanner.h /^ int _total_groups; \/\/ groups in a orc file$/;" m class:doris::ORCScanner +_total_groups src/exec/parquet_reader.h /^ int _total_groups; \/\/ groups in a parquet file$/;" m class:doris::ParquetReaderWrap +_total_index_size src/olap/rowset/beta_rowset_writer.h /^ int64_t _total_index_size;$/;" m class:doris::BetaRowsetWriter +_total_mem_consumption src/exprs/agg_fn_evaluator.h /^ uint64_t _total_mem_consumption;$/;" m class:doris::AggFnEvaluator +_total_mem_consumption src/exprs/new_agg_fn_evaluator.h /^ uint64_t _total_mem_consumption;$/;" m class:doris::NewAggFnEvaluator +_total_num_rows src/exec/row_batch_list.h /^ int64_t _total_num_rows;$/;" m class:doris::RowBatchList +_total_pages_num_counter src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _total_pages_num_counter = nullptr;$/;" m class:doris::OlapScanNode +_total_pinned_buffers src/runtime/buffered_block_mgr2.h /^ int _total_pinned_buffers;$/;" m class:doris::BufferedBlockMgr2 +_total_put_wait_time src/util/blocking_priority_queue.hpp /^ uint64_t _total_put_wait_time;$/;" m class:doris::BlockingPriorityQueue +_total_put_wait_time src/util/blocking_queue.hpp /^ uint64_t _total_put_wait_time;$/;" m class:doris::BlockingQueue +_total_queued_tasks src/util/threadpool.h /^ int _total_queued_tasks;$/;" m class:doris::ThreadPool +_total_range_queue_capacity src/runtime/disk_io_mgr_internal.h /^ AtomicInt _total_range_queue_capacity;$/;" m class:doris::DiskIoMgr::RequestContext +_total_read_bytes src/exec/plain_text_line_reader.h /^ size_t _total_read_bytes;$/;" m class:doris::PlainTextLineReader +_total_throughput_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* _total_throughput_counter;$/;" m class:doris::ScanNode +_total_time src/util/runtime_profile.h /^ Counter* _total_time; \/\/ total wall clock time$/;" m class:doris::RuntimeProfile::ThreadCounters +_total_time src/util/stopwatch.hpp /^ uint64_t _total_time; \/\/ in nanosec$/;" m class:doris::MonotonicStopWatch +_tproto src/util/thrift_util.h /^ boost::shared_ptr _tproto;$/;" m class:doris::ThriftDeserializer +_tracker src/olap/olap_index.h /^ std::unique_ptr _tracker;$/;" m class:doris::MemIndex +_tracker src/olap/reader.h /^ std::unique_ptr _tracker;$/;" m class:doris::Reader +_tracker src/olap/row_block.h /^ std::unique_ptr _tracker;$/;" m class:doris::RowBlock +_tracker src/olap/row_block2.h /^ MemTracker _tracker;$/;" m class:doris::RowBlockV2 +_tracker src/olap/rowset/segment_reader.h /^ std::unique_ptr _tracker;$/;" m class:doris::SegmentReader +_tracker src/olap/rowset/segment_v2/binary_dict_page.h /^ MemTracker _tracker;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +_tracker src/olap/rowset/segment_v2/bitmap_index_reader.h /^ MemTracker _tracker;$/;" m class:doris::segment_v2::BitmapIndexIterator +_tracker src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^ MemTracker _tracker;$/;" m class:doris::segment_v2::__anon54::BitmapIndexWriterImpl file: +_tracker src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ MemTracker _tracker;$/;" m class:doris::segment_v2::BloomFilterIndexIterator +_tracker src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ MemTracker _tracker;$/;" m class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl file: +_tracker src/olap/rowset/segment_v2/column_reader.h /^ MemTracker _tracker;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_tracker src/olap/rowset/segment_v2/zone_map_index.h /^ MemTracker _tracker;$/;" m class:doris::segment_v2::ZoneMapIndexWriter +_tracker src/runtime/buffered_block_mgr2.cc /^ MemTracker* _tracker;$/;" m class:doris::BufferedBlockMgr2::Client file: +_tracker src/runtime/vectorized_row_batch.h /^ std::unique_ptr _tracker;$/;" m class:doris::VectorizedRowBatch +_tracker src/util/arrow/row_batch.cpp /^ MemTracker* _tracker;$/;" m class:doris::ToRowBatchConverter file: +_tracker test/exec/broker_scanner_test.cpp /^ MemTracker _tracker;$/;" m class:doris::BrokerScannerTest file: +_tracker test/exec/partitioned_hash_table_test.cpp /^ MemTracker _tracker;$/;" m class:doris::PartitionedHashTableTest file: +_tracker test/olap/key_coder_test.cpp /^ MemTracker _tracker;$/;" m class:doris::KeyCoderTest file: +_tracker test/olap/rowset/segment_v2/bitmap_index_test.cpp /^ MemTracker _tracker;$/;" m class:doris::segment_v2::BitmapIndexTest file: +_tracker test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^ MemTracker _tracker;$/;" m class:doris::segment_v2::ColumnReaderWriterTest file: +_tracker test/runtime/buffered_tuple_stream2_test.cpp /^ MemTracker _tracker;$/;" m class:doris::SimpleTupleStreamTest file: +_tracker test/runtime/data_stream_test.cpp /^ MemTracker _tracker;$/;" m class:doris::DataStreamTest file: +_transfer_done src/exec/olap_scan_node.h /^ bool _transfer_done;$/;" m class:doris::OlapScanNode +_transfer_thread src/exec/olap_scan_node.h /^ boost::thread_group _transfer_thread;$/;" m class:doris::OlapScanNode +_translator_count src/runtime/dpp_sink.h /^ int _translator_count;$/;" m class:doris::DppSink +_translator_map src/runtime/dpp_sink.h /^ std::unordered_map> _translator_map;$/;" m class:doris::DppSink +_transport src/util/thrift_client.h /^ boost::shared_ptr _transport;$/;" m class:doris::ThriftClientImpl +_travel_length src/exec/partitioned_hash_table.h /^ int64_t _travel_length;$/;" m class:doris::PartitionedHashTable +_try_cache src/runtime/disk_io_mgr.h /^ bool _try_cache;$/;" m class:doris::DiskIoMgr::ScanRange +_tsink test/runtime/memory_scratch_sink_test.cpp /^ TMemoryScratchSink _tsink;$/;" m class:doris::MemoryScratchSinkTest file: +_tsink test/runtime/result_sink_test.cpp /^ TResultSink _tsink;$/;" m class:doris::ResultSinkTest file: +_tuple src/exec/csv_scan_node.h /^ Tuple* _tuple;$/;" m class:doris::CsvScanNode +_tuple src/exec/mysql_scan_node.h /^ Tuple* _tuple = nullptr;$/;" m class:doris::MysqlScanNode +_tuple_data_pool src/runtime/row_batch.h /^ boost::scoped_ptr _tuple_data_pool;$/;" m class:doris::RowBatch +_tuple_desc src/exec/broker_scan_node.h /^ TupleDescriptor* _tuple_desc;$/;" m class:doris::BrokerScanNode +_tuple_desc src/exec/csv_scan_node.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::CsvScanNode +_tuple_desc src/exec/es/es_predicate.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::EsPredicate +_tuple_desc src/exec/es_http_scan_node.h /^ TupleDescriptor* _tuple_desc;$/;" m class:doris::EsHttpScanNode +_tuple_desc src/exec/es_http_scanner.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::EsHttpScanner +_tuple_desc src/exec/es_scan_node.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::EsScanNode +_tuple_desc src/exec/except_node.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::ExceptNode +_tuple_desc src/exec/kudu_scan_node.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::KuduScanNode +_tuple_desc src/exec/merge_node.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::MergeNode +_tuple_desc src/exec/mysql_scan_node.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::MysqlScanNode +_tuple_desc src/exec/olap_scan_node.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::OlapScanNode +_tuple_desc src/exec/olap_scanner.h /^ const TupleDescriptor* _tuple_desc; \/**< tuple descripter *\/$/;" m class:doris::OlapScanner +_tuple_desc src/exec/repeat_node.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::RepeatNode +_tuple_desc src/exec/schema_scanner.h /^ TupleDescriptor* _tuple_desc;$/;" m class:doris::SchemaScanner +_tuple_desc src/exec/tablet_info.h /^ TupleDescriptor* _tuple_desc = nullptr;$/;" m class:doris::OlapTableSchemaParam +_tuple_desc src/exec/tablet_sink.h /^ TupleDescriptor* _tuple_desc = nullptr;$/;" m class:doris::stream_load::NodeChannel +_tuple_desc src/exec/union_node.h /^ const TupleDescriptor* _tuple_desc;$/;" m class:doris::UnionNode +_tuple_desc src/olap/memtable.h /^ TupleDescriptor* _tuple_desc;$/;" m class:doris::MemTable +_tuple_desc src/runtime/descriptor_helper.h /^ TTupleDescriptor _tuple_desc;$/;" m class:doris::TTupleDescriptorBuilder +_tuple_desc src/runtime/tablets_channel.h /^ TupleDescriptor* _tuple_desc = nullptr;$/;" m class:doris::TabletsChannel +_tuple_desc_id src/exec/tablet_sink.h /^ int _tuple_desc_id = -1;$/;" m class:doris::stream_load::OlapTableSink +_tuple_desc_map src/runtime/descriptors.h /^ TupleDescriptorMap _tuple_desc_map;$/;" m class:doris::DescriptorTbl +_tuple_desc_map src/runtime/descriptors.h /^ std::vector _tuple_desc_map;$/;" m class:doris::RowDescriptor +_tuple_id src/exec/broker_scan_node.h /^ TupleId _tuple_id;$/;" m class:doris::BrokerScanNode +_tuple_id src/exec/csv_scan_node.h /^ TupleId _tuple_id;$/;" m class:doris::CsvScanNode +_tuple_id src/exec/es_http_scan_node.h /^ TupleId _tuple_id;$/;" m class:doris::EsHttpScanNode +_tuple_id src/exec/es_http_scanner.h /^ TupleId _tuple_id;$/;" m class:doris::EsHttpScanner +_tuple_id src/exec/es_scan_node.h /^ TupleId _tuple_id;$/;" m class:doris::EsScanNode +_tuple_id src/exec/except_node.h /^ const int _tuple_id;$/;" m class:doris::ExceptNode +_tuple_id src/exec/kudu_scan_node.h /^ const TupleId _tuple_id;$/;" m class:doris::KuduScanNode +_tuple_id src/exec/merge_node.h /^ int _tuple_id;$/;" m class:doris::MergeNode +_tuple_id src/exec/mysql_scan_node.h /^ TupleId _tuple_id;$/;" m class:doris::MysqlScanNode +_tuple_id src/exec/olap_scan_node.h /^ TupleId _tuple_id;$/;" m class:doris::OlapScanNode +_tuple_id src/exec/schema_scan_node.h /^ TupleId _tuple_id;$/;" m class:doris::SchemaScanNode +_tuple_id src/exec/union_node.h /^ const int _tuple_id;$/;" m class:doris::UnionNode +_tuple_id src/exprs/slot_ref.h /^ TupleId _tuple_id; \/\/ used for desc this slot from$/;" m class:doris::SlotRef +_tuple_id src/runtime/descriptor_helper.h /^ TTupleId _tuple_id;$/;" m class:doris::TTupleDescriptorBuilder +_tuple_ids src/exec/exec_node.h /^ std::vector _tuple_ids;$/;" m class:doris::ExecNode +_tuple_ids src/exprs/tuple_is_null_predicate.h /^ std::vector _tuple_ids;$/;" m class:doris::TupleIsNullPredicate +_tuple_idx src/exec/olap_scan_node.h /^ int _tuple_idx;$/;" m class:doris::OlapScanNode +_tuple_idx src/exec/olap_scanner.h /^ int _tuple_idx = 0;$/;" m class:doris::OlapScanner +_tuple_idx src/exec/schema_scan_node.h /^ int _tuple_idx;$/;" m class:doris::SchemaScanNode +_tuple_idx src/exprs/slot_ref.h /^ int _tuple_idx; \/\/ within row$/;" m class:doris::SlotRef +_tuple_idx_map src/runtime/descriptors.h /^ std::vector _tuple_idx_map;$/;" m class:doris::RowDescriptor +_tuple_idx_nullable_map src/runtime/descriptors.h /^ std::vector _tuple_idx_nullable_map;$/;" m class:doris::RowDescriptor +_tuple_idxs src/exprs/tuple_is_null_predicate.h /^ std::vector _tuple_idxs;$/;" m class:doris::TupleIsNullPredicate +_tuple_is_nullable src/exprs/slot_ref.h /^ bool _tuple_is_nullable; \/\/ true if the tuple is nullable.$/;" m class:doris::SlotRef +_tuple_mem test/runtime/data_stream_test.cpp /^ int64_t* _tuple_mem;$/;" m class:doris::DataStreamTest file: +_tuple_offset src/runtime/descriptors.h /^ const int _tuple_offset;$/;" m class:doris::SlotDescriptor +_tuple_pool src/exec/aggregation_node.h /^ boost::scoped_ptr _tuple_pool;$/;" m class:doris::AggregationNode +_tuple_pool src/exec/csv_scan_node.h /^ boost::scoped_ptr _tuple_pool;$/;" m class:doris::CsvScanNode +_tuple_pool src/exec/mysql_scan_node.h /^ std::unique_ptr _tuple_pool;$/;" m class:doris::MysqlScanNode +_tuple_pool src/exec/pre_aggregation_node.h /^ boost::scoped_ptr _tuple_pool;$/;" m class:doris::PreAggregationNode +_tuple_pool src/exec/schema_scan_node.h /^ boost::scoped_ptr _tuple_pool;$/;" m class:doris::SchemaScanNode +_tuple_pool src/exec/sort_node.h /^ boost::scoped_ptr _tuple_pool;$/;" m class:doris::SortNode +_tuple_pool src/exec/topn_node.h /^ boost::scoped_ptr _tuple_pool;$/;" m class:doris::TopNNode +_tuple_pool src/runtime/buffered_block_mgr.h /^ boost::scoped_ptr _tuple_pool;$/;" m class:doris::BufferedBlockMgr +_tuple_pool src/runtime/qsorter.h /^ boost::scoped_ptr _tuple_pool;$/;" m class:doris::QSorter +_tuple_pool test/runtime/data_spliter_test.cpp /^ MemPool _tuple_pool;$/;" m class:doris::DataSplitTest file: +_tuple_pool test/runtime/dpp_sink_test.cpp /^ MemPool _tuple_pool;$/;" m class:doris::DppSinkTest file: +_tuple_pool test/runtime/qsorter_test.cpp /^ MemPool _tuple_pool;$/;" m class:doris::QSorterTest file: +_tuple_ptrs src/runtime/row_batch.h /^ Tuple** _tuple_ptrs;$/;" m class:doris::RowBatch +_tuple_ptrs_size src/runtime/row_batch.h /^ int _tuple_ptrs_size;$/;" m class:doris::RowBatch +_tuple_row test/exprs/in_predicate_test.cpp /^ TupleRow _tuple_row;$/;" m class:doris::InPredicateTest file: +_tuple_row_less_than src/exec/topn_node.h /^ boost::scoped_ptr _tuple_row_less_than;$/;" m class:doris::TopNNode +_tuple_row_size src/exec/pre_aggregation_node.h /^ int _tuple_row_size;$/;" m class:doris::PreAggregationNode +_tuple_size src/runtime/merge_sorter.cpp /^ const int _tuple_size;$/;" m class:doris::MergeSorter::TupleSorter file: +_tuple_size src/runtime/spill_sorter.cc /^ const int _tuple_size;$/;" m class:doris::SpillSorter::TupleSorter file: +_tuple_stream src/exec/partitioned_hash_table.h /^ BufferedTupleStream2* _tuple_stream;$/;" m class:doris::PartitionedHashTable +_tuple_streams src/runtime/row_batch.h /^ std::vector _tuple_streams;$/;" m class:doris::RowBatch +_tuple_to_row src/olap/memtable.cpp /^void MemTable::_tuple_to_row(const Tuple* tuple, ContiguousRow* row, MemPool* mem_pool) {$/;" f class:doris::MemTable +_tuples src/runtime/tuple_row.h /^ Tuple* _tuples[1];$/;" m class:doris::TupleRow +_tuples_descs src/testutil/desc_tbl_builder.h /^ std::vector _tuples_descs;$/;" m class:doris::DescriptorTblBuilder +_txn_id src/exec/tablet_sink.h /^ int64_t _txn_id = -1;$/;" m class:doris::stream_load::OlapTableSink +_txn_id src/olap/rowset/segment_group.h /^ TTransactionId _txn_id;$/;" m class:doris::SegmentGroup +_txn_id src/runtime/tablets_channel.h /^ int64_t _txn_id = -1;$/;" m class:doris::TabletsChannel +_txn_lock_mask src/olap/txn_manager.h /^ const int32_t _txn_lock_mask = _txn_lock_num - 1;$/;" m class:doris::TxnManager +_txn_lock_num src/olap/txn_manager.h /^ const int32_t _txn_lock_num = 1 << 7;$/;" m class:doris::TxnManager +_txn_locks src/olap/txn_manager.h /^ std::map> _txn_locks;$/;" m class:doris::TxnManager +_txn_manager src/olap/data_dir.h /^ TxnManager* _txn_manager;$/;" m class:doris::DataDir +_txn_manager src/olap/storage_engine.h /^ std::unique_ptr _txn_manager;$/;" m class:doris::StorageEngine +_txn_map_lock src/olap/txn_manager.h /^ RWMutex _txn_map_lock;$/;" m class:doris::TxnManager +_txn_mgr test/olap/olap_snapshot_converter_test.cpp /^ TxnManager _txn_mgr;$/;" m class:doris::OlapSnapshotConverterTest file: +_txn_mgr test/olap/tablet_mgr_test.cpp /^ TxnManager _txn_mgr;$/;" m class:doris::TabletMgrTest file: +_txn_mgr test/olap/txn_manager_test.cpp /^ TxnManager _txn_mgr;$/;" m class:doris::TxnManagerTest file: +_txn_partition_map src/olap/txn_manager.h /^ std::unordered_map> _txn_partition_map;$/;" m class:doris::TxnManager +_type src/exec/es/es_predicate.h /^ PrimitiveType _type;$/;" m class:doris::ExtLiteral +_type src/exec/es/es_scan_reader.h /^ std::string _type;$/;" m class:doris::ESScanReader +_type src/exec/exec_node.h /^ TPlanNodeType::type _type;$/;" m class:doris::ExecNode +_type src/exec/schema_scanner/schema_variables_scanner.h /^ TVarType::type _type;$/;" m class:doris::SchemaVariablesScanner +_type src/exprs/aggregate_functions.cpp /^ FunctionContext::Type _type;$/;" m class:doris::MultiDistinctCountDateState file: +_type src/exprs/aggregate_functions.cpp /^ FunctionContext::Type _type;$/;" m class:doris::MultiDistinctDecimalState file: +_type src/exprs/aggregate_functions.cpp /^ FunctionContext::Type _type;$/;" m class:doris::MultiDistinctDecimalV2State file: +_type src/exprs/aggregate_functions.cpp /^ FunctionContext::Type _type;$/;" m class:doris::MultiDistinctNumericState file: +_type src/exprs/aggregate_functions.cpp /^ FunctionContext::Type _type;$/;" m class:doris::MultiDistinctStringCountState file: +_type src/exprs/expr.h /^ TypeDescriptor _type;$/;" m class:doris::Expr +_type src/exprs/hybird_map.h /^ PrimitiveType _type;$/;" m class:doris::HybirdMap +_type src/http/action/compaction_action.h /^ CompactionActionType _type;$/;" m class:doris::CompactionAction +_type src/olap/hll.h /^ HllDataType _type = HLL_DATA_EMPTY;$/;" m class:doris::HyperLogLog +_type src/olap/rowset/column_reader.h /^ FieldType _type;$/;" m class:doris::DefaultValueReader +_type src/olap/rowset/segment_v2/column_reader.h /^ FieldType _type;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_type src/olap/rowset/segment_v2/encoding_info.h /^ FieldType _type;$/;" m class:doris::segment_v2::EncodingInfo +_type src/olap/tablet_schema.h /^ FieldType _type;$/;" m class:doris::TabletColumn +_type src/runtime/descriptors.h /^ const TypeDescriptor _type;$/;" m class:doris::SlotDescriptor +_type src/runtime/dpp_sink_internal.h /^ PrimitiveType _type;$/;" m class:doris::PartRangeKey +_type src/util/bitmap_value.h /^ BitmapDataType _type;$/;" m class:doris::BitmapValue +_type src/util/metrics.h /^ MetricType _type = MetricType::UNTYPED;$/;" m class:doris::MetricCollector +_type src/util/metrics.h /^ MetricType _type;$/;" m class:doris::Metric +_type src/util/runtime_profile.h /^ TUnit::type _type;$/;" m class:doris::RuntimeProfile::Counter +_type_info src/olap/column_block.h /^ const TypeInfo* _type_info;$/;" m class:doris::ColumnBlock +_type_info src/olap/field.h /^ const TypeInfo* _type_info;$/;" m class:doris::Field +_type_info src/olap/rowset/segment_v2/column_reader.h /^ const TypeInfo* _type_info = nullptr;$/;" m class:doris::segment_v2::ColumnReader +_type_info src/olap/rowset/segment_v2/indexed_column_reader.h /^ const TypeInfo* _type_info = nullptr;$/;" m class:doris::segment_v2::IndexedColumnReader +_type_max src/exec/olap_common.h /^ T _type_max; \/\/ Column type's max value$/;" m class:doris::ColumnValueRange +_type_min src/exec/olap_common.h /^ T _type_min; \/\/ Column type's min value$/;" m class:doris::ColumnValueRange +_type_size src/olap/rowset/segment_v2/column_reader.h /^ size_t _type_size;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +_typeinfo src/olap/rowset/segment_v2/bitmap_index_reader.h /^ const TypeInfo* _typeinfo;$/;" m class:doris::segment_v2::BitmapIndexReader +_typeinfo src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^ const TypeInfo* _typeinfo;$/;" m class:doris::segment_v2::__anon54::BitmapIndexWriterImpl file: +_typeinfo src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ const TypeInfo* _typeinfo;$/;" m class:doris::segment_v2::BloomFilterIndexReader +_typeinfo src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ const TypeInfo* _typeinfo;$/;" m class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl file: +_typeinfo src/olap/rowset/segment_v2/indexed_column_writer.h /^ const TypeInfo* _typeinfo;$/;" m class:doris::segment_v2::IndexedColumnWriter +_uncompressed src/olap/file_stream.h /^ StorageByteBuffer* _uncompressed;$/;" m class:doris::ReadOnlyFileStream +_uncompressed src/olap/in_stream.h /^ StorageByteBuffer* _uncompressed;$/;" m class:doris::InStream +_uncompressed_bytes_counter src/runtime/data_stream_sender.h /^ RuntimeProfile::Counter* _uncompressed_bytes_counter;$/;" m class:doris::DataStreamSender +_unexpected_remote_bytes src/runtime/disk_io_mgr_internal.h /^ AtomicInt _unexpected_remote_bytes;$/;" m class:doris::DiskIoMgr::RequestContext +_unfullfilled_reserved_buffers src/runtime/buffered_block_mgr2.h /^ int _unfullfilled_reserved_buffers;$/;" m class:doris::BufferedBlockMgr2 +_union_block src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::_union_block(RowBlock** block) {$/;" f class:doris::AlphaRowsetReader +_unique_column_id src/olap/rowset/column_writer.h /^ uint32_t _unique_column_id;$/;" m class:doris::IntegerColumnWriter +_unique_column_id src/olap/stream_name.h /^ uint32_t _unique_column_id;$/;" m class:doris::StreamName +_unique_id src/olap/tablet_schema.h /^ int32_t _unique_id;$/;" m class:doris::TabletColumn +_unique_id_to_segment_id_map src/olap/rowset/segment_reader.h /^ UniqueIdToColumnIdMap _unique_id_to_segment_id_map; \/\/ uniqid到segment id的映射$/;" m class:doris::SegmentReader +_unique_id_to_tablet_id_map src/olap/rowset/segment_reader.h /^ UniqueIdToColumnIdMap _unique_id_to_tablet_id_map; \/\/ unique id到tablet id的映射$/;" m class:doris::SegmentReader +_unique_key_next_row src/olap/reader.cpp /^OLAPStatus Reader::_unique_key_next_row(RowCursor* row_cursor, MemPool* mem_pool, ObjectPool* agg_pool, bool* eof) {$/;" f class:doris::Reader +_unpin_timer src/runtime/buffered_tuple_stream.h /^ RuntimeProfile::Counter* _unpin_timer;$/;" m class:doris::BufferedTupleStream +_unpin_timer src/runtime/buffered_tuple_stream2.h /^ RuntimeProfile::Counter* _unpin_timer;$/;" m class:doris::BufferedTupleStream2 +_unpinned_blocks src/runtime/buffered_block_mgr2.h /^ InternalQueue _unpinned_blocks;$/;" m class:doris::BufferedBlockMgr2 +_unprocessed src/util/tdigest.h /^ std::vector _unprocessed;$/;" m class:doris::TDigest +_unprocessed_weight src/util/tdigest.h /^ Value _unprocessed_weight = 0.0;$/;" m class:doris::TDigest +_unref src/olap/lru_cache.cpp /^bool LRUCache::_unref(LRUHandle* e) {$/;" f class:doris::LRUCache +_unreported_error_idx src/runtime/runtime_state.h /^ int _unreported_error_idx;$/;" m class:doris::RuntimeState +_unsorted_run src/runtime/merge_sorter.h /^ Run* _unsorted_run;$/;" m class:doris::MergeSorter +_unsorted_run src/runtime/spill_sorter.h /^ Run* _unsorted_run;$/;" m class:doris::SpillSorter +_unspecified_colomn_slot_vec src/exec/csv_scan_node.h /^ std::vector _unspecified_colomn_slot_vec;$/;" m class:doris::CsvScanNode +_unspecified_colomn_type_vec src/exec/csv_scan_node.h /^ std::vector _unspecified_colomn_type_vec;$/;" m class:doris::CsvScanNode +_unspecified_columns src/exec/csv_scan_node.h /^ std::vector _unspecified_columns;$/;" m class:doris::CsvScanNode +_unstarted_scan_ranges src/runtime/disk_io_mgr_internal.h /^ InternalQueue _unstarted_scan_ranges;$/;" m class:doris::DiskIoMgr::RequestContext::PerDiskState +_unstarted_write_ranges src/runtime/disk_io_mgr_internal.h /^ InternalQueue _unstarted_write_ranges;$/;" m class:doris::DiskIoMgr::RequestContext::PerDiskState +_unused_blocks src/runtime/buffered_block_mgr2.h /^ InternalQueue _unused_blocks;$/;" m class:doris::BufferedBlockMgr2 +_unused_rowset_monitor_thread src/olap/storage_engine.h /^ std::thread _unused_rowset_monitor_thread;$/;" m class:doris::StorageEngine +_unused_rowset_monitor_thread_callback src/olap/olap_server.cpp /^void* StorageEngine::_unused_rowset_monitor_thread_callback(void* arg) {$/;" f class:doris::StorageEngine +_unused_rowsets src/olap/storage_engine.h /^ std::unordered_map _unused_rowsets;$/;" m class:doris::StorageEngine +_unz64_H src/util/minizip/unzip.h 44;" d +_update_cgroups_mtx src/agent/cgroups_mgr.h /^ std::mutex _update_cgroups_mtx; $/;" m class:doris::CgroupsMgr +_update_cpu_metrics src/util/system_metrics.cpp /^void SystemMetrics::_update_cpu_metrics() {$/;" f class:doris::SystemMetrics +_update_disk_metrics src/util/system_metrics.cpp /^void SystemMetrics::_update_disk_metrics() {$/;" f class:doris::SystemMetrics +_update_fd_metrics src/util/system_metrics.cpp /^void SystemMetrics::_update_fd_metrics() {$/;" f class:doris::SystemMetrics +_update_fn output/udf/include/uda_test_harness.h /^ UpdateFn _update_fn;$/;" m class:doris_udf::UdaTestHarness +_update_fn output/udf/include/uda_test_harness.h /^ UpdateFn _update_fn;$/;" m class:doris_udf::UdaTestHarness2 +_update_fn output/udf/include/uda_test_harness.h /^ UpdateFn _update_fn;$/;" m class:doris_udf::UdaTestHarness3 +_update_fn output/udf/include/uda_test_harness.h /^ UpdateFn _update_fn;$/;" m class:doris_udf::UdaTestHarness4 +_update_fn src/exprs/agg_fn_evaluator.h /^ void* _update_fn;$/;" m class:doris::AggFnEvaluator +_update_fn src/olap/aggregate_func.h /^ void (*_update_fn)(RowCursorCell* dst, const RowCursorCell& src, MemPool* mem_pool);$/;" m class:doris::AggregateInfo +_update_fn src/udf/uda_test_harness.h /^ UpdateFn _update_fn;$/;" m class:doris_udf::UdaTestHarness +_update_fn src/udf/uda_test_harness.h /^ UpdateFn _update_fn;$/;" m class:doris_udf::UdaTestHarness2 +_update_fn src/udf/uda_test_harness.h /^ UpdateFn _update_fn;$/;" m class:doris_udf::UdaTestHarness3 +_update_fn src/udf/uda_test_harness.h /^ UpdateFn _update_fn;$/;" m class:doris_udf::UdaTestHarness4 +_update_fn_symbol src/exprs/agg_fn_evaluator.h /^ std::string _update_fn_symbol;$/;" m class:doris::AggFnEvaluator +_update_memory_metrics src/util/system_metrics.cpp /^void SystemMetrics::_update_memory_metrics() {$/;" f class:doris::SystemMetrics +_update_merge_ctx_and_build_merge_heap src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::_update_merge_ctx_and_build_merge_heap(AlphaMergeContext* merge_ctx) {$/;" f class:doris::AlphaRowsetReader +_update_net_metrics src/util/system_metrics.cpp /^void SystemMetrics::_update_net_metrics() {$/;" f class:doris::SystemMetrics +_update_period src/util/progress_updater.h /^ int _update_period;$/;" m class:doris::ProgressUpdater +_update_process_fd_num src/util/doris_metrics.cpp /^void DorisMetrics::_update_process_fd_num() {$/;" f class:doris::DorisMetrics +_update_process_thread_num src/util/doris_metrics.cpp /^void DorisMetrics::_update_process_thread_num() {$/;" f class:doris::DorisMetrics +_update_realtime_counter src/exec/olap_scanner.cpp /^void OlapScanner::_update_realtime_counter() {$/;" f class:doris::OlapScanner +_update_registers src/olap/hll.h /^ void _update_registers(uint64_t hash_value) {$/;" f class:doris::HyperLogLog +_update_storage_medium_type_count src/olap/storage_engine.cpp /^void StorageEngine::_update_storage_medium_type_count() {$/;" f class:doris::StorageEngine +_update_tablet_meta_info_workers src/agent/agent_server.h /^ std::unique_ptr _update_tablet_meta_info_workers;$/;" m class:doris::AgentServer +_update_tablet_meta_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_update_tablet_meta_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_upgrade_counter src/util/blocking_priority_queue.hpp /^ int _upgrade_counter;$/;" m class:doris::BlockingPriorityQueue +_upload_worker_thread_callback src/agent/task_worker_pool.cpp /^void* TaskWorkerPool::_upload_worker_thread_callback(void* arg_this) {$/;" f class:doris::TaskWorkerPool +_upload_workers src/agent/agent_server.h /^ std::unique_ptr _upload_workers;$/;" m class:doris::AgentServer +_uri src/http/http_request.h /^ std::string _uri;$/;" m class:doris::HttpRequest +_usage src/olap/lru_cache.h /^ size_t _usage;$/;" m class:doris::CachePriority::LRUCache +_use_aggregate src/exec/pre_aggregation_node.h /^ bool _use_aggregate;$/;" m class:doris::PreAggregationNode +_use_dictionary_encoding src/olap/rowset/column_writer.h /^ bool _use_dictionary_encoding;$/;" m class:doris::VarStringColumnWriter +_use_page_cache src/olap/rowset/segment_v2/indexed_column_reader.h /^ bool _use_page_cache;$/;" m class:doris::segment_v2::IndexedColumnReader +_use_pushdown_conjuncts src/exec/olap_scanner.h /^ bool _use_pushdown_conjuncts = false;$/;" m class:doris::OlapScanner +_use_small_buffers src/runtime/buffered_tuple_stream.h /^ bool _use_small_buffers;$/;" m class:doris::BufferedTupleStream +_use_small_buffers src/runtime/buffered_tuple_stream2.h /^ bool _use_small_buffers;$/;" m class:doris::BufferedTupleStream2 +_used src/olap/file_stream.h /^ size_t _used;$/;" m class:doris::ReadOnlyFileStream::FileCursor +_used src/olap/rowset/run_length_byte_reader.h /^ int32_t _used;$/;" m class:doris::RunLengthByteReader +_used src/olap/rowset/run_length_integer_reader.h /^ int32_t _used;$/;" m class:doris::RunLengthIntegerReader +_used_clients src/runtime/client_cache.h /^ std::unique_ptr _used_clients;$/;" m class:doris::ClientCacheHelper +_used_columns src/olap/rowset/segment_reader.h /^ std::vector _used_columns;$/;" m class:doris::SegmentReader +_user src/runtime/descriptors.h /^ std::string _user;$/;" m class:doris::MySQLTableDescriptor +_user src/runtime/fragment_mgr.cpp /^ std::string _user;$/;" m class:doris::FragmentExecState file: +_user src/runtime/runtime_state.h /^ std::string _user;$/;" m class:doris::RuntimeState +_user_mutex src/util/condition_variable.h /^ pthread_mutex_t* _user_mutex;$/;" m class:doris::ConditionVariable +_user_name src/exec/es/es_scan_reader.h /^ std::string _user_name;$/;" m class:doris::ESScanReader +_user_time src/util/runtime_profile.h /^ Counter* _user_time; \/\/ user CPU time$/;" m class:doris::RuntimeProfile::ThreadCounters +_uuid_gen_lock src/util/uuid_generator.h /^ SpinLock _uuid_gen_lock;$/;" m class:doris::UUIDGenerator +_val src/util/runtime_profile.h /^ int64_t _val;$/;" m class:doris::ScopedCounter +_valid src/olap/generic_iterators.cpp /^ bool _valid = false;$/;" m class:doris::MergeIteratorContext file: +_valid_data_len src/runtime/buffered_block_mgr.h /^ int64_t _valid_data_len;$/;" m class:doris::BufferedBlockMgr::Block +_valid_data_len src/runtime/buffered_block_mgr2.h /^ int64_t _valid_data_len;$/;" m class:doris::BufferedBlockMgr2::Block +_valid_rowset_ids src/olap/rowset/unique_rowset_id_generator.h /^ std::set _valid_rowset_ids; $/;" m class:doris::UniqueRowsetIdGenerator +_validate_alter_result src/olap/schema_change.cpp /^OLAPStatus SchemaChangeHandler::_validate_alter_result(TabletSharedPtr new_tablet, const TAlterTabletReqV2& request) {$/;" f class:doris::SchemaChangeHandler +_validate_data src/exec/tablet_sink.cpp /^int OlapTableSink::_validate_data(RuntimeState* state, RowBatch* batch, Bitmap* filter_bitmap) {$/;" f class:doris::stream_load::OlapTableSink +_validate_data_ns src/exec/tablet_sink.h /^ int64_t _validate_data_ns = 0;$/;" m class:doris::stream_load::OlapTableSink +_validate_data_timer src/exec/tablet_sink.h /^ RuntimeProfile::Counter* _validate_data_timer = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_validate_options src/olap/storage_engine.cpp /^static Status _validate_options(const EngineOptions& options) {$/;" f namespace:doris +_validate_rowset src/olap/rowset/alpha_rowset_writer.cpp /^bool AlphaRowsetWriter::_validate_rowset() {$/;" f class:doris::AlphaRowsetWriter +_validx_key_coder src/olap/rowset/segment_v2/indexed_column_reader.h /^ const KeyCoder* _validx_key_coder = nullptr;$/;" m class:doris::segment_v2::IndexedColumnReader +_validx_key_coder src/olap/rowset/segment_v2/indexed_column_writer.h /^ const KeyCoder* _validx_key_coder;$/;" m class:doris::segment_v2::IndexedColumnWriter +_value src/common/atomic.h /^ T _value;$/;" m class:doris::AtomicInt +_value src/exec/es/es_predicate.h /^ void* _value;$/;" m class:doris::ExtLiteral +_value src/exec/es/es_query_builder.h /^ std::string _value;$/;" m class:doris::RangeQueryBuilder +_value src/exprs/hybird_set.h /^ StringValue _value;$/;" m class:doris::StringValueSet::Iterator +_value src/exprs/literal.h /^ ExprValue _value;$/;" m class:doris::Literal +_value src/runtime/decimalv2_value.h /^ int128_t _value;$/;" m class:doris::DecimalV2Value +_value src/util/metrics.h /^ CoreLocalValue _value;$/;" m class:doris::CoreLocalCounter +_value src/util/metrics.h /^ T _value;$/;" m class:doris::LockSimpleMetric +_value src/util/path_trie.hpp /^ T* _value;$/;" m class:doris::PathTrie::TrieNode +_value src/util/runtime_profile.h /^ AtomicInt64 _value;$/;" m class:doris::RuntimeProfile::Counter +_value_cids src/olap/reader.h /^ std::vector _value_cids;$/;" m class:doris::Reader +_value_ctxs src/runtime/dpp_sink_internal.h /^ std::vector _value_ctxs;$/;" m class:doris::RollupSchema +_value_index_builder src/olap/rowset/segment_v2/indexed_column_writer.h /^ std::unique_ptr _value_index_builder;$/;" m class:doris::segment_v2::IndexedColumnWriter +_value_index_page_handle src/olap/rowset/segment_v2/indexed_column_reader.h /^ PageHandle _value_index_page_handle;$/;" m class:doris::segment_v2::IndexedColumnReader +_value_index_reader src/olap/rowset/segment_v2/indexed_column_reader.h /^ IndexPageReader _value_index_reader;$/;" m class:doris::segment_v2::IndexedColumnReader +_value_iter src/olap/rowset/segment_v2/indexed_column_reader.h /^ IndexPageIterator _value_iter;$/;" m class:doris::segment_v2::IndexedColumnIterator +_value_ops src/runtime/dpp_sink_internal.h /^ std::vector _value_ops;$/;" m class:doris::RollupSchema +_value_present src/olap/rowset/column_reader.h /^ bool _value_present;$/;" m class:doris::ColumnReader +_value_seek_encoding_map src/olap/rowset/segment_v2/encoding_info.cpp /^ std::unordered_map> _value_seek_encoding_map;$/;" m class:doris::segment_v2::EncodingInfoResolver file: +_value_separator src/exec/broker_scanner.h /^ char _value_separator;$/;" m class:doris::BrokerScanner +_value_updaters src/runtime/dpp_sink.cpp /^ std::vector _value_updaters;$/;" m class:doris::Translator file: +_values src/exec/es/es_query_builder.h /^ std::vector _values;$/;" m class:doris::TermsInSetQueryBuilder +_values src/olap/rowset/column_reader.h /^ FLOAT_TYPE* _values;$/;" m class:doris::FloatintPointColumnReader +_values src/olap/rowset/column_reader.h /^ Slice* _values;$/;" m class:doris::StringColumnDictionaryReader +_values src/olap/rowset/column_reader.h /^ Slice* _values;$/;" m class:doris::StringColumnDirectReader +_values src/olap/rowset/column_reader.h /^ T* _values;$/;" m class:doris::IntegerColumnReaderWrapper +_values src/olap/rowset/column_reader.h /^ char* _values;$/;" m class:doris::TinyColumnReader +_values src/olap/rowset/column_reader.h /^ decimal12_t* _values;$/;" m class:doris::DecimalColumnReader +_values src/olap/rowset/column_reader.h /^ int128_t* _values;$/;" m class:doris::LargeIntColumnReader +_values src/olap/rowset/column_reader.h /^ void* _values;$/;" m class:doris::DefaultValueReader +_values src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ ValueDict _values;$/;" m class:doris::segment_v2::__anon52::BloomFilterIndexWriterImpl file: +_values src/olap/rowset/segment_v2/index_page.h /^ std::vector _values;$/;" m class:doris::segment_v2::IndexPageReader +_values src/olap/rowset/segment_v2/zone_map_index.h /^ std::vector _values;$/;" m class:doris::segment_v2::ZoneMapIndexWriter +_values src/olap/tuple.h /^ std::vector _values;$/;" m class:doris::OlapTuple +_values src/util/core_local.h /^ std::vector _values;$/;" m class:doris::CoreLocalValue +_values_num src/util/frame_of_reference_coding.h /^ uint32_t _values_num = 0;$/;" m class:doris::ForDecoder +_values_num src/util/frame_of_reference_coding.h /^ uint32_t _values_num = 0;$/;" m class:doris::ForEncoder +_var_len_blocks src/runtime/merge_sorter.cpp /^ std::vector _var_len_blocks;$/;" m class:doris::MergeSorter::Run file: +_var_len_blocks src/runtime/spill_sorter.cc /^ vector _var_len_blocks;$/;" m class:doris::SpillSorter::Run file: +_var_len_blocks_index src/runtime/spill_sorter.cc /^ int _var_len_blocks_index;$/;" m class:doris::SpillSorter::Run file: +_var_len_copy_block src/runtime/spill_sorter.cc /^ BufferedBlockMgr2::Block* _var_len_copy_block;$/;" m class:doris::SpillSorter::Run file: +_var_length src/olap/wrapper_field.h /^ size_t _var_length;$/;" m class:doris::WrapperField +_var_result src/exec/schema_scanner/schema_variables_scanner.h /^ TShowVariableResult _var_result;$/;" m class:doris::SchemaVariablesScanner +_var_result_begin src/exec/hash_table.h /^ int _var_result_begin;$/;" m class:doris::HashTable +_var_result_begin src/exec/partitioned_hash_table.h /^ int _var_result_begin;$/;" m class:doris::PartitionedHashTableCtx +_var_run_length src/olap/rowset/run_length_integer_writer.h /^ uint32_t _var_run_length;$/;" m class:doris::RunLengthIntegerWriter +_vararg_start_idx src/exprs/agg_fn.h /^ int _vararg_start_idx;$/;" m class:doris::AggFn +_vararg_start_idx src/exprs/scalar_fn_call.h /^ int _vararg_start_idx;$/;" m class:doris::ScalarFnCall +_varargs_buffer src/udf/udf_internal.h /^ uint8_t* _varargs_buffer;$/;" m class:doris::FunctionContextImpl +_varargs_buffer_size src/udf/udf_internal.h /^ int _varargs_buffer_size;$/;" m class:doris::FunctionContextImpl +_variable_buf src/olap/row_cursor.h /^ char* _variable_buf = nullptr;$/;" m class:doris::RowCursor +_variable_len src/olap/row_cursor.h /^ size_t _variable_len;$/;" m class:doris::RowCursor +_vec_cond_timer src/exec/olap_scan_node.h /^ RuntimeProfile::Counter* _vec_cond_timer = nullptr;$/;" m class:doris::OlapScanNode +_vector_compute_fn src/exprs/expr.h /^ VectorComputeFn _vector_compute_fn;$/;" m class:doris::Expr +_vectorized_batch test/olap/in_list_predicate_test.cpp /^ VectorizedRowBatch* _vectorized_batch;$/;" m class:doris::TestInListPredicate file: +_vectorized_batch test/olap/null_predicate_test.cpp /^ VectorizedRowBatch* _vectorized_batch;$/;" m class:doris::TestNullPredicate file: +_version src/exec/olap_scanner.h /^ int64_t _version;$/;" m class:doris::OlapScanner +_version src/exec/tablet_info.h /^ int64_t _version;$/;" m class:doris::OlapTableSchemaParam +_version src/olap/reader.h /^ Version _version;$/;" m class:doris::Reader +_version src/olap/rowset/segment_group.h /^ Version _version; \/\/ version of associated data file$/;" m class:doris::SegmentGroup +_version src/olap/rowset/unique_rowset_id_generator.h /^ const int64_t _version = 2; \/\/ modify it when create new version id generator$/;" m class:doris::UniqueRowsetIdGenerator +_version src/olap/task/engine_checksum_task.h /^ TVersion _version;$/;" m class:doris::EngineChecksumTask +_version src/udf/udf_internal.h /^ doris_udf::FunctionContext::DorisVersion _version;$/;" m class:doris::FunctionContextImpl +_version_graph src/olap/rowset_graph.h /^ std::vector _version_graph;$/;" m class:doris::RowsetGraph +_version_hash src/olap/rowset/segment_group.h /^ VersionHash _version_hash; \/\/ version hash for this segmentgroup$/;" m class:doris::SegmentGroup +_version_hash src/olap/task/engine_checksum_task.h /^ TVersionHash _version_hash;$/;" m class:doris::EngineChecksumTask +_version_locked src/olap/reader.h /^ bool _version_locked;$/;" m class:doris::Reader +_vertex_index_map src/olap/rowset_graph.h /^ std::unordered_map _vertex_index_map;$/;" m class:doris::RowsetGraph +_visit src/util/arrow/row_batch.cpp /^ _visit(const T& array) {$/;" f class:doris::ToRowBatchConverter file: +_visit src/util/arrow/row_batch.cpp /^ _visit(const T& type) {$/;" f class:doris::FromRowBatchConverter file: +_visit src/util/arrow/row_block.cpp /^ _visit(const T& array) {$/;" f class:doris::ToRowBlockConverter file: +_visit src/util/arrow/row_block.cpp /^ _visit(const T& type) {$/;" f class:doris::FromRowBlockConverter file: +_visit_simple_metric src/http/action/metrics_action.cpp /^void PrometheusMetricsVisitor::_visit_simple_metric($/;" f class:doris::PrometheusMetricsVisitor +_voluntary_context_switches src/util/runtime_profile.h /^ Counter* _voluntary_context_switches;$/;" m class:doris::RuntimeProfile::ThreadCounters +_wait_duration src/exec/olap_scan_node.h /^ boost::posix_time::time_duration _wait_duration;$/;" m class:doris::OlapScanNode +_wait_in_flight_packet src/exec/tablet_sink.cpp /^Status NodeChannel::_wait_in_flight_packet() {$/;" f class:doris::stream_load::NodeChannel +_wait_in_flight_packet_ns src/exec/tablet_sink.h /^ int64_t _wait_in_flight_packet_ns = 0;$/;" m class:doris::stream_load::OlapTableSink +_wait_in_flight_packet_timer src/exec/tablet_sink.h /^ RuntimeProfile::Counter* _wait_in_flight_packet_timer = nullptr;$/;" m class:doris::stream_load::OlapTableSink +_wait_last_brpc src/runtime/data_stream_sender.cpp /^ inline Status _wait_last_brpc() {$/;" f class:doris::DataStreamSender::Channel file: +_wait_scanner_timer src/exec/broker_scan_node.h /^ RuntimeProfile::Counter* _wait_scanner_timer;$/;" m class:doris::BrokerScanNode +_wait_scanner_timer src/exec/es_http_scan_node.h /^ RuntimeProfile::Counter* _wait_scanner_timer;$/;" m class:doris::EsHttpScanNode +_waiting_rpc src/runtime/buffer_control_block.h /^ std::deque _waiting_rpc;$/;" m class:doris::BufferControlBlock +_warn_after_ms src/util/thread.h /^ int _warn_after_ms;$/;" m class:doris::ThreadJoiner +_warn_every_ms src/util/thread.h /^ int _warn_every_ms;$/;" m class:doris::ThreadJoiner +_wblock src/olap/rowset/segment_v2/column_writer.h /^ fs::WritableBlock* _wblock = nullptr;$/;" m class:doris::segment_v2::ColumnWriter +_wblock src/olap/rowset/segment_v2/indexed_column_writer.h /^ fs::WritableBlock* _wblock;$/;" m class:doris::segment_v2::IndexedColumnWriter +_wblock src/olap/rowset/segment_v2/segment_writer.h /^ fs::WritableBlock* _wblock;$/;" m class:doris::segment_v2::SegmentWriter +_wblocks src/olap/rowset/beta_rowset_writer.h /^ std::vector> _wblocks;$/;" m class:doris::BetaRowsetWriter +_web_page_handler src/service/http_service.h /^ std::unique_ptr _web_page_handler;$/;" m class:doris::HttpService +_weight src/util/tdigest.h /^ Weight _weight = 0;$/;" m class:doris::Centroid +_wild test/exec/schema_scanner/schema_authors_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaAuthorScannerTest file: +_wild test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaCharsetsScannerTest file: +_wild test/exec/schema_scanner/schema_collations_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaCollationsScannerTest file: +_wild test/exec/schema_scanner/schema_columns_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaColumnsScannerTest file: +_wild test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaCreateTableScannerTest file: +_wild test/exec/schema_scanner/schema_engines_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaEnginesScannerTest file: +_wild test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaOpenTablesScannerTest file: +_wild test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaSchemataScannerTest file: +_wild test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaTableNamesScannerTest file: +_wild test/exec/schema_scanner/schema_tables_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaTablesScannerTest file: +_wild test/exec/schema_scanner/schema_variables_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaVariablesScannerTest file: +_wild test/exec/schema_scanner_test.cpp /^ std::string _wild;$/;" m class:doris::SchemaScannerTest file: +_wildcard src/util/path_trie.hpp /^ std::string _wildcard;$/;" m class:doris::PathTrie::TrieNode +_window src/exec/analytic_eval_node.h /^ const TAnalyticWindow _window;$/;" m class:doris::AnalyticEvalNode +_window_tuples src/exec/analytic_eval_node.h /^ std::list > _window_tuples;$/;" m class:doris::AnalyticEvalNode +_without_filter src/olap/rowset/segment_reader.h /^ bool _without_filter = false;$/;" m class:doris::SegmentReader +_work_func src/util/batch_process_thread_pool.hpp /^ BatchProcessFunction _work_func;$/;" m class:doris::BatchProcessThreadPool +_work_queue src/util/batch_process_thread_pool.hpp /^ BlockingPriorityQueue _work_queue;$/;" m class:doris::BatchProcessThreadPool +_work_queue src/util/priority_thread_pool.hpp /^ BlockingPriorityQueue _work_queue;$/;" m class:doris::PriorityThreadPool +_worker_count src/agent/task_worker_pool.h /^ uint32_t _worker_count;$/;" m class:doris::TaskWorkerPool +_worker_thread_condition_variable src/agent/task_worker_pool.h /^ ConditionVariable _worker_thread_condition_variable;$/;" m class:doris::TaskWorkerPool +_worker_thread_lock src/agent/task_worker_pool.h /^ Mutex _worker_thread_lock;$/;" m class:doris::TaskWorkerPool +_workers src/http/ev_http_server.h /^ std::vector _workers;$/;" m class:doris::EvHttpServer +_wr_length src/olap/file_helper.h /^ off_t _wr_length;$/;" m class:doris::FileHandler +_write_bitmap_index src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::_write_bitmap_index() {$/;" f class:doris::segment_v2::SegmentWriter +_write_block src/runtime/buffered_tuple_stream.h /^ BufferedBlockMgr::Block* _write_block;$/;" m class:doris::BufferedTupleStream +_write_block src/runtime/buffered_tuple_stream2.h /^ BufferedBlockMgr2::Block* _write_block;$/;" m class:doris::BufferedTupleStream2 +_write_bloom_filter_index src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::_write_bloom_filter_index() {$/;" f class:doris::segment_v2::SegmentWriter +_write_buf src/runtime/stream_load/stream_load_pipe.h /^ ByteBufferPtr _write_buf;$/;" m class:doris::StreamLoadPipe +_write_byte src/olap/rowset/bit_field_writer.cpp /^OLAPStatus BitFieldWriter::_write_byte() {$/;" f class:doris::BitFieldWriter +_write_bytes src/olap/push_handler.h /^ int64_t _write_bytes = 0;$/;" m class:doris::PushHandler +_write_cluster_id_to_path src/olap/data_dir.cpp /^Status DataDir::_write_cluster_id_to_path(const std::string& path, int32_t cluster_id) {$/;" f class:doris::DataDir +_write_complete_cv src/runtime/buffered_block_mgr2.h /^ boost::condition_variable _write_complete_cv;$/;" m class:doris::BufferedBlockMgr2::Block +_write_data src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::_write_data() {$/;" f class:doris::segment_v2::SegmentWriter +_write_data_page src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::_write_data_page(Page* page) {$/;" f class:doris::segment_v2::ColumnWriter +_write_delta_values src/olap/rowset/run_length_integer_writer.cpp /^OLAPStatus RunLengthIntegerWriter::_write_delta_values() {$/;" f class:doris::RunLengthIntegerWriter +_write_direct_values src/olap/rowset/run_length_integer_writer.cpp /^OLAPStatus RunLengthIntegerWriter::_write_direct_values() {$/;" f class:doris::RunLengthIntegerWriter +_write_footer src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::_write_footer() {$/;" f class:doris::segment_v2::SegmentWriter +_write_head src/olap/out_stream.cpp /^OLAPStatus OutStream::_write_head(StorageByteBuffer* buf,$/;" f class:doris::OutStream +_write_len src/runtime/dpp_writer.h /^ int64_t _write_len;$/;" m class:doris::DppWriter +_write_mbytes_per_sec src/olap/rowset/segment_writer.h /^ uint32_t _write_mbytes_per_sec;$/;" m class:doris::SegmentWriter +_write_ordinal_index src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::_write_ordinal_index() {$/;" f class:doris::segment_v2::SegmentWriter +_write_patched_base_values src/olap/rowset/run_length_integer_writer.cpp /^OLAPStatus RunLengthIntegerWriter::_write_patched_base_values() {$/;" f class:doris::RunLengthIntegerWriter +_write_range src/runtime/buffered_block_mgr2.h /^ DiskIoMgr::WriteRange* _write_range;$/;" m class:doris::BufferedBlockMgr2::Block +_write_raw_data src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::_write_raw_data(const std::vector& slices) {$/;" f class:doris::segment_v2::SegmentWriter +_write_row_block src/olap/schema_change.cpp /^bool SchemaChangeDirectly::_write_row_block(RowsetWriter* rowset_writer, RowBlock* row_block) {$/;" f class:doris::SchemaChangeDirectly +_write_rows src/olap/push_handler.h /^ int64_t _write_rows = 0;$/;" m class:doris::PushHandler +_write_short_key_index src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::_write_short_key_index() {$/;" f class:doris::segment_v2::SegmentWriter +_write_short_repeat_values src/olap/rowset/run_length_integer_writer.cpp /^OLAPStatus RunLengthIntegerWriter::_write_short_repeat_values() {$/;" f class:doris::RunLengthIntegerWriter +_write_timer src/runtime/export_sink.h /^ RuntimeProfile::Counter* _write_timer;$/;" m class:doris::ExportSink +_write_tuple_idx src/runtime/buffered_tuple_stream.h /^ uint32_t _write_tuple_idx;$/;" m class:doris::BufferedTupleStream +_write_tuple_idx src/runtime/buffered_tuple_stream2.h /^ uint32_t _write_tuple_idx;$/;" m class:doris::BufferedTupleStream2 +_write_values src/olap/rowset/run_length_byte_writer.cpp /^OLAPStatus RunLengthByteWriter::_write_values() {$/;" f class:doris::RunLengthByteWriter +_write_values src/olap/rowset/run_length_integer_writer.cpp /^OLAPStatus RunLengthIntegerWriter::_write_values() {$/;" f class:doris::RunLengthIntegerWriter +_write_zone_map src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::_write_zone_map() {$/;" f class:doris::segment_v2::SegmentWriter +_writer src/olap/fs/file_block_manager.cpp /^ shared_ptr _writer;$/;" m class:doris::fs::internal::FileWritableBlock file: +_writer src/olap/rowset/column_writer.h /^ IntegerColumnWriter _writer;$/;" m class:doris::IntegerColumnWriterWrapper +_writer src/olap/rowset/column_writer.h /^ RunLengthByteWriter* _writer;$/;" m class:doris::ByteColumnWriter +_writer src/olap/rowset/column_writer.h /^ RunLengthIntegerWriter* _writer;$/;" m class:doris::IntegerColumnWriter +_writer src/runtime/dpp_sink.cpp /^ DppWriter* _writer;$/;" m class:doris::Translator file: +_writer src/runtime/mysql_table_sink.h /^ MysqlTableWriter* _writer;$/;" m class:doris::MysqlTableSink +_writer src/runtime/result_sink.h /^ boost::shared_ptr _writer;$/;" m class:doris::ResultSink +_writer test/olap/bit_field_test.cpp /^ BitFieldWriter* _writer;$/;" m class:doris::TestBitField file: +_writer test/olap/run_length_byte_test.cpp /^ RunLengthByteWriter* _writer;$/;" m class:doris::TestRunLengthByte file: +_writer test/olap/run_length_integer_test.cpp /^ RunLengthIntegerWriter* _writer;$/;" m class:doris::TestRunLengthSignInteger file: +_writer test/olap/run_length_integer_test.cpp /^ RunLengthIntegerWriter* _writer;$/;" m class:doris::TestRunLengthUnsignInteger file: +_writer_state src/olap/rowset/alpha_rowset_writer.h /^ WriterState _writer_state;$/;" m class:doris::AlphaRowsetWriter +_writer_timer src/runtime/dpp_sink.cpp /^ RuntimeProfile::Counter* _writer_timer;$/;" m class:doris::Translator file: +_writes_done test/runtime/disk_io_mgr_test.cpp /^ condition_variable _writes_done;$/;" m class:doris::DiskIoMgrTest file: +_writes_issued src/runtime/buffered_block_mgr2.h /^ int _writes_issued;$/;" m class:doris::BufferedBlockMgr2 +_written_mutex test/runtime/disk_io_mgr_test.cpp /^ mutex _written_mutex;$/;" m class:doris::DiskIoMgrTest file: +_year src/runtime/datetime_value.h /^ uint16_t _year;$/;" m class:doris::DateTimeValue +_z_strm src/exec/decompressor.h /^ z_stream _z_strm;$/;" m class:doris::GzipDecompressor +_zig_zag_literals src/olap/rowset/run_length_integer_writer.h /^ int64_t _zig_zag_literals[MAX_SCOPE]; \/\/ for direct encoding$/;" m class:doris::RunLengthIntegerWriter +_zip_file src/util/zip_util.h /^ unzFile _zip_file;$/;" m class:doris::ZipFile +_zip_path src/util/zip_util.h /^ std::string _zip_path;$/;" m class:doris::ZipFile +_zone_map_index src/olap/rowset/segment_v2/column_reader.h /^ std::unique_ptr _zone_map_index;$/;" m class:doris::segment_v2::ColumnReader +_zone_map_index_builder src/olap/rowset/segment_v2/column_writer.h /^ std::unique_ptr _zone_map_index_builder;$/;" m class:doris::segment_v2::ColumnWriter +_zone_map_index_meta src/olap/rowset/segment_v2/column_reader.h /^ const ZoneMapIndexPB* _zone_map_index_meta = nullptr;$/;" m class:doris::segment_v2::ColumnReader +_zone_map_match_condition src/olap/rowset/segment_v2/column_reader.cpp /^bool ColumnReader::_zone_map_match_condition(const ZoneMapPB& zone_map,$/;" f class:doris::segment_v2::ColumnReader +_zone_maps src/olap/rowset/column_data_writer.h /^ std::vector> _zone_maps;$/;" m class:doris::ColumnDataWriter +_zone_maps src/olap/rowset/segment_group.h /^ std::vector> _zone_maps;$/;" m class:doris::SegmentGroup +_zz_bits_100p src/olap/rowset/run_length_integer_writer.h /^ uint32_t _zz_bits_100p;$/;" m class:doris::RunLengthIntegerWriter +_zz_bits_90p src/olap/rowset/run_length_integer_writer.h /^ uint32_t _zz_bits_90p;$/;" m class:doris::RunLengthIntegerWriter +a2b_bin src/gutil/strings/escaping.cc /^string a2b_bin(const string& a, bool byte_order_msb) {$/;" f namespace:strings +a2b_hex src/gutil/strings/escaping.cc /^string a2b_hex(const string& a) {$/;" f namespace:strings +a2b_hex src/gutil/strings/escaping.cc /^void a2b_hex(const char* a, char* b, int num) {$/;" f namespace:strings +a2b_hex src/gutil/strings/escaping.cc /^void a2b_hex(const char* a, unsigned char* b, int num) {$/;" f namespace:strings +a2b_hex src/gutil/strings/escaping.cc /^void a2b_hex(const char* from, string* to, int num) {$/;" f namespace:strings +a2b_hex_t src/gutil/strings/escaping.cc /^static void a2b_hex_t(const char* a, T b, int num) {$/;" f namespace:strings +abort src/olap/fs/file_block_manager.cpp /^Status FileWritableBlock::abort() {$/;" f class:doris::fs::internal::FileWritableBlock +abort_on_default_limit_exceeded src/runtime/runtime_state.h /^ bool abort_on_default_limit_exceeded() const {$/;" f class:doris::RuntimeState +abort_on_error src/runtime/runtime_state.h /^ bool abort_on_error() const {$/;" f class:doris::RuntimeState +abort_tarns src/runtime/mysql_table_writer.h /^ Status abort_tarns() {$/;" f class:doris::MysqlTableWriter +abs src/exprs/udf_builtins.cpp /^DoubleVal UdfBuiltins::abs(FunctionContext* context, const DoubleVal& v) {$/;" f class:doris::UdfBuiltins +abs src/runtime/decimalv2_value.cpp /^static inline int128_t abs(const int128_t& x) { return (x < 0) ? -x : x; }$/;" f namespace:doris +acceptable_terminators src/gutil/strings/numbers.h /^ const char* acceptable_terminators;$/;" m struct:DoubleRangeOptions +access src/util/core_local.h /^ inline T* access() const {$/;" f class:doris::CoreLocalValue +access_at_core src/util/core_local.h /^ inline T* access_at_core(size_t core_idx) const {$/;" f class:doris::CoreLocalValue +acquire src/olap/rowset/segment_group.cpp /^void SegmentGroup::acquire() {$/;" f class:doris::SegmentGroup +acquire_data src/common/object_pool.h /^ void acquire_data(ObjectPool* src) {$/;" f class:doris::ObjectPool +acquire_data src/runtime/mem_pool.cpp /^void MemPool::acquire_data(MemPool* src, bool keep_current) {$/;" f class:doris::MemPool +acquire_inc_rs_meta_by_version src/olap/tablet_meta.cpp /^RowsetMetaSharedPtr TabletMeta::acquire_inc_rs_meta_by_version(const Version& version) const {$/;" f class:doris::TabletMeta +acquire_rs_meta_by_version src/olap/tablet_meta.cpp /^RowsetMetaSharedPtr TabletMeta::acquire_rs_meta_by_version(const Version& version) const {$/;" f class:doris::TabletMeta +acquire_state src/runtime/row_batch.cpp /^void RowBatch::acquire_state(RowBatch* src) {$/;" f class:doris::RowBatch +acquire_thread_token src/runtime/thread_resource_mgr.h /^inline void ThreadResourceMgr::ResourcePool::acquire_thread_token() {$/;" f class:doris::ThreadResourceMgr::ResourcePool +active_tmp_devices src/runtime/tmp_file_mgr.cc /^vector TmpFileMgr::active_tmp_devices() {$/;" f class:doris::TmpFileMgr +actual_consume src/runtime/routine_load/data_consumer_group.cpp /^void KafkaDataConsumerGroup::actual_consume($/;" f class:doris::KafkaDataConsumerGroup +add src/common/atomic.h /^ ALWAYS_INLINE T add(T x) {$/;" f class:doris::AtomicInt +add src/common/object_pool.h /^ T* add(T* t) {$/;" f class:doris::ObjectPool +add src/exprs/agg_fn_evaluator.h /^inline void AggFnEvaluator::add($/;" f class:doris::AggFnEvaluator +add src/exprs/agg_fn_evaluator.h /^inline void AggFnEvaluator::add(const std::vector& evaluators,$/;" f class:doris::AggFnEvaluator +add src/geo/wkt_parse_type.h /^ void add(GeoCoordinateList* coordinates) {$/;" f struct:doris::GeoCoordinateListList +add src/geo/wkt_parse_type.h /^ void add(const GeoCoordinate& coordinate) {$/;" f struct:doris::GeoCoordinateList +add src/olap/rowset/segment_v2/binary_dict_page.cpp /^Status BinaryDictPageBuilder::add(const uint8_t* vals, size_t* count) {$/;" f class:doris::segment_v2::BinaryDictPageBuilder +add src/olap/rowset/segment_v2/binary_prefix_page.cpp /^Status BinaryPrefixPageBuilder::add(const uint8_t* vals, size_t* add_count) {$/;" f class:doris::segment_v2::BinaryPrefixPageBuilder +add src/olap/rowset/segment_v2/index_page.cpp /^void IndexPageBuilder::add(const Slice& key, const PagePointer& ptr) {$/;" f class:doris::segment_v2::IndexPageBuilder +add src/olap/rowset/segment_v2/indexed_column_writer.cpp /^Status IndexedColumnWriter::add(const void* value) {$/;" f class:doris::segment_v2::IndexedColumnWriter +add src/olap/rowset/segment_v2/row_ranges.h /^ void add(const RowRange& range) {$/;" f class:doris::segment_v2::RowRanges +add src/olap/stream_index_common.h /^ inline void add(const CellType& cell) {$/;" f class:doris::ColumnStatistics +add src/runtime/decimal_value.cpp /^inline void add(const int32_t value1, const int32_t value2, int32_t* to, int32_t* carry) {$/;" f namespace:doris +add src/runtime/free_list.hpp /^ void add(uint8_t* memory, int size) {$/;" f class:doris::FreeList +add src/util/bitmap_value.h /^ void add(uint32_t x) {$/;" f class:doris::detail::Roaring64Map +add src/util/bitmap_value.h /^ void add(uint64_t value) {$/;" f class:doris::BitmapValue +add src/util/bitmap_value.h /^ void add(uint64_t x) {$/;" f class:doris::detail::Roaring64Map +add src/util/metrics.h /^ MetricLabels& add(const std::string& name, const std::string& value) {$/;" f struct:doris::MetricLabels +add src/util/runtime_profile.h /^ virtual void add(int64_t delta) {$/;" f class:doris::RuntimeProfile::HighWaterMarkCounter +add src/util/tdigest.h /^ inline bool add(Value x, Weight w) {$/;" f class:doris::TDigest +add src/util/tdigest.h /^ inline void add(const Centroid &c) {$/;" f class:doris::Centroid +add src/util/tdigest.h /^ inline void add(std::vector::const_iterator iter, std::vector::const_iterator end) {$/;" f class:doris::TDigest +add src/util/tdigest.h /^ inline void add(std::vector digests) { add(digests.cbegin(), digests.cend()); }$/;" f class:doris::TDigest +add src/util/tdigest.h /^ void add(Value x) { add(x, 1); }$/;" f class:doris::TDigest +add src/util/tdigest.h /^ void add(std::vector::const_iterator iter, std::vector::const_iterator end) {$/;" f class:doris::TDigest +add2 src/runtime/decimal_value.cpp /^inline void add2(const int32_t value1, const int32_t value2, int32_t* to, int32_t* carry) {$/;" f namespace:doris +addChecked src/util/bitmap_value.h /^ bool addChecked(uint32_t x) {$/;" f class:doris::detail::Roaring64Map +addChecked src/util/bitmap_value.h /^ bool addChecked(uint64_t x) {$/;" f class:doris::detail::Roaring64Map +addMany src/util/bitmap_value.h /^ void addMany(size_t n_args, const uint32_t* vals) {$/;" f class:doris::detail::Roaring64Map +addMany src/util/bitmap_value.h /^ void addMany(size_t n_args, const uint64_t* vals) {$/;" f class:doris::detail::Roaring64Map +add_aggregate_mapping src/olap/aggregate_func.cpp /^ void add_aggregate_mapping() {$/;" f class:doris::AggregateFuncResolver +add_alter_task src/olap/tablet.cpp /^OLAPStatus Tablet::add_alter_task(int64_t related_tablet_id,$/;" f class:doris::Tablet +add_alter_task src/olap/tablet_meta.cpp /^void TabletMeta::add_alter_task(const AlterTabletTask& alter_task) {$/;" f class:doris::TabletMeta +add_batch src/runtime/buffer_control_block.cpp /^Status BufferControlBlock::add_batch(TFetchDataResult* result) {$/;" f class:doris::BufferControlBlock +add_batch src/runtime/data_stream_recvr.cc /^void DataStreamRecvr::SenderQueue::add_batch($/;" f class:doris::DataStreamRecvr::SenderQueue +add_batch src/runtime/data_stream_recvr.cc /^void DataStreamRecvr::add_batch($/;" f class:doris::DataStreamRecvr +add_batch src/runtime/dpp_sink.cpp /^Status DppSink::add_batch($/;" f class:doris::DppSink +add_batch src/runtime/dpp_sink.cpp /^Status Translator::add_batch(RowBatch* batch) {$/;" f class:doris::Translator +add_batch src/runtime/dpp_writer.cpp /^Status DppWriter::add_batch(RowBatch* batch) {$/;" f class:doris::DppWriter +add_batch src/runtime/load_channel.cpp /^Status LoadChannel::add_batch($/;" f class:doris::LoadChannel +add_batch src/runtime/load_channel_mgr.cpp /^Status LoadChannelMgr::add_batch($/;" f class:doris::LoadChannelMgr +add_batch src/runtime/merge_sorter.cpp /^Status MergeSorter::Run::add_batch(RowBatch* batch, int start_index, int* num_processed) {$/;" f class:doris::MergeSorter::Run +add_batch src/runtime/merge_sorter.cpp /^Status MergeSorter::add_batch(RowBatch* batch) {$/;" f class:doris::MergeSorter +add_batch src/runtime/qsorter.cpp /^Status QSorter::add_batch(RowBatch* batch) {$/;" f class:doris::QSorter +add_batch src/runtime/sorter.h /^ virtual Status add_batch(RowBatch* batch) {$/;" f class:doris::Sorter +add_batch src/runtime/spill_sorter.cc /^Status SpillSorter::Run::add_batch(RowBatch* batch, int start_index, int* num_processed) {$/;" f class:doris::SpillSorter::Run +add_batch src/runtime/spill_sorter.cc /^Status SpillSorter::add_batch(RowBatch* batch) {$/;" f class:doris::SpillSorter +add_batch src/runtime/tablets_channel.cpp /^Status TabletsChannel::add_batch(const PTabletWriterAddBatchRequest& params) {$/;" f class:doris::TabletsChannel +add_batch_execution_time_ns src/exec/tablet_sink.h /^ int64_t add_batch_execution_time_ns = 0;$/;" m struct:doris::stream_load::AddBatchCounter +add_batch_num src/exec/tablet_sink.h /^ int64_t add_batch_num = 0;$/;" m struct:doris::stream_load::AddBatchCounter +add_batch_wait_lock_time_ns src/exec/tablet_sink.h /^ int64_t add_batch_wait_lock_time_ns = 0;$/;" m struct:doris::stream_load::AddBatchCounter +add_block src/runtime/row_batch.cpp /^void RowBatch::add_block(BufferedBlockMgr2::Block* block) {$/;" f class:doris::RowBatch +add_bloom_filter src/olap/bloom_filter_writer.cpp /^OLAPStatus BloomFilterIndexWriter::add_bloom_filter(BloomFilter* bf) {$/;" f class:doris::BloomFilterIndexWriter +add_bucketing_counters src/util/runtime_profile.cpp /^void RuntimeProfile::add_bucketing_counters(const std::string& name,$/;" f class:doris::RuntimeProfile +add_buffer src/runtime/row_batch.cpp /^void RowBatch::add_buffer(BufferPool::ClientHandle* client,$/;" f class:doris::RowBatch +add_bytes src/olap/bloom_filter.hpp /^ void add_bytes(const char* buf, uint32_t len) {$/;" f class:doris::BloomFilter +add_bytes src/olap/rowset/segment_v2/bloom_filter.h /^ void add_bytes(char* buf, uint32_t size) {$/;" f class:doris::segment_v2::BloomFilter +add_callback src/http/web_page_handler.h /^ void add_callback(const PageHandlerCallback& callback) {$/;" f class:doris::WebPageHandler::PageHandlers +add_child src/exprs/expr.h /^ void add_child(Expr* expr) {$/;" f class:doris::Expr +add_child src/olap/reader.cpp /^OLAPStatus CollectIterator::add_child(RowsetReaderSharedPtr rs_reader) {$/;" f class:doris::CollectIterator +add_child src/util/runtime_profile.cpp /^void RuntimeProfile::add_child(RuntimeProfile* child, bool indent, RuntimeProfile* loc) {$/;" f class:doris::RuntimeProfile +add_child_tracker src/runtime/mem_tracker.h /^ void add_child_tracker(MemTracker* tracker) {$/;" f class:doris::MemTracker +add_child_unlock src/util/runtime_profile.cpp /^void RuntimeProfile::add_child_unlock(RuntimeProfile* child, bool indent, RuntimeProfile* loc) {$/;" f class:doris::RuntimeProfile +add_cond src/olap/olap_cond.cpp /^OLAPStatus CondColumn::add_cond(const TCondition& tcond, const TabletColumn& column) {$/;" f class:doris::CondColumn +add_consumer src/runtime/routine_load/data_consumer_group.h /^ void add_consumer(std::shared_ptr consumer) {$/;" f class:doris::DataConsumerGroup +add_counter src/util/perf_counters.cpp /^bool PerfCounters::add_counter(Counter counter) {$/;" f class:doris::PerfCounters +add_counter src/util/runtime_profile.cpp /^RuntimeProfile::Counter* RuntimeProfile::add_counter($/;" f class:doris::RuntimeProfile +add_counter src/util/runtime_profile.h /^ Counter* add_counter(const std::string& name, TUnit::type type) {$/;" f class:doris::RuntimeProfile +add_default_counters src/util/perf_counters.cpp /^bool PerfCounters::add_default_counters() {$/;" f class:doris::PerfCounters +add_default_path_handlers src/http/default_path_handlers.cpp /^void add_default_path_handlers(WebPageHandler* web_page_handler, MemTracker* process_mem_tracker) {$/;" f namespace:doris +add_delete_predicate src/olap/tablet.cpp /^void Tablet::add_delete_predicate($/;" f class:doris::Tablet +add_delete_predicate src/olap/tablet_meta.cpp /^void TabletMeta::add_delete_predicate(const DeletePredicatePB& delete_predicate, int64_t version) {$/;" f class:doris::TabletMeta +add_derived_counter src/util/runtime_profile.cpp /^RuntimeProfile::DerivedCounter* RuntimeProfile::add_derived_counter($/;" f class:doris::RuntimeProfile +add_event_sequence src/util/runtime_profile.cpp /^RuntimeProfile::EventSequence* RuntimeProfile::add_event_sequence(const std::string& name) {$/;" f class:doris::RuntimeProfile +add_export_output_file src/runtime/runtime_state.h /^ void add_export_output_file(const std::string& file) {$/;" f class:doris::RuntimeState +add_filtered_rows src/olap/schema_change.h /^ void add_filtered_rows(uint64_t filtered_rows) {$/;" f class:doris::SchemaChange +add_fixed_value src/exec/olap_common.h /^Status ColumnValueRange::add_fixed_value(T value) {$/;" f class:doris::ColumnValueRange +add_hash src/olap/bloom_filter.hpp /^ void add_hash(uint64_t hash) {$/;" f class:doris::BloomFilter +add_hash src/olap/rowset/segment_v2/block_split_bloom_filter.cpp /^void BlockSplitBloomFilter::add_hash(uint64_t hash) {$/;" f class:doris::segment_v2::BlockSplitBloomFilter +add_header src/http/http_response.cpp /^void HttpResponse::add_header(const std::string& key, const std::string& value) {$/;" f class:doris::HttpResponse +add_inc_rowset src/olap/tablet.cpp /^OLAPStatus Tablet::add_inc_rowset(const RowsetSharedPtr& rowset) {$/;" f class:doris::Tablet +add_inc_rs_meta src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::add_inc_rs_meta(const RowsetMetaSharedPtr& rs_meta) {$/;" f class:doris::TabletMeta +add_index_entry src/olap/stream_index_writer.cpp /^OLAPStatus StreamIndexWriter::add_index_entry(const PositionEntryWriter& entry) {$/;" f class:doris::StreamIndexWriter +add_info_string src/util/runtime_profile.cpp /^void RuntimeProfile::add_info_string(const std::string& key, const std::string& value) {$/;" f class:doris::RuntimeProfile +add_intermediate_tuple src/exec/partitioned_aggregation_node_ir.cc /^Status PartitionedAggregationNode::add_intermediate_tuple($/;" f class:doris::PartitionedAggregationNode +add_io_buffer src/runtime/row_batch.cpp /^void RowBatch::add_io_buffer(DiskIoMgr::BufferDescriptor* buffer) {$/;" f class:doris::RowBatch +add_item src/olap/short_key_index.cpp /^Status ShortKeyIndexBuilder::add_item(const Slice& key) {$/;" f class:doris::ShortKeyIndexBuilder +add_key src/exprs/bitmap_function.cpp /^ void add_key(const T key) {$/;" f struct:doris::BitmapIntersect +add_mapping src/olap/key_coder.cpp /^ void add_mapping() {$/;" f class:doris::KeyCoderResolver file: +add_mapping src/olap/types.cpp /^ template void add_mapping() {$/;" f class:doris::TypeInfoResolver file: +add_merged_rows src/olap/schema_change.h /^ void add_merged_rows(uint64_t merged_rows) {$/;" f class:doris::SchemaChange +add_metic src/util/metrics.cpp /^bool MetricCollector::add_metic(const MetricLabels& labels, Metric* metric) {$/;" f class:doris::MetricCollector +add_null src/olap/tuple.h /^ void add_null() {$/;" f class:doris::OlapTuple +add_nulls src/olap/rowset/segment_v2/zone_map_index.h /^ void add_nulls(uint32_t count) {$/;" f class:doris::segment_v2::ZoneMapIndexWriter +add_one_batch src/exec/olap_scan_node.cpp /^Status OlapScanNode::add_one_batch(RowBatchInterface* row_batch) {$/;" f class:doris::OlapScanNode +add_one_row src/runtime/result_writer.cpp /^Status ResultWriter::add_one_row(TupleRow* row) {$/;" f class:doris::ResultWriter +add_output_header src/http/http_request.cpp /^void HttpRequest::add_output_header(const char* key, const char* value) {$/;" f class:doris::HttpRequest +add_pending_ids src/olap/data_dir.cpp /^void DataDir::add_pending_ids(const std::string& id) {$/;" f class:doris::DataDir +add_position src/olap/stream_index_writer.cpp /^OLAPStatus PositionEntryWriter::add_position(uint32_t position) {$/;" f class:doris::PositionEntryWriter +add_range src/exec/olap_common.h /^Status ColumnValueRange::add_range(SQLFilterOp op, T value) {$/;" f class:doris::ColumnValueRange +add_rate_counter src/util/runtime_profile.cpp /^RuntimeProfile::Counter* RuntimeProfile::add_rate_counter($/;" f class:doris::RuntimeProfile +add_reference src/gutil/type_traits.h /^template struct add_reference { typedef T& type; };$/;" s namespace:base +add_reference src/gutil/type_traits.h /^template struct add_reference { typedef T& type; };$/;" s namespace:base +add_request_range src/runtime/disk_io_mgr_reader_context.cc /^void DiskIoMgr::RequestContext::add_request_range($/;" f class:doris::DiskIoMgr::RequestContext +add_result_tuple src/exec/analytic_eval_node.cpp /^void AnalyticEvalNode::add_result_tuple(int64_t stream_idx) {$/;" f class:doris::AnalyticEvalNode +add_row src/exec/tablet_sink.cpp /^Status IndexChannel::add_row(Tuple* tuple, int64_t tablet_id) {$/;" f class:doris::stream_load::IndexChannel +add_row src/exec/tablet_sink.cpp /^Status NodeChannel::add_row(Tuple* input_tuple, int64_t tablet_id) {$/;" f class:doris::stream_load::NodeChannel +add_row src/runtime/buffered_block_mgr.h /^ void add_row() {$/;" f class:doris::BufferedBlockMgr::Block +add_row src/runtime/buffered_block_mgr2.h /^ void add_row() {$/;" f class:doris::BufferedBlockMgr2::Block +add_row src/runtime/buffered_tuple_stream.h /^ bool add_row(TupleRow* row) {$/;" f class:doris::BufferedTupleStream +add_row src/runtime/buffered_tuple_stream.h /^inline bool BufferedTupleStream::add_row(TupleRow* row, uint8_t** dst) {$/;" f class:doris::BufferedTupleStream +add_row src/runtime/buffered_tuple_stream2.inline.h /^inline bool BufferedTupleStream2::add_row(TupleRow* row, Status* status) {$/;" f class:doris::BufferedTupleStream2 +add_row src/runtime/data_stream_sender.cpp /^Status DataStreamSender::Channel::add_row(TupleRow* row) {$/;" f class:doris::DataStreamSender::Channel +add_row src/runtime/row_batch.h /^ int add_row() {$/;" f class:doris::RowBatch +add_row_batch src/exec/row_batch_list.h /^ void add_row_batch(RowBatch* row_batch) {$/;" f class:doris::RowBatchList +add_row_block src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::add_row_block(const RowBlock& row_block, const uint32_t data_offset) {$/;" f class:doris::SegmentGroup +add_rows src/runtime/row_batch.h /^ int add_rows(int n) {$/;" f class:doris::RowBatch +add_rowset src/olap/rowset/alpha_rowset_writer.cpp /^OLAPStatus AlphaRowsetWriter::add_rowset(RowsetSharedPtr rowset) {$/;" f class:doris::AlphaRowsetWriter +add_rowset src/olap/rowset/beta_rowset_writer.cpp /^OLAPStatus BetaRowsetWriter::add_rowset(RowsetSharedPtr rowset) {$/;" f class:doris::BetaRowsetWriter +add_rowset src/olap/tablet.cpp /^OLAPStatus Tablet::add_rowset(RowsetSharedPtr rowset, bool need_persist) {$/;" f class:doris::Tablet +add_rowset_for_linked_schema_change src/olap/rowset/alpha_rowset_writer.cpp /^OLAPStatus AlphaRowsetWriter::add_rowset_for_linked_schema_change($/;" f class:doris::AlphaRowsetWriter +add_rowset_for_linked_schema_change src/olap/rowset/beta_rowset_writer.cpp /^OLAPStatus BetaRowsetWriter::add_rowset_for_linked_schema_change(RowsetSharedPtr rowset,$/;" f class:doris::BetaRowsetWriter +add_rs_meta src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::add_rs_meta(const RowsetMetaSharedPtr& rs_meta) {$/;" f class:doris::TabletMeta +add_run src/olap/rowset/segment_v2/column_writer.cpp /^ void add_run(bool value, size_t run) {$/;" f class:doris::segment_v2::NullBitmapBuilder +add_runtime_exec_option src/exec/exec_node.cpp /^void ExecNode::add_runtime_exec_option(const std::string& str) {$/;" f class:doris::ExecNode +add_sampling_counter src/util/runtime_profile.cpp /^RuntimeProfile::Counter* RuntimeProfile::add_sampling_counter($/;" f class:doris::RuntimeProfile +add_scan_bytes src/runtime/query_statistics.h /^ void add_scan_bytes(int64_t scan_bytes) {$/;" f class:doris::QueryStatistics +add_scan_ranges src/runtime/disk_io_mgr.cc /^Status DiskIoMgr::add_scan_ranges(RequestContext* reader,$/;" f class:doris::DiskIoMgr +add_scan_rows src/runtime/query_statistics.h /^ void add_scan_rows(int64_t scan_rows) {$/;" f class:doris::QueryStatistics +add_segment src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::add_segment() {$/;" f class:doris::SegmentGroup +add_segment_group src/olap/rowset/alpha_rowset_meta.cpp /^void AlphaRowsetMeta::add_segment_group(const SegmentGroupPB& segment_group) {$/;" f class:doris::AlphaRowsetMeta +add_short_key src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::add_short_key(const RowCursor& short_key, const uint32_t data_offset) {$/;" f class:doris::SegmentGroup +add_slot src/runtime/descriptor_helper.h /^ TTupleDescriptorBuilder& add_slot(const TSlotDescriptor& slot_desc) {$/;" f class:doris::TTupleDescriptorBuilder +add_slot src/runtime/descriptors.cpp /^void TupleDescriptor::add_slot(SlotDescriptor* slot) {$/;" f class:doris::TupleDescriptor +add_slots src/runtime/descriptor_helper.h /^ void add_slots(const std::vector& slots) {$/;" f class:doris::TDescriptorTableBuilder +add_status test/runtime/load_channel_mgr_test.cpp /^OLAPStatus add_status;$/;" m namespace:doris file: +add_sub_plan_statistics src/runtime/data_stream_recvr.h /^ void add_sub_plan_statistics(const PQueryStatistics& statistics, int sender_id) {$/;" f class:doris::DataStreamRecvr +add_tablet src/exec/tablet_sink.h /^ void add_tablet(const TTabletWithPartition& tablet) {$/;" f class:doris::stream_load::NodeChannel +add_thread src/util/thread.cpp /^void ThreadMgr::add_thread(const pthread_t& pthread_id, const std::string& name,$/;" f class:doris::ThreadMgr +add_thread test/runtime/buffer_control_block_test.cpp /^void* add_thread(void* param) {$/;" f namespace:doris +add_thread_counters src/util/runtime_profile.cpp /^RuntimeProfile::ThreadCounters* RuntimeProfile::add_thread_counters($/;" f class:doris::RuntimeProfile +add_to_bucket src/exec/hash_table.hpp /^inline void HashTable::add_to_bucket(Bucket* bucket, int64_t node_idx, Node* node) {$/;" f class:doris::HashTable +add_to_debug_string src/exec/blocking_join_node.h /^ virtual void add_to_debug_string(int indentation_level, std::stringstream* out) const {$/;" f class:doris::BlockingJoinNode +add_tuple src/runtime/descriptor_helper.h /^ void add_tuple(const TTupleDescriptor& tuple) {$/;" f class:doris::TDescriptorTableBuilder +add_tuple_stream src/runtime/row_batch.cpp /^void RowBatch::add_tuple_stream(BufferedTupleStream2* stream) {$/;" f class:doris::RowBatch +add_two_number src/exprs/udf_builtins.cpp /^BigIntVal UdfBuiltins::add_two_number($/;" f class:doris::UdfBuiltins +add_unused_rowset src/olap/storage_engine.cpp /^void StorageEngine::add_unused_rowset(RowsetSharedPtr rowset) {$/;" f class:doris::StorageEngine +add_value src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^ void add_value(const CppType& value) {$/;" f class:doris::segment_v2::__anon54::BitmapIndexWriterImpl +add_value src/olap/tuple.h /^ void add_value(const std::string& value, bool is_null = false) {$/;" f class:doris::OlapTuple +add_values src/olap/rowset/segment_v2/zone_map_index.cpp /^void ZoneMapIndexWriter::add_values(const void* values, size_t count) {$/;" f class:doris::segment_v2::ZoneMapIndexWriter +add_version_to_graph src/olap/rowset_graph.cpp /^OLAPStatus RowsetGraph::add_version_to_graph(const Version& version) {$/;" f class:doris::RowsetGraph +add_warning src/udf/udf.cpp /^bool FunctionContext::add_warning(const char* warning_msg) {$/;" f class:doris_udf::FunctionContext +add_write_range src/runtime/disk_io_mgr.cc /^Status DiskIoMgr::add_write_range(RequestContext* writer, WriteRange* write_range) {$/;" f class:doris::DiskIoMgr +add_zone_map src/olap/rowset/rowset_meta.h /^ void add_zone_map(const ZoneMap& zone_map) {$/;" f class:doris::RowsetMeta +add_zone_maps src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::add_zone_maps($/;" f class:doris::SegmentGroup +add_zone_maps_for_linked_schema_change src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::add_zone_maps_for_linked_schema_change($/;" f class:doris::SegmentGroup +addr src/util/network_util.h /^ struct sockaddr_in addr;$/;" m class:doris::InetAddress typeref:struct:doris::InetAddress::sockaddr_in +addrs src/util/broker_load_error_hub.h /^ std::vector addrs;$/;" m struct:doris::BrokerLoadErrorHub::BrokerInfo +advance src/olap/column_block.h /^ void advance(size_t skip) { _row_offset += skip; }$/;" f class:doris::ColumnBlockView +advance src/olap/generic_iterators.cpp /^Status MergeIteratorContext::advance() {$/;" f class:doris::MergeIteratorContext +advance src/olap/selection_vector.h /^ void advance(size_t skip) {$/;" f class:doris::SelectionVectorView +advance src/util/tdigest.h /^ bool advance() { return ++iter != end; }$/;" f struct:doris::CentroidList +advance_row_block src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::advance_row_block(int64_t num_row_blocks, RowBlockPosition* position) const {$/;" f class:doris::SegmentGroup +aes_create_key src/util/aes_util.cpp /^static void aes_create_key(const unsigned char* origin_key, uint32_t key_length,$/;" f namespace:doris +aes_decrypt src/exprs/encryption_functions.cpp /^StringVal EncryptionFunctions::aes_decrypt(FunctionContext* ctx,$/;" f class:doris::EncryptionFunctions +aes_encrypt src/exprs/encryption_functions.cpp /^StringVal EncryptionFunctions::aes_encrypt(FunctionContext* ctx,$/;" f class:doris::EncryptionFunctions +aes_mode_key_sizes src/util/aes_util.cpp /^static uint aes_mode_key_sizes[]= {$/;" m namespace:doris file: +agg_finalize src/olap/field.h /^ inline void agg_finalize(RowCursorCell* dst, MemPool* mem_pool) const {$/;" f class:doris::Field +agg_finalize_row src/olap/row.h /^void agg_finalize_row(RowType* row, MemPool* mem_pool) {$/;" f namespace:doris +agg_finalize_row src/olap/row.h /^void agg_finalize_row(const std::vector& ids, RowType* row, MemPool* mem_pool) {$/;" f namespace:doris +agg_fn src/exprs/new_agg_fn_evaluator.h /^ const AggFn& agg_fn() const { return agg_fn_; }$/;" f class:doris::NewAggFnEvaluator +agg_fn_ src/exprs/new_agg_fn_evaluator.h /^ const AggFn& agg_fn_;$/;" m class:doris::NewAggFnEvaluator +agg_fn_ctx src/exprs/new_agg_fn_evaluator_ir.cc /^FunctionContext* NewAggFnEvaluator::agg_fn_ctx() const {$/;" f class:NewAggFnEvaluator +agg_fn_ctx_ src/exprs/new_agg_fn_evaluator.h /^ boost::scoped_ptr agg_fn_ctx_;$/;" m class:doris::NewAggFnEvaluator +agg_fn_ctxs src/exec/partitioned_aggregation_node.h /^ std::vector agg_fn_ctxs;$/;" m struct:doris::PartitionedAggregationNode::Partition +agg_fn_evals src/exec/new_partitioned_aggregation_node.h /^ std::vector agg_fn_evals;$/;" m struct:doris::NewPartitionedAggregationNode::Partition +agg_fn_evals_ src/exec/new_partitioned_aggregation_node.h /^ std::vector agg_fn_evals_;$/;" m class:doris::NewPartitionedAggregationNode +agg_fn_pool src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr agg_fn_pool;$/;" m struct:doris::NewPartitionedAggregationNode::Partition +agg_fn_pool src/exec/partitioned_aggregation_node.h /^ boost::scoped_ptr agg_fn_pool;$/;" m struct:doris::PartitionedAggregationNode::Partition +agg_fn_pool_ src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr agg_fn_pool_;$/;" m class:doris::NewPartitionedAggregationNode +agg_fns_ src/exec/new_partitioned_aggregation_node.h /^ std::vector agg_fns_;$/;" m class:doris::NewPartitionedAggregationNode +agg_init src/olap/field.h /^ virtual void agg_init(RowCursorCell* dst, const RowCursorCell& src, MemPool* mem_pool, ObjectPool* agg_pool) const {$/;" f class:doris::Field +agg_merge src/udf/udf.cpp /^void HllVal::agg_merge(const HllVal &other) {$/;" f class:doris_udf::HllVal +agg_method src/olap/aggregate_func.h /^ FieldAggregationMethod agg_method() const { return _agg_method; }$/;" f class:doris::AggregateInfo +agg_method src/olap/aggregate_func.h /^ static const FieldAggregationMethod agg_method = aggMethod;$/;" m struct:doris::AggregateTraits +agg_object_pool src/runtime/row_batch.h /^ ObjectPool* agg_object_pool() {$/;" f class:doris::RowBatch +agg_op src/exprs/agg_fn.h /^ AggregationOp agg_op() const { return agg_op_; }$/;" f class:doris::AggFn +agg_op src/exprs/agg_fn_evaluator.h /^ AggregationOp agg_op() const {$/;" f class:doris::AggFnEvaluator +agg_op_ src/exprs/agg_fn.h /^ AggregationOp agg_op_;$/;" m class:doris::AggFn +agg_parse_and_cal src/udf/udf.cpp /^void HllVal::agg_parse_and_cal(FunctionContext* ctx, const HllVal& other) {$/;" f class:doris_udf::HllVal +agg_update src/olap/field.h /^ inline void agg_update(RowCursorCell* dest, const RowCursorCell& src, MemPool* mem_pool = nullptr) const {$/;" f class:doris::Field +agg_update_row src/olap/row.h /^void agg_update_row(DstRowType* dst, const SrcRowType& src, MemPool* mem_pool) {$/;" f namespace:doris +agg_update_row src/olap/row.h /^void agg_update_row(const std::vector& cids, DstRowType* dst, const SrcRowType& src) {$/;" f namespace:doris +aggregated_partitions_ src/exec/new_partitioned_aggregation_node.h /^ std::deque aggregated_partitions_;$/;" m class:doris::NewPartitionedAggregationNode +aggregated_row_stream src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr aggregated_row_stream;$/;" m struct:doris::NewPartitionedAggregationNode::Partition +aggregated_row_stream src/exec/partitioned_aggregation_node.h /^ boost::scoped_ptr aggregated_row_stream;$/;" m struct:doris::PartitionedAggregationNode::Partition +aggregation src/olap/field.h /^ FieldAggregationMethod aggregation() const { return _agg_info->agg_method();}$/;" f class:doris::Field +aggregation src/olap/reader.h /^ bool aggregation;$/;" m struct:doris::ReaderParams +aggregation src/olap/tablet_schema.h /^ FieldAggregationMethod aggregation() const { return _aggregation; }$/;" f class:doris::TabletColumn +aligned_malloc src/gutil/port.h /^inline void *aligned_malloc(size_t size, int minimum_alignment) {$/;" f +all src/runtime/dpp_sink_internal.h /^ static const PartRange& all() {$/;" f class:doris::PartRange +all_buffers_it src/runtime/buffered_block_mgr.h /^ std::list::iterator all_buffers_it;$/;" m struct:doris::BufferedBlockMgr::BufferDescriptor +all_buffers_it src/runtime/buffered_block_mgr2.h /^ std::list::iterator all_buffers_it;$/;" m struct:doris::BufferedBlockMgr2::BufferDescriptor +all_inc_rs_metas src/olap/tablet_meta.h /^inline const std::vector& TabletMeta::all_inc_rs_metas() const {$/;" f class:doris::TabletMeta +all_null src/olap/stream_index_reader.h /^ bool all_null() const {$/;" f class:doris::PositionEntryReader +all_null src/olap/stream_index_reader.h /^ bool all_null() const {$/;" f class:doris::PositionProvider +all_rs_metas src/olap/tablet_meta.h /^inline const std::vector& TabletMeta::all_rs_metas() const {$/;" f class:doris::TabletMeta +alloc_time src/runtime/bufferpool/buffer_pool_counters.h /^ RuntimeProfile::Counter* alloc_time;$/;" m struct:doris::BufferPoolClientCounters +allocate src/gutil/stl_util.h /^ pointer allocate(size_type n, std::allocator::const_pointer hint = 0) {$/;" f class:STLCountingAllocator +allocate src/olap/schema_change.cpp /^OLAPStatus RowBlockAllocator::allocate(RowBlock** row_block,$/;" f class:doris::RowBlockAllocator +allocate src/runtime/buffered_block_mgr.h /^ template T* allocate(int size) {$/;" f class:doris::BufferedBlockMgr::Block +allocate src/runtime/buffered_block_mgr2.h /^ template T* allocate(int size) {$/;" f class:doris::BufferedBlockMgr2::Block +allocate src/runtime/free_list.hpp /^ uint8_t* allocate(int size, int* buffer_size) {$/;" f class:doris::FreeList +allocate src/runtime/free_pool.hpp /^ uint8_t* allocate(int size) {$/;" f class:doris::FreePool +allocate src/runtime/mem_pool.h /^ uint8_t* ALWAYS_INLINE allocate(int64_t size, int alignment) {$/;" f class:doris::MemPool +allocate src/runtime/mem_pool.h /^ uint8_t* allocate(int64_t size) {$/;" f class:doris::MemPool +allocate src/runtime/memory/chunk_allocator.cpp /^bool ChunkAllocator::allocate(size_t size, Chunk* chunk) {$/;" f class:doris::ChunkAllocator +allocate src/runtime/memory/system_allocator.cpp /^uint8_t* SystemAllocator::allocate(size_t length) {$/;" f class:doris::SystemAllocator +allocate src/udf/udf.cpp /^ uint8_t* allocate(int byte_size) {$/;" f class:doris::FreePool +allocate src/udf/udf.cpp /^uint8_t* FunctionContext::allocate(int byte_size) { $/;" f class:doris_udf::FunctionContext +allocate src/util/byte_buffer.h /^ static ByteBufferPtr allocate(size_t size) {$/;" f struct:doris::ByteBuffer +allocate src/util/radix_sort.h /^ void * allocate(size_t size) {$/;" f struct:doris::RadixSortMallocAllocator +allocate_any_val src/exprs/anyval_util.cpp /^Status allocate_any_val(RuntimeState* state, MemPool* pool, const TypeDescriptor& type,$/;" f namespace:doris +allocate_dir src/runtime/load_path_mgr.cpp /^Status LoadPathMgr::allocate_dir($/;" f class:doris::LoadPathMgr +allocate_local src/udf/udf.cpp /^uint8_t* FunctionContextImpl::allocate_local(int byte_size) {$/;" f class:doris::FunctionContextImpl +allocate_memory src/olap/field.h /^ virtual char* allocate_memory(char* cell_ptr, char* variable_ptr) const {$/;" f class:doris::Field +allocate_memory_for_string_type src/olap/row_cursor.cpp /^OLAPStatus RowCursor::allocate_memory_for_string_type(const TabletSchema& schema) {$/;" f class:doris::RowCursor +allocate_row src/runtime/buffered_tuple_stream.cpp /^inline uint8_t* BufferedTupleStream::allocate_row(int size) {$/;" f class:doris::BufferedTupleStream +allocate_row src/runtime/buffered_tuple_stream2.inline.h /^inline uint8_t* BufferedTupleStream2::allocate_row(int size, Status *status) {$/;" f class:doris::BufferedTupleStream2 +allocate_scratch_space src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::allocate_scratch_space(int64_t block_size,$/;" f class:doris::BufferedBlockMgr2 +allocate_space src/runtime/tmp_file_mgr.cc /^Status TmpFileMgr::File::allocate_space(int64_t write_size, int64_t* offset) {$/;" f class:doris::TmpFileMgr::File +allocate_string_value src/olap/field.h /^ char* allocate_string_value(MemPool* pool) const {$/;" f class:doris::Field +allocate_value src/olap/field.h /^ virtual inline char* allocate_value(MemPool* pool) const { return (char*)pool->allocate(_type_info->size()); }$/;" f class:doris::Field +allocate_via_malloc src/runtime/memory/system_allocator.cpp /^uint8_t* SystemAllocator::allocate_via_malloc(size_t length) {$/;" f class:doris::SystemAllocator +allocate_via_mmap src/runtime/memory/system_allocator.cpp /^uint8_t* SystemAllocator::allocate_via_mmap(size_t length) {$/;" f class:doris::SystemAllocator +allocated_ src/runtime/bufferpool/suballocator.h /^ int64_t allocated_;$/;" m class:doris::Suballocator +allocated_bytes src/runtime/mem_pool.h /^ int64_t allocated_bytes;$/;" m struct:doris::MemPool::ChunkInfo +allocated_bytes src/util/system_metrics.cpp /^ IntGauge allocated_bytes;$/;" m struct:doris::MemoryMetrics file: +allocator src/runtime/bufferpool/buffer_pool.h /^ BufferAllocator* allocator() { return allocator_.get(); }$/;" f class:doris::BufferPool +allocator src/util/core_local.h /^ CoreDataAllocator* allocator(int i) const {$/;" f class:doris::CoreLocalValueController +allocator_ src/exec/new_partitioned_hash_table.h /^ Suballocator* allocator_;$/;" m class:doris::NewPartitionedHashTable +allocator_ src/runtime/bufferpool/buffer_pool.h /^ boost::scoped_ptr allocator_;$/;" m class:doris::BufferPool +allow_comparators src/gutil/strings/numbers.h /^ bool allow_comparators;$/;" m struct:DoubleRangeOptions +allow_currency src/gutil/strings/numbers.h /^ bool allow_currency;$/;" m struct:DoubleRangeOptions +allow_unbounded_markers src/gutil/strings/numbers.h /^ bool allow_unbounded_markers;$/;" m struct:DoubleRangeOptions +alpha_rowset_extra_meta_pb src/olap/rowset/rowset_meta.h /^ const AlphaRowsetExtraMetaPB& alpha_rowset_extra_meta_pb() const {$/;" f class:doris::RowsetMeta +already_failed src/exec/tablet_sink.h /^ bool already_failed() const { return _already_failed; }$/;" f class:doris::stream_load::NodeChannel +alter_state src/olap/tablet_meta.h /^ inline const AlterTabletState& alter_state() const { return _alter_state; }$/;" f class:doris::AlterTabletTask +alter_tablet_type src/olap/schema_change.h /^ AlterTabletType alter_tablet_type;$/;" m struct:doris::SchemaChangeHandler::SchemaChangeParams +alter_task src/olap/tablet.cpp /^AlterTabletTaskSharedPtr Tablet::alter_task() {$/;" f class:doris::Tablet +alter_task src/olap/tablet_meta.cpp /^AlterTabletTaskSharedPtr TabletMeta::alter_task() {$/;" f class:doris::TabletMeta +alter_type src/olap/tablet_meta.h /^ inline const AlterTabletType& alter_type() const { return _alter_type; }$/;" f class:doris::AlterTabletTask +analyze_priority_cidrs src/service/backend_options.cpp /^bool BackendOptions::analyze_priority_cidrs() {$/;" f class:doris::BackendOptions +and_ src/gutil/template_util.h /^struct and_ : public integral_constant {$/;" s namespace:base +any_limit_exceeded src/runtime/mem_tracker.h /^ bool any_limit_exceeded() {$/;" f class:doris::MemTracker +any_selected src/olap/selection_vector.h /^inline bool SelectionVector::any_selected() const {$/;" f class:doris::SelectionVector +any_val_alignment src/exprs/anyval_util.h /^ static int any_val_alignment(const TypeDescriptor& t) {$/;" f class:doris::AnyValUtil +any_val_eq src/exprs/case_expr.cpp /^bool CaseExpr::any_val_eq(const TypeDescriptor& type, const AnyVal* v1, const AnyVal* v2) {$/;" f class:doris::CaseExpr +any_val_size src/exprs/anyval_util.h /^ static int any_val_size(const TypeDescriptor& t) {$/;" f class:doris::AnyValUtil +append src/olap/fs/file_block_manager.cpp /^Status FileWritableBlock::append(const Slice& data) {$/;" f class:doris::fs::internal::FileWritableBlock +append src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::append(const void* data, size_t num_rows) {$/;" f class:doris::segment_v2::ColumnWriter +append src/olap/rowset/segment_v2/column_writer.h /^ Status append(bool is_null, void* data) {$/;" f class:doris::segment_v2::ColumnWriter +append src/olap/rowset/segment_v2/column_writer.h /^ Status append(const CellType& cell) {$/;" f class:doris::segment_v2::ColumnWriter +append src/runtime/message_body_sink.cpp /^Status MessageBodyFileSink::append(const char* data, size_t size) {$/;" f class:doris::MessageBodyFileSink +append src/runtime/message_body_sink.h /^ virtual Status append(const ByteBufferPtr& buf) {$/;" f class:doris::MessageBodySink +append src/runtime/mysql_table_writer.cpp /^Status MysqlTableWriter::append(RowBatch* batch) {$/;" f class:doris::MysqlTableWriter +append src/runtime/string_buffer.hpp /^ void append(const char* str, int len) {$/;" f class:doris::StringBuffer +append src/runtime/string_buffer.hpp /^ void append(const uint8_t* str, int len) {$/;" f class:doris::StringBuffer +append src/udf/udf.cpp /^void StringVal::append(FunctionContext* ctx, const uint8_t* buf, size_t buf_len) {$/;" f class:doris_udf::StringVal +append src/udf/udf.cpp /^void StringVal::append(FunctionContext* ctx, const uint8_t* buf, size_t buf_len,$/;" f class:doris_udf::StringVal +append src/util/buffer_builder.hpp /^ inline void append(const T& v) {$/;" f class:doris::BufferBuilder +append src/util/buffer_builder.hpp /^ inline void append(const void* buffer, int len) {$/;" f class:doris::BufferBuilder +append src/util/faststring.h /^ void append(const std::string &str) {$/;" f class:doris::faststring +append src/util/faststring.h /^ void append(const void *src_v, size_t count) {$/;" f class:doris::faststring +append_any_val_type src/util/symbols_util.cpp /^static void append_any_val_type($/;" f namespace:doris +append_condition src/olap/olap_cond.cpp /^OLAPStatus Conditions::append_condition(const TCondition& tcond) {$/;" f class:doris::Conditions +append_date_string src/runtime/datetime_value.cpp /^char* DateTimeValue::append_date_string(char *to) const {$/;" f class:doris::DateTimeValue +append_entry src/olap/rowset/segment_v2/ordinal_page_index.cpp /^void OrdinalIndexWriter::append_entry(ordinal_t ordinal, const PagePointer& data_pp) {$/;" f class:doris::segment_v2::OrdinalIndexWriter +append_error_msg_to_file src/runtime/runtime_state.cpp /^void RuntimeState::append_error_msg_to_file($/;" f class:doris::RuntimeState +append_exec_option src/util/runtime_profile.h /^ void append_exec_option(const std::string& option) {$/;" f class:doris::RuntimeProfile +append_mangled_token src/util/symbols_util.cpp /^static void append_mangled_token(const std::string& s, std::stringstream* out) {$/;" f namespace:doris +append_next_node src/exec/partitioned_hash_table.inline.h /^inline PartitionedHashTable::DuplicateNode* PartitionedHashTable::append_next_node($/;" f class:doris::PartitionedHashTable +append_nullable src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::append_nullable($/;" f class:doris::segment_v2::ColumnWriter +append_nulls src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::append_nulls(size_t num_rows) {$/;" f class:doris::segment_v2::ColumnWriter +append_one_row src/runtime/dpp_writer.cpp /^Status DppWriter::append_one_row(TupleRow* row) {$/;" f class:doris::DppWriter +append_row src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::append_row(const RowType& row) {$/;" f class:doris::segment_v2::SegmentWriter +append_row_batch src/runtime/result_writer.cpp /^Status ResultWriter::append_row_batch(RowBatch* batch) {$/;" f class:doris::ResultWriter +append_seq_id src/util/symbols_util.cpp /^static void append_seq_id(int seq_id, std::stringstream* out) {$/;" f namespace:doris +append_spilled_row src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::append_spilled_row(BufferedTupleStream2* stream, TupleRow* row) {$/;" f class:doris::PartitionedAggregationNode +append_spilled_row src/exec/partitioned_aggregation_node_ir.cc /^Status PartitionedAggregationNode::append_spilled_row(Partition* partition, TupleRow* row) {$/;" f class:doris::PartitionedAggregationNode +append_string src/runtime/datetime_value.cpp /^static char* append_string(const char* from, char* to) {$/;" f namespace:doris +append_time_string src/runtime/datetime_value.cpp /^char* DateTimeValue::append_time_string(char *to) const {$/;" f class:doris::DateTimeValue +append_to_buf src/runtime/dpp_writer.cpp /^void DppWriter::append_to_buf(const void* ptr, int len) {$/;" f class:doris::DppWriter +append_with_line_delimiter src/runtime/routine_load/kafka_consumer_pipe.h /^ Status append_with_line_delimiter(const char* data, size_t size) {$/;" f class:doris::KafkaConsumerPipe +append_with_prefix src/runtime/datetime_value.cpp /^static char* append_with_prefix(const char* str, int str_len,$/;" f namespace:doris +appendv src/olap/fs/file_block_manager.cpp /^Status FileWritableBlock::appendv(const Slice* data, size_t data_cnt) {$/;" f class:doris::fs::internal::FileWritableBlock +apply_cgroup src/agent/cgroups_mgr.cpp /^void CgroupsMgr::apply_cgroup(const string& user_name, const string& level) {$/;" f class:doris::CgroupsMgr +apply_system_cgroup src/agent/cgroups_mgr.h /^ static void apply_system_cgroup() {$/;" f class:doris::CgroupsMgr +aquire src/olap/rowset/rowset.h /^ void aquire() {$/;" f class:doris::Rowset +arg_type_descs src/exprs/agg_fn.h /^ const std::vector& arg_type_descs() const {$/;" f class:doris::AggFn +arg_type_descs_ src/exprs/agg_fn.h /^ const std::vector arg_type_descs_;$/;" m class:doris::AggFn +array src/olap/byte_buffer.h /^ char* array() {$/;" f class:doris::StorageByteBuffer +array src/olap/byte_buffer.h /^ const char* array() const {$/;" f class:doris::StorageByteBuffer +array src/olap/byte_buffer.h /^ const char* array(size_t position) const {$/;" f class:doris::StorageByteBuffer +array_ src/gutil/gscoped_ptr.h /^ C* array_;$/;" m class:gscoped_array +arraysize src/gutil/macros.h 144;" d +arraysize src/service/brpc.h 32;" d +arrow src/runtime/memory_scratch_sink.h /^namespace arrow {$/;" n +arrow src/runtime/record_batch_queue.h /^namespace arrow {$/;" n +arrow src/runtime/result_queue_mgr.h /^namespace arrow {$/;" n +arrow src/util/arrow/row_batch.h /^namespace arrow {$/;" n +arrow src/util/arrow/row_block.h /^namespace arrow {$/;" n +arrow src/util/arrow/utils.h /^namespace arrow {$/;" n +arrow_pretty_print src/util/arrow/utils.cpp /^Status arrow_pretty_print(const arrow::Array& arr, std::ostream* os) {$/;" f namespace:doris +arrow_pretty_print src/util/arrow/utils.cpp /^Status arrow_pretty_print(const arrow::RecordBatch& rb, std::ostream* os) {$/;" f namespace:doris +as_string src/gutil/strings/stringpiece.h /^ std::string as_string() const {$/;" f class:StringPiece +as_wkt src/geo/geo_types.cpp /^std::string GeoCircle::as_wkt() const {$/;" f class:doris::GeoCircle +as_wkt src/geo/geo_types.cpp /^std::string GeoLine::as_wkt() const {$/;" f class:doris::GeoLine +as_wkt src/geo/geo_types.cpp /^std::string GeoPoint::as_wkt() const {$/;" f class:doris::GeoPoint +as_wkt src/geo/geo_types.cpp /^std::string GeoPolygon::as_wkt() const {$/;" f class:doris::GeoPolygon +ascii src/exprs/string_functions.cpp /^IntVal StringFunctions::ascii(FunctionContext* context, const StringVal& str) {$/;" f class:doris::StringFunctions +ascii_isalnum src/gutil/strings/ascii_ctype.h /^static inline bool ascii_isalnum(unsigned char c) { return kApb[c] & 0x04; }$/;" f +ascii_isalpha src/gutil/strings/ascii_ctype.h /^static inline bool ascii_isalpha(unsigned char c) { return kApb[c] & 0x01; }$/;" f +ascii_isascii src/gutil/strings/ascii_ctype.h /^static inline bool ascii_isascii(unsigned char c) {$/;" f +ascii_isblank src/gutil/strings/ascii_ctype.h /^static inline bool ascii_isblank(unsigned char c) { return kApb[c] & 0x20; }$/;" f +ascii_iscntrl src/gutil/strings/ascii_ctype.h /^static inline bool ascii_iscntrl(unsigned char c) { return kApb[c] & 0x40; }$/;" f +ascii_isdigit src/gutil/strings/ascii_ctype.h /^static inline bool ascii_isdigit(unsigned char c) {$/;" f +ascii_isgraph src/gutil/strings/ascii_ctype.h /^static inline bool ascii_isgraph(unsigned char c) {$/;" f +ascii_islower src/gutil/strings/ascii_ctype.h /^static inline bool ascii_islower(unsigned char c) {$/;" f +ascii_isprint src/gutil/strings/ascii_ctype.h /^static inline bool ascii_isprint(unsigned char c) {$/;" f +ascii_ispunct src/gutil/strings/ascii_ctype.h /^static inline bool ascii_ispunct(unsigned char c) { return kApb[c] & 0x10; }$/;" f +ascii_isspace src/gutil/strings/ascii_ctype.h /^static inline bool ascii_isspace(unsigned char c) { return kApb[c] & 0x08; }$/;" f +ascii_isupper src/gutil/strings/ascii_ctype.h /^static inline bool ascii_isupper(unsigned char c) {$/;" f +ascii_isxdigit src/gutil/strings/ascii_ctype.h /^static inline bool ascii_isxdigit(unsigned char c) { return kApb[c] & 0x80; }$/;" f +ascii_tolower src/gutil/strings/ascii_ctype.h /^static inline char ascii_tolower(unsigned char c) { return kAsciiToLower[c]; }$/;" f +ascii_toupper src/gutil/strings/ascii_ctype.h /^static inline char ascii_toupper(unsigned char c) { return kAsciiToUpper[c]; }$/;" f +assemble_state src/common/status.cpp /^inline const char* assemble_state($/;" f namespace:doris +asserter_ src/gutil/threading/thread_collision_warner.h /^ AsserterBase* asserter_;$/;" m class:base::ThreadCollisionWarner +assign src/runtime/string_buffer.hpp /^ void assign(const char* str, int len) {$/;" f class:doris::StringBuffer +assign_copy src/util/faststring.h /^ void assign_copy(const std::string &str) {$/;" f class:doris::faststring +assign_copy src/util/faststring.h /^ void assign_copy(const uint8_t *src, size_t len) {$/;" f class:doris::faststring +assign_fn_ctx_idx src/exprs/expr.cpp /^void Expr::assign_fn_ctx_idx(int* next_fn_ctx_idx) {$/;" f class:doris::Expr +assign_from_double src/runtime/decimal_value.h /^ DecimalValue& assign_from_double(const double double_value) {$/;" f class:doris::DecimalValue +assign_from_double src/runtime/decimalv2_value.h /^ DecimalV2Value& assign_from_double(const double double_value) {$/;" f class:doris::DecimalV2Value +assign_from_float src/runtime/decimal_value.h /^ DecimalValue& assign_from_float(const float float_value) {$/;" f class:doris::DecimalValue +assign_from_float src/runtime/decimalv2_value.h /^ DecimalV2Value& assign_from_float(const float float_value) {$/;" f class:doris::DecimalV2Value +assign_thread_to_cgroups src/agent/cgroups_mgr.cpp /^AgentStatus CgroupsMgr::assign_thread_to_cgroups(int64_t thread_id,$/;" f class:doris::CgroupsMgr +assign_to_cgroups src/agent/cgroups_mgr.cpp /^AgentStatus CgroupsMgr::assign_to_cgroups(const string& user_name,$/;" f class:doris::CgroupsMgr +assign_topic_partitions src/runtime/routine_load/data_consumer.cpp /^Status KafkaDataConsumer::assign_topic_partitions($/;" f class:doris::KafkaDataConsumer +assign_topic_partitions src/runtime/routine_load/data_consumer_group.cpp /^Status KafkaDataConsumerGroup::assign_topic_partitions(StreamLoadContext* ctx) {$/;" f class:doris::KafkaDataConsumerGroup +at src/util/faststring.h /^ const uint8_t &at(size_t i) const {$/;" f class:doris::faststring +at_capacity src/runtime/row_batch.h /^ bool at_capacity() {$/;" f class:doris::RowBatch +at_capacity src/runtime/row_batch.h /^ bool at_capacity(MemPool* tuple_pool) {$/;" f class:doris::RowBatch +at_end src/exec/hash_table.h /^ bool at_end() {$/;" f class:doris::HashTable::Iterator +at_end src/exec/partitioned_hash_table.h /^ bool at_end() const { return _bucket_idx == BUCKET_NOT_FOUND; }$/;" f class:doris::PartitionedHashTable::Iterator +at_end src/exec/row_batch_list.h /^ bool at_end() {$/;" f class:doris::RowBatchList::TupleRowIterator +at_end src/runtime/row_batch.h /^ bool IR_ALWAYS_INLINE at_end() { return _row >= _row_batch_end; }$/;" f class:doris::RowBatch::Iterator +at_resource_limit src/runtime/row_batch.h /^ bool at_resource_limit() {$/;" f class:doris::RowBatch +atoi32 src/gutil/strtoint.h /^inline int32 atoi32(const char *nptr) {$/;" f +atoi32 src/gutil/strtoint.h /^inline int32 atoi32(const string &s) {$/;" f +atoi64 src/gutil/strtoint.h /^inline int64 atoi64(const char *nptr) {$/;" f +atoi64 src/gutil/strtoint.h /^inline int64 atoi64(const string &s) {$/;" f +atoi_kmgt src/gutil/strings/numbers.cc /^uint64 atoi_kmgt(const char* s) {$/;" f +atoi_kmgt src/gutil/strings/numbers.h /^inline uint64 atoi_kmgt(const string& s) { return atoi_kmgt(s.c_str()); }$/;" f +atoll src/gutil/port.h 814;" d +attach src/olap/row_cursor.h /^ inline void attach(char* buf) { _fixed_buf = buf; }$/;" f class:doris::RowCursor +attach src/olap/stream_index_common.cpp /^void ColumnStatistics::attach(char* buffer) {$/;" f class:doris::ColumnStatistics +attach src/olap/stream_index_reader.cpp /^void PositionEntryReader::attach(char* buffer) {$/;" f class:doris::PositionEntryReader +attach_buf src/olap/wrapper_field.h /^ void attach_buf(char* buf) {$/;" f class:doris::WrapperField +attach_field src/olap/wrapper_field.h /^ void attach_field(char* field) {$/;" f class:doris::WrapperField +auth src/runtime/stream_load/stream_load_context.h /^ AuthInfo auth;$/;" m class:doris::StreamLoadContext +auth_code src/common/utils.h /^ int64_t auth_code = -1;$/;" m struct:doris::AuthInfo +autodigit_greater src/gutil/strings/numbers.h /^struct autodigit_greater$/;" s +autodigit_less src/gutil/strings/numbers.h /^struct autodigit_less$/;" s +available src/olap/file_stream.cpp /^uint64_t ReadOnlyFileStream::available() {$/;" f class:doris::ReadOnlyFileStream +available src/olap/in_stream.cpp /^uint64_t InStream::available() {$/;" f class:doris::InStream +available src/olap/olap_common.h /^ int64_t available; \/\/ 可用空间,单位字节$/;" m struct:doris::DataDirInfo +available_allocated_buffers src/runtime/buffered_block_mgr2.h /^ int available_allocated_buffers() const {$/;" f class:doris::BufferedBlockMgr2 +available_buffer src/olap/in_stream.h /^ const char* available_buffer() {$/;" f class:doris::InStream +available_buffers src/runtime/buffered_block_mgr2.cc /^int64_t BufferedBlockMgr2::available_buffers(Client* client) const {$/;" f class:doris::BufferedBlockMgr2 +available_storage_medium_type_count src/olap/storage_engine.h /^ uint32_t available_storage_medium_type_count() {$/;" f class:doris::StorageEngine +avg_finalize src/exprs/aggregate_functions.cpp /^DoubleVal AggregateFunctions::avg_finalize(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::AggregateFunctions +avg_get_value src/exprs/aggregate_functions.cpp /^DoubleVal AggregateFunctions::avg_get_value(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::AggregateFunctions +avg_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::avg_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +avg_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::avg_merge(FunctionContext* ctx, const StringVal& src,$/;" f class:doris::AggregateFunctions +avg_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::avg_remove(FunctionContext* ctx, const T& src, StringVal* dst) {$/;" f class:doris::AggregateFunctions +avg_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::avg_update(FunctionContext* ctx, const T& src, StringVal* dst) {$/;" f class:doris::AggregateFunctions +avx_count_nibbles src/util/simdutf8check.h /^static inline void avx_count_nibbles(__m256i bytes,$/;" f +avx_processed_utf_bytes src/util/simdutf8check.h /^struct avx_processed_utf_bytes {$/;" s +avxcarryContinuations src/util/simdutf8check.h /^static inline __m256i avxcarryContinuations(__m256i initial_lengths,$/;" f +avxcheckContinuations src/util/simdutf8check.h /^static inline void avxcheckContinuations(__m256i initial_lengths,$/;" f +avxcheckFirstContinuationMax src/util/simdutf8check.h /^static inline void avxcheckFirstContinuationMax(__m256i current_bytes,$/;" f +avxcheckOverlong src/util/simdutf8check.h /^static inline void avxcheckOverlong(__m256i current_bytes,$/;" f +avxcheckSmallerThan0xF4 src/util/simdutf8check.h /^static inline void avxcheckSmallerThan0xF4(__m256i current_bytes,$/;" f +avxcheckUTF8Bytes src/util/simdutf8check.h /^avxcheckUTF8Bytes(__m256i current_bytes,$/;" f +avxcheckUTF8Bytes_asciipath src/util/simdutf8check.h /^avxcheckUTF8Bytes_asciipath(__m256i current_bytes,$/;" f +avxcontinuationLengths src/util/simdutf8check.h /^static inline __m256i avxcontinuationLengths(__m256i high_nibbles) {$/;" f +b2a_bin src/gutil/strings/escaping.cc /^string b2a_bin(const string& b, bool byte_order_msb) {$/;" f namespace:strings +b2a_hex src/gutil/strings/escaping.cc /^string b2a_hex(const StringPiece& b) {$/;" f namespace:strings +b2a_hex src/gutil/strings/escaping.cc /^string b2a_hex(const char* b, int len) {$/;" f namespace:strings +b2a_hex src/gutil/strings/escaping.cc /^void b2a_hex(const unsigned char* b, char* a, int num) {$/;" f namespace:strings +b2a_hex src/gutil/strings/escaping.cc /^void b2a_hex(const unsigned char* from, string* to, int num) {$/;" f namespace:strings +b2a_hex_t src/gutil/strings/escaping.cc /^static void b2a_hex_t(const unsigned char* b, T a, int num) {$/;" f namespace:strings +backward src/util/radix_sort.h /^ static KeyBits backward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); }$/;" f struct:doris::RadixSortSignedTransform +backward src/util/radix_sort.h /^ static KeyBits backward(KeyBits x) { return x; }$/;" f struct:doris::RadixSortIdentityTransform +backward src/util/radix_sort.h /^ static KeyBits backward(KeyBits x) {$/;" f struct:doris::RadixSortFloatTransform +baidu_crc32_byte src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline int baidu_crc32_byte(char const* src, int crc,$/;" f namespace:doris +baidu_crc32_qw src/olap/bhp_lib.h /^inline int baidu_crc32_qw(char const* src, int crc, unsigned int length) {$/;" f namespace:doris +base src/gutil/atomic_refcount.h /^namespace base {$/;" n +base src/gutil/atomicops-internals-gcc.h /^namespace base {$/;" n +base src/gutil/atomicops-internals-tsan.h /^namespace base {$/;" n +base src/gutil/atomicops-internals-x86.h /^namespace base {$/;" n +base src/gutil/atomicops.h /^namespace base {$/;" n +base src/gutil/casts.h /^namespace base {$/;" n +base src/gutil/cpu.cc /^namespace base {$/;" n file: +base src/gutil/cpu.h /^namespace base {$/;" n +base src/gutil/macros.h /^namespace base {$/;" n +base src/gutil/spinlock_internal.cc /^namespace base { namespace internal { static int SuggestedDelayNS(int loop); }}$/;" n file: +base src/gutil/spinlock_internal.cc /^namespace base {$/;" n file: +base src/gutil/spinlock_internal.h /^namespace base {$/;" n +base src/gutil/spinlock_linux-inl.h /^namespace base {$/;" n +base src/gutil/spinlock_posix-inl.h /^namespace base {$/;" n +base src/gutil/spinlock_win32-inl.h /^namespace base {$/;" n +base src/gutil/sysinfo.cc /^namespace base {$/;" n file: +base src/gutil/sysinfo.h /^namespace base {$/;" n +base src/gutil/template_util.h /^namespace base {$/;" n +base src/gutil/threading/thread_collision_warner.cc /^namespace base {$/;" n file: +base src/gutil/threading/thread_collision_warner.h /^namespace base {$/;" n +base src/gutil/type_traits.h /^namespace base {$/;" n +base64_decode src/util/url_coding.cpp /^bool base64_decode(const std::string& in, std::string* out) {$/;" f namespace:doris +base64_decode src/util/url_coding.cpp /^static inline int64_t base64_decode($/;" f namespace:doris +base64_decode2 src/exprs/base64.cpp /^int64_t base64_decode2($/;" f namespace:doris +base64_encode src/exec/olap_utils.h /^inline size_t base64_encode($/;" f namespace:doris +base64_encode src/util/url_coding.cpp /^size_t base64_encode(const unsigned char *data,$/;" f namespace:doris +base64_encode src/util/url_coding.cpp /^void base64_encode(const std::string& in, std::string* out) {$/;" f namespace:doris +base64_encode2 src/exprs/base64.cpp /^size_t base64_encode2(const unsigned char *data,$/;" f namespace:doris +base64_pad src/exec/olap_utils.h /^static const char base64_pad = '=';$/;" m namespace:doris +base64_pad src/exprs/base64.cpp /^static const char base64_pad = '=';$/;" v file: +base64_pad src/util/url_coding.cpp /^static const char base64_pad = '=';$/;" m namespace:doris file: +base64url_encode src/util/url_coding.cpp /^void base64url_encode(const std::string& in, std::string *out) {$/;" f namespace:doris +base_bytes src/olap/rowset/run_length_integer_writer.h /^ base_bytes: 3;$/;" m struct:doris::RunLengthIntegerWriter::PatchedBaseHead +base_compaction_bytes_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::base_compaction_bytes_total;$/;" m class:doris::DorisMetrics file: +base_compaction_bytes_total src/util/doris_metrics.h /^ static IntCounter base_compaction_bytes_total;$/;" m class:doris::DorisMetrics +base_compaction_deltas_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::base_compaction_deltas_total;$/;" m class:doris::DorisMetrics file: +base_compaction_deltas_total src/util/doris_metrics.h /^ static IntCounter base_compaction_deltas_total;$/;" m class:doris::DorisMetrics +base_compaction_request_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::base_compaction_request_failed;$/;" m class:doris::DorisMetrics file: +base_compaction_request_failed src/util/doris_metrics.h /^ static IntCounter base_compaction_request_failed;$/;" m class:doris::DorisMetrics +base_compaction_request_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::base_compaction_request_total;$/;" m class:doris::DorisMetrics file: +base_compaction_request_total src/util/doris_metrics.h /^ static IntCounter base_compaction_request_total;$/;" m class:doris::DorisMetrics +base_name src/util/path_util.cpp /^string base_name(const string& path) {$/;" f namespace:doris::path_util +base_tablet src/olap/schema_change.h /^ TabletSharedPtr base_tablet;$/;" m struct:doris::SchemaChangeHandler::SchemaChangeParams +batch src/exec/merge_join_node.h /^ RowBatch batch;$/;" m struct:doris::MergeJoinNode::ChildReaderContext +batch_size src/runtime/runtime_state.h /^ int batch_size() const {$/;" f class:doris::RuntimeState +bcopy src/gutil/port.h 854;" d +be_number src/runtime/runtime_state.h /^ int be_number(void) {$/;" f class:doris::RuntimeState +begin src/exec/csv_scan_node.cpp /^ char const* begin() const {$/;" f class:doris::StringRef +begin src/exec/hash_table.hpp /^inline HashTable::Iterator HashTable::begin() {$/;" f class:doris::HashTable +begin src/exec/partitioned_hash_table.inline.h /^inline PartitionedHashTable::Iterator PartitionedHashTable::begin(PartitionedHashTableCtx* ctx) {$/;" f class:doris::PartitionedHashTable +begin src/exprs/hybird_set.h /^ IteratorBase* begin() {$/;" f class:doris::HybirdSet +begin src/exprs/hybird_set.h /^ IteratorBase* begin() {$/;" f class:doris::StringValueSet +begin src/gutil/strings/split_internal.h /^ const Iterator& begin() const { return begin_; }$/;" f class:strings::internal::Splitter +begin src/gutil/strings/stringpiece.h /^ iterator begin() const { return ptr_; }$/;" f class:StringPiece +begin src/olap/olap_index.h /^ const OLAPIndexOffset begin() const {$/;" f class:doris::MemIndex +begin src/olap/rowset/segment_v2/ordinal_page_index.h /^OrdinalPageIndexIterator OrdinalIndexReader::begin() {$/;" f class:doris::segment_v2::OrdinalIndexReader +begin src/olap/short_key_index.h /^ ShortKeyIndexIterator begin() const {$/;" f class:doris::ShortKeyIndexDecoder +begin src/util/bitmap_value.h /^inline Roaring64MapSetBitForwardIterator Roaring64Map::begin() const {$/;" f class:doris::detail::Roaring64Map +begin src/util/lru_cache.hpp /^ Iterator begin() {$/;" f class:doris::LruCache +begin_ src/gutil/strings/split_internal.h /^ const Iterator begin_;$/;" m class:strings::internal::Splitter +begin_include src/exec/olap_common.h /^ bool begin_include() const {$/;" f class:doris::OlapScanKeys +begin_include src/exec/olap_utils.h /^ bool begin_include;$/;" m struct:doris::OlapScanRange +begin_offset src/runtime/stream_load/stream_load_context.h /^ std::map begin_offset;$/;" m class:doris::KafkaLoadInfo +begin_scan_range src/exec/olap_utils.h /^ OlapTuple begin_scan_range;$/;" m struct:doris::OlapScanRange +begin_trans src/runtime/mysql_table_writer.h /^ Status begin_trans() {$/;" f class:doris::MysqlTableWriter +begin_txn src/runtime/stream_load/stream_load_executor.cpp /^Status StreamLoadExecutor::begin_txn(StreamLoadContext* ctx) {$/;" f class:doris::StreamLoadExecutor +bfd_parser src/runtime/exec_env.h /^ BfdParser* bfd_parser() const { return _bfd_parser; }$/;" f class:doris::ExecEnv +big_ src/gutil/template_util.h /^struct big_ {$/;" s namespace:base +big_endian src/util/bit_util.h /^ static inline int16_t big_endian(int16_t val) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline int16_t big_endian(int16_t value) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline int32_t big_endian(int32_t val) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline int32_t big_endian(int32_t value) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline int64_t big_endian(int64_t val) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline int64_t big_endian(int64_t value) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline uint16_t big_endian(uint16_t val) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline uint16_t big_endian(uint16_t value) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline uint32_t big_endian(uint32_t val) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline uint32_t big_endian(uint32_t value) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline uint64_t big_endian(uint64_t val) {$/;" f class:doris::BitUtil +big_endian src/util/bit_util.h /^ static inline uint64_t big_endian(uint64_t value) {$/;" f class:doris::BitUtil +big_val test/runtime/dpp_writer_test.cpp /^ int64_t big_val;$/;" m struct:doris::TestDataTuple file: +big_val test/runtime/mysql_table_writer_test.cpp /^ int64_t big_val;$/;" m struct:doris::TestDataTuple file: +bigger src/runtime/decimal_value.h /^ bool bigger(const DecimalValue& other) const {$/;" f class:doris::DecimalValue +bigint_val src/exprs/expr_value.h /^ int64_t bigint_val;$/;" m struct:doris::ExprValue +bin src/exprs/math_functions.cpp /^StringVal MathFunctions::bin(FunctionContext* ctx, const BigIntVal& v) {$/;" f class:doris::MathFunctions +binary_find_partition src/runtime/data_spliter.cpp /^int DataSpliter::binary_find_partition(const PartRangeKey& key) const {$/;" f class:doris::DataSpliter +binary_find_partition src/runtime/data_stream_sender.cpp /^int DataStreamSender::binary_find_partition(const PartRangeKey& key) const {$/;" f class:doris::DataStreamSender +binary_find_partition_id src/exec/broker_scan_node.cpp /^int64_t BrokerScanNode::binary_find_partition_id(const PartRangeKey& key) const {$/;" f class:doris::BrokerScanNode +bit_cast src/gutil/casts.h /^inline Dest bit_cast(const Source& source) {$/;" f +bit_cast src/util/radix_sort.h /^decay_t bit_cast(const From& from) {$/;" f namespace:doris +bit_ceil src/runtime/disk_io_mgr.cc /^static int64_t bit_ceil(int64_t value, int64_t divisor) {$/;" f file: +bit_log2 src/runtime/disk_io_mgr.cc /^static int bit_log2(uint64_t x) {$/;" f file: +bit_mask src/runtime/descriptors.h /^ uint8_t bit_mask; \/\/ to extract null indicator$/;" m struct:doris::NullIndicatorOffset +bit_num src/olap/bloom_filter.hpp /^ uint32_t bit_num() const {$/;" f class:doris::BitSet +bit_num src/olap/bloom_filter.hpp /^ uint32_t bit_num() const {$/;" f class:doris::BloomFilter +bit_offset src/runtime/descriptors.h /^ uint8_t bit_offset; \/\/ only used to serialize, from 1 to 8$/;" m struct:doris::NullIndicatorOffset +bit_offset_ src/util/bit_stream_utils.h /^ int bit_offset_; \/\/ Offset in buffered_values_$/;" m class:doris::BitReader +bit_offset_ src/util/bit_stream_utils.h /^ int bit_offset_; \/\/ Offset in buffered_values_$/;" m class:doris::BitWriter +bit_or src/util/runtime_profile.h /^ void bit_or(int64_t delta) {$/;" f class:doris::RuntimeProfile::Counter +bit_pack src/util/frame_of_reference_coding.cpp /^void ForEncoder::bit_pack(const T *input, uint8_t in_num, int bit_width, uint8_t *output) {$/;" f class:doris::ForEncoder +bit_packing_one_frame_value src/util/frame_of_reference_coding.cpp /^void ForEncoder::bit_packing_one_frame_value(const T* input) {$/;" f class:doris::ForEncoder +bit_reader_ src/util/rle_encoding.h /^ BitReader bit_reader_;$/;" m class:doris::RleDecoder +bit_reverse_table src/gutil/bits.h /^ static const unsigned char bit_reverse_table[];$/;" m class:Bits +bit_set src/olap/bloom_filter.hpp /^ const BitSet& bit_set() const {$/;" f class:doris::BloomFilter +bit_set_data src/olap/bloom_filter.hpp /^ uint64_t* bit_set_data() const {$/;" f class:doris::BloomFilter +bit_set_data_len src/olap/bloom_filter.hpp /^ uint32_t bit_set_data_len() const {$/;" f class:doris::BloomFilter +bit_unpack src/util/frame_of_reference_coding.cpp /^void ForDecoder::bit_unpack(const uint8_t *input, uint8_t in_num, int bit_width, T *output) {$/;" f class:doris::ForDecoder +bit_width src/olap/rowset/run_length_integer_writer.h /^ bit_width: 5,$/;" m struct:doris::RunLengthIntegerWriter::DeltaHead +bit_width src/olap/rowset/run_length_integer_writer.h /^ bit_width: 5,$/;" m struct:doris::RunLengthIntegerWriter::DirectHead +bit_width src/olap/rowset/run_length_integer_writer.h /^ bit_width: 5,$/;" m struct:doris::RunLengthIntegerWriter::PatchedBaseHead +bit_width_ src/util/rle_encoding.h /^ const int bit_width_;$/;" m class:doris::RleEncoder +bit_width_ src/util/rle_encoding.h /^ int bit_width_;$/;" m class:doris::RleDecoder +bit_writer_ src/util/rle_encoding.h /^ BitWriter bit_writer_;$/;" m class:doris::RleEncoder +bitmap src/olap/selection_vector.h /^ const uint8_t* bitmap() const { return _bitmap.get(); }$/;" f class:doris::SelectionVector +bitmapOf src/util/bitmap_value.h /^ static Roaring64Map bitmapOf(size_t n...) {$/;" f class:doris::detail::Roaring64Map +bitmap_and src/exprs/bitmap_function.cpp /^StringVal BitmapFunctions::bitmap_and(FunctionContext* ctx, const StringVal& lhs, const StringVal& rhs){$/;" f class:doris::BitmapFunctions +bitmap_contains src/exprs/bitmap_function.cpp /^BooleanVal BitmapFunctions::bitmap_contains(FunctionContext* ctx, const StringVal& src, const BigIntVal& input) {$/;" f class:doris::BitmapFunctions +bitmap_count src/exprs/bitmap_function.cpp /^BigIntVal BitmapFunctions::bitmap_count(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::BitmapFunctions +bitmap_empty src/exprs/bitmap_function.cpp /^StringVal BitmapFunctions::bitmap_empty(FunctionContext* ctx) {$/;" f class:doris::BitmapFunctions +bitmap_finalize src/exprs/bitmap_function.cpp /^BigIntVal BitmapFunctions::bitmap_finalize(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::BitmapFunctions +bitmap_from_string src/exprs/bitmap_function.cpp /^StringVal BitmapFunctions::bitmap_from_string(FunctionContext* ctx, const StringVal& input) {$/;" f class:doris::BitmapFunctions +bitmap_get_value src/exprs/bitmap_function.cpp /^BigIntVal BitmapFunctions::bitmap_get_value(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::BitmapFunctions +bitmap_has_any src/exprs/bitmap_function.cpp /^BooleanVal BitmapFunctions::bitmap_has_any(FunctionContext* ctx, const StringVal& lhs, const StringVal& rhs) {$/;" f class:doris::BitmapFunctions +bitmap_hash src/exprs/bitmap_function.cpp /^StringVal BitmapFunctions::bitmap_hash(doris_udf::FunctionContext* ctx, const doris_udf::StringVal& src) {$/;" f class:doris::BitmapFunctions +bitmap_index_filter_count src/olap/olap_common.h /^ int64_t bitmap_index_filter_count = 0;$/;" m struct:doris::OlapReaderStatistics +bitmap_index_filter_timer src/olap/olap_common.h /^ int64_t bitmap_index_filter_timer = 0;$/;" m struct:doris::OlapReaderStatistics +bitmap_init src/exprs/bitmap_function.cpp /^void BitmapFunctions::bitmap_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::BitmapFunctions +bitmap_intersect_finalize src/exprs/bitmap_function.cpp /^BigIntVal BitmapFunctions::bitmap_intersect_finalize(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::BitmapFunctions +bitmap_intersect_init src/exprs/bitmap_function.cpp /^void BitmapFunctions::bitmap_intersect_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::BitmapFunctions +bitmap_intersect_merge src/exprs/bitmap_function.cpp /^void BitmapFunctions::bitmap_intersect_merge(FunctionContext* ctx, const StringVal& src, const StringVal* dst) {$/;" f class:doris::BitmapFunctions +bitmap_intersect_serialize src/exprs/bitmap_function.cpp /^StringVal BitmapFunctions::bitmap_intersect_serialize(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::BitmapFunctions +bitmap_intersect_update src/exprs/bitmap_function.cpp /^void BitmapFunctions::bitmap_intersect_update(FunctionContext* ctx, const StringVal& src, const ValType& key,$/;" f class:doris::BitmapFunctions +bitmap_nums src/olap/rowset/segment_v2/bitmap_index_reader.h /^ inline rowid_t bitmap_nums() const {$/;" f class:doris::segment_v2::BitmapIndexIterator +bitmap_nums src/olap/rowset/segment_v2/bitmap_index_reader.h /^ int64_t bitmap_nums() {$/;" f class:doris::segment_v2::BitmapIndexReader +bitmap_or src/exprs/bitmap_function.cpp /^StringVal BitmapFunctions::bitmap_or(FunctionContext* ctx, const StringVal& lhs, const StringVal& rhs){$/;" f class:doris::BitmapFunctions +bitmap_serialize src/exprs/bitmap_function.cpp /^StringVal BitmapFunctions::bitmap_serialize(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::BitmapFunctions +bitmap_to_string src/exprs/bitmap_function.cpp /^StringVal BitmapFunctions::bitmap_to_string(FunctionContext* ctx, const StringVal& input) {$/;" f class:doris::BitmapFunctions +bitmap_union src/exprs/bitmap_function.cpp /^void BitmapFunctions::bitmap_union(FunctionContext* ctx, const StringVal& src, StringVal* dst) {$/;" f class:doris::BitmapFunctions +bitmap_update_int src/exprs/bitmap_function.cpp /^void BitmapFunctions::bitmap_update_int(FunctionContext* ctx, const T& src, StringVal* dst) {$/;" f class:doris::BitmapFunctions +bits src/util/frame_of_reference_coding.h /^static inline uint8_t bits(const T v) {$/;" f namespace:doris +bitsToKey src/util/radix_sort.h /^ static Key bitsToKey(KeyBits x) { return bit_cast(x); }$/;" f struct:doris::RadixSort +bits_ src/gutil/strings/charset.h /^ uint64 bits_[4];$/;" m class:strings::CharSet +bits_less_than_64 src/util/frame_of_reference_coding.h /^static inline uint8_t bits_less_than_64(const uint64_t v) {$/;" f namespace:doris +bits_may_more_than_64 src/util/frame_of_reference_coding.h /^static inline uint8_t bits_may_more_than_64(const uint128_t v) {$/;" f namespace:doris +bitshuffle src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp /^namespace bitshuffle {$/;" n namespace:doris file: +bitshuffle src/olap/rowset/segment_v2/bitshuffle_wrapper.h /^namespace bitshuffle {$/;" n namespace:doris +blacklist src/runtime/tmp_file_mgr.h /^ bool blacklist() {$/;" f class:doris::TmpFileMgr::Dir +blacklist_device src/runtime/tmp_file_mgr.cc /^void TmpFileMgr::blacklist_device(DeviceId device_id) {$/;" f class:doris::TmpFileMgr +blob src/olap/rowset/run_length_integer_writer.h /^ char blob[];$/;" m struct:doris::RunLengthIntegerWriter::DeltaHead +blob src/olap/rowset/run_length_integer_writer.h /^ char blob[];$/;" m struct:doris::RunLengthIntegerWriter::DirectHead +blob src/olap/rowset/run_length_integer_writer.h /^ char blob[];$/;" m struct:doris::RunLengthIntegerWriter::PatchedBaseHead +blob src/olap/rowset/run_length_integer_writer.h /^ char blob[];$/;" m struct:doris::RunLengthIntegerWriter::ShortRepeatHead +block src/runtime/buffered_block_mgr.h /^ Block* block;$/;" m struct:doris::BufferedBlockMgr::BufferDescriptor +block src/runtime/buffered_block_mgr2.h /^ Block* block;$/;" m struct:doris::BufferedBlockMgr2::BufferDescriptor +block src/runtime/buffered_tuple_stream.h /^ uint64_t block() const {$/;" f struct:doris::BufferedTupleStream::RowIdx +block src/runtime/buffered_tuple_stream2.h /^ uint64_t block() const {$/;" f struct:doris::BufferedTupleStream2::RowIdx +block_convert_ns src/olap/olap_common.h /^ int64_t block_convert_ns = 0;$/;" m struct:doris::OlapReaderStatistics +block_count src/olap/bloom_filter.hpp /^ uint64_t block_count;$/;" m struct:doris::BloomFilterIndexHeader +block_count src/olap/rowset/segment_reader.h /^ uint32_t block_count() const {$/;" f class:doris::SegmentReader +block_count src/olap/stream_index_common.h /^ uint64_t block_count; \/\/ 本index中block的个数$/;" m struct:doris::StreamIndexHeader +block_fetch_ns src/olap/olap_common.h /^ int64_t block_fetch_ns = 0;$/;" m struct:doris::OlapReaderStatistics +block_load_ns src/olap/olap_common.h /^ int64_t block_load_ns = 0;$/;" m struct:doris::OlapReaderStatistics +block_manager src/olap/fs/file_block_manager.cpp /^BlockManager* FileReadableBlock::block_manager() const {$/;" f class:doris::fs::internal::FileReadableBlock +block_manager src/olap/fs/file_block_manager.cpp /^BlockManager* FileWritableBlock::block_manager() const {$/;" f class:doris::fs::internal::FileWritableBlock +block_manager_preflush_control src/olap/fs/block_manager.cpp /^const std::string BlockManager::block_manager_preflush_control = "finalize";$/;" m class:doris::fs::BlockManager file: +block_manager_preflush_control src/olap/fs/block_manager.h /^ static const std::string block_manager_preflush_control;$/;" m class:doris::fs::BlockManager +block_manager_types src/olap/fs/block_manager.h /^ static std::vector block_manager_types() {$/;" f class:doris::fs::BlockManager +block_mgr src/runtime/runtime_state.h /^ BufferedBlockMgr* block_mgr() {$/;" f class:doris::RuntimeState +block_mgr2 src/runtime/runtime_state.h /^ BufferedBlockMgr2* block_mgr2() {$/;" f class:doris::RuntimeState +block_mgr_for_ut src/olap/fs/fs_util.cpp /^BlockManager* block_mgr_for_ut() {$/;" f namespace:doris::fs::fs_util +block_mgr_parent_tracker src/runtime/test_env.h /^ MemTracker* block_mgr_parent_tracker() {$/;" f class:doris::TestEnv +block_seek_ns src/olap/olap_common.h /^ int64_t block_seek_ns = 0;$/;" m struct:doris::OlapReaderStatistics +block_seek_num src/olap/olap_common.h /^ int64_t block_seek_num = 0;$/;" m struct:doris::OlapReaderStatistics +block_size src/olap/olap_index.h /^ uint32_t block_size;$/;" m struct:doris::RowBlockPosition +block_statistics src/olap/rowset/column_writer.h /^ ColumnStatistics* block_statistics() {$/;" f class:doris::ColumnWriter +block_status src/olap/row_block.h /^ uint8_t block_status() const { return _block_status; }$/;" f class:doris::RowBlock +block_status src/runtime/vectorized_row_batch.h /^ uint8_t block_status() const { return _block_status; }$/;" f class:doris::VectorizedRowBatch +block_wait src/util/counter_cond_variable.hpp /^ void block_wait() {$/;" f class:doris::CounterCondVariable +blocking_get src/runtime/record_batch_queue.h /^ bool blocking_get(std::shared_ptr* result) {$/;" f class:doris::RecordBatchQueue +blocking_get src/util/blocking_priority_queue.hpp /^ bool blocking_get(T* out) {$/;" f class:doris::BlockingPriorityQueue +blocking_get src/util/blocking_queue.hpp /^ bool blocking_get(T* out) {$/;" f class:doris::BlockingQueue +blocking_put src/runtime/record_batch_queue.h /^ bool blocking_put(const std::shared_ptr& val) {$/;" f class:doris::RecordBatchQueue +blocking_put src/util/blocking_priority_queue.hpp /^ bool blocking_put(const T& val) {$/;" f class:doris::BlockingPriorityQueue +blocking_put src/util/blocking_queue.hpp /^ bool blocking_put(const T& val) {$/;" f class:doris::BlockingQueue +blocks_created_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::blocks_created_total;$/;" m class:doris::DorisMetrics file: +blocks_created_total src/util/doris_metrics.h /^ static IntCounter blocks_created_total;$/;" m class:doris::DorisMetrics +blocks_deleted_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::blocks_deleted_total;$/;" m class:doris::DorisMetrics file: +blocks_deleted_total src/util/doris_metrics.h /^ static IntCounter blocks_deleted_total;$/;" m class:doris::DorisMetrics +blocks_load src/olap/olap_common.h /^ int64_t blocks_load = 0;$/;" m struct:doris::OlapReaderStatistics +blocks_open_reading src/olap/fs/block_manager_metrics.h /^ IntGauge* blocks_open_reading;$/;" m struct:doris::fs::internal::BlockManagerMetrics +blocks_open_reading src/util/doris_metrics.cpp /^IntGauge DorisMetrics::blocks_open_reading;$/;" m class:doris::DorisMetrics file: +blocks_open_reading src/util/doris_metrics.h /^ static IntGauge blocks_open_reading;$/;" m class:doris::DorisMetrics +blocks_open_writing src/olap/fs/block_manager_metrics.h /^ IntGauge* blocks_open_writing;$/;" m struct:doris::fs::internal::BlockManagerMetrics +blocks_open_writing src/util/doris_metrics.cpp /^IntGauge DorisMetrics::blocks_open_writing;$/;" m class:doris::DorisMetrics file: +blocks_open_writing src/util/doris_metrics.h /^ static IntGauge blocks_open_writing;$/;" m class:doris::DorisMetrics +blocks_pinned src/runtime/buffered_tuple_stream2.h /^ int blocks_pinned() const { return _num_pinned; }$/;" f class:doris::BufferedTupleStream2 +blocks_push_remote_duration_us src/util/doris_metrics.cpp /^IntCounter DorisMetrics::blocks_push_remote_duration_us;$/;" m class:doris::DorisMetrics file: +blocks_push_remote_duration_us src/util/doris_metrics.h /^ static IntCounter blocks_push_remote_duration_us;$/;" m class:doris::DorisMetrics +blocks_unpinned src/runtime/buffered_tuple_stream2.h /^ int blocks_unpinned() const { return _blocks.size() - _num_pinned - _num_small_blocks; }$/;" f class:doris::BufferedTupleStream2 +bloom_add src/runtime/string_search.hpp /^ void bloom_add(char c) {$/;" f class:doris::StringSearch +bloom_filter_fpp src/olap/tablet.h /^inline double Tablet::bloom_filter_fpp() const {$/;" f class:doris::Tablet +bloom_filter_fpp src/olap/tablet_schema.h /^ inline double bloom_filter_fpp() const { return _bf_fpp; }$/;" f class:doris::TabletSchema +bloom_query src/runtime/string_search.hpp /^ bool bloom_query(char c) const {$/;" f class:doris::StringSearch +body_bytes src/http/action/mini_load.cpp /^ size_t body_bytes = 0;$/;" m struct:doris::MiniLoadAsyncCtx file: +body_bytes src/runtime/stream_load/stream_load_context.h /^ size_t body_bytes = 0;$/;" m class:doris::StreamLoadContext +body_sink src/runtime/stream_load/stream_load_context.h /^ std::shared_ptr body_sink;$/;" m class:doris::StreamLoadContext +bool_val src/exprs/expr_value.h /^ bool bool_val;$/;" m struct:doris::ExprValue +bootstrap_page_footer src/http/web_page_handler.cpp /^void WebPageHandler::bootstrap_page_footer(std::stringstream* output) {$/;" f class:doris::WebPageHandler +bootstrap_page_header src/http/web_page_handler.cpp /^void WebPageHandler::bootstrap_page_header(std::stringstream* output) {$/;" f class:doris::WebPageHandler +brief src/runtime/stream_load/stream_load_context.cpp /^std::string StreamLoadContext::brief(bool detail) const {$/;" f class:doris::StreamLoadContext +broker_client_cache src/runtime/exec_env.h /^ ClientCache* broker_client_cache() { return _broker_client_cache; }$/;" f class:doris::ExecEnv +broker_mgr src/runtime/exec_env.h /^ BrokerMgr* broker_mgr() const { return _broker_mgr; }$/;" f class:doris::ExecEnv +brokers src/runtime/stream_load/stream_load_context.h /^ std::string brokers;$/;" m class:doris::KafkaLoadInfo +brpc src/runtime/buffer_control_block.h /^namespace brpc {$/;" n +brpc src/service/brpc_service.cpp /^namespace brpc {$/;" n file: +brpc src/service/brpc_service.h /^namespace brpc {$/;" n +brpc src/service/internal_service.h /^namespace brpc {$/;" n +brpc_port src/exec/tablet_info.h /^ int32_t brpc_port;$/;" m struct:doris::NodeInfo +brpc_stub_cache src/runtime/exec_env.h /^ BrpcStubCache* brpc_stub_cache() const { return _brpc_stub_cache; }$/;" f class:doris::ExecEnv +bshuf_compress_lz4 src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp 30;" d file: +bshuf_compress_lz4 src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp 34;" d file: +bshuf_compress_lz4_bound src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp 29;" d file: +bshuf_compress_lz4_bound src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp 33;" d file: +bshuf_decompress_lz4 src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp 31;" d file: +bshuf_decompress_lz4 src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp 35;" d file: +bstream src/util/minizip/unzip.c /^ bz_stream bstream; \/* bzLib stream structure for bziped *\/$/;" m struct:__anon33 file: +bswap_16 src/gutil/port.h /^static inline uint16 bswap_16(uint16 x) {$/;" f +bswap_16 src/gutil/port.h 108;" d +bswap_16 src/gutil/port.h 115;" d +bswap_16 src/gutil/port.h 127;" d +bswap_24 src/gutil/endian.h /^inline uint32_t bswap_24(uint32_t x) {$/;" f +bswap_32 src/gutil/port.h /^static inline uint32 bswap_32(uint32 x) {$/;" f +bswap_32 src/gutil/port.h 109;" d +bswap_32 src/gutil/port.h 116;" d +bswap_32 src/gutil/port.h 134;" d +bswap_64 src/gutil/port.h /^static inline uint64 bswap_64(uint64 x) {$/;" f +bswap_64 src/gutil/port.h 110;" d +bswap_64 src/gutil/port.h 117;" d +bswap_64 src/gutil/port.h 145;" d +bucketData src/exec/new_partitioned_hash_table.h /^ } bucketData;$/;" m struct:doris::NewPartitionedHashTable::Bucket typeref:union:doris::NewPartitionedHashTable::Bucket::__anon41 +bucketData src/exec/partitioned_hash_table.h /^ } bucketData;$/;" m struct:doris::PartitionedHashTable::Bucket typeref:union:doris::PartitionedHashTable::Bucket::__anon42 +bucket_allocation_ src/exec/new_partitioned_hash_table.h /^ std::unique_ptr bucket_allocation_;$/;" m class:doris::NewPartitionedHashTable +bucket_id src/runtime/dpp_sink_internal.h /^ uint32_t bucket_id;$/;" m struct:doris::TabletDesc +bucket_idx_ src/exec/new_partitioned_hash_table.h /^ int64_t bucket_idx_;$/;" m class:doris::NewPartitionedHashTable::Iterator +bucket_size src/gutil/hash/hash.h /^ static const size_t bucket_size = 4; \/\/ These are required by MSVC$/;" m struct:GoodFastHash +bucket_size src/gutil/hash/hash.h /^ static const size_t bucket_size = 4; \/\/ These are required by MSVC$/;" m struct:__gnu_cxx::hash +bucket_size src/gutil/strings/stringpiece.h /^ static const size_t bucket_size = 4; \/\/ These are required by MSVC$/;" m struct:GoodFastHash +bucketing_counters src/util/runtime_profile.h /^ BucketCountersMap bucketing_counters;$/;" m struct:doris::RuntimeProfile::PeriodicCounterUpdateState +buckets_ src/exec/new_partitioned_hash_table.h /^ Bucket* buckets_;$/;" m class:doris::NewPartitionedHashTable +buddy_ src/runtime/bufferpool/suballocator.h /^ Suballocation* buddy_;$/;" m class:doris::Suballocation +buf src/util/mysql_row_buffer.h /^ const char* buf() const {$/;" f class:doris::MysqlRowBuffer +buffer output/udf/include/udf.h /^ int32_t buffer[9] = {0};$/;" m struct:doris_udf::DecimalVal +buffer src/olap/olap_index.h /^ EntrySlice buffer;$/;" m struct:doris::SegmentMetaInfo +buffer src/runtime/buffered_block_mgr.h /^ uint8_t* buffer() const {$/;" f class:doris::BufferedBlockMgr::Block +buffer src/runtime/buffered_block_mgr.h /^ uint8_t* buffer;$/;" m struct:doris::BufferedBlockMgr::BufferDescriptor +buffer src/runtime/buffered_block_mgr2.h /^ uint8_t* buffer() const {$/;" f class:doris::BufferedBlockMgr2::Block +buffer src/runtime/buffered_block_mgr2.h /^ uint8_t* buffer;$/;" m struct:doris::BufferedBlockMgr2::BufferDescriptor +buffer src/runtime/bufferpool/buffer_pool_internal.h /^ BufferHandle buffer;$/;" m class:doris::BufferPool::Page +buffer src/runtime/disk_io_mgr.h /^ char* buffer() { return _buffer; }$/;" f class:doris::DiskIoMgr::BufferDescriptor +buffer src/runtime/row_batch.h /^ BufferPool::BufferHandle buffer;$/;" m struct:doris::RowBatch::BufferInfo +buffer src/udf/udf.h /^ int32_t buffer[9] = {0};$/;" m struct:doris_udf::DecimalVal +buffer src/util/bit_stream_utils.h /^ faststring *buffer() const { return buffer_; }$/;" f class:doris::BitWriter +buffer_ src/runtime/bufferpool/suballocator.h /^ BufferPool::BufferHandle buffer_;$/;" m class:doris::Suballocation +buffer_ src/util/bit_stream_utils.h /^ const uint8_t* buffer_;$/;" m class:doris::BitReader +buffer_ src/util/bit_stream_utils.h /^ faststring *buffer_;$/;" m class:doris::BitWriter +buffer_ src/util/bitmap.h /^ std::vector buffer_;$/;" m class:doris::Bitmap +buffer_len src/runtime/buffered_block_mgr.h /^ int64_t buffer_len() const {$/;" f class:doris::BufferedBlockMgr::Block +buffer_len src/runtime/buffered_block_mgr2.h /^ int64_t buffer_len() const {$/;" f class:doris::BufferedBlockMgr2::Block +buffer_len src/runtime/disk_io_mgr.h /^ int64_t buffer_len() { return _buffer_len; }$/;" f class:doris::DiskIoMgr::BufferDescriptor +buffer_len test/udf/uda_test.cpp /^ int buffer_len;$/;" m struct:doris_udf::MinState file: +buffer_lock src/runtime/bufferpool/buffer_pool_internal.h /^ SpinLock buffer_lock;$/;" m class:doris::BufferPool::Page +buffer_pool src/runtime/exec_env.h /^ BufferPool* buffer_pool() { return _buffer_pool; }$/;" f class:doris::ExecEnv +buffer_pool_ src/runtime/buffered_tuple_stream3.h /^ BufferPool* buffer_pool_;$/;" m class:doris::BufferedTupleStream3 +buffer_pool_client_ src/runtime/buffered_tuple_stream3.h /^ BufferPool::ClientHandle* buffer_pool_client_;$/;" m class:doris::BufferedTupleStream3 +buffer_reservation src/runtime/exec_env.h /^ ReservationTracker* buffer_reservation() { return _buffer_reservation; }$/;" f class:doris::ExecEnv +buffer_reservation src/runtime/runtime_state.h /^ ReservationTracker* buffer_reservation() const { $/;" f class:doris::RuntimeState +buffer_size src/runtime/string_buffer.hpp /^ int buffer_size() const {$/;" f class:doris::StringBuffer +buffer_sizes_ src/runtime/bufferpool/buffer_allocator.cc /^ PerSizeLists buffer_sizes_[LOG_MAX_BUFFER_BYTES + 1];$/;" m class:doris::BufferPool::FreeBufferArena file: +buffered_values_ src/util/bit_stream_utils.h /^ uint64_t buffered_values_;$/;" m class:doris::BitReader +buffered_values_ src/util/bit_stream_utils.h /^ uint64_t buffered_values_;$/;" m class:doris::BitWriter +buffered_values_ src/util/rle_encoding.h /^ uint64_t buffered_values_[8];$/;" m class:doris::RleEncoder +buffers_allocated_bytes_ src/runtime/bufferpool/buffer_pool_internal.h /^ int64_t buffers_allocated_bytes_;$/;" m class:doris::BufferPool::Client +build src/exec/es/es_scroll_query.cpp /^std::string ESScrollQueryBuilder::build(const std::map& properties,$/;" f class:doris::ESScrollQueryBuilder +build src/olap/rowset/alpha_rowset_writer.cpp /^RowsetSharedPtr AlphaRowsetWriter::build() {$/;" f class:doris::AlphaRowsetWriter +build src/olap/rowset/beta_rowset_writer.cpp /^RowsetSharedPtr BetaRowsetWriter::build() {$/;" f class:doris::BetaRowsetWriter +build src/runtime/descriptor_helper.h /^ TSlotDescriptor build() {$/;" f class:doris::TSlotDescriptorBuilder +build src/runtime/descriptor_helper.h /^ void build(TDescriptorTableBuilder* tb) {$/;" f class:doris::TTupleDescriptorBuilder +build src/testutil/desc_tbl_builder.cc /^DescriptorTbl* DescriptorTblBuilder::build() {$/;" f class:doris::DescriptorTblBuilder +build src/util/faststring.h /^ OwnedSlice build() {$/;" f class:doris::faststring +build src/util/threadpool.cpp /^Status ThreadPoolBuilder::build(std::unique_ptr* pool) const {$/;" f class:doris::ThreadPoolBuilder +build_clear_scroll_body src/exec/es/es_scroll_query.cpp /^std::string ESScrollQueryBuilder::build_clear_scroll_body(const std::string& scroll_id) {$/;" f class:doris::ESScrollQueryBuilder +build_conjuncts_list src/exec/es_http_scan_node.cpp /^Status EsHttpScanNode::build_conjuncts_list() {$/;" f class:doris::EsHttpScanNode +build_disjuncts_list src/exec/es/es_predicate.cpp /^Status EsPredicate::build_disjuncts_list() {$/;" f class:doris::EsPredicate +build_disjuncts_list src/exec/es/es_predicate.cpp /^Status EsPredicate::build_disjuncts_list(const Expr* conjunct) {$/;" f class:doris::EsPredicate +build_expire_txn_map src/olap/txn_manager.cpp /^void TxnManager::build_expire_txn_map(std::map>* expire_txn_map) {$/;" f class:doris::TxnManager +build_expr_context_list test/exec/es_predicate_test.cpp /^Status EsPredicateTest::build_expr_context_list(std::vector& conjunct_ctxs) {$/;" f class:doris::EsPredicateTest +build_expr_evals src/exec/new_partitioned_hash_table_ir.cc /^ExprContext* const* NewPartitionedHashTableCtx::build_expr_evals() const {$/;" f class:NewPartitionedHashTableCtx +build_expr_evals_ src/exec/new_partitioned_hash_table.h /^ std::vector build_expr_evals_;$/;" m class:doris::NewPartitionedHashTableCtx +build_exprs_ src/exec/new_partitioned_aggregation_node.h /^ std::vector build_exprs_;$/;" m class:doris::NewPartitionedAggregationNode +build_exprs_ src/exec/new_partitioned_hash_table.h /^ const std::vector& build_exprs_;$/;" m class:doris::NewPartitionedHashTableCtx +build_list_debug_string src/exec/cross_join_node.cpp /^std::string CrossJoinNode::build_list_debug_string() {$/;" f class:doris::CrossJoinNode +build_max_key src/olap/row_cursor.cpp /^OLAPStatus RowCursor::build_max_key() {$/;" f class:doris::RowCursor +build_min_key src/olap/row_cursor.cpp /^OLAPStatus RowCursor::build_min_key() {$/;" f class:doris::RowCursor +build_next_scroll_body src/exec/es/es_scroll_query.cpp /^std::string ESScrollQueryBuilder::build_next_scroll_body(const std::string& scroll_id, const std::string& scroll) {$/;" f class:doris::ESScrollQueryBuilder +build_olap_filters src/exec/olap_scan_node.cpp /^Status OlapScanNode::build_olap_filters() {$/;" f class:doris::OlapScanNode +build_scan_key src/exec/olap_scan_node.cpp /^Status OlapScanNode::build_scan_key() {$/;" f class:doris::OlapScanNode +build_segment test/olap/rowset/segment_v2/segment_test.cpp /^ void build_segment(SegmentWriterOptions opts,$/;" f class:doris::segment_v2::SegmentReaderWriterTest +build_side_thread src/exec/blocking_join_node.cpp /^void BlockingJoinNode::build_side_thread(RuntimeState* state, boost::promise* status) {$/;" f class:doris::BlockingJoinNode +build_side_thread src/exec/hash_join_node.cpp /^void HashJoinNode::build_side_thread(RuntimeState* state, boost::promise* status) {$/;" f class:doris::HashJoinNode +build_tablet_report_info src/olap/tablet.cpp /^void Tablet::build_tablet_report_info(TTabletInfo* tablet_info) {$/;" f class:doris::Tablet +build_timer_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* build_timer_;$/;" m class:doris::NewPartitionedAggregationNode +build_tuple src/testutil/desc_tbl_builder.cc /^TTupleDescriptor DescriptorTblBuilder::build_tuple($/;" f class:doris::DescriptorTblBuilder +byte_before_the_zipfile src/util/minizip/unzip.c /^ ZPOS64_T byte_before_the_zipfile;\/* byte before the zipfile, (>0 for sfx)*\/$/;" m struct:__anon33 file: +byte_before_the_zipfile src/util/minizip/unzip.c /^ ZPOS64_T byte_before_the_zipfile;\/* byte before the zipfile, (>0 for sfx)*\/$/;" m struct:__anon34 file: +byte_offset src/runtime/descriptors.h /^ int byte_offset;$/;" m struct:doris::NullIndicatorOffset +byte_offset_ src/util/bit_stream_utils.h /^ int byte_offset_; \/\/ Offset in buffer_$/;" m class:doris::BitReader +byte_offset_ src/util/bit_stream_utils.h /^ int byte_offset_; \/\/ Offset in buffer_$/;" m class:doris::BitWriter +byte_size src/exec/hash_table.h /^ int64_t byte_size() const {$/;" f class:doris::HashTable +byte_size src/exec/partitioned_hash_table.h /^ int64_t byte_size() const { return _total_data_page_size; }$/;" f class:doris::PartitionedHashTable +byte_size src/exprs/expr.cpp /^ int byte_size;$/;" m struct:doris::MemLayoutData file: +byte_size src/runtime/buffered_tuple_stream.h /^ int64_t byte_size() const {$/;" f class:doris::BufferedTupleStream +byte_size src/runtime/buffered_tuple_stream2.h /^ int64_t byte_size() const { return _total_byte_size; }$/;" f class:doris::BufferedTupleStream2 +byte_size src/runtime/buffered_tuple_stream3.h /^ int64_t byte_size() const { return total_byte_size_; }$/;" f class:doris::BufferedTupleStream3 +byte_size src/runtime/descriptors.h /^ int byte_size() const {$/;" f class:doris::TupleDescriptor +byte_swap src/util/bit_util.h /^ static inline int16_t byte_swap(int16_t value) {$/;" f class:doris::BitUtil +byte_swap src/util/bit_util.h /^ static inline int32_t byte_swap(int32_t value) {$/;" f class:doris::BitUtil +byte_swap src/util/bit_util.h /^ static inline int64_t byte_swap(int64_t value) {$/;" f class:doris::BitUtil +byte_swap src/util/bit_util.h /^ static inline uint16_t byte_swap(uint16_t value) {$/;" f class:doris::BitUtil +byte_swap src/util/bit_util.h /^ static inline uint32_t byte_swap(uint32_t value) {$/;" f class:doris::BitUtil +byte_swap src/util/bit_util.h /^ static inline uint64_t byte_swap(uint64_t value) {$/;" f class:doris::BitUtil +byte_swap src/util/bit_util.h /^ static inline void byte_swap(void* dst, void* src, int len) {$/;" f class:doris::BitUtil +bytes src/runtime/bufferpool/buffer_pool_internal.h /^ int64_t bytes() const { return bytes_; }$/;" f class:doris::BufferPool::PageList +bytes_ src/runtime/bufferpool/buffer_pool_internal.h /^ int64_t bytes_;$/;" m class:doris::BufferPool::PageList +bytes_allocated src/runtime/buffered_block_mgr2.cc /^int64_t BufferedBlockMgr2::bytes_allocated() const {$/;" f class:doris::BufferedBlockMgr2 +bytes_appended src/olap/fs/file_block_manager.cpp /^size_t FileWritableBlock::bytes_appended() const {$/;" f class:doris::fs::internal::FileWritableBlock +bytes_in_mem src/runtime/buffered_tuple_stream2.cc /^int64_t BufferedTupleStream2::bytes_in_mem(bool ignore_current) const {$/;" f class:doris::BufferedTupleStream2 +bytes_left src/util/bit_stream_utils.h /^ int bytes_left() { return max_bytes_ - (byte_offset_ + BitUtil::Ceil(bit_offset_, 8)); }$/;" f class:doris::BitReader +bytes_pinned_ src/runtime/buffered_tuple_stream3.h /^ int64_t bytes_pinned_;$/;" m class:doris::BufferedTupleStream3 +bytes_read src/olap/olap_common.h /^ int64_t bytes_read = 0;$/;" m struct:doris::OlapReaderStatistics +bytes_read src/runtime/bufferpool/buffer_pool_counters.h /^ RuntimeProfile::Counter* bytes_read;$/;" m struct:doris::BufferPoolClientCounters +bytes_read src/util/system_metrics.cpp /^ IntLockCounter bytes_read;$/;" m struct:doris::DiskMetrics file: +bytes_read_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* bytes_read_counter() const {$/;" f class:doris::ScanNode +bytes_read_dn_cache src/runtime/disk_io_mgr.cc /^int64_t DiskIoMgr::bytes_read_dn_cache(RequestContext* reader) const {$/;" f class:doris::DiskIoMgr +bytes_read_local src/runtime/disk_io_mgr.cc /^int64_t DiskIoMgr::bytes_read_local(RequestContext* reader) const {$/;" f class:doris::DiskIoMgr +bytes_read_short_circuit src/runtime/disk_io_mgr.cc /^int64_t DiskIoMgr::bytes_read_short_circuit(RequestContext* reader) const {$/;" f class:doris::DiskIoMgr +bytes_read_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::bytes_read_total;$/;" m class:doris::DorisMetrics file: +bytes_read_total src/util/doris_metrics.h /^ static IntCounter bytes_read_total;$/;" m class:doris::DorisMetrics +bytes_remaining src/runtime/buffered_block_mgr.h /^ int bytes_remaining() const {$/;" f class:doris::BufferedBlockMgr::Block +bytes_remaining src/runtime/buffered_block_mgr2.h /^ int bytes_remaining() const {$/;" f class:doris::BufferedBlockMgr2::Block +bytes_to_long_be src/olap/serialize.cpp /^OLAPStatus bytes_to_long_be(ReadOnlyFileStream* stream, int32_t n, int64_t* value) {$/;" f namespace:doris::ser +bytes_used src/gutil/stl_util.h /^ int64* bytes_used() const { return bytes_used_; }$/;" f class:STLCountingAllocator +bytes_used_ src/gutil/stl_util.h /^ int64* bytes_used_;$/;" m class:STLCountingAllocator +bytes_written src/http/action/mini_load.cpp /^ size_t bytes_written = 0;$/;" m struct:doris::MiniLoadAsyncCtx file: +bytes_written src/runtime/bufferpool/buffer_pool_counters.h /^ RuntimeProfile::Counter* bytes_written;$/;" m struct:doris::BufferPoolClientCounters +bytes_written src/util/bit_stream_utils.h /^ int bytes_written() const { return byte_offset_ + BitUtil::Ceil(bit_offset_, 8); }$/;" f class:doris::BitWriter +bytes_written src/util/system_metrics.cpp /^ IntLockCounter bytes_written;$/;" m struct:doris::DiskMetrics file: +bytes_written_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::bytes_written_total;$/;" m class:doris::DorisMetrics file: +bytes_written_total src/util/doris_metrics.h /^ static IntCounter bytes_written_total;$/;" m class:doris::DorisMetrics +bzero src/gutil/port.h /^inline void bzero(void *s, int n) {$/;" f +c_str src/exec/csv_scan_node.cpp /^ char const* c_str() const {$/;" f class:doris::StringRef +cache src/olap/page_cache.h /^ Cache* cache() const { return _cache; }$/;" f class:doris::PageCacheHandle +cached_pages_num src/olap/olap_common.h /^ int64_t cached_pages_num = 0;$/;" m struct:doris::OlapReaderStatistics +calc_base_compaction_score src/olap/tablet.cpp /^const uint32_t Tablet::calc_base_compaction_score() const {$/;" f class:doris::Tablet +calc_cumulative_compaction_score src/olap/tablet.cpp /^const uint32_t Tablet::calc_cumulative_compaction_score() const {$/;" f class:doris::Tablet +calc_daynr src/runtime/datetime_value.cpp /^uint64_t DateTimeValue::calc_daynr(uint32_t year, uint32_t month, uint32_t day) {$/;" f class:doris::DateTimeValue +calc_days_in_year src/runtime/datetime_value.cpp /^static uint32_t calc_days_in_year(uint32_t year) {$/;" f namespace:doris +calc_job_max_load_memory src/runtime/load_channel_mgr.cpp /^static int64_t calc_job_max_load_memory(int64_t mem_limit_in_req, int64_t total_mem_limit) {$/;" f namespace:doris +calc_job_timeout_s src/runtime/load_channel_mgr.cpp /^static int64_t calc_job_timeout_s(int64_t timeout_in_req_s) {$/;" f namespace:doris +calc_missed_versions src/olap/tablet.cpp /^void Tablet::calc_missed_versions(int64_t spec_version, vector* missed_versions) {$/;" f class:doris::Tablet +calc_missed_versions_unlocked src/olap/tablet.cpp /^void Tablet::calc_missed_versions_unlocked(int64_t spec_version,$/;" f class:doris::Tablet +calc_process_max_load_memory src/runtime/load_channel_mgr.cpp /^static int64_t calc_process_max_load_memory(int64_t process_mem_limit) {$/;" f namespace:doris +calc_week src/runtime/datetime_value.cpp /^uint8_t DateTimeValue::calc_week(const DateTimeValue& value, uint8_t mode, uint32_t *year) {$/;" f class:doris::DateTimeValue +calc_weekday src/runtime/datetime_value.cpp /^uint8_t DateTimeValue::calc_weekday(uint64_t day_nr, bool is_sunday_first_day) {$/;" f class:doris::DateTimeValue +calculate_cumulative_point src/olap/tablet.cpp /^OLAPStatus Tablet::calculate_cumulative_point() {$/;" f class:doris::Tablet +calculate_mem_tracker src/runtime/test_env.cc /^int64_t TestEnv::calculate_mem_tracker(int max_buffers, int block_size) {$/;" f class:doris::TestEnv +calculate_metrics src/common/daemon.cpp /^void* calculate_metrics(void* dummy) {$/;" f namespace:doris +call src/util/once.h /^ ReturnType call(Fn fn) {$/;" f class:doris::DorisCallOnce +call_zopen64 src/util/minizip/ioapi.c /^voidpf call_zopen64 (const zlib_filefunc64_32_def* pfilefunc,const void*filename,int mode)$/;" f +call_zseek64 src/util/minizip/ioapi.c /^long call_zseek64 (const zlib_filefunc64_32_def* pfilefunc,voidpf filestream, ZPOS64_T offset, int origin)$/;" f +call_ztell64 src/util/minizip/ioapi.c /^ZPOS64_T call_ztell64 (const zlib_filefunc64_32_def* pfilefunc,voidpf filestream)$/;" f +callback src/runtime/fragment_mgr.cpp /^void FragmentExecState::callback(const Status& status, RuntimeProfile* profile, bool done) {$/;" f class:doris::FragmentExecState +callbacks src/http/web_page_handler.h /^ const std::vector& callbacks() const {$/;" f class:doris::WebPageHandler::PageHandlers +can_do_bloom_filter src/olap/olap_cond.h /^ bool can_do_bloom_filter() const {$/;" f class:doris::CondColumn +can_do_bloom_filter src/olap/olap_cond.h /^ bool can_do_bloom_filter() const {$/;" f struct:doris::Cond +can_do_compaction src/olap/tablet.cpp /^bool Tablet::can_do_compaction() {$/;" f class:doris::Tablet +cancel src/exec/tablet_sink.cpp /^void IndexChannel::cancel() {$/;" f class:doris::stream_load::IndexChannel +cancel src/exec/tablet_sink.cpp /^void NodeChannel::cancel() {$/;" f class:doris::stream_load::NodeChannel +cancel src/olap/delta_writer.cpp /^OLAPStatus DeltaWriter::cancel() {$/;" f class:doris::DeltaWriter +cancel src/olap/memtable_flush_executor.cpp /^void FlushToken::cancel() {$/;" f class:doris::FlushToken +cancel src/olap/task/engine_task.h /^ virtual OLAPStatus cancel() { return OLAP_SUCCESS; }$/;" f class:doris::EngineTask +cancel src/runtime/buffer_control_block.cpp /^Status BufferControlBlock::cancel() {$/;" f class:doris::BufferControlBlock +cancel src/runtime/buffered_block_mgr2.cc /^void BufferedBlockMgr2::cancel() {$/;" f class:doris::BufferedBlockMgr2 +cancel src/runtime/data_stream_mgr.cpp /^void DataStreamMgr::cancel(const TUniqueId& fragment_instance_id) {$/;" f class:doris::DataStreamMgr +cancel src/runtime/data_stream_recvr.cc /^void DataStreamRecvr::SenderQueue::cancel() {$/;" f class:doris::DataStreamRecvr::SenderQueue +cancel src/runtime/disk_io_mgr_reader_context.cc /^void DiskIoMgr::RequestContext::cancel(const Status& status) {$/;" f class:doris::DiskIoMgr::RequestContext +cancel src/runtime/disk_io_mgr_scan_range.cc /^void DiskIoMgr::ScanRange::cancel(const Status& status) {$/;" f class:doris::DiskIoMgr::ScanRange +cancel src/runtime/fragment_mgr.cpp /^Status FragmentExecState::cancel(const PPlanFragmentCancelReason& reason) {$/;" f class:doris::FragmentExecState +cancel src/runtime/fragment_mgr.cpp /^Status FragmentMgr::cancel(const TUniqueId& id, const PPlanFragmentCancelReason& reason) {$/;" f class:doris::FragmentMgr +cancel src/runtime/fragment_mgr.h /^ Status cancel(const TUniqueId& fragment_id) {$/;" f class:doris::FragmentMgr +cancel src/runtime/load_channel.cpp /^Status LoadChannel::cancel() {$/;" f class:doris::LoadChannel +cancel src/runtime/load_channel_mgr.cpp /^Status LoadChannelMgr::cancel(const PTabletWriterCancelRequest& params) {$/;" f class:doris::LoadChannelMgr +cancel src/runtime/message_body_sink.cpp /^void MessageBodyFileSink::cancel() {$/;" f class:doris::MessageBodyFileSink +cancel src/runtime/message_body_sink.h /^ virtual void cancel() { }$/;" f class:doris::MessageBodySink +cancel src/runtime/plan_fragment_executor.cpp /^void PlanFragmentExecutor::cancel() {$/;" f class:doris::PlanFragmentExecutor +cancel src/runtime/result_buffer_mgr.cpp /^Status ResultBufferMgr::cancel(const TUniqueId& query_id) {$/;" f class:doris::ResultBufferMgr +cancel src/runtime/result_queue_mgr.cpp /^Status ResultQueueMgr::cancel(const TUniqueId& fragment_instance_id) {$/;" f class:doris::ResultQueueMgr +cancel src/runtime/routine_load/data_consumer.cpp /^Status KafkaDataConsumer::cancel(StreamLoadContext* ctx) {$/;" f class:doris::KafkaDataConsumer +cancel src/runtime/tablets_channel.cpp /^Status TabletsChannel::cancel() {$/;" f class:doris::TabletsChannel +cancel src/util/scoped_cleanup.h /^ void cancel() { cancelled_ = true; }$/;" f class:doris::ScopedCleanup +cancel test/runtime/fragment_mgr_test.cpp /^void PlanFragmentExecutor::cancel() {$/;" f class:doris::PlanFragmentExecutor +cancel test/runtime/load_channel_mgr_test.cpp /^OLAPStatus DeltaWriter::cancel() {$/;" f class:doris::DeltaWriter +cancel_at_time src/runtime/result_buffer_mgr.cpp /^Status ResultBufferMgr::cancel_at_time(time_t cancel_time, const TUniqueId& query_id) {$/;" f class:doris::ResultBufferMgr +cancel_context src/runtime/disk_io_mgr.cc /^void DiskIoMgr::cancel_context(RequestContext* context, bool wait_for_disks_completion) {$/;" f class:doris::DiskIoMgr +cancel_job src/runtime/etl_job_mgr.cpp /^Status EtlJobMgr::cancel_job(const TUniqueId& id) {$/;" f class:doris::EtlJobMgr +cancel_plan_fragment src/service/backend_service.cpp /^void BackendService::cancel_plan_fragment(TCancelPlanFragmentResult& return_val,$/;" f class:doris::BackendService +cancel_plan_fragment src/service/internal_service.cpp /^void PInternalServiceImpl::cancel_plan_fragment($/;" f class:doris::PInternalServiceImpl +cancel_plan_fragment test/runtime/data_stream_test.cpp /^ virtual void cancel_plan_fragment($/;" f class:doris::DorisTestBackend +cancel_stream src/runtime/data_stream_recvr.cc /^void DataStreamRecvr::cancel_stream() {$/;" f class:doris::DataStreamRecvr +cancel_task src/runtime/export_task_mgr.cpp /^Status ExportTaskMgr::cancel_task(const TUniqueId& id) {$/;" f class:doris::ExportTaskMgr +cancel_thread src/runtime/result_buffer_mgr.cpp /^void ResultBufferMgr::cancel_thread() {$/;" f class:doris::ResultBufferMgr +cancel_thread test/runtime/buffer_control_block_test.cpp /^void* cancel_thread(void* param) {$/;" f namespace:doris +cancel_worker src/runtime/fragment_mgr.cpp /^void FragmentMgr::cancel_worker() {$/;" f class:doris::FragmentMgr +cancelled_ src/util/scoped_cleanup.h /^ bool cancelled_;$/;" m class:doris::ScopedCleanup +canonicalize src/util/file_utils.cpp /^Status FileUtils::canonicalize(const std::string& path, std::string* real_path) {$/;" f class:doris::FileUtils +capacity src/exec/new_partitioned_hash_table.h /^ int ALWAYS_INLINE capacity() const { return capacity_; }$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +capacity src/gutil/strings/stringpiece.h /^ int capacity() const { return length_; }$/;" f class:StringPiece +capacity src/olap/byte_buffer.h /^ inline uint64_t capacity() const {$/;" f class:doris::StorageByteBuffer +capacity src/olap/row_block.h /^ size_t capacity() const { return _capacity; }$/;" f class:doris::RowBlock +capacity src/olap/row_block2.h /^ size_t capacity() const { return _capacity; }$/;" f class:doris::RowBlockV2 +capacity src/runtime/row_batch.h /^ int capacity() const {$/;" f class:doris::RowBatch +capacity src/runtime/vectorized_row_batch.h /^ uint16_t capacity() {$/;" f class:doris::VectorizedRowBatch +capacity src/util/buffer_builder.hpp /^ int capacity() const {$/;" f class:doris::BufferBuilder +capacity src/util/byte_buffer.h /^ size_t capacity;$/;" m struct:doris::ByteBuffer +capacity src/util/faststring.h /^ size_t capacity() const {$/;" f class:doris::faststring +capacity_ src/exec/new_partitioned_hash_table.h /^ int capacity_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +capacity_ src/util/faststring.h /^ size_t capacity_;$/;" m class:doris::faststring +capacity_bytes src/olap/options.h /^ int64_t capacity_bytes;$/;" m struct:doris::StorePath +capture_consistent_rowsets src/olap/tablet.cpp /^OLAPStatus Tablet::capture_consistent_rowsets(const Version& spec_version,$/;" f class:doris::Tablet +capture_consistent_versions src/olap/rowset_graph.cpp /^OLAPStatus RowsetGraph::capture_consistent_versions(const Version& spec_version,$/;" f class:doris::RowsetGraph +capture_consistent_versions src/olap/tablet.cpp /^OLAPStatus Tablet::capture_consistent_versions(const Version& spec_version,$/;" f class:doris::Tablet +capture_rs_readers src/olap/tablet.cpp /^OLAPStatus Tablet::capture_rs_readers(const Version& spec_version,$/;" f class:doris::Tablet +capture_rs_readers src/olap/tablet.cpp /^OLAPStatus Tablet::capture_rs_readers(const vector& version_path,$/;" f class:doris::Tablet +cardinality src/util/bitmap_value.h /^ int64_t cardinality() const {$/;" f class:doris::BitmapValue +cardinality src/util/bitmap_value.h /^ uint64_t cardinality() const {$/;" f class:doris::detail::Roaring64Map +carried_continuations src/util/simdutf8check.h /^ __m128i carried_continuations;$/;" m struct:processed_utf_bytes +carried_continuations src/util/simdutf8check.h /^ __m256i carried_continuations;$/;" m struct:avx_processed_utf_bytes +carryContinuations src/util/simdutf8check.h /^static inline __m128i carryContinuations(__m128i initial_lengths,$/;" f +case_val src/exprs/case_expr.cpp /^ AnyVal* case_val;$/;" m struct:doris::CaseExprState file: +cast_to_date src/runtime/datetime_value.h /^ void cast_to_date() {$/;" f class:doris::DateTimeValue +cast_to_date_val src/exprs/cast_functions.cpp /^DateTimeVal CastFunctions::cast_to_date_val($/;" f class:doris::CastFunctions +cast_to_datetime_val src/exprs/cast_functions.cpp /^DateTimeVal CastFunctions::cast_to_datetime_val($/;" f class:doris::CastFunctions +cast_to_datetime_val src/exprs/decimal_operators.cpp /^DateTimeVal DecimalOperators::cast_to_datetime_val($/;" f class:doris::DecimalOperators +cast_to_datetime_val src/exprs/decimalv2_operators.cpp /^DateTimeVal DecimalV2Operators::cast_to_datetime_val($/;" f class:doris::DecimalV2Operators +cast_to_datetime_val src/exprs/time_operators.cpp /^DateTimeVal TimeOperators::cast_to_datetime_val($/;" f class:doris::TimeOperators +cast_to_decimal_val src/exprs/decimal_operators.cpp /^DecimalVal DecimalOperators::cast_to_decimal_val($/;" f class:doris::DecimalOperators +cast_to_decimal_val src/exprs/decimalv2_operators.cpp /^DecimalVal DecimalV2Operators::cast_to_decimal_val($/;" f class:doris::DecimalV2Operators +cast_to_decimal_val src/exprs/time_operators.cpp /^DecimalVal TimeOperators::cast_to_decimal_val($/;" f class:doris::TimeOperators +cast_to_decimalv2_val src/exprs/decimalv2_operators.cpp /^DecimalV2Val DecimalV2Operators::cast_to_decimalv2_val($/;" f class:doris::DecimalV2Operators +cast_to_string src/exec/olap_common.cpp /^std::string cast_to_string(__int128 value) {$/;" f namespace:doris +cast_to_string src/exec/olap_common.h /^std::string cast_to_string(T value) {$/;" f namespace:doris +cast_to_string_val src/exprs/cast_functions.cpp /^StringVal CastFunctions::cast_to_string_val(FunctionContext* ctx, const DateTimeVal& val) {$/;" f class:doris::CastFunctions +cast_to_string_val src/exprs/cast_functions.cpp /^StringVal CastFunctions::cast_to_string_val(FunctionContext* ctx, const LargeIntVal& val) {$/;" f class:doris::CastFunctions +cast_to_string_val src/exprs/cast_functions.cpp /^StringVal CastFunctions::cast_to_string_val(FunctionContext* ctx, const StringVal& val) {$/;" f class:doris::CastFunctions +cast_to_string_val src/exprs/cast_functions.cpp /^StringVal CastFunctions::cast_to_string_val(FunctionContext* ctx, const TinyIntVal& val) {$/;" f class:doris::CastFunctions +cast_to_string_val src/exprs/decimal_operators.cpp /^StringVal DecimalOperators::cast_to_string_val($/;" f class:doris::DecimalOperators +cast_to_string_val src/exprs/decimalv2_operators.cpp /^StringVal DecimalV2Operators::cast_to_string_val($/;" f class:doris::DecimalV2Operators +cast_to_string_val src/exprs/time_operators.cpp /^StringVal TimeOperators::cast_to_string_val($/;" f class:doris::TimeOperators +cast_to_time src/runtime/datetime_value.h /^ void cast_to_time() {$/;" f class:doris::DateTimeValue +category src/util/thread.cpp /^ const std::string& category() const { return _category; }$/;" f class:doris::ThreadMgr::ThreadDescriptor +category src/util/thread.cpp /^const std::string& Thread::category() const {$/;" f class:doris::Thread +cdf src/util/tdigest.h /^ Value cdf(Value x) {$/;" f class:doris::TDigest +cdfProcessed src/util/tdigest.h /^ Value cdfProcessed(Value x) const {$/;" f class:doris::TDigest +ceil src/util/bit_util.h /^ static inline int64_t ceil(int64_t value, int64_t divisor) {$/;" f class:doris::BitUtil +cell src/olap/column_block.h /^inline ColumnBlockCell ColumnBlock::cell(size_t idx) const {$/;" f class:doris::ColumnBlock +cell src/olap/row.h /^ RowCursorCell cell(uint32_t cid) const {$/;" f struct:doris::ContiguousRow +cell src/olap/row_block2.h /^ ColumnBlockCell cell(uint32_t cid) const {$/;" f class:doris::RowBlockRow +cell src/olap/row_cursor.h /^ RowCursorCell cell(uint32_t cid) const {$/;" f class:doris::RowCursor +cell src/olap/rowset/segment_v2/bitshuffle_page.h /^ CppType cell(int idx) const {$/;" f class:doris::segment_v2::BitshufflePageBuilder +cell_ptr src/olap/column_block.h /^ const uint8_t* cell_ptr() const { return _block.cell_ptr(_idx); }$/;" f struct:doris::ColumnBlockCell +cell_ptr src/olap/column_block.h /^ const uint8_t* cell_ptr(size_t idx) const { return _data + idx * _type_info->size(); }$/;" f class:doris::ColumnBlock +cell_ptr src/olap/row_block2.h /^ const uint8_t* cell_ptr(size_t col_idx) const {$/;" f class:doris::RowBlockRow +cell_ptr src/olap/row_cursor.h /^ char* cell_ptr(uint32_t cid) const {$/;" f class:doris::RowCursor +cell_ptr src/olap/row_cursor_cell.h /^ const void* cell_ptr() const { return (char*)_ptr + 1; }$/;" f struct:doris::RowCursorCell +cell_ptr src/olap/wrapper_field.h /^ void* cell_ptr() const { return _field_buf + 1; }$/;" f class:doris::WrapperField +central_pos src/util/minizip/unzip.c /^ ZPOS64_T central_pos; \/* position of the beginning of the central dir*\/$/;" m struct:__anon34 file: +cgroups_mgr src/runtime/exec_env.h /^ CgroupsMgr* cgroups_mgr() { return _cgroups_mgr; }$/;" f class:doris::ExecEnv +change test/olap/skiplist_test.cpp /^ void change(ReaderState s) {$/;" f class:doris::TestState +change_row_block src/olap/schema_change.cpp /^bool RowBlockChanger::change_row_block($/;" f class:doris::RowBlockChanger +char2unsigned src/gutil/hash/hash.cc /^static inline uint32 char2unsigned(char c) {$/;" f file: +char2unsigned src/gutil/hash/jenkins.cc /^static inline uint32 char2unsigned(char c) {$/;" f file: +char2unsigned64 src/gutil/hash/jenkins.cc /^static inline uint64 char2unsigned64(char c) {$/;" f file: +char32 src/gutil/integral_types.h /^typedef signed int char32;$/;" t +char_val test/runtime/dpp_writer_test.cpp /^ std::string char_val;$/;" m struct:doris::TestDataTuple file: +char_val test/runtime/mysql_table_writer_test.cpp /^ std::string char_val;$/;" m struct:doris::TestDataTuple file: +charge src/olap/lru_cache.h /^ size_t charge;$/;" m struct:doris::CachePriority::LRUHandle +charntorune src/gutil/utf/rune.c /^charntorune(Rune *rune, const char *str, int length)$/;" f +charset src/exec/schema_scanner/schema_charsets_scanner.h /^ const char *charset;$/;" m struct:doris::SchemaCharsetsScanner::CharsetStruct +charset src/exec/schema_scanner/schema_collations_scanner.h /^ const char *charset;$/;" m struct:doris::SchemaCollationsScanner::CollationStruct +chartorune src/gutil/utf/rune.c /^chartorune(Rune *rune, const char *str)$/;" f +chdir src/gutil/port.h 844;" d +check src/olap/rowset/segment_group.cpp /^bool SegmentGroup::check() {$/;" f class:doris::SegmentGroup +checkContinuations src/util/simdutf8check.h /^static inline void checkContinuations(__m128i initial_lengths, __m128i carries,$/;" f +checkFirstContinuationMax src/util/simdutf8check.h /^static inline void checkFirstContinuationMax(__m128i current_bytes,$/;" f +checkOverlong src/util/simdutf8check.h /^static inline void checkOverlong(__m128i current_bytes,$/;" f +checkSmallerThan0xF4 src/util/simdutf8check.h /^static inline void checkSmallerThan0xF4(__m128i current_bytes,$/;" f +checkUTF8Bytes src/util/simdutf8check.h /^checkUTF8Bytes(__m128i current_bytes, struct processed_utf_bytes *previous,$/;" f +checkWeights src/util/tdigest.h /^ inline int checkWeights() { return checkWeights(_processed, _processed_weight); }$/;" f class:doris::TDigest +checkWeights src/util/tdigest.h /^ size_t checkWeights(const std::vector& sorted, Value total) {$/;" f class:doris::TDigest +check_allocations_empty src/udf/udf.cpp /^bool FunctionContextImpl::check_allocations_empty() {$/;" f class:doris::FunctionContextImpl +check_and_resize src/exec/partitioned_hash_table.cc /^bool PartitionedHashTable::check_and_resize($/;" f class:doris::PartitionedHashTable +check_and_resize_hash_partitions src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::check_and_resize_hash_partitions(int num_rows,$/;" f class:doris::PartitionedAggregationNode +check_and_write_text_slot src/exec/csv_scan_node.cpp /^bool CsvScanNode::check_and_write_text_slot($/;" f class:doris::CsvScanNode +check_auth src/http/action/mini_load.cpp /^Status MiniLoadAction::check_auth($/;" f class:doris::MiniLoadAction +check_condition_valid src/olap/delete_handler.cpp /^OLAPStatus DeleteConditionHandler::check_condition_valid($/;" f class:doris::DeleteConditionHandler +check_correctness src/olap/compaction.cpp /^OLAPStatus Compaction::check_correctness(const Merger::Statistics& stats) {$/;" f class:doris::Compaction +check_date src/runtime/datetime_value.cpp /^bool DateTimeValue::check_date() const {$/;" f class:doris::DateTimeValue +check_decimal_input src/exec/broker_scanner.cpp /^bool BrokerScanner::check_decimal_input($/;" f class:doris::BrokerScanner +check_decimal_input src/exec/csv_scan_node.cpp /^bool CsvScanNode::check_decimal_input($/;" f class:doris::CsvScanNode +check_es_query src/exec/es/es_query_builder.cpp /^Status BooleanQueryBuilder::check_es_query(const ExtFunction& extFunction) {$/;" f class:doris::BooleanQueryBuilder +check_exist src/util/file_utils.cpp /^bool FileUtils::check_exist(const std::string& path) {$/;" f class:doris::FileUtils +check_exist src/util/file_utils.cpp /^bool FileUtils::check_exist(const std::string& path, Env* env) {$/;" f class:doris::FileUtils +check_format src/exprs/timestamp_functions.cpp /^bool TimestampFunctions::check_format(const StringVal& format, DateTimeValue& t) {$/;" f class:doris::TimestampFunctions +check_hll_function src/exec/csv_scan_node.cpp /^bool CsvScanNode::check_hll_function(TMiniLoadEtlFunction& function) {$/;" f class:doris::CsvScanNode +check_integrity src/runtime/mem_pool.cpp /^bool MemPool::check_integrity(bool check_current_chunk_empty) {$/;" f class:doris::MemPool +check_left_conjuncts src/exec/es_scan_node.cpp /^bool EsScanNode::check_left_conjuncts(Expr* conjunct) {$/;" f class:doris::EsScanNode +check_local_allocations_empty src/udf/udf.cpp /^bool FunctionContextImpl::check_local_allocations_empty() {$/;" f class:doris::FunctionContextImpl +check_log_path_is_allowed src/http/download_action.cpp /^Status DownloadAction::check_log_path_is_allowed(const std::string& file_path) {$/;" f class:doris::DownloadAction +check_not_pool_thread_unlocked src/util/threadpool.cpp /^void ThreadPool::check_not_pool_thread_unlocked() {$/;" f class:doris::ThreadPool +check_path src/olap/rowset/alpha_rowset.cpp /^bool AlphaRowset::check_path(const std::string& path) {$/;" f class:doris::AlphaRowset +check_path src/olap/rowset/beta_rowset.cpp /^bool BetaRowset::check_path(const std::string& path) {$/;" f class:doris::BetaRowset +check_path src/olap/tablet.cpp /^bool Tablet::check_path(const std::string& path_to_check) {$/;" f class:doris::Tablet +check_path_is_allowed src/http/download_action.cpp /^Status DownloadAction::check_path_is_allowed(const std::string& file_path) {$/;" f class:doris::DownloadAction +check_query_state src/runtime/runtime_state.cpp /^Status RuntimeState::check_query_state(const std::string& msg) {$/;" f class:doris::RuntimeState +check_range src/runtime/datetime_value.cpp /^bool DateTimeValue::check_range() const {$/;" f class:doris::DateTimeValue +check_receivers test/runtime/data_stream_test.cpp /^ void check_receivers(TPartitionType::type stream_type, int num_senders) {$/;" f class:doris::DataStreamTest +check_request src/http/action/mini_load.cpp /^static Status check_request(HttpRequest* req) {$/;" f namespace:doris +check_rowset_id src/olap/tablet.cpp /^bool Tablet::check_rowset_id(const RowsetId& rowset_id) {$/;" f class:doris::Tablet +check_rowset_id_in_unused_rowsets src/olap/storage_engine.cpp /^bool StorageEngine::check_rowset_id_in_unused_rowsets(const RowsetId& rowset_id) {$/;" f class:doris::StorageEngine +check_rowset_meta src/olap/rowset/rowset_meta_manager.cpp /^bool RowsetMetaManager::check_rowset_meta(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id) {$/;" f class:doris::RowsetMetaManager +check_senders test/runtime/data_stream_test.cpp /^ void check_senders() {$/;" f class:doris::DataStreamTest +check_sse_support src/runtime/disk_io_mgr.cc /^static void check_sse_support() {$/;" f namespace:doris +check_tablet_id_exist src/olap/tablet_manager.cpp /^bool TabletManager::check_tablet_id_exist(TTabletId tablet_id) {$/;" f class:doris::TabletManager +check_token src/http/download_action.cpp /^Status DownloadAction::check_token(HttpRequest *req) {$/;" f class:doris::DownloadAction +check_valid_allocation src/runtime/free_pool.hpp /^ void check_valid_allocation(FreeListNode* computed_list_ptr) {$/;" f class:doris::FreePool +check_validation src/olap/reader.h /^ void check_validation() const {$/;" f struct:doris::ReaderParams +check_version_continuity src/olap/compaction.cpp /^OLAPStatus Compaction::check_version_continuity(const vector& rowsets) {$/;" f class:doris::Compaction +check_version_exist src/olap/tablet.cpp /^bool Tablet::check_version_exist(const Version& version) const {$/;" f class:doris::Tablet +check_version_integrity src/olap/tablet.cpp /^OLAPStatus Tablet::check_version_integrity(const Version& version) {$/;" f class:doris::Tablet +check_word src/runtime/datetime_value.cpp /^static int check_word(const char* lib[], const char* str, const char* end, const char** endptr) {$/;" f namespace:doris +checksum src/exec/lzo_decompressor.cpp /^Status LzopDecompressor::checksum(LzoChecksum type, const std::string& source,$/;" f class:doris::LzopDecompressor +checksum src/olap/file_helper.h /^ uint32_t checksum() const {$/;" f class:doris::FileHeader +checksum src/olap/file_helper.h /^ uint32_t checksum;$/;" m struct:doris::_FixedFileHeader +checksum src/olap/file_helper.h /^ uint32_t checksum;$/;" m struct:doris::_FixedFileHeaderV2 +checksum src/olap/out_stream.h /^ uint32_t checksum; \/\/ 32位校验值$/;" m struct:doris::StreamHead +checksum src/olap/push_handler.h /^ uint32_t checksum() const {$/;" f class:doris::BinaryFile +checksum src/olap/row_block.h /^ uint32_t checksum;$/;" m struct:doris::RowBlockInfo +checksum src/runtime/user_function_cache.cpp /^ std::string checksum;$/;" m struct:doris::UserFunctionCacheEntry file: +child src/exec/exec_node.h /^ ExecNode* child(int i) {$/;" f class:doris::ExecNode +child_batch_ src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr child_batch_;$/;" m class:doris::NewPartitionedAggregationNode +child_eos_ src/exec/new_partitioned_aggregation_node.h /^ bool child_eos_;$/;" m class:doris::NewPartitionedAggregationNode +child_reservations_ src/runtime/bufferpool/reservation_tracker.h /^ int64_t child_reservations_;$/;" m class:doris::ReservationTracker +children src/exprs/expr.h /^ const std::vector& children() const {$/;" f class:doris::Expr +children src/runtime/types.h /^ std::vector children;$/;" m struct:doris::TypeDescriptor +choose_update_or_merge src/exprs/agg_fn_evaluator.cpp /^void AggFnEvaluator::choose_update_or_merge($/;" f class:doris::AggFnEvaluator +chunk src/runtime/mem_pool.h /^ Chunk chunk;$/;" m struct:doris::MemPool::ChunkInfo +chunks_ src/runtime/mem_pool.h /^ std::vector chunks_;$/;" m class:doris::MemPool +claim_buffer_reservation src/exec/exec_node.cpp /^Status ExecNode::claim_buffer_reservation(RuntimeState* state) {$/;" f class:doris::ExecNode +clean src/runtime/load_path_mgr.cpp /^void LoadPathMgr::clean() {$/;" f class:doris::LoadPathMgr +clean_error_log src/runtime/load_path_mgr.cpp /^void LoadPathMgr::clean_error_log() {$/;" f class:doris::LoadPathMgr +clean_one_path src/runtime/load_path_mgr.cpp /^void LoadPathMgr::clean_one_path(const std::string& path) {$/;" f class:doris::LoadPathMgr +clean_page_bytes_limit_ src/runtime/bufferpool/buffer_allocator.h /^ const int64_t clean_page_bytes_limit_;$/;" m struct:doris::BufferPool::BufferAllocator +clean_page_bytes_remaining_ src/runtime/bufferpool/buffer_allocator.h /^ AtomicInt64 clean_page_bytes_remaining_;$/;" m struct:doris::BufferPool::BufferAllocator +clean_pages src/runtime/bufferpool/buffer_allocator.cc /^ InternalList clean_pages;$/;" m struct:doris::BufferPool::FreeBufferArena::PerSizeLists file: +clean_up src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::Partition::clean_up() {$/;" f class:doris::PartitionedAggregationNode::Partition +cleaner src/runtime/load_path_mgr.cpp /^void* LoadPathMgr::cleaner(void* param) {$/;" f class:doris::LoadPathMgr +cleanup_hash_tbl src/exec/partitioned_aggregation_node.cc /^void PartitionedAggregationNode::cleanup_hash_tbl($/;" f class:doris::PartitionedAggregationNode +cleanup_queue_ src/exec/exec_node.h /^ std::list cleanup_queue_;$/;" m class:doris::ExecNode::RowBatchQueue +cleanup_queued_buffers src/runtime/disk_io_mgr_scan_range.cc /^void DiskIoMgr::ScanRange::cleanup_queued_buffers() {$/;" f class:doris::DiskIoMgr::ScanRange +clear src/common/object_pool.h /^ void clear() {$/;" f class:doris::ObjectPool +clear src/exec/olap_common.h /^ void clear() {$/;" f class:doris::ColumnValueRange +clear src/exec/olap_common.h /^ void clear() {$/;" f class:doris::OlapScanKeys +clear src/gutil/strings/stringpiece.h /^ void clear() {$/;" f class:StringPiece +clear src/olap/bloom_filter.hpp /^ void clear() {$/;" f class:doris::BitSet +clear src/olap/bloom_filter.hpp /^ void clear() {$/;" f class:doris::BloomFilter +clear src/olap/lru_cache.h /^ void clear() {$/;" f class:doris::CacheKey +clear src/olap/reader.cpp /^void CollectIterator::clear() {$/;" f class:doris::CollectIterator +clear src/olap/row_block.cpp /^void RowBlock::clear() {$/;" f class:doris::RowBlock +clear src/olap/row_block2.h /^ void clear() {$/;" f class:doris::RowBlockV2 +clear src/olap/rowset/segment_v2/row_ranges.h /^ void clear() {$/;" f class:doris::segment_v2::RowRanges +clear src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::clear() {$/;" f class:doris::StorageEngine +clear src/runtime/mem_pool.cpp /^void MemPool::clear() {$/;" f class:doris::MemPool +clear src/runtime/query_statistics.h /^ void clear() {$/;" f class:doris::QueryStatistics +clear src/runtime/row_batch.cpp /^void RowBatch::clear() {$/;" f class:doris::RowBatch +clear src/runtime/string_buffer.hpp /^ void clear() {$/;" f class:doris::StringBuffer +clear src/runtime/vectorized_row_batch.h /^ inline void clear() {$/;" f class:doris::VectorizedRowBatch +clear src/util/bitmap_value.h /^ void clear() { roarings.clear(); }$/;" f class:doris::detail::Roaring64Map +clear src/util/faststring.h /^ void clear() {$/;" f class:doris::faststring +clear src/util/frame_of_reference_coding.h /^ void clear() {$/;" f class:doris::ForEncoder +clear src/util/internal_queue.h /^ void clear() {$/;" f class:doris::InternalQueueBase +clear src/util/slice.h /^ void clear() {$/;" f struct:doris::Slice +clear_bit src/olap/selection_vector.h /^ void clear_bit(size_t row) {$/;" f class:doris::SelectionVector +clear_bit src/olap/selection_vector.h /^ void clear_bit(size_t row_idx) {$/;" f class:doris::SelectionVectorView +clear_bits src/olap/selection_vector.h /^ void clear_bits(size_t nrows) {$/;" f class:doris::SelectionVectorView +clear_error_msg src/exprs/expr_context.cpp /^void ExprContext::clear_error_msg() {$/;" f class:doris::ExprContext +clear_error_msg src/udf/udf.cpp /^void FunctionContext::clear_error_msg() {$/;" f class:doris_udf::FunctionContext +clear_reservations src/runtime/buffered_block_mgr2.cc /^void BufferedBlockMgr2::clear_reservations(Client* client) {$/;" f class:doris::BufferedBlockMgr2 +clear_row src/runtime/row_batch.h /^ void clear_row(TupleRow* row) {$/;" f class:doris::RowBatch +clear_scan_context src/runtime/external_scan_context_mgr.cpp /^Status ExternalScanContextMgr::clear_scan_context(const std::string& context_id) {$/;" f class:doris::ExternalScanContextMgr +clear_scratch_dir test/runtime/buffered_block_mgr2_test.cpp /^static int clear_scratch_dir() {$/;" f namespace:doris +clear_segment_group src/olap/rowset/alpha_rowset_meta.cpp /^void AlphaRowsetMeta::clear_segment_group() {$/;" f class:doris::AlphaRowsetMeta +clear_tablets src/olap/data_dir.cpp /^void DataDir::clear_tablets(std::vector* tablet_infos) {$/;" f class:doris::DataDir +clear_transaction_task src/olap/storage_engine.cpp /^void StorageEngine::clear_transaction_task(const TTransactionId transaction_id) {$/;" f class:doris::StorageEngine +clear_transaction_task src/olap/storage_engine.cpp /^void StorageEngine::clear_transaction_task(const TTransactionId transaction_id,$/;" f class:doris::StorageEngine +client src/runtime/bufferpool/buffer_pool_internal.h /^ Client* const client;$/;" m class:doris::BufferPool::Page +client src/runtime/row_batch.h /^ BufferPool::ClientHandle* client;$/;" m struct:doris::RowBatch::BufferInfo +client_ src/runtime/bufferpool/buffer_pool.h /^ ClientHandle* client_;$/;" m class:doris::BufferPool::PageHandle +client_ src/runtime/bufferpool/buffer_pool.h /^ const ClientHandle* client_;$/;" m class:doris::BufferPool::BufferHandle +client_ src/runtime/bufferpool/suballocator.h /^ BufferPool::ClientHandle* client_;$/;" m class:doris::Suballocator +client_cache src/exec/broker_reader.cpp /^inline BrokerServiceClientCache* client_cache(ExecEnv* env) {$/;" f namespace:doris +client_cache src/exec/broker_writer.cpp /^inline BrokerServiceClientCache* client_cache(ExecEnv* env) {$/;" f namespace:doris +client_cache src/runtime/exec_env.h /^ ClientCache* client_cache() { return _backend_client_cache; }$/;" f class:doris::ExecEnv +client_cache src/runtime/snapshot_loader.cpp /^inline BrokerServiceClientCache* client_cache(ExecEnv* env) {$/;" f namespace:doris +client_factory src/runtime/client_cache.h /^ void** client_key) > client_factory;$/;" t class:doris::ClientCacheHelper +client_flag src/exec/mysql_scanner.h /^ unsigned long client_flag;$/;" m struct:doris::MysqlScannerParam +client_id src/exec/broker_reader.cpp /^inline const std::string& client_id(ExecEnv* env, const TNetworkAddress& addr) {$/;" f namespace:doris +client_id src/exec/broker_writer.cpp /^inline const std::string& client_id(ExecEnv* env, const TNetworkAddress& addr) {$/;" f namespace:doris +client_id src/runtime/snapshot_loader.cpp /^inline const std::string& client_id(ExecEnv* env, const TNetworkAddress& addr) {$/;" f namespace:doris +clock_name_ src/util/os_info.cpp /^std::string OsInfo::clock_name_ =$/;" m class:doris::OsInfo file: +clock_name_ src/util/os_info.h /^ static std::string clock_name_;$/;" m class:doris::OsInfo +clone src/exprs/agg_fn.h /^ Expr* clone(ObjectPool* pool) const {$/;" f class:doris::AggFn +clone src/exprs/expr_context.cpp /^Status ExprContext::clone(RuntimeState* state, ExprContext** new_ctx) {$/;" f class:doris::ExprContext +clone src/exprs/expr_context.cpp /^Status ExprContext::clone(RuntimeState* state, ExprContext** new_ctx, Expr* root) {$/;" f class:doris::ExprContext +clone src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(clone)(int (*fn)(void *), void *child_stack,$/;" f +clone src/olap/field.h /^ virtual Field* clone() const {$/;" f class:doris::Field +clone src/udf/udf.cpp /^FunctionContext* FunctionContextImpl::clone(MemPool* pool) {$/;" f class:doris::FunctionContextImpl +clone_and_append src/common/status.cpp /^Status Status::clone_and_append(const Slice& msg) const {$/;" f class:doris::Status +clone_and_prepend src/common/status.cpp /^Status Status::clone_and_prepend(const Slice& msg) const {$/;" f class:doris::Status +clone_if_not_exists src/exprs/expr.cpp /^Status Expr::clone_if_not_exists($/;" f class:doris::Expr +clone_requests_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::clone_requests_failed;$/;" m class:doris::DorisMetrics file: +clone_requests_failed src/util/doris_metrics.h /^ static IntCounter clone_requests_failed;$/;" m class:doris::DorisMetrics +clone_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::clone_requests_total;$/;" m class:doris::DorisMetrics file: +clone_requests_total src/util/doris_metrics.h /^ static IntCounter clone_requests_total;$/;" m class:doris::DorisMetrics +close src/exec/aggregation_node.cpp /^Status AggregationNode::close(RuntimeState* state) {$/;" f class:doris::AggregationNode +close src/exec/analytic_eval_node.cpp /^Status AnalyticEvalNode::close(RuntimeState* state) {$/;" f class:doris::AnalyticEvalNode +close src/exec/assert_num_rows_node.cpp /^Status AssertNumRowsNode::close(RuntimeState* state) {$/;" f class:doris::AssertNumRowsNode +close src/exec/blocking_join_node.cpp /^Status BlockingJoinNode::close(RuntimeState* state) {$/;" f class:doris::BlockingJoinNode +close src/exec/broker_reader.cpp /^void BrokerReader::close() {$/;" f class:doris::BrokerReader +close src/exec/broker_scan_node.cpp /^Status BrokerScanNode::close(RuntimeState* state) {$/;" f class:doris::BrokerScanNode +close src/exec/broker_scanner.cpp /^void BrokerScanner::close() {$/;" f class:doris::BrokerScanner +close src/exec/broker_writer.cpp /^Status BrokerWriter::close() {$/;" f class:doris::BrokerWriter +close src/exec/cross_join_node.cpp /^Status CrossJoinNode::close(RuntimeState* state) {$/;" f class:doris::CrossJoinNode +close src/exec/csv_scan_node.cpp /^Status CsvScanNode::close(RuntimeState* state) {$/;" f class:doris::CsvScanNode +close src/exec/data_sink.h /^ virtual Status close(RuntimeState* state, Status exec_status) {$/;" f class:doris::DataSink +close src/exec/es/es_scan_reader.cpp /^Status ESScanReader::close() {$/;" f class:doris::ESScanReader +close src/exec/es_http_scan_node.cpp /^Status EsHttpScanNode::close(RuntimeState* state) {$/;" f class:doris::EsHttpScanNode +close src/exec/es_http_scanner.cpp /^void EsHttpScanner::close() {$/;" f class:doris::EsHttpScanner +close src/exec/es_scan_node.cpp /^Status EsScanNode::close(RuntimeState* state) {$/;" f class:doris::EsScanNode +close src/exec/exchange_node.cpp /^Status ExchangeNode::close(RuntimeState* state) {$/;" f class:doris::ExchangeNode +close src/exec/exec_node.cpp /^Status ExecNode::close(RuntimeState* state) {$/;" f class:doris::ExecNode +close src/exec/hash_join_node.cpp /^Status HashJoinNode::close(RuntimeState* state) {$/;" f class:doris::HashJoinNode +close src/exec/hash_table.cpp /^void HashTable::close() {$/;" f class:doris::HashTable +close src/exec/intersect_node.cpp /^Status IntersectNode::close(RuntimeState* state) {$/;" f class:doris::IntersectNode +close src/exec/kudu_scan_node.cpp /^Status KuduScanNode::close(RuntimeState* state) {$/;" f class:doris::KuduScanNode +close src/exec/kudu_scanner.cpp /^void KuduScanner::close() {$/;" f class:doris::KuduScanner +close src/exec/local_file_reader.cpp /^void LocalFileReader::close() {$/;" f class:doris::LocalFileReader +close src/exec/local_file_writer.cpp /^Status LocalFileWriter::close() {$/;" f class:doris::LocalFileWriter +close src/exec/merge_join_node.cpp /^Status MergeJoinNode::close(RuntimeState* state) {$/;" f class:doris::MergeJoinNode +close src/exec/merge_node.cpp /^Status MergeNode::close(RuntimeState* state) {$/;" f class:doris::MergeNode +close src/exec/mysql_scan_node.cpp /^Status MysqlScanNode::close(RuntimeState* state) {$/;" f class:doris::MysqlScanNode +close src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::close(RuntimeState* state) {$/;" f class:doris::NewPartitionedAggregationNode +close src/exec/olap_rewrite_node.cpp /^Status OlapRewriteNode::close(RuntimeState* state) {$/;" f class:doris::OlapRewriteNode +close src/exec/olap_scan_node.cpp /^Status OlapScanNode::close(RuntimeState* state) {$/;" f class:doris::OlapScanNode +close src/exec/olap_scanner.cpp /^Status OlapScanner::close(RuntimeState* state) {$/;" f class:doris::OlapScanner +close src/exec/orc_scanner.cpp /^void ORCScanner::close() {$/;" f class:doris::ORCScanner +close src/exec/parquet_reader.cpp /^void ParquetReaderWrap::close() {$/;" f class:doris::ParquetReaderWrap +close src/exec/parquet_scanner.cpp /^void ParquetScanner::close() {$/;" f class:doris::ParquetScanner +close src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::close(RuntimeState* state) {$/;" f class:doris::PartitionedAggregationNode +close src/exec/partitioned_aggregation_node.cc /^void PartitionedAggregationNode::Partition::close(bool finalize_rows) {$/;" f class:doris::PartitionedAggregationNode::Partition +close src/exec/partitioned_hash_table.cc /^void PartitionedHashTable::close() {$/;" f class:doris::PartitionedHashTable +close src/exec/partitioned_hash_table.cc /^void PartitionedHashTableCtx::close() {$/;" f class:doris::PartitionedHashTableCtx +close src/exec/pl_task_root.cpp /^Status ExchangeNode::close(RuntimeState* state) {$/;" f class:doris::ExchangeNode +close src/exec/plain_text_line_reader.cpp /^void PlainTextLineReader::close() {$/;" f class:doris::PlainTextLineReader +close src/exec/pre_aggregation_node.cpp /^Status PreAggregationNode::close(RuntimeState* state) {$/;" f class:doris::PreAggregationNode +close src/exec/repeat_node.cpp /^Status RepeatNode::close(RuntimeState* state) {$/;" f class:doris::RepeatNode +close src/exec/schema_scan_node.cpp /^Status SchemaScanNode::close(RuntimeState* state) {$/;" f class:doris::SchemaScanNode +close src/exec/select_node.cpp /^Status SelectNode::close(RuntimeState* state) {$/;" f class:doris::SelectNode +close src/exec/sort_exec_exprs.cpp /^void SortExecExprs::close(RuntimeState* state) {$/;" f class:doris::SortExecExprs +close src/exec/sort_node.cpp /^Status SortNode::close(RuntimeState* state) {$/;" f class:doris::SortNode +close src/exec/spill_sort_node.cc /^Status SpillSortNode::close(RuntimeState* state) {$/;" f class:doris::SpillSortNode +close src/exec/tablet_sink.cpp /^Status IndexChannel::close(RuntimeState* state) {$/;" f class:doris::stream_load::IndexChannel +close src/exec/tablet_sink.cpp /^Status NodeChannel::close(RuntimeState* state) {$/;" f class:doris::stream_load::NodeChannel +close src/exec/tablet_sink.cpp /^Status OlapTableSink::close(RuntimeState* state, Status close_status) {$/;" f class:doris::stream_load::OlapTableSink +close src/exec/topn_node.cpp /^Status TopNNode::close(RuntimeState* state) {$/;" f class:doris::TopNNode +close src/exec/union_node.cpp /^Status UnionNode::close(RuntimeState* state) {$/;" f class:doris::UnionNode +close src/exprs/agg_fn_evaluator.cpp /^void AggFnEvaluator::close(RuntimeState* state) {$/;" f class:doris::AggFnEvaluator +close src/exprs/case_expr.cpp /^void CaseExpr::close($/;" f class:doris::CaseExpr +close src/exprs/expr.cpp /^void Expr::close($/;" f class:doris::Expr +close src/exprs/expr.cpp /^void Expr::close() {$/;" f class:doris::Expr +close src/exprs/expr.cpp /^void Expr::close(const std::vector& ctxs, RuntimeState* state) {$/;" f class:doris::Expr +close src/exprs/expr.cpp /^void Expr::close(const vector& exprs) {$/;" f class:doris::Expr +close src/exprs/expr.h /^ void close($/;" f class:doris::Expr +close src/exprs/expr_context.cpp /^void ExprContext::close(RuntimeState* state) {$/;" f class:doris::ExprContext +close src/exprs/scalar_fn_call.cpp /^void ScalarFnCall::close($/;" f class:doris::ScalarFnCall +close src/olap/delta_writer.cpp /^OLAPStatus DeltaWriter::close() {$/;" f class:doris::DeltaWriter +close src/olap/file_helper.cpp /^OLAPStatus FileHandler::close() {$/;" f class:doris::FileHandler +close src/olap/file_helper.cpp /^OLAPStatus FileHandlerWithBuf::close() {$/;" f class:doris::FileHandlerWithBuf +close src/olap/fs/file_block_manager.cpp /^Status FileReadableBlock::close() {$/;" f class:doris::fs::internal::FileReadableBlock +close src/olap/fs/file_block_manager.cpp /^Status FileWritableBlock::close() {$/;" f class:doris::fs::internal::FileWritableBlock +close src/olap/memtable.cpp /^OLAPStatus MemTable::close() {$/;" f class:doris::MemTable +close src/olap/reader.cpp /^void Reader::close() {$/;" f class:doris::Reader +close src/olap/rowset/rowset.h /^ void close() {$/;" f class:doris::Rowset +close src/plugin/plugin.h /^ int (*close)(void *);$/;" m struct:doris::Plugin +close src/runtime/buffer_control_block.cpp /^Status BufferControlBlock::close(Status exec_status) {$/;" f class:doris::BufferControlBlock +close src/runtime/buffered_tuple_stream.cpp /^void BufferedTupleStream::close() {$/;" f class:doris::BufferedTupleStream +close src/runtime/buffered_tuple_stream2.cc /^void BufferedTupleStream2::close() {$/;" f class:doris::BufferedTupleStream2 +close src/runtime/data_spliter.cpp /^Status DataSpliter::close(RuntimeState* state, Status close_status) {$/;" f class:doris::DataSpliter +close src/runtime/data_stream_recvr.cc /^void DataStreamRecvr::SenderQueue::close() {$/;" f class:doris::DataStreamRecvr::SenderQueue +close src/runtime/data_stream_recvr.cc /^void DataStreamRecvr::close() {$/;" f class:doris::DataStreamRecvr +close src/runtime/data_stream_sender.cpp /^Status DataStreamSender::close(RuntimeState* state, Status exec_status) {$/;" f class:doris::DataStreamSender +close src/runtime/data_stream_sender.cpp /^void DataStreamSender::Channel::close(RuntimeState* state) {$/;" f class:doris::DataStreamSender::Channel +close src/runtime/disk_io_mgr_scan_range.cc /^void DiskIoMgr::ScanRange::close() {$/;" f class:doris::DiskIoMgr::ScanRange +close src/runtime/dpp_sink.cpp /^Status Translator::close(RuntimeState* state) {$/;" f class:doris::Translator +close src/runtime/dpp_sink.cpp /^void HllDppSinkMerge::close() {$/;" f class:doris::HllDppSinkMerge +close src/runtime/dpp_sink_internal.cpp /^Status PartitionInfo::close(RuntimeState* state) {$/;" f class:doris::PartitionInfo +close src/runtime/dpp_sink_internal.cpp /^Status RollupSchema::close(RuntimeState* state) {$/;" f class:doris::RollupSchema +close src/runtime/dpp_writer.cpp /^Status DppWriter::close() {$/;" f class:doris::DppWriter +close src/runtime/export_sink.cpp /^Status ExportSink::close(RuntimeState* state, Status exec_status) {$/;" f class:doris::ExportSink +close src/runtime/mem_tracker.cpp /^void MemTracker::close() {$/;" f class:doris::MemTracker +close src/runtime/memory_scratch_sink.cpp /^Status MemoryScratchSink::close(RuntimeState* state, Status exec_status) {$/;" f class:doris::MemoryScratchSink +close src/runtime/mysql_table_sink.cpp /^Status MysqlTableSink::close(RuntimeState* state, Status exec_status) {$/;" f class:doris::MysqlTableSink +close src/runtime/plan_fragment_executor.cpp /^void PlanFragmentExecutor::close() {$/;" f class:doris::PlanFragmentExecutor +close src/runtime/qsorter.cpp /^Status QSorter::close(RuntimeState* state) {$/;" f class:doris::QSorter +close src/runtime/result_sink.cpp /^Status ResultSink::close(RuntimeState* state, Status exec_status) {$/;" f class:doris::ResultSink +close src/runtime/sorter.h /^ virtual Status close(RuntimeState* state) {$/;" f class:doris::Sorter +close src/runtime/tablets_channel.cpp /^Status TabletsChannel::close(int sender_id, bool* finished,$/;" f class:doris::TabletsChannel +close src/udf/udf.cpp /^void FunctionContextImpl::close() {$/;" f class:doris::FunctionContextImpl +close src/util/broker_load_error_hub.cpp /^Status BrokerLoadErrorHub::close() {$/;" f class:doris::BrokerLoadErrorHub +close src/util/mysql_load_error_hub.cpp /^Status MysqlLoadErrorHub::close() {$/;" f class:doris::MysqlLoadErrorHub +close src/util/null_load_error_hub.cpp /^Status NullLoadErrorHub::close() {$/;" f class:doris::NullLoadErrorHub +close src/util/thrift_client.cpp /^Status ThriftClientImpl::close() {$/;" f class:doris::ThriftClientImpl +close src/util/zip_util.cpp /^Status ZipFile::close() {$/;" f class:doris::ZipFile +close test/runtime/fragment_mgr_test.cpp /^void PlanFragmentExecutor::close() {$/;" f class:doris::PlanFragmentExecutor +close test/runtime/load_channel_mgr_test.cpp /^OLAPStatus DeltaWriter::close() {$/;" f class:doris::DeltaWriter +close_connections src/runtime/client_cache.cpp /^void ClientCacheHelper::close_connections(const TNetworkAddress& hostport) {$/;" f class:doris::ClientCacheHelper +close_connections src/runtime/client_cache.h /^ void close_connections(const TNetworkAddress& hostport) {$/;" f class:doris::ClientCache +close_current_client_scanner src/exec/kudu_scanner.cpp /^void KuduScanner::close_current_client_scanner() {$/;" f class:doris::KuduScanner +close_file_func src/util/minizip/ioapi.h /^typedef int (ZCALLBACK *close_file_func) OF((voidpf opaque, voidpf stream));$/;" t +close_internal src/runtime/data_stream_sender.cpp /^Status DataStreamSender::Channel::close_internal() {$/;" f class:doris::DataStreamSender::Channel +close_partitions src/exec/partitioned_aggregation_node.cc /^void PartitionedAggregationNode::close_partitions() {$/;" f class:doris::PartitionedAggregationNode +close_scanner src/service/backend_service.cpp /^void BackendService::close_scanner(TScanCloseResult& result_, const TScanCloseParams& params) {$/;" f class:doris::BackendService +close_status test/runtime/load_channel_mgr_test.cpp /^OLAPStatus close_status;$/;" m namespace:doris file: +close_thread test/runtime/buffer_control_block_test.cpp /^void* close_thread(void* param) {$/;" f namespace:doris +close_tuple_streams src/runtime/row_batch.cpp /^void RowBatch::close_tuple_streams() {$/;" f class:doris::RowBatch +close_wait src/exec/tablet_sink.cpp /^Status NodeChannel::close_wait(RuntimeState* state) {$/;" f class:doris::stream_load::NodeChannel +close_wait src/olap/delta_writer.cpp /^OLAPStatus DeltaWriter::close_wait(google::protobuf::RepeatedPtrField* tablet_vec) {$/;" f class:doris::DeltaWriter +close_wait src/runtime/data_stream_sender.cpp /^void DataStreamSender::Channel::close_wait(RuntimeState* state) {$/;" f class:doris::DataStreamSender::Channel +close_wait test/runtime/load_channel_mgr_test.cpp /^OLAPStatus DeltaWriter::close_wait(google::protobuf::RepeatedPtrField* tablet_vec) {$/;" f class:doris::DeltaWriter +closed src/exec/broker_reader.cpp /^bool BrokerReader::closed() {$/;" f class:doris::BrokerReader +closed src/exec/local_file_reader.cpp /^bool LocalFileReader::closed() {$/;" f class:doris::LocalFileReader +closed src/exec/parquet_reader.cpp /^bool ParquetFile::closed() const {$/;" f class:doris::ParquetFile +closed src/exprs/expr_context.h /^ bool closed() { $/;" f class:doris::ExprContext +closed src/udf/udf_internal.h /^ bool closed() const {$/;" f class:doris::FunctionContextImpl +closed_ src/exprs/new_agg_fn_evaluator.h /^ bool closed_ = false;$/;" m class:doris::NewAggFnEvaluator +closed_ src/olap/fs/file_block_manager.cpp /^ std::atomic_bool closed_;$/;" m class:doris::fs::internal::FileReadableBlock file: +closed_ src/runtime/buffered_tuple_stream3.h /^ bool closed_; \/\/ Used for debugging.$/;" m class:doris::BufferedTupleStream3 +closing_ src/gutil/strings/split.cc /^ char closing_[256];$/;" m class:__anon31::ClosingSymbolLookup file: +cluster src/common/utils.h /^ std::string cluster;$/;" m struct:doris::AuthInfo +cluster_id src/olap/data_dir.h /^ int32_t cluster_id() const { return _cluster_id; }$/;" f class:doris::DataDir +clz128 src/runtime/decimalv2_value.cpp /^static int clz128(unsigned __int128 v) {$/;" f namespace:doris +cmp src/olap/decimal12.h /^ int32_t cmp(const decimal12_t& other) const {$/;" f struct:doris::decimal12_t +cmp src/olap/types.h /^ inline int cmp(const void* left, const void* right) const {$/;" f class:doris::TypeInfo +cmp src/olap/types.h /^ static inline int cmp(const void* left, const void* right) {$/;" f struct:doris::BaseFieldtypeTraits +cmp src/olap/types.h /^ static int cmp(const void* left, const void* right) {$/;" f struct:doris::FieldTypeTraits +cmp src/olap/uint24.h /^ int32_t cmp(const uint24_t& other) const {$/;" f struct:doris::uint24_t +cmp src/olap/wrapper_field.h /^ int cmp(const WrapperField* field) const {$/;" f class:doris::WrapperField +cmt_offset src/runtime/stream_load/stream_load_context.h /^ std::map cmt_offset;$/;" m class:doris::KafkaLoadInfo +cntl src/runtime/buffer_control_block.h /^ brpc::Controller* cntl = nullptr;$/;" m struct:doris::GetResultBatchCtx +cntl src/util/ref_count_closure.h /^ brpc::Controller cntl;$/;" m class:doris::RefCountClosure +code src/common/status.h /^ TStatusCode::type code() const {$/;" f class:doris::Status +code_as_string src/common/status.cpp /^std::string Status::code_as_string() const {$/;" f class:doris::Status +codec src/olap/rowset/segment_v2/page_io.h /^ const BlockCompressionCodec* codec = nullptr;$/;" m struct:doris::segment_v2::PageReadOptions +codegen_enabled src/runtime/runtime_state.h /^ bool codegen_enabled() const {$/;" f class:doris::RuntimeState +codegen_level src/runtime/runtime_state.h /^ int codegen_level() const {$/;" f class:doris::RuntimeState +col src/exec/es/es_predicate.h /^ ExtColumnDesc col;$/;" m struct:doris::ExtBinaryPredicate +col src/exec/es/es_predicate.h /^ ExtColumnDesc col;$/;" m struct:doris::ExtInPredicate +col src/exec/es/es_predicate.h /^ ExtColumnDesc col;$/;" m struct:doris::ExtIsNullPredicate +col src/exec/es/es_predicate.h /^ ExtColumnDesc col;$/;" m struct:doris::ExtLikePredicate +col_data src/runtime/vectorized_row_batch.h /^ void* col_data() {$/;" f class:doris::ColumnVector +col_name src/runtime/descriptors.h /^ const std::string& col_name() const {$/;" f class:doris::SlotDescriptor +col_pos src/runtime/descriptors.h /^ int col_pos() const {$/;" f class:doris::SlotDescriptor +collect src/util/metrics.h /^ void collect(MetricsVisitor* visitor) {$/;" f class:doris::MetricRegistry +collect src/util/metrics.h /^ void collect(const std::string& prefix, const std::string& name, MetricsVisitor* visitor) {$/;" f class:doris::MetricCollector +collect_load_stat src/runtime/stream_load/stream_load_executor.cpp /^bool StreamLoadExecutor::collect_load_stat(StreamLoadContext* ctx, TTxnCommitAttachment* attach) {$/;" f class:doris::StreamLoadExecutor +collect_nodes src/exec/exec_node.cpp /^void ExecNode::collect_nodes(TPlanNodeType::type node_type, vector* nodes) {$/;" f class:doris::ExecNode +collect_non_null_varslots src/runtime/spill_sorter.cc /^void SpillSorter::Run::collect_non_null_varslots($/;" f class:doris::SpillSorter::Run +collect_output src/runtime/dpp_sink.cpp /^void DppSink::collect_output(std::vector* files) {$/;" f class:doris::DppSink +collect_query_statistics src/exec/exchange_node.cpp /^Status ExchangeNode::collect_query_statistics(QueryStatistics* statistics) {$/;" f class:doris::ExchangeNode +collect_query_statistics src/exec/exec_node.cpp /^Status ExecNode::collect_query_statistics(QueryStatistics* statistics) {$/;" f class:doris::ExecNode +collect_query_statistics src/exec/olap_scan_node.cpp /^Status OlapScanNode::collect_query_statistics(QueryStatistics* statistics) {$/;" f class:doris::OlapScanNode +collect_query_statistics src/runtime/plan_fragment_executor.cpp /^void PlanFragmentExecutor::collect_query_statistics() {$/;" f class:doris::PlanFragmentExecutor +collect_scan_nodes src/exec/exec_node.cpp /^void ExecNode::collect_scan_nodes(vector* nodes) {$/;" f class:doris::ExecNode +collect_scanners_status src/exec/es_http_scan_node.cpp /^Status EsHttpScanNode::collect_scanners_status() {$/;" f class:doris::EsHttpScanNode +cols src/exec/es/es_predicate.h /^ std::vector cols;$/;" m struct:doris::ExtFunction +column src/olap/schema.h /^ const Field* column(ColumnId cid) const { return _cols[cid]; }$/;" f class:doris::Schema +column src/olap/tablet_schema.cpp /^const TabletColumn& TabletSchema::column(size_t ordinal) const {$/;" f class:doris::TabletSchema +column src/runtime/vectorized_row_batch.h /^ ColumnVector* column(int column_index) {$/;" f class:doris::VectorizedRowBatch +column_block src/olap/column_block.h /^ ColumnBlock* column_block() { return _block; }$/;" f class:doris::ColumnBlockView +column_block src/olap/row_block2.h /^ ColumnBlock column_block(ColumnId cid) const {$/;" f class:doris::RowBlockV2 +column_block src/olap/row_block2.h /^ ColumnBlock column_block(size_t col_idx) const {$/;" f class:doris::RowBlockRow +column_contains_index test/olap/rowset/segment_v2/segment_test.cpp /^static bool column_contains_index(ColumnMetaPB column_meta, ColumnIndexTypePB type) {$/;" f namespace:doris::segment_v2 +column_data src/olap/rowset/alpha_rowset_reader.h /^ std::unique_ptr column_data = nullptr;$/;" m struct:doris::AlphaMergeContext +column_id src/olap/column_predicate.h /^ uint32_t column_id() const { return _column_id; }$/;" f class:doris::ColumnPredicate +column_id src/olap/rowset/column_reader.h /^ uint32_t column_id() {$/;" f class:doris::ColumnReader +column_id src/olap/rowset/column_writer.h /^ uint32_t column_id() const {$/;" f class:doris::ColumnWriter +column_id src/olap/rowset/segment_reader.h /^ uint32_t column_id;$/;" m struct:doris::SegmentReader::VectorizedPositionInfo +column_ids src/olap/row_block.h /^ std::vector column_ids;$/;" m struct:doris::RowBlockInfo +column_ids src/olap/schema.h /^ const std::vector& column_ids() const { return _col_ids; }$/;" f class:doris::Schema +column_indices src/exec/parquet_reader.cpp /^Status ParquetReaderWrap::column_indices(const std::vector& tuple_slot_descs)$/;" f class:doris::ParquetReaderWrap +column_name src/runtime/descriptor_helper.h /^ TSlotDescriptorBuilder& column_name(const std::string& name) {$/;" f class:doris::TSlotDescriptorBuilder +column_offset src/olap/schema.h /^ size_t column_offset(ColumnId cid) const {$/;" f class:doris::Schema +column_pos src/runtime/descriptor_helper.h /^ TSlotDescriptorBuilder& column_pos(int column_pos) {$/;" f class:doris::TSlotDescriptorBuilder +column_position src/olap/rowset/segment_reader.h /^ uint32_t column_position;$/;" m struct:doris::SegmentReader::VectorizedPositionInfo +column_predicates src/olap/iterators.h /^ const std::vector* column_predicates = nullptr;$/;" m class:doris::StorageReadOptions +column_schema src/olap/row_cursor.h /^ const Field* column_schema(uint32_t cid) const {$/;" f class:doris::RowCursor +column_size src/olap/row_cursor.h /^ size_t column_size(uint32_t cid) const {$/;" f class:doris::RowCursor +column_size src/olap/schema.h /^ size_t column_size(ColumnId cid) const {$/;" f class:doris::Schema +column_statistic src/olap/stream_index_reader.cpp /^const ColumnStatistics& PositionEntryReader::column_statistic() const {$/;" f class:doris::PositionEntryReader +column_type_to_type_desc src/exprs/anyval_util.cpp /^FunctionContext::TypeDesc AnyValUtil::column_type_to_type_desc(const TypeDescriptor& type) {$/;" f class:doris::AnyValUtil +column_unique_id src/olap/rowset/column_reader.h /^ uint32_t column_unique_id() {$/;" f class:doris::ColumnReader +columns src/olap/olap_cond.h /^ const CondColumns& columns() const {$/;" f class:doris::Conditions +columns src/olap/schema.h /^ const std::vector& columns() const { return _cols; }$/;" f class:doris::Schema +columns src/olap/tablet_schema.cpp /^const std::vector& TabletSchema::columns() const {$/;" f class:doris::TabletSchema +columns src/runtime/vectorized_row_batch.h /^ const std::vector& columns() const { return _cols; }$/;" f class:doris::VectorizedRowBatch +commit src/runtime/routine_load/data_consumer.cpp /^Status KafkaDataConsumer::commit(std::vector& offset) {$/;" f class:doris::KafkaDataConsumer +commit_infos src/runtime/stream_load/stream_load_context.h /^ std::vector commit_infos;$/;" m class:doris::StreamLoadContext +commit_last_row src/runtime/row_batch.h /^ void commit_last_row() {$/;" f class:doris::RowBatch +commit_rows src/runtime/row_batch.h /^ void commit_rows(int n) {$/;" f class:doris::RowBatch +commit_txn src/olap/txn_manager.cpp /^OLAPStatus TxnManager::commit_txn($/;" f class:doris::TxnManager +commit_txn src/olap/txn_manager.cpp /^OLAPStatus TxnManager::commit_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, TTransactionId transaction_id,$/;" f class:doris::TxnManager +commit_txn src/runtime/stream_load/stream_load_executor.cpp /^Status StreamLoadExecutor::commit_txn(StreamLoadContext* ctx) {$/;" f class:doris::StreamLoadExecutor +common_test test/olap/storage_types_test.cpp /^void common_test(typename TypeTraits::CppType src_val) {$/;" f namespace:doris +common_test test/olap/storage_types_test.cpp /^void common_test(Slice src_val) {$/;" f namespace:doris +common_test test/olap/storage_types_test.cpp /^void common_test(Slice src_val) {$/;" f namespace:doris +compact src/olap/base_compaction.cpp /^OLAPStatus BaseCompaction::compact() {$/;" f class:doris::BaseCompaction +compact src/olap/cumulative_compaction.cpp /^OLAPStatus CumulativeCompaction::compact() {$/;" f class:doris::CumulativeCompaction +compaction_rowset_type src/olap/storage_engine.h /^ RowsetTypePB compaction_rowset_type() const {$/;" f class:doris::StorageEngine +comparator src/olap/rowset/rowset.h /^ static bool comparator(const RowsetSharedPtr& left, const RowsetSharedPtr& right) {$/;" f class:doris::Rowset +compare src/gutil/strings/stringpiece.h /^ int compare(StringPiece x) const {$/;" f class:StringPiece +compare src/olap/field.h /^ int compare(const void* left, const void* right) const {$/;" f class:doris::Field +compare src/olap/lru_cache.h /^ inline int compare(const CacheKey& b) const {$/;" f class:doris::CacheKey +compare src/runtime/raw_value_ir.cpp /^int RawValue::compare(const void* v1, const void* v2, const TypeDescriptor& type) {$/;" f class:doris::RawValue +compare src/runtime/string_value.hpp /^inline int StringValue::compare(const StringValue& other) const {$/;" f class:doris::StringValue +compare src/util/metrics.h /^ int compare(const MetricLabel& other) const {$/;" f struct:doris::MetricLabel +compare src/util/slice.h /^inline int Slice::compare(const Slice& b) const {$/;" f class:doris::Slice +compare src/util/tuple_row_compare.h /^ int compare(TupleRow* lhs, TupleRow* rhs) const {$/;" f class:doris::TupleRowComparator +compare_ src/olap/skiplist.h /^ Comparator const compare_;$/;" m class:doris::SkipList +compare_and_swap src/common/atomic.h /^ bool compare_and_swap(T old_val, T new_val) {$/;" f class:doris::AtomicInt +compare_and_swap_val src/common/atomic.h /^ T compare_and_swap_val(T old_val, T new_val) {$/;" f class:doris::AtomicInt +compare_cell src/olap/field.h /^ int compare_cell(const LhsCellType& lhs,$/;" f class:doris::Field +compare_double_with_epsilon test/util/radix_sort_test.cpp /^bool compare_double_with_epsilon(double a, double b, double E) {$/;" f namespace:doris +compare_float_with_epsilon test/util/radix_sort_test.cpp /^bool compare_float_with_epsilon(float a, float b, float E) {$/;" f namespace:doris +compare_key src/runtime/dpp_sink_internal.h /^ int compare_key(const PartRangeKey& key) const {$/;" f class:doris::PartRange +compare_large src/exec/olap_utils.h /^inline bool compare_large(const void* lhs, const void* rhs) {$/;" f namespace:doris +compare_part_use_range src/exec/broker_scan_node.cpp /^static bool compare_part_use_range(const PartitionInfo* v1, const PartitionInfo* v2) {$/;" f namespace:doris +compare_part_use_range src/runtime/data_spliter.cpp /^static bool compare_part_use_range(const PartitionInfo* v1, const PartitionInfo* v2) {$/;" f namespace:doris +compare_part_use_range src/runtime/data_stream_sender.cpp /^static bool compare_part_use_range(const PartitionInfo* v1, const PartitionInfo* v2) {$/;" f namespace:doris +compare_row src/exec/merge_join_node.cpp /^Status MergeJoinNode::compare_row(TupleRow* left_row, TupleRow* right_row, bool* is_lt) {$/;" f class:doris::MergeJoinNode +compare_row src/olap/row.h /^int compare_row(const LhsRowType& lhs, const RhsRowType& rhs) {$/;" f namespace:doris +compare_row_key src/olap/row.h /^int compare_row_key(const LhsRowType& lhs, const RhsRowType& rhs) {$/;" f namespace:doris +compare_row_with_lhs_columns src/olap/rowset/segment_v2/segment_iterator.cpp /^int compare_row_with_lhs_columns(const LhsRowType& lhs, const RhsRowType& rhs) {$/;" f namespace:doris::segment_v2 +compare_value src/exec/merge_join_node.cpp /^int compare_value(const StringValue* left_value, const StringValue* right_value) {$/;" f namespace:doris +compare_value src/exec/merge_join_node.cpp /^int compare_value(const void* left_value, const void* right_value) {$/;" f namespace:doris +compile_regex src/exprs/string_functions.cpp /^static re2::RE2* compile_regex($/;" f namespace:doris +compiler_barrier src/common/atomic.h /^ static ALWAYS_INLINE void compiler_barrier() {$/;" f class:doris::AtomicUtil +compound_not src/exprs/compound_predicate.cpp /^BooleanVal CompoundPredicate::compound_not(FunctionContext* context, const BooleanVal& v) {$/;" f class:doris::CompoundPredicate +compress src/util/block_compression.cpp /^Status BlockCompressionCodec::compress(const std::vector& inputs, Slice* output) const {$/;" f class:doris::BlockCompressionCodec +compress src/util/tdigest.h /^ inline void compress() { process(); }$/;" f class:doris::TDigest +compress_and_write_page src/olap/rowset/segment_v2/page_io.h /^ static Status compress_and_write_page(const BlockCompressionCodec* codec,$/;" f class:doris::segment_v2::PageIO +compress_kind src/olap/rowset/column_data_writer.cpp /^CompressKind ColumnDataWriter::compress_kind() {$/;" f class:doris::ColumnDataWriter +compress_kind src/olap/tablet.h /^inline CompressKind Tablet::compress_kind() const {$/;" f class:doris::Tablet +compress_kind src/olap/tablet_schema.h /^ inline CompressKind compress_kind() const { return _compress_kind; }$/;" f class:doris::TabletSchema +compress_lz4 src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp /^int64_t compress_lz4(void* in, void* out, size_t size,$/;" f namespace:doris::bitshuffle +compress_lz4_bound src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp /^size_t compress_lz4_bound(size_t size, size_t elem_size, size_t block_size) {$/;" f namespace:doris::bitshuffle +compress_page_body src/olap/rowset/segment_v2/page_io.cpp /^Status PageIO::compress_page_body(const BlockCompressionCodec* codec,$/;" f class:doris::segment_v2::PageIO +compressed_bytes_read src/olap/olap_common.h /^ int64_t compressed_bytes_read = 0;$/;" m struct:doris::OlapReaderStatistics +compressed_size src/util/minizip/unzip.h /^ ZPOS64_T compressed_size; \/* compressed size 8 bytes *\/$/;" m struct:unz_file_info64_s +compressed_size src/util/minizip/unzip.h /^ uLong compressed_size; \/* compressed size 4 bytes *\/$/;" m struct:unz_file_info_s +compression src/olap/rowset/segment_v2/indexed_column_writer.h /^ CompressionTypePB compression = NO_COMPRESSION;$/;" m struct:doris::segment_v2::IndexedColumnWriterOptions +compression src/util/tdigest.h /^ Value compression() const { return _compression; }$/;" f class:doris::TDigest +compression_method src/util/minizip/unzip.c /^ uLong compression_method; \/* compression method (0==store) *\/$/;" m struct:__anon33 file: +compression_method src/util/minizip/unzip.h /^ uLong compression_method; \/* compression method 2 bytes *\/$/;" m struct:unz_file_info64_s +compression_method src/util/minizip/unzip.h /^ uLong compression_method; \/* compression method 2 bytes *\/$/;" m struct:unz_file_info_s +compression_min_space_saving src/olap/rowset/segment_v2/column_writer.h /^ double compression_min_space_saving = 0.1;$/;" m struct:doris::segment_v2::ColumnWriterOptions +compression_min_space_saving src/olap/rowset/segment_v2/indexed_column_writer.h /^ double compression_min_space_saving = 0.1;$/;" m struct:doris::segment_v2::IndexedColumnWriterOptions +compute_distance src/olap/rowset/segment_group.cpp /^uint32_t SegmentGroup::compute_distance(const RowBlockPosition& position1,$/;" f class:doris::SegmentGroup +compute_fn src/exprs/info_func.cpp /^void* InfoFunc::compute_fn(Expr* e, TupleRow* row) {$/;" f class:doris::InfoFunc +compute_format_len src/runtime/datetime_value.cpp /^int DateTimeValue::compute_format_len(const char* format, int len) const {$/;" f class:doris::DateTimeValue +compute_hists src/olap/serialize.h /^inline void compute_hists(int64_t* data, uint16_t count, uint16_t hists[65]) {$/;" f namespace:doris::ser +compute_knuth_variance src/exprs/aggregate_functions.cpp /^static double compute_knuth_variance(const KnuthVarianceState& state, bool pop) {$/;" f namespace:doris +compute_md5 test/runtime/user_function_cache_test.cpp /^static std::string compute_md5(const std::string& file) {$/;" f namespace:doris +compute_mean_stddev src/util/stat_util.hpp /^ static void compute_mean_stddev(const T* values, int N, double* mean, double* stddev) {$/;" f class:doris::StatUtil +compute_num_null_indicator_bytes src/runtime/buffered_tuple_stream.cpp /^int BufferedTupleStream::compute_num_null_indicator_bytes(int block_size) const {$/;" f class:doris::BufferedTupleStream +compute_num_null_indicator_bytes src/runtime/buffered_tuple_stream2.cc /^int BufferedTupleStream2::compute_num_null_indicator_bytes(int block_size) const {$/;" f class:doris::BufferedTupleStream2 +compute_range_part_code src/runtime/data_stream_sender.cpp /^Status DataStreamSender::compute_range_part_code($/;" f class:doris::DataStreamSender +compute_results_layout src/exprs/expr.cpp /^int Expr::compute_results_layout($/;" f class:doris::Expr +compute_row_size src/runtime/buffered_tuple_stream.cpp /^int BufferedTupleStream::compute_row_size(TupleRow* row) const {$/;" f class:doris::BufferedTupleStream +compute_row_size src/runtime/buffered_tuple_stream2.cc /^int64_t BufferedTupleStream2::compute_row_size(TupleRow* row) const {$/;" f class:doris::BufferedTupleStream2 +compute_time_in_profile src/util/runtime_profile.cpp /^void RuntimeProfile::compute_time_in_profile() {$/;" f class:doris::RuntimeProfile +compute_time_in_profile src/util/runtime_profile.cpp /^void RuntimeProfile::compute_time_in_profile(int64_t total) {$/;" f class:doris::RuntimeProfile +compute_total_size src/util/slice.h /^ static size_t compute_total_size(const std::vector& slices) {$/;" f struct:doris::Slice +compute_version_hash_from_rowsets src/olap/tablet.cpp /^void Tablet::compute_version_hash_from_rowsets($/;" f class:doris::Tablet +concat src/exprs/string_functions.cpp /^StringVal StringFunctions::concat($/;" f class:doris::StringFunctions +concat test/udf/udf_test.cpp /^StringVal concat(FunctionContext* context, int n, const StringVal* args) {$/;" f namespace:doris_udf +concat_ws src/exprs/string_functions.cpp /^StringVal StringFunctions::concat_ws($/;" f class:doris::StringFunctions +concurrent_reader test/olap/skiplist_test.cpp /^static void concurrent_reader(void* arg) {$/;" f namespace:doris +cond_ src/util/countdown_latch.h /^ ConditionVariable cond_;$/;" m class:doris::CountDownLatch +cond_num_t src/olap/delete_handler.h /^ typedef std::vector::size_type cond_num_t;$/;" t class:doris::DeleteHandler +conditions src/olap/iterators.h /^ const Conditions* conditions = nullptr;$/;" m class:doris::StorageReadOptions +conditions src/olap/reader.h /^ std::vector conditions;$/;" m struct:doris::ReaderParams +conditions src/olap/rowset/rowset_reader_context.h /^ const Conditions* conditions = nullptr;$/;" m struct:doris::RowsetReaderContext +conditions_num src/olap/delete_handler.h /^ cond_num_t conditions_num() const{$/;" f class:doris::DeleteHandler +conds src/olap/olap_cond.h /^ const std::vector& conds() const {$/;" f class:doris::CondColumn +config src/common/config.h /^namespace config {$/;" n namespace:doris +config src/common/configbase.cpp /^namespace config {$/;" n namespace:doris file: +config src/common/configbase.h /^namespace config {$/;" n namespace:doris +config_handler src/http/default_path_handlers.cpp /^void config_handler(const WebPageHandler::ArgumentMap& args, std::stringstream* output) {$/;" f namespace:doris +confmap src/common/configbase.cpp /^std::map* confmap = NULL;$/;" m namespace:doris::config file: +conjunct_ctxs src/exec/exec_node.h /^ const std::vector& conjunct_ctxs() const {$/;" f class:doris::ExecNode +conjunct_ctxs src/exec/olap_scanner.h /^ std::vector* conjunct_ctxs() {$/;" f class:doris::OlapScanner +const build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 10;" d file: +const_iterator src/gutil/strings/stringpiece.h /^ typedef const char* const_iterator;$/;" t class:StringPiece +const_iterator src/util/bitmap_value.h /^ typedef Roaring64MapSetBitForwardIterator const_iterator;$/;" t class:doris::detail::Roaring64Map +const_reference src/gutil/strings/stringpiece.h /^ typedef const char& const_reference;$/;" t class:StringPiece +const_reverse_iterator src/gutil/strings/stringpiece.h /^ typedef std::reverse_iterator const_reverse_iterator;$/;" t class:StringPiece +constant_ends_with_fn src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::constant_ends_with_fn($/;" f class:doris::LikePredicate +constant_equals_fn src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::constant_equals_fn($/;" f class:doris::LikePredicate +constant_regex_fn src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::constant_regex_fn($/;" f class:doris::LikePredicate +constant_regex_fn_partial src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::constant_regex_fn_partial($/;" f class:doris::LikePredicate +constant_starts_with_fn src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::constant_starts_with_fn($/;" f class:doris::LikePredicate +constant_substring_fn src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::constant_substring_fn($/;" f class:doris::LikePredicate +construct_build_side src/exec/cross_join_node.cpp /^Status CrossJoinNode::construct_build_side(RuntimeState* state) {$/;" f class:doris::CrossJoinNode +construct_data_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::construct_data_file_path(const std::string& snapshot_path, int32_t segment_id) const {$/;" f class:doris::SegmentGroup +construct_data_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::construct_data_file_path(int32_t segment_id) const {$/;" f class:doris::SegmentGroup +construct_hash_table src/exec/hash_join_node.cpp /^Status HashJoinNode::construct_hash_table(RuntimeState* state) {$/;" f class:doris::HashJoinNode +construct_header_file_path src/olap/tablet_meta.cpp /^string TabletMeta::construct_header_file_path(const string& schema_hash_path,$/;" f class:doris::TabletMeta +construct_index_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::construct_index_file_path(const std::string& snapshot_path, int32_t segment_id) const {$/;" f class:doris::SegmentGroup +construct_index_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::construct_index_file_path(int32_t segment_id) const {$/;" f class:doris::SegmentGroup +construct_input_rowset_readers src/olap/compaction.cpp /^OLAPStatus Compaction::construct_input_rowset_readers() {$/;" f class:doris::Compaction +construct_intermediate_tuple src/exec/aggregation_node.cpp /^Tuple* AggregationNode::construct_intermediate_tuple() {$/;" f class:doris::AggregationNode +construct_intermediate_tuple src/exec/partitioned_aggregation_node.cc /^Tuple* PartitionedAggregationNode::construct_intermediate_tuple($/;" f class:doris::PartitionedAggregationNode +construct_is_null_pred_in_where_pred src/exec/olap_scan_node.cpp /^void OlapScanNode::construct_is_null_pred_in_where_pred(Expr* expr, SlotDescriptor* slot, std::string is_null_str) {$/;" f class:doris::OlapScanNode +construct_old_data_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::construct_old_data_file_path(const std::string& path_prefix, int32_t segment_id) const {$/;" f class:doris::SegmentGroup +construct_old_index_file_path src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::construct_old_index_file_path(const std::string& path_prefix, int32_t segment_id) const {$/;" f class:doris::SegmentGroup +construct_one_tuple test/exec/olap_scanner_test.cpp /^void construct_one_tuple(TupleDescriptor& tuple_desc) {$/;" f namespace:doris +construct_output_rowset_writer src/olap/compaction.cpp /^OLAPStatus Compaction::construct_output_rowset_writer() {$/;" f class:doris::Compaction +construct_row src/exec/pre_aggregation_node.cpp /^TupleRow* PreAggregationNode::construct_row(TupleRow* in_row) {$/;" f class:doris::PreAggregationNode +construct_rowset_graph src/olap/rowset_graph.cpp /^OLAPStatus RowsetGraph::construct_rowset_graph(const std::vector& rs_metas) {$/;" f class:doris::RowsetGraph +construct_scan_range test/exec/olap_common_test.cpp /^void construct_scan_range(TPaloScanRange* doris_scan_range) {$/;" f namespace:doris +construct_scan_ranges test/exec/olap_scanner_test.cpp /^boost::shared_ptr construct_scan_ranges() {$/;" f namespace:doris +construct_single_row src/exec/pre_aggregation_node.cpp /^Status PreAggregationNode::construct_single_row() {$/;" f class:doris::PreAggregationNode +construct_sub_predicates src/olap/delete_handler.cpp /^string DeleteConditionHandler::construct_sub_predicates(const TCondition& condition) {$/;" f class:doris::DeleteConditionHandler +consume src/olap/field.h /^ virtual void consume(RowCursorCell* dst, const char* src, bool src_null, MemPool* mem_pool, ObjectPool* agg_pool) const {$/;" f class:doris::Field +consume src/runtime/mem_tracker.h /^ void consume(int64_t bytes) {$/;" f class:doris::MemTracker +consume_local src/runtime/mem_tracker.h /^ void consume_local(int64_t bytes, MemTracker* end_tracker) {$/;" f class:doris::MemTracker +consume_memory src/runtime/buffered_block_mgr2.cc /^bool BufferedBlockMgr2::consume_memory(Client* client, int64_t size) {$/;" f class:doris::BufferedBlockMgr2 +consumer_id src/runtime/stream_load/stream_load_context.h /^ int64_t consumer_id;$/;" m class:doris::StreamLoadContext +consumers src/runtime/routine_load/data_consumer_group.h /^ const std::vector>& consumers() {$/;" f class:doris::DataConsumerGroup +consumption src/runtime/mem_tracker.h /^ int64_t consumption() const {$/;" f class:doris::MemTracker +contain src/olap/rowset/segment_v2/row_ranges.h /^ bool contain(rowid_t from, rowid_t to) {$/;" f class:doris::segment_v2::RowRanges +contain_path src/util/filesystem_util.cc /^bool FileSystemUtil::contain_path($/;" f class:doris::FileSystemUtil +container_ptr_ src/gutil/stl_util.h /^ STLContainer *container_ptr_;$/;" m class:STLElementDeleter +container_ptr_ src/gutil/stl_util.h /^ STLContainer *container_ptr_;$/;" m class:STLValueDeleter +container_ptr_ src/gutil/stl_util.h /^ STLContainer *container_ptr_;$/;" m class:TemplatedElementDeleter +container_ptr_ src/gutil/stl_util.h /^ STLContainer *container_ptr_;$/;" m class:TemplatedValueDeleter +contains src/geo/geo_types.cpp /^bool GeoCircle::contains(const GeoShape* rhs) const {$/;" f class:doris::GeoCircle +contains src/geo/geo_types.cpp /^bool GeoPolygon::contains(const GeoShape* rhs) const {$/;" f class:doris::GeoPolygon +contains src/geo/geo_types.h /^ virtual bool contains(const GeoShape* rhs) const { return false; }$/;" f class:doris::GeoShape +contains src/gutil/charmap.h /^ bool contains(unsigned char c) const {$/;" f class:Charmap +contains src/gutil/strings/stringpiece.cc /^bool StringPiece::contains(StringPiece s) const {$/;" f class:StringPiece +contains src/olap/olap_common.h /^ bool contains(const Version& other) const {$/;" f struct:doris::Version +contains src/olap/rowset/segment_v2/parsed_page.h /^ bool contains(ordinal_t ord) { return ord >= first_ordinal && ord < (first_ordinal + num_rows); }$/;" f struct:doris::segment_v2::ParsedPage +contains src/runtime/bufferpool/buffer_pool_internal.h /^ bool contains(Page* page) { return list_.contains(page); }$/;" f class:doris::BufferPool::PageList +contains src/util/bitmap_value.h /^ bool contains(uint32_t x) const {$/;" f class:doris::detail::Roaring64Map +contains src/util/bitmap_value.h /^ bool contains(uint64_t x) const {$/;" f class:doris::detail::Roaring64Map +contains src/util/bitmap_value.h /^ bool contains(uint64_t x) {$/;" f class:doris::BitmapValue +contains src/util/cidr.cpp /^bool CIDR::contains(const std::string& ip) {$/;" f class:doris::CIDR +contains src/util/cidr.cpp /^bool CIDR::contains(uint32_t ip_int) {$/;" f class:doris::CIDR +contains src/util/internal_queue.h /^ bool contains(const T* target) const {$/;" f class:doris::InternalQueueBase +contains_null src/exprs/new_in_predicate.h /^ bool contains_null;$/;" m struct:doris::InPredicate::SetLookupState +contains_version src/olap/rowset/rowset.h /^ bool contains_version(Version version) {$/;" f class:doris::Rowset +content src/http/http_response.h /^ const std::string* content() const {$/;" f class:doris::HttpResponse +content_type src/http/http_response.h /^ const std::string& content_type() const {$/;" f class:doris::HttpResponse +context_id src/runtime/external_scan_context_mgr.h /^ std::string context_id;$/;" m struct:doris::ScanContext +context_status src/runtime/disk_io_mgr.cc /^Status DiskIoMgr::context_status(RequestContext* context) const {$/;" f class:doris::DiskIoMgr +continuationLengths src/util/simdutf8check.h /^static inline __m128i continuationLengths(__m128i high_nibbles) {$/;" f +conv_int src/exprs/math_functions.cpp /^StringVal MathFunctions::conv_int($/;" f class:doris::MathFunctions +conv_string src/exprs/math_functions.cpp /^StringVal MathFunctions::conv_string($/;" f class:doris::MathFunctions +convert src/util/arrow/row_batch.cpp /^Status FromRowBatchConverter::convert(std::shared_ptr* out) {$/;" f class:doris::FromRowBatchConverter +convert src/util/arrow/row_batch.cpp /^Status ToRowBatchConverter:: convert(std::shared_ptr* result) {$/;" f class:doris::ToRowBatchConverter +convert src/util/arrow/row_block.cpp /^Status FromRowBlockConverter::convert(std::shared_ptr* out) {$/;" f class:doris::FromRowBlockConverter +convert src/util/arrow/row_block.cpp /^Status ToRowBlockConverter::convert(std::shared_ptr* result) {$/;" f class:doris::ToRowBlockConverter +convert_alpha_to_beta src/olap/rowset/rowset_converter.cpp /^OLAPStatus RowsetConverter::convert_alpha_to_beta(const RowsetMetaSharedPtr& src_rowset_meta,$/;" f class:doris::RowsetConverter +convert_beta_to_alpha src/olap/rowset/rowset_converter.cpp /^OLAPStatus RowsetConverter::convert_beta_to_alpha(const RowsetMetaSharedPtr& src_rowset_meta,$/;" f class:doris::RowsetConverter +convert_bitmap_intersect_to_string test/exprs/bitmap_function_test.cpp /^StringVal convert_bitmap_intersect_to_string(FunctionContext* ctx, BitmapIntersect& intersect) {$/;" f namespace:doris +convert_bitmap_to_string test/exprs/bitmap_function_test.cpp /^StringVal convert_bitmap_to_string(FunctionContext* ctx, BitmapValue& bitmap) {$/;" f namespace:doris +convert_bitmap_to_string test/util/bitmap_value_test.cpp /^std::string convert_bitmap_to_string(BitmapValue& bitmap) {$/;" f namespace:doris +convert_float_from_varchar src/olap/types.h /^OLAPStatus convert_float_from_varchar(void* dest, const void* src) {$/;" f namespace:doris +convert_format src/exprs/timestamp_functions.cpp /^StringVal TimestampFunctions::convert_format(FunctionContext* ctx, const StringVal& format) {$/;" f class:doris::TimestampFunctions +convert_from src/olap/field.h /^ inline OLAPStatus convert_from(char* dest, const char* src, const TypeInfo* src_type, MemPool* mem_pool) const {$/;" f class:doris::Field +convert_from src/olap/row_cursor.h /^ OLAPStatus convert_from(size_t index, const char* src, const TypeInfo* src_type, MemPool* mem_pool) {$/;" f class:doris::RowCursor +convert_from src/olap/types.h /^ OLAPStatus convert_from(void* dest, const void* src, const TypeInfo* src_type, MemPool* mem_pool) const{$/;" f class:doris::TypeInfo +convert_from src/olap/types.h /^ static OLAPStatus convert_from(void* dest, const void* src, const TypeInfo* src_type, MemPool* memPool) {$/;" f struct:doris::FieldTypeTraits +convert_from src/olap/types.h /^ static OLAPStatus convert_from(void* dest, const void* src, const TypeInfo* src_type, MemPool* mem_pool) {$/;" f struct:doris::BaseFieldtypeTraits +convert_from src/olap/types.h /^ static OLAPStatus convert_from(void* dest, const void* src, const TypeInfo* src_type, MemPool* mem_pool) {$/;" f struct:doris::FieldTypeTraits +convert_from_old_files src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::convert_from_old_files(const std::string& snapshot_path,$/;" f class:doris::AlphaRowset +convert_from_old_files src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::convert_from_old_files(const std::string& snapshot_path,$/;" f class:doris::SegmentGroup +convert_hll_to_string test/exprs/hll_function_test.cpp /^StringVal convert_hll_to_string(FunctionContext* ctx, HyperLogLog& hll) {$/;" f namespace:doris +convert_int_from_varchar src/olap/types.h /^OLAPStatus convert_int_from_varchar(void* dest, const void* src) {$/;" f namespace:doris +convert_like_pattern src/exprs/like_predicate.cpp /^void LikePredicate::convert_like_pattern($/;" f class:doris::LikePredicate +convert_old_data_success src/olap/data_dir.cpp /^bool DataDir::convert_old_data_success() {$/;" f class:doris::DataDir +convert_one_row src/exec/broker_scanner.cpp /^bool BrokerScanner::convert_one_row($/;" f class:doris::BrokerScanner +convert_rowset_ids src/olap/snapshot_manager.cpp /^OLAPStatus SnapshotManager::convert_rowset_ids(const string& clone_dir, int64_t tablet_id,$/;" f class:doris::SnapshotManager +convert_to_arrow_batch src/util/arrow/row_batch.cpp /^Status convert_to_arrow_batch($/;" f namespace:doris +convert_to_arrow_batch src/util/arrow/row_block.cpp /^Status convert_to_arrow_batch(const RowBlockV2& block,$/;" f namespace:doris +convert_to_arrow_field src/util/arrow/row_batch.cpp /^Status convert_to_arrow_field(SlotDescriptor* desc,$/;" f namespace:doris +convert_to_arrow_field src/util/arrow/row_block.cpp /^Status convert_to_arrow_field(uint32_t cid,$/;" f namespace:doris +convert_to_arrow_schema src/util/arrow/row_batch.cpp /^Status convert_to_arrow_schema($/;" f namespace:doris +convert_to_arrow_schema src/util/arrow/row_block.cpp /^Status convert_to_arrow_schema(const Schema& schema,$/;" f namespace:doris +convert_to_arrow_type src/util/arrow/row_batch.cpp /^Status convert_to_arrow_type(const TypeDescriptor& type,$/;" f namespace:doris +convert_to_arrow_type src/util/arrow/row_block.cpp /^Status convert_to_arrow_type(FieldType type, std::shared_ptr *result) {$/;" f namespace:doris +convert_to_doris_schema src/util/arrow/row_block.cpp /^Status convert_to_doris_schema(const arrow::Schema& schema,$/;" f namespace:doris +convert_to_doris_type src/util/arrow/row_batch.cpp /^Status convert_to_doris_type(const arrow::DataType& type,$/;" f namespace:doris +convert_to_fixed_value src/exec/olap_common.cpp /^void ColumnValueRange::convert_to_fixed_value() {$/;" f class:doris::ColumnValueRange +convert_to_fixed_value src/exec/olap_common.cpp /^void ColumnValueRange::convert_to_fixed_value() {$/;" f class:doris::ColumnValueRange +convert_to_fixed_value src/exec/olap_common.cpp /^void ColumnValueRange::convert_to_fixed_value() {$/;" f class:doris::ColumnValueRange +convert_to_fixed_value src/exec/olap_common.cpp /^void ColumnValueRange<__int128>::convert_to_fixed_value() {$/;" f class:doris::ColumnValueRange +convert_to_fixed_value src/exec/olap_common.h /^void ColumnValueRange::convert_to_fixed_value() {$/;" f class:doris::ColumnValueRange +convert_to_old_files src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::convert_to_old_files(const std::string& snapshot_path, $/;" f class:doris::AlphaRowset +convert_to_old_files src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::convert_to_old_files(const std::string& snapshot_path,$/;" f class:doris::SegmentGroup +convert_to_pdelta src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::convert_to_pdelta(const RowsetMetaPB& rowset_meta_pb, PDelta* delta) {$/;" f class:doris::OlapSnapshotConverter +convert_to_range_value src/exec/olap_common.h /^void ColumnValueRange::convert_to_range_value() {$/;" f class:doris::ColumnValueRange +convert_to_row_batch src/util/arrow/row_batch.cpp /^Status convert_to_row_batch(const arrow::RecordBatch& batch,$/;" f namespace:doris +convert_to_row_block src/olap/row_block2.cpp /^Status RowBlockV2::convert_to_row_block(RowCursor* helper, RowBlock* dst) {$/;" f class:doris::RowBlockV2 +convert_to_row_block src/util/arrow/row_block.cpp /^Status convert_to_row_block(const arrow::RecordBatch& batch,$/;" f namespace:doris +convert_to_row_desc src/util/arrow/row_batch.cpp /^Status convert_to_row_desc($/;" f namespace:doris +convert_to_rowset_meta src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::convert_to_rowset_meta(const PDelta& delta,$/;" f class:doris::OlapSnapshotConverter +convert_to_rowset_meta src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::convert_to_rowset_meta(const PPendingDelta& pending_delta,$/;" f class:doris::OlapSnapshotConverter +convert_to_slot_desc src/util/arrow/row_batch.cpp /^Status convert_to_slot_desc(const arrow::Field& field,$/;" f namespace:doris +convert_to_tablet_column src/util/arrow/row_block.cpp /^Status convert_to_tablet_column(const arrow::Field& field,$/;" f namespace:doris +convert_to_type_name src/util/arrow/row_block.cpp /^Status convert_to_type_name(const arrow::DataType& type,$/;" f namespace:doris +convert_tz src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::convert_tz(FunctionContext* ctx, const DateTimeVal& ts_val,$/;" f class:doris::TimestampFunctions +coordinate src/geo/wkt_yacc.y /^coordinate:$/;" l +coordinate_list src/geo/wkt_yacc.y /^coordinate_list:$/;" l +coordinate_list_list src/geo/wkt_yacc.y /^coordinate_list_list:$/;" l +coordinator_callback src/runtime/fragment_mgr.cpp /^void FragmentExecState::coordinator_callback($/;" f class:doris::FragmentExecState +copy src/exprs/expr.cpp /^Expr* Expr::copy(ObjectPool* pool, Expr* old_expr) {$/;" f class:doris::Expr +copy src/gutil/strings/stringpiece.cc /^int StringPiece::copy(char* buf, size_type n, size_type pos) const {$/;" f class:StringPiece +copy src/olap/wrapper_field.h /^ void copy(const WrapperField* field) {$/;" f class:doris::WrapperField +copyOnWrite src/util/bitmap_value.h /^ bool copyOnWrite = false;$/;" m class:doris::detail::Roaring64Map +copy_dir src/olap/utils.cpp /^OLAPStatus copy_dir(const string &src_dir, const string &dst_dir) {$/;" f namespace:doris +copy_file src/olap/utils.cpp /^OLAPStatus copy_file(const string& src, const string& dest) {$/;" f namespace:doris +copy_file src/util/file_utils.cpp /^Status FileUtils::copy_file(const std::string& src_path, const std::string& dest_path) {$/;" f class:doris::FileUtils +copy_files_to src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::copy_files_to(const std::string& dir) {$/;" f class:doris::AlphaRowset +copy_files_to src/olap/rowset/beta_rowset.cpp /^OLAPStatus BetaRowset::copy_files_to(const std::string& dir) {$/;" f class:doris::BetaRowset +copy_files_to src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::copy_files_to(const std::string& dir) {$/;" f class:doris::SegmentGroup +copy_from src/udf/udf.cpp /^StringVal StringVal::copy_from(FunctionContext* ctx, const uint8_t* buf, size_t len) {$/;" f class:doris_udf::StringVal +copy_int_to_decimal_frac src/runtime/decimal_value.h /^inline int32_t DecimalValue::copy_int_to_decimal_frac(int64_t frac_value, int32_t* buff) {$/;" f class:doris::DecimalValue +copy_int_to_decimal_int src/runtime/decimal_value.h /^inline int DecimalValue::copy_int_to_decimal_int(int64_t int_value, int32_t* buff) {$/;" f class:doris::DecimalValue +copy_object src/olap/field.h /^ void copy_object(DstCellType* dst, const SrcCellType& src, MemPool* pool) const {$/;" f class:doris::Field +copy_object src/olap/types.h /^ inline void copy_object(void* dest, const void* src, MemPool* mem_pool) const {$/;" f class:doris::TypeInfo +copy_object src/olap/types.h /^ static inline void copy_object(void* dest, const void* src, MemPool* mem_pool) {$/;" f struct:doris::BaseFieldtypeTraits +copy_object src/olap/types.h /^ static void copy_object(void* dest, const void* src, MemPool* mem_pool) {$/;" f struct:doris::FieldTypeTraits +copy_one test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^ void copy_one(PageDecoderType* decoder, typename TypeTraits::CppType* ret) {$/;" f class:doris::BitShufflePageTest +copy_one test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^ void copy_one(PageDecoderType* decoder, typename TypeTraits::CppType* ret) {$/;" f class:doris::FrameOfReferencePageTest +copy_one test/olap/rowset/segment_v2/plain_page_test.cpp /^ void copy_one(PageDecoderType* decoder, typename TypeTraits::CppType* ret) {$/;" f class:doris::segment_v2::PlainPageTest +copy_one test/olap/rowset/segment_v2/rle_page_test.cpp /^ void copy_one(PageDecoderType* decoder, typename TypeTraits::CppType* ret) {$/;" f class:doris::RlePageTest +copy_one_row src/exec/olap_rewrite_node.cpp /^bool OlapRewriteNode::copy_one_row(TupleRow* src_row, Tuple* tuple, $/;" f class:doris::OlapRewriteNode +copy_one_row src/exec/schema_scan_node.cpp /^void SchemaScanNode::copy_one_row() {$/;" f class:doris::SchemaScanNode +copy_row src/olap/row.h /^void copy_row(DstRowType* dst, const SrcRowType& src, MemPool* pool) {$/;" f namespace:doris +copy_row src/runtime/dpp_sink.cpp /^void Translator::copy_row(TupleRow* row) {$/;" f class:doris::Translator +copy_row src/runtime/row_batch.h /^ void copy_row(TupleRow* src, TupleRow* dest) {$/;" f class:doris::RowBatch +copy_row_in_memtable src/olap/row.h /^void copy_row_in_memtable(DstRowType* dst, const SrcRowType& src, MemPool* pool) {$/;" f namespace:doris +copy_rows src/exec/olap_rewrite_node.cpp /^bool OlapRewriteNode::copy_rows(RuntimeState* state, RowBatch* output_batch) {$/;" f class:doris::OlapRewriteNode +copy_rows src/exec/select_node.cpp /^bool SelectNode::copy_rows(RowBatch* output_batch) {$/;" f class:doris::SelectNode +copy_rows src/runtime/row_batch.h /^ void copy_rows(int dest, int src, int num_rows) {$/;" f class:doris::RowBatch +copy_state src/common/status.cpp /^const char* Status::copy_state(const char* state) {$/;" f class:doris::Status +copy_strings src/runtime/buffered_tuple_stream2_ir.cc /^bool BufferedTupleStream2::copy_strings(const Tuple* tuple,$/;" f class:doris::BufferedTupleStream2 +copy_value src/util/frame_of_reference_coding.cpp /^T* ForDecoder::copy_value(T* val, size_t count) {$/;" f class:doris::ForDecoder +copy_value src/util/frame_of_reference_coding.cpp /^const T* ForEncoder::copy_value(const T *p_data, size_t count) {$/;" f class:doris::ForEncoder +copy_var_len_data src/runtime/merge_sorter.cpp /^void MergeSorter::Run::copy_var_len_data(char* dest, const std::vector& var_values) {$/;" f class:doris::MergeSorter::Run +copy_var_len_data src/runtime/spill_sorter.cc /^void SpillSorter::Run::copy_var_len_data(char* dest, const vector& string_values) {$/;" f class:doris::SpillSorter::Run +copy_var_len_data_convert_offset src/runtime/spill_sorter.cc /^void SpillSorter::Run::copy_var_len_data_convert_offset(char* dest, int64_t offset,$/;" f class:doris::SpillSorter::Run +core_id src/runtime/memory/chunk.h /^ int core_id = 0;$/;" m struct:doris::Chunk +core_to_numa_node_ src/util/cpu_info.cpp /^unique_ptr CpuInfo::core_to_numa_node_;$/;" m class:doris::CpuInfo file: +core_to_numa_node_ src/util/cpu_info.h /^ static std::unique_ptr core_to_numa_node_;$/;" m class:doris::CpuInfo +count src/exprs/aggregate_functions.cpp /^ int64_t count = 0;$/;" m struct:doris::AvgState file: +count src/exprs/aggregate_functions.cpp /^ int64_t count = 0;$/;" m struct:doris::DecimalV2AvgState file: +count src/exprs/aggregate_functions.cpp /^ int64_t count;$/;" m struct:doris::DecimalAvgState file: +count src/exprs/aggregate_functions.cpp /^ int64_t count;$/;" m struct:doris::KnuthVarianceState file: +count src/exprs/aggregate_functions.cpp /^ int64_t count;$/;" m struct:doris::RankState file: +count src/olap/olap_index.h /^ const size_t count() const {$/;" f struct:doris::SegmentMetaInfo +count src/olap/olap_index.h /^ size_t count() const {$/;" f class:doris::MemIndex +count src/olap/rowset/segment_v2/binary_dict_page.cpp /^size_t BinaryDictPageBuilder::count() const {$/;" f class:doris::segment_v2::BinaryDictPageBuilder +count src/olap/rowset/segment_v2/bitshuffle_page.h /^ size_t count() const {$/;" f class:doris::segment_v2::BitshufflePageBuilder +count src/olap/rowset/segment_v2/index_page.h /^ inline size_t count() const {$/;" f class:doris::segment_v2::IndexPageReader +count src/olap/rowset/segment_v2/index_page.h /^ size_t count() const { return _count; }$/;" f class:doris::segment_v2::IndexPageBuilder +count src/olap/rowset/segment_v2/plain_page.h /^ size_t count() const {$/;" f class:doris::segment_v2::PlainPageBuilder +count src/olap/rowset/segment_v2/row_ranges.h /^ size_t count() const {$/;" f class:doris::segment_v2::RowRange +count src/olap/rowset/segment_v2/row_ranges.h /^ size_t count() {$/;" f class:doris::segment_v2::RowRanges +count src/util/countdown_latch.h /^ uint64_t count() const {$/;" f class:doris::CountDownLatch +count src/util/frame_of_reference_coding.h /^ uint32_t count() const {$/;" f class:doris::ForDecoder +count test/util/thread_pool_test.cpp /^void count(int thread_id, const int& i) {$/;" f namespace:doris +count_ src/gutil/strings/split.h /^ int count_;$/;" m class:strings::delimiter::LimitImpl +count_ src/util/countdown_latch.h /^ uint64_t count_;$/;" m class:doris::CountDownLatch +count_distinct_data_filter src/exprs/agg_fn_evaluator.cpp /^bool AggFnEvaluator::count_distinct_data_filter(TupleRow* row, Tuple* dst) {$/;" f class:doris::AggFnEvaluator +count_distinct_date_finalize src/exprs/aggregate_functions.cpp /^BigIntVal AggregateFunctions::count_distinct_date_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +count_distinct_date_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_distinct_date_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +count_distinct_date_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_distinct_date_merge(FunctionContext* ctx, StringVal& src,$/;" f class:doris::AggregateFunctions +count_distinct_date_serialize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::count_distinct_date_serialize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +count_distinct_date_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_distinct_date_update(FunctionContext* ctx, DateTimeVal& src,$/;" f class:doris::AggregateFunctions +count_distinct_decimal_finalize src/exprs/aggregate_functions.cpp /^BigIntVal AggregateFunctions::count_distinct_decimal_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +count_distinct_decimalv2_finalize src/exprs/aggregate_functions.cpp /^BigIntVal AggregateFunctions::count_distinct_decimalv2_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +count_distinct_string_finalize src/exprs/aggregate_functions.cpp /^BigIntVal AggregateFunctions::count_distinct_string_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +count_distinct_string_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_distinct_string_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +count_distinct_string_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_distinct_string_merge(FunctionContext* ctx, StringVal& src,$/;" f class:doris::AggregateFunctions +count_distinct_string_serialize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::count_distinct_string_serialize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +count_distinct_string_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_distinct_string_update(FunctionContext* ctx, StringVal& src,$/;" f class:doris::AggregateFunctions +count_down src/util/countdown_latch.h /^ void count_down() {$/;" f class:doris::CountDownLatch +count_down src/util/countdown_latch.h /^ void count_down(int amount) {$/;" f class:doris::CountDownLatch +count_finalize src/exprs/aggregate_functions.cpp /^ BigIntVal count_finalize() {$/;" f class:doris::MultiDistinctCountDateState +count_finalize src/exprs/aggregate_functions.cpp /^ BigIntVal count_finalize() {$/;" f class:doris::MultiDistinctDecimalState +count_finalize src/exprs/aggregate_functions.cpp /^ BigIntVal count_finalize() {$/;" f class:doris::MultiDistinctDecimalV2State +count_finalize src/exprs/aggregate_functions.cpp /^ BigIntVal count_finalize() {$/;" f class:doris::MultiDistinctNumericState +count_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_merge(FunctionContext*, const BigIntVal& src,$/;" f class:doris::AggregateFunctions +count_nibbles src/util/simdutf8check.h /^static inline void count_nibbles(__m128i bytes,$/;" f +count_or_sum_distinct_decimal_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_or_sum_distinct_decimal_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_decimal_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_or_sum_distinct_decimal_merge(FunctionContext* ctx, StringVal& src,$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_decimal_serialize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::count_or_sum_distinct_decimal_serialize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_decimal_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_or_sum_distinct_decimal_update(FunctionContext* ctx, DecimalVal& src,$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_decimalv2_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_or_sum_distinct_decimalv2_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_decimalv2_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_or_sum_distinct_decimalv2_merge(FunctionContext* ctx, StringVal& src,$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_decimalv2_serialize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::count_or_sum_distinct_decimalv2_serialize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_decimalv2_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_or_sum_distinct_decimalv2_update(FunctionContext* ctx, DecimalV2Val& src,$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_numeric_finalize src/exprs/aggregate_functions.cpp /^BigIntVal AggregateFunctions::count_or_sum_distinct_numeric_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_numeric_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_or_sum_distinct_numeric_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_numeric_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_or_sum_distinct_numeric_merge(FunctionContext* ctx, StringVal& src,$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_numeric_serialize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::count_or_sum_distinct_numeric_serialize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +count_or_sum_distinct_numeric_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_or_sum_distinct_numeric_update(FunctionContext* ctx, T& src,$/;" f class:doris::AggregateFunctions +count_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_remove($/;" f class:doris::AggregateFunctions +count_selected src/olap/selection_vector.h /^ size_t count_selected() const {$/;" f class:doris::SelectionVector +count_star_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_star_remove(FunctionContext*, BigIntVal* dst) {$/;" f class:doris::AggregateFunctions +count_star_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_star_update(FunctionContext*, BigIntVal* dst) {$/;" f class:doris::AggregateFunctions +count_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::count_update($/;" f class:doris::AggregateFunctions +counter src/util/perf_counters.h /^ Counter counter;$/;" m struct:doris::PerfCounters::CounterData +counter test/runtime/thread_resource_mgr_test.cpp /^ int counter() const {$/;" f class:doris::NotifiedCounter +counter_ src/gutil/threading/thread_collision_warner.h /^ volatile subtle::Atomic64 counter_;$/;" m class:base::ThreadCollisionWarner +counter_names src/util/perf_counters.h /^ const std::vector* counter_names() const {$/;" f class:doris::PerfCounters +counter_sum src/util/runtime_profile.cpp /^int64_t RuntimeProfile::counter_sum(const std::vector* counters) {$/;" f class:doris::RuntimeProfile +counters src/runtime/bufferpool/buffer_pool_internal.h /^ const BufferPoolClientCounters& counters() const { return counters_; }$/;" f class:doris::BufferPool::Client +counters src/util/perf_counters.cpp /^const vector* PerfCounters::counters(int snapshot) const {$/;" f class:doris::PerfCounters +counters_ src/runtime/bufferpool/buffer_pool_internal.h /^ BufferPoolClientCounters counters_;$/;" m class:doris::BufferPool::Client +counters_ src/runtime/bufferpool/reservation_tracker.h /^ ReservationTrackerCounters counters_;$/;" m class:doris::ReservationTracker +cpu_brand src/gutil/cpu.h /^ const std::string& cpu_brand() const { return cpu_brand_; }$/;" f class:base::final +cpu_brand_ src/gutil/cpu.h /^ std::string cpu_brand_;$/;" m class:base::final +cpu_vendor_ src/gutil/cpu.h /^ std::string cpu_vendor_;$/;" m class:base::final +cpu_wait src/common/atomic.h /^ static ALWAYS_INLINE void cpu_wait() {$/;" f class:doris::AtomicUtil +cpuid src/gutil/atomicops-internals-x86.cc 44;" d file: +cpuid src/gutil/atomicops-internals-x86.cc 50;" d file: +cpuinfo_cycles_per_second src/gutil/sysinfo.cc /^static double cpuinfo_cycles_per_second = 1.0; \/\/ 0.0 might be dangerous$/;" m namespace:base file: +cpuinfo_max_cpu_index src/gutil/sysinfo.cc /^static int cpuinfo_max_cpu_index = -1;$/;" m namespace:base file: +cpuinfo_num_cpus src/gutil/sysinfo.cc /^static int cpuinfo_num_cpus = 1; \/\/ Conservative guess$/;" m namespace:base file: +crc src/util/minizip/unzip.h /^ uLong crc; \/* crc-32 4 bytes *\/$/;" m struct:unz_file_info64_s +crc src/util/minizip/unzip.h /^ uLong crc; \/* crc-32 4 bytes *\/$/;" m struct:unz_file_info_s +crc32 src/olap/out_stream.cpp /^uint32_t OutStream::crc32(uint32_t checksum) const {$/;" f class:doris::OutStream +crc32 src/util/minizip/unzip.c /^ uLong crc32; \/* crc32 of all data uncompressed *\/$/;" m struct:__anon33 file: +crc32_wait src/util/minizip/unzip.c /^ uLong crc32_wait; \/* crc32 we must obtain after decompress all *\/$/;" m struct:__anon33 file: +crc32c src/util/crc32c.cpp /^namespace crc32c {$/;" n namespace:doris file: +crc32c src/util/crc32c.h /^namespace crc32c {$/;" n namespace:doris +crc32c test/util/crc32c_test.cpp /^namespace crc32c {$/;" n namespace:doris file: +crc32c_lut src/olap/utils.cpp /^unsigned int crc32c_lut(char const * b, unsigned int off, unsigned int len, unsigned int crc) {$/;" f namespace:doris +crc32c_qw src/olap/bhp_lib.h /^inline int crc32c_qw(char const* src, int crc, unsigned int qwlen) {$/;" f namespace:doris +crc_hash src/util/hash_util.hpp /^ static uint32_t crc_hash(const void* data, int32_t bytes, uint32_t hash) {$/;" f class:doris::HashUtil +crc_hash64 src/util/hash_util.hpp /^ static uint64_t crc_hash64(const void* data, int32_t bytes, uint64_t hash) {$/;" f class:doris::HashUtil +create src/exec/partitioned_hash_table.cc /^PartitionedHashTable* PartitionedHashTable::create(RuntimeState* state,$/;" f class:doris::PartitionedHashTable +create src/exec/schema_scanner.cpp /^SchemaScanner* SchemaScanner::create(TSchemaTableType::type type) {$/;" f class:doris::SchemaScanner +create src/exprs/agg_fn_evaluator.cpp /^Status AggFnEvaluator::create($/;" f class:doris::AggFnEvaluator +create src/exprs/agg_fn_evaluator.cpp /^Status AggFnEvaluator::create(ObjectPool* pool,$/;" f class:doris::AggFnEvaluator +create src/exprs/aggregate_functions.cpp /^ static void create(StringVal* dst) {$/;" f class:doris::MultiDistinctCountDateState +create src/exprs/aggregate_functions.cpp /^ static void create(StringVal* dst) {$/;" f class:doris::MultiDistinctDecimalState +create src/exprs/aggregate_functions.cpp /^ static void create(StringVal* dst) {$/;" f class:doris::MultiDistinctDecimalV2State +create src/exprs/aggregate_functions.cpp /^ static void create(StringVal* dst) {$/;" f class:doris::MultiDistinctNumericState +create src/exprs/aggregate_functions.cpp /^ static void create(StringVal* dst) {$/;" f class:doris::MultiDistinctStringCountState +create src/exprs/expr.cpp /^Status Expr::create(const TExpr& texpr, const RowDescriptor& row_desc,$/;" f class:doris::Expr +create src/exprs/expr.cpp /^Status Expr::create(const vector& texprs, const RowDescriptor& row_desc,$/;" f class:doris::Expr +create src/olap/byte_buffer.cpp /^StorageByteBuffer* StorageByteBuffer::create(uint64_t capacity) {$/;" f class:doris::StorageByteBuffer +create src/olap/field.h /^ static Field* create(const TabletColumn& column) {$/;" f class:doris::FieldFactory +create src/olap/push_handler.cpp /^IBinaryReader* IBinaryReader::create(bool need_decompress) {$/;" f class:doris::IBinaryReader +create src/olap/rowset/column_data.cpp /^ColumnData* ColumnData::create(SegmentGroup* segment_group) {$/;" f class:doris::ColumnData +create src/olap/rowset/column_data_writer.cpp /^ColumnDataWriter* ColumnDataWriter::create(SegmentGroup* segment_group, bool is_push_write,$/;" f class:doris::ColumnDataWriter +create src/olap/rowset/column_reader.cpp /^ ColumnReader* ColumnReader::create(uint32_t column_id,$/;" f class:doris::ColumnReader +create src/olap/rowset/column_reader.cpp /^ColumnReader* ColumnReader::create(uint32_t column_id,$/;" f class:doris::ColumnReader +create src/olap/rowset/column_writer.cpp /^ColumnWriter* ColumnWriter::create(uint32_t column_id,$/;" f class:doris::ColumnWriter +create src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^Status BitmapIndexWriter::create(const TypeInfo* typeinfo, std::unique_ptr* res) {$/;" f class:doris::segment_v2::BitmapIndexWriter +create src/olap/rowset/segment_v2/bloom_filter.cpp /^Status BloomFilter::create(BloomFilterAlgorithmPB algorithm, std::unique_ptr* bf) {$/;" f class:doris::segment_v2::BloomFilter +create src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^Status BloomFilterIndexWriter::create(const BloomFilterOptions& bf_options,$/;" f class:doris::segment_v2::BloomFilterIndexWriter +create src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::create(const ColumnReaderOptions& opts,$/;" f class:doris::segment_v2::ColumnReader +create src/olap/rowset/segment_v2/parsed_page.h /^ static Status create(PageHandle handle,$/;" f struct:doris::segment_v2::ParsedPage +create src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::create(const TCreateTabletReq& request, const TabletUid& tablet_uid,$/;" f class:doris::TabletMeta +create src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::create(int64_t table_id, int64_t partition_id,$/;" f class:doris::TabletMeta +create src/olap/wrapper_field.cpp /^WrapperField* WrapperField::create(const TabletColumn& column, uint32_t len) {$/;" f class:doris::WrapperField +create src/runtime/buffered_block_mgr.cpp /^Status BufferedBlockMgr::create(RuntimeState* state,$/;" f class:doris::BufferedBlockMgr +create src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::create($/;" f class:doris::BufferedBlockMgr2 +create src/runtime/descriptors.cpp /^Status DescriptorTbl::create(ObjectPool* pool, const TDescriptorTable& thrift_tbl,$/;" f class:doris::DescriptorTbl +create src/runtime/tuple.h /^ static Tuple* create(int size, MemPool* pool) {$/;" f class:doris::Tuple +create src/runtime/tuple_row.h /^ static TupleRow* create(const std::vector& descs, MemPool* pool) {$/;" f class:doris::TupleRow +create src/util/bfd_parser.cpp /^BfdParser* BfdParser::create() {$/;" f class:doris::BfdParser +create src/util/bfd_parser.cpp /^BfdParser* BfdParser::create(const std::string& prog_name) {$/;" f class:doris::BfdParser +create src/util/thread.h /^ static Status create(const std::string& category, const std::string& name, const F& f,$/;" f class:doris::Thread +createContext src/util/thrift_server.cpp /^void* ThriftServer::ThriftServerEventProcessor::createContext($/;" f class:doris::ThriftServer::ThriftServerEventProcessor +create_and_init_rowset_reader test/olap/rowset/beta_rowset_test.cpp /^ void create_and_init_rowset_reader(Rowset* rowset, RowsetReaderContext& context, RowsetReaderSharedPtr* result) {$/;" f class:doris::BetaRowsetTest +create_and_save_last_position test/olap/column_reader_test.cpp /^ void create_and_save_last_position() {$/;" f class:doris::TestColumn +create_and_save_last_position test/olap/schema_change_test.cpp /^ void create_and_save_last_position() {$/;" f class:doris::TestColumn +create_any_val src/exprs/anyval_util.cpp /^AnyVal* create_any_val(ObjectPool* pool, const TypeDescriptor& type) {$/;" f namespace:doris +create_binary_predicate test/olap/vectorized_olap_reader_test.cpp /^ Expr* create_binary_predicate() {$/;" f class:doris::TestVectorizedOLAPReader +create_block src/olap/fs/file_block_manager.cpp /^Status FileBlockManager::create_block(const CreateBlockOptions& opts,$/;" f class:doris::fs::FileBlockManager +create_block_mgr src/runtime/runtime_state.cpp /^Status RuntimeState::create_block_mgr() {$/;" f class:doris::RuntimeState +create_by_type src/olap/field.h /^ static Field* create_by_type(const FieldType& type) {$/;" f class:doris::FieldFactory +create_by_type src/olap/wrapper_field.cpp /^WrapperField* WrapperField::create_by_type(const FieldType& type, int32_t var_length) {$/;" f class:doris::WrapperField +create_by_type src/olap/wrapper_field.h /^ static WrapperField* create_by_type(const FieldType& type) {$/;" f class:doris::WrapperField +create_char_key test/olap/tablet_schema_helper.h /^TabletColumn create_char_key(int32_t id, bool is_nullable = true) {$/;" f namespace:doris +create_char_type src/runtime/types.h /^ static TypeDescriptor create_char_type(int len) {$/;" f struct:doris::TypeDescriptor +create_child src/util/runtime_profile.cpp /^RuntimeProfile* RuntimeProfile::create_child(const std::string& name, bool indent,$/;" f class:doris::RuntimeProfile +create_client src/runtime/client_cache.cpp /^Status ClientCacheHelper::create_client($/;" f class:doris::ClientCacheHelper +create_comparetor src/runtime/dpp_sink.cpp /^Status Translator::create_comparetor(RuntimeState* state) {$/;" f class:doris::Translator +create_context src/udf/udf.cpp /^doris_udf::FunctionContext* FunctionContextImpl::create_context($/;" f class:doris::FunctionContextImpl +create_data_sink src/exec/data_sink.cpp /^Status DataSink::create_data_sink($/;" f class:doris::DataSink +create_decimal_type src/runtime/types.h /^ static TypeDescriptor create_decimal_type(int precision, int scale) {$/;" f struct:doris::TypeDescriptor +create_decimalv2_type src/runtime/types.h /^ static TypeDescriptor create_decimalv2_type(int precision, int scale) {$/;" f struct:doris::TypeDescriptor +create_decompressor src/exec/broker_scanner.cpp /^Status BrokerScanner::create_decompressor(TFileFormatType::type type) {$/;" f class:doris::BrokerScanner +create_decompressor src/exec/decompressor.cpp /^Status Decompressor::create_decompressor(CompressType type,$/;" f class:doris::Decompressor +create_descriptor_table test/runtime/load_channel_mgr_test.cpp /^TDescriptorTable create_descriptor_table() {$/;" f namespace:doris +create_descriptor_tablet test/olap/delta_writer_test.cpp /^TDescriptorTable create_descriptor_tablet() {$/;" f namespace:doris +create_descriptors test/runtime/buffered_tuple_stream2_test.cpp /^ virtual void create_descriptors() {$/;" f class:doris::MultiNullableTupleStreamTest +create_descriptors test/runtime/buffered_tuple_stream2_test.cpp /^ virtual void create_descriptors() {$/;" f class:doris::MultiTupleStreamTest +create_descriptors test/runtime/buffered_tuple_stream2_test.cpp /^ virtual void create_descriptors() {$/;" f class:doris::SimpleNullStreamTest +create_descriptors test/runtime/buffered_tuple_stream2_test.cpp /^ virtual void create_descriptors() {$/;" f class:doris::SimpleTupleStreamTest +create_deserialize_protocol src/util/thrift_util.cpp /^boost::shared_ptr create_deserialize_protocol($/;" f namespace:doris +create_dir src/util/file_utils.cpp /^Status FileUtils::create_dir(const std::string& dir_path) {$/;" f class:doris::FileUtils +create_dir src/util/file_utils.cpp /^Status FileUtils::create_dir(const std::string& path, Env* env) {$/;" f class:doris::FileUtils +create_directory src/util/filesystem_util.cc /^Status FileSystemUtil::create_directory(const string& directory) {$/;" f class:doris::FileSystemUtil +create_dst_tuple test/exec/parquet_scanner_test.cpp /^int ParquetSannerTest::create_dst_tuple(TDescriptorTable& t_desc_table, int next_slot_id) {$/;" f class:doris::ParquetSannerTest +create_error_log_file src/runtime/runtime_state.cpp /^Status RuntimeState::create_error_log_file() {$/;" f class:doris::RuntimeState +create_expr src/exprs/expr.cpp /^Status Expr::create_expr(ObjectPool* pool, const TExprNode& texpr_node, Expr** expr) {$/;" f class:doris::Expr +create_expr test/exprs/binary_predicate_test.cpp /^ Expr* create_expr() {$/;" f class:doris::BinaryOpTest +create_expr test/exprs/in_op_test.cpp /^ Expr* create_expr() {$/;" f class:doris::InOpTest +create_expr_info test/exec/parquet_scanner_test.cpp /^void ParquetSannerTest::create_expr_info() {$/;" f class:doris::ParquetSannerTest +create_expr_tree src/exprs/expr.cpp /^Status Expr::create_expr_tree(ObjectPool* pool, const TExpr& texpr, ExprContext** ctx) {$/;" f class:doris::Expr +create_expr_trees src/exprs/expr.cpp /^Status Expr::create_expr_trees($/;" f class:doris::Expr +create_file src/util/filesystem_util.cc /^Status FileSystemUtil::create_file(const string& file_path) {$/;" f class:doris::FileSystemUtil +create_flush_token src/olap/memtable_flush_executor.cpp /^OLAPStatus MemTableFlushExecutor::create_flush_token(std::unique_ptr* flush_token) {$/;" f class:doris::MemTableFlushExecutor +create_from_file src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::create_from_file(const string& file_path) {$/;" f class:doris::TabletMeta +create_from_thrift src/util/runtime_profile.cpp /^RuntimeProfile* RuntimeProfile::create_from_thrift($/;" f class:doris::RuntimeProfile +create_global_cache src/olap/page_cache.cpp /^void StoragePageCache::create_global_cache(size_t capacity) {$/;" f class:doris::StoragePageCache +create_hash_partitions src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::create_hash_partitions(int level) {$/;" f class:doris::PartitionedAggregationNode +create_heartbeat_server src/agent/heartbeat_server.cpp /^AgentStatus create_heartbeat_server($/;" f namespace:doris +create_hll_type src/runtime/types.h /^ static TypeDescriptor create_hll_type() {$/;" f struct:doris::TypeDescriptor +create_hub src/util/load_error_hub.cpp /^Status LoadErrorHub::create_hub($/;" f class:doris::LoadErrorHub +create_in_predicate test/olap/vectorized_olap_reader_test.cpp /^ ExprContent* create_in_predicate(int v1, int v2) {$/;" f class:doris::TestVectorizedOLAPReader +create_int_key test/olap/tablet_schema_helper.h /^TabletColumn create_int_key(int32_t id, bool is_nullable = true,$/;" f namespace:doris +create_int_value test/olap/tablet_schema_helper.h /^TabletColumn create_int_value($/;" f namespace:doris +create_load_dir src/runtime/runtime_state.cpp /^Status RuntimeState::create_load_dir() {$/;" f class:doris::RuntimeState +create_merger src/runtime/data_stream_recvr.cc /^Status DataStreamRecvr::create_merger(const TupleRowComparator& less_than) {$/;" f class:doris::DataStreamRecvr +create_merger src/runtime/spill_sorter.cc /^Status SpillSorter::create_merger(int num_runs) {$/;" f class:doris::SpillSorter +create_node src/exec/exec_node.cpp /^Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanNode& tnode,$/;" f class:doris::ExecNode +create_output_row src/exec/blocking_join_node.cpp /^void BlockingJoinNode::create_output_row(TupleRow* out, TupleRow* left, TupleRow* build) {$/;" f class:doris::BlockingJoinNode +create_output_row src/exec/hash_join_node.cpp /^void HashJoinNode::create_output_row(TupleRow* out, TupleRow* probe, TupleRow* build) {$/;" f class:doris::HashJoinNode +create_output_row src/exec/merge_join_node.cpp /^void MergeJoinNode::create_output_row(TupleRow* out, TupleRow* left, TupleRow* right) {$/;" f class:doris::MergeJoinNode +create_page_builder src/olap/rowset/segment_v2/encoding_info.cpp /^ static Status create_page_builder(const PageBuilderOptions& opts, PageBuilder** builder) {$/;" f struct:doris::segment_v2::TypeEncodingTraits +create_page_builder src/olap/rowset/segment_v2/encoding_info.h /^ Status create_page_builder(const PageBuilderOptions& opts, PageBuilder** builder) const {$/;" f class:doris::segment_v2::EncodingInfo +create_page_decoder src/olap/rowset/segment_v2/encoding_info.cpp /^ static Status create_page_decoder(const Slice& data, const PageDecoderOptions& opts, PageDecoder** decoder) {$/;" f struct:doris::segment_v2::TypeEncodingTraits +create_page_decoder src/olap/rowset/segment_v2/encoding_info.h /^ Status create_page_decoder(const Slice& data, const PageDecoderOptions& opts, PageDecoder** decoder) const {$/;" f class:doris::segment_v2::EncodingInfo +create_profile src/runtime/dpp_sink.cpp /^Status Translator::create_profile(RuntimeState* state) {$/;" f class:doris::Translator +create_query_state src/runtime/test_env.cc /^Status TestEnv::create_query_state(int64_t query_id, int max_buffers, int block_size,$/;" f class:doris::TestEnv +create_query_states src/runtime/test_env.cc /^Status TestEnv::create_query_states(int64_t start_query_id, int num_mgrs,$/;" f class:doris::TestEnv +create_queue src/runtime/result_queue_mgr.cpp /^void ResultQueueMgr::create_queue(const TUniqueId& fragment_instance_id, BlockQueueSharedPtr* queue) {$/;" f class:doris::ResultQueueMgr +create_reader src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::create_reader(std::shared_ptr* result) {$/;" f class:doris::AlphaRowset +create_reader src/olap/rowset/beta_rowset.cpp /^OLAPStatus BetaRowset::create_reader(RowsetReaderSharedPtr* result) {$/;" f class:doris::BetaRowset +create_recvr src/runtime/data_stream_mgr.cpp /^shared_ptr DataStreamMgr::create_recvr(RuntimeState* state,$/;" f class:doris::DataStreamMgr +create_rollup_requests_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::create_rollup_requests_failed;$/;" m class:doris::DorisMetrics file: +create_rollup_requests_failed src/util/doris_metrics.h /^ static IntCounter create_rollup_requests_failed;$/;" m class:doris::DorisMetrics +create_rollup_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::create_rollup_requests_total;$/;" m class:doris::DorisMetrics file: +create_rollup_requests_total src/util/doris_metrics.h /^ static IntCounter create_rollup_requests_total;$/;" m class:doris::DorisMetrics +create_row_batch test/runtime/buffered_tuple_stream_test.cpp /^RowBatch* BufferedTupleStreamTest::create_row_batch(int num_rows) {$/;" f class:doris::BufferedTupleStreamTest +create_row_batch test/runtime/data_stream_test.cpp /^ RowBatch* create_row_batch() {$/;" f class:doris::DataStreamTest +create_row_desc test/runtime/data_stream_test.cpp /^ void create_row_desc() {$/;" f class:doris::DataStreamTest +create_row_index_entry src/olap/rowset/column_writer.cpp /^OLAPStatus ColumnWriter::create_row_index_entry() {$/;" f class:doris::ColumnWriter +create_rowset src/olap/rowset/rowset_factory.cpp /^OLAPStatus RowsetFactory::create_rowset(const TabletSchema* schema,$/;" f class:doris::RowsetFactory +create_rowset_reader_context test/olap/rowset/alpha_rowset_test.cpp /^void create_rowset_reader_context(TabletSchema* tablet_schema, const std::vector* return_columns,$/;" f namespace:doris +create_rowset_reader_context test/olap/rowset/rowset_converter_test.cpp /^void create_rowset_reader_context(TabletSchema* tablet_schema, const std::vector* return_columns,$/;" f namespace:doris +create_rowset_writer src/olap/rowset/rowset_factory.cpp /^OLAPStatus RowsetFactory::create_rowset_writer(const RowsetWriterContext& context,$/;" f class:doris::RowsetFactory +create_rowset_writer_context test/olap/rowset/alpha_rowset_test.cpp /^void create_rowset_writer_context(TabletSchema* tablet_schema,$/;" f namespace:doris +create_rowset_writer_context test/olap/rowset/beta_rowset_test.cpp /^ void create_rowset_writer_context(TabletSchema* tablet_schema,$/;" f class:doris::BetaRowsetTest +create_rowset_writer_context test/olap/rowset/rowset_converter_test.cpp /^void create_rowset_writer_context(TabletSchema* tablet_schema, RowsetTypePB dst_type,$/;" f namespace:doris +create_runtime_state src/runtime/test_env.cc /^RuntimeState* TestEnv::create_runtime_state(int64_t query_id) {$/;" f class:doris::TestEnv +create_scan_context src/runtime/external_scan_context_mgr.cpp /^Status ExternalScanContextMgr::create_scan_context(std::shared_ptr* p_context) {$/;" f class:doris::ExternalScanContextMgr +create_scanner src/exec/broker_scan_node.cpp /^std::unique_ptr BrokerScanNode::create_scanner(const TBrokerScanRange& scan_range,$/;" f class:doris::BrokerScanNode +create_schema test/olap/generic_iterators_test.cpp /^Schema create_schema() {$/;" f namespace:doris +create_schema test/olap/memtable_flush_executor_test.cpp /^Schema create_schema() {$/;" f namespace:doris +create_schema test/olap/rowset/segment_v2/segment_test.cpp /^ TabletSchema create_schema(const vector& columns, int num_short_key_columns = -1) {$/;" f class:doris::segment_v2::SegmentReaderWriterTest +create_schema test/runtime/load_channel_mgr_test.cpp /^void create_schema(DescriptorTbl* desc_tbl, POlapTableSchemaParam* pschema) {$/;" f namespace:doris +create_sender src/runtime/result_buffer_mgr.cpp /^Status ResultBufferMgr::create_sender($/;" f class:doris::ResultBufferMgr +create_service src/service/backend_service.cpp /^Status BackendService::create_service(ExecEnv* exec_env, int port, ThriftServer** server) {$/;" f class:doris::BackendService +create_set src/exprs/hybird_set.cpp /^HybirdSetBase* HybirdSetBase::create_set(PrimitiveType type) {$/;" f class:doris::HybirdSetBase +create_single src/olap/rowset/segment_v2/row_ranges.h /^ static RowRanges create_single(int64_t from, int64_t to) {$/;" f class:doris::segment_v2::RowRanges +create_single src/olap/rowset/segment_v2/row_ranges.h /^ static RowRanges create_single(uint64_t row_count) {$/;" f class:doris::segment_v2::RowRanges +create_sorter src/runtime/dpp_sink.cpp /^Status Translator::create_sorter(RuntimeState* state) {$/;" f class:doris::Translator +create_src_tuple test/exec/parquet_scanner_test.cpp /^int ParquetSannerTest::create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id) {$/;" f class:doris::ParquetSannerTest +create_stream src/olap/out_stream.cpp /^OutStream* OutStreamFactory::create_stream($/;" f class:doris::OutStreamFactory +create_tablet src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::create_tablet(const TCreateTabletReq& request) {$/;" f class:doris::StorageEngine +create_tablet src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::create_tablet(const TCreateTabletReq& request,$/;" f class:doris::TabletManager +create_tablet_from_meta src/olap/tablet.cpp /^TabletSharedPtr Tablet::create_tablet_from_meta(TabletMetaSharedPtr tablet_meta,$/;" f class:doris::Tablet +create_tablet_meta test/olap/rowset/rowset_converter_test.cpp /^void create_tablet_meta(TabletSchema* tablet_schema, TabletMeta* tablet_meta) {$/;" f namespace:doris +create_tablet_request test/olap/delta_writer_test.cpp /^void create_tablet_request(int64_t tablet_id, int32_t schema_hash, TCreateTabletReq* request) {$/;" f namespace:doris +create_tablet_requests_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::create_tablet_requests_failed;$/;" m class:doris::DorisMetrics file: +create_tablet_requests_failed src/util/doris_metrics.h /^ static IntCounter create_tablet_requests_failed;$/;" m class:doris::DorisMetrics +create_tablet_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::create_tablet_requests_total;$/;" m class:doris::DorisMetrics file: +create_tablet_requests_total src/util/doris_metrics.h /^ static IntCounter create_tablet_requests_total;$/;" m class:doris::DorisMetrics +create_tablet_schema test/olap/rowset/alpha_rowset_test.cpp /^void create_tablet_schema(KeysType keys_type, TabletSchema* tablet_schema) {$/;" f namespace:doris +create_tablet_schema test/olap/rowset/beta_rowset_test.cpp /^ void create_tablet_schema(TabletSchema* tablet_schema) {$/;" f class:doris::BetaRowsetTest +create_tablet_schema test/olap/rowset/rowset_converter_test.cpp /^void create_tablet_schema(KeysType keys_type, TabletSchema* tablet_schema) {$/;" f namespace:doris +create_temp_string_val src/udf/udf.cpp /^StringVal StringVal::create_temp_string_val(FunctionContext* ctx, int len) {$/;" f class:doris_udf::StringVal +create_test_context src/udf/udf.cpp /^FunctionContext* FunctionContext::create_test_context() {$/;" f class:doris_udf::FunctionContext +create_thread src/util/threadpool.cpp /^Status ThreadPool::create_thread() {$/;" f class:doris::ThreadPool +create_tree src/exec/exec_node.cpp /^Status ExecNode::create_tree(RuntimeState* state, ObjectPool* pool, const TPlan& plan,$/;" f class:doris::ExecNode +create_tree src/exprs/expr.cpp /^Status Expr::create_tree(const TExpr& texpr, ObjectPool* pool, Expr* root) {$/;" f class:doris::Expr +create_tree_from_thrift src/exprs/expr.cpp /^Status Expr::create_tree_from_thrift($/;" f class:doris::Expr +create_tree_helper src/exec/exec_node.cpp /^Status ExecNode::create_tree_helper($/;" f class:doris::ExecNode +create_tree_internal src/exprs/expr.cpp /^Status Expr::create_tree_internal(const vector& nodes, ObjectPool* pool,$/;" f class:doris::Expr +create_tuple_comparator test/runtime/data_stream_test.cpp /^ void create_tuple_comparator() {$/;" f class:doris::DataStreamTest +create_tuple_desc src/exec/schema_scanner.cpp /^Status SchemaScanner::create_tuple_desc(ObjectPool* pool) {$/;" f class:doris::SchemaScanner +create_tuple_row test/exec/hash_table_test.cpp /^TupleRow* HashTableTest::create_tuple_row(int32_t val) {$/;" f class:doris::HashTableTest +create_value_updaters src/runtime/dpp_sink.cpp /^Status Translator::create_value_updaters() {$/;" f class:doris::Translator +create_varchar_key test/olap/tablet_schema_helper.h /^TabletColumn create_varchar_key(int32_t id, bool is_nullable = true) {$/;" f namespace:doris +create_varchar_type src/runtime/types.h /^ static TypeDescriptor create_varchar_type(int len) {$/;" f struct:doris::TypeDescriptor +create_with_default_value test/olap/tablet_schema_helper.h /^TabletColumn create_with_default_value(std::string default_value) {$/;" f namespace:doris +create_with_flags src/util/thread.h /^ static Status create_with_flags(const std::string& category, const std::string& name,$/;" f class:doris::Thread +create_writer src/runtime/dpp_sink.cpp /^Status Translator::create_writer(RuntimeState* state) {$/;" f class:doris::Translator +creation_time src/olap/rowset/rowset.h /^ int64_t creation_time() { return rowset_meta()->creation_time(); }$/;" f class:doris::Rowset +creation_time src/olap/rowset/rowset_meta.h /^ int64_t creation_time() const {$/;" f class:doris::RowsetMeta +creation_time src/olap/tablet.h /^inline const int64_t Tablet::creation_time() const {$/;" f class:doris::Tablet +creation_time src/olap/tablet_meta.h /^inline int64_t TabletMeta::creation_time() const {$/;" f class:doris::TabletMeta +creation_time src/olap/txn_manager.h /^ int64_t creation_time = 0;$/;" m struct:doris::TabletTxnInfo +ctx test/exprs/bitmap_function_test.cpp /^ FunctionContext* ctx;$/;" m class:doris::BitmapFunctionsTest file: +ctx test/exprs/hll_function_test.cpp /^ FunctionContext* ctx;$/;" m class:doris::HllFunctionsTest file: +ctx test/exprs/timestamp_functions_test.cpp /^ FunctionContext* ctx;$/;" m class:doris::TimestampFunctionsTest file: +cumulative_allocations src/runtime/bufferpool/buffer_pool_counters.h /^ RuntimeProfile::Counter* cumulative_allocations;$/;" m struct:doris::BufferPoolClientCounters +cumulative_bytes_alloced src/runtime/bufferpool/buffer_pool_counters.h /^ RuntimeProfile::Counter* cumulative_bytes_alloced;$/;" m struct:doris::BufferPoolClientCounters +cumulative_compaction_bytes_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::cumulative_compaction_bytes_total;$/;" m class:doris::DorisMetrics file: +cumulative_compaction_bytes_total src/util/doris_metrics.h /^ static IntCounter cumulative_compaction_bytes_total;$/;" m class:doris::DorisMetrics +cumulative_compaction_deltas_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::cumulative_compaction_deltas_total;$/;" m class:doris::DorisMetrics file: +cumulative_compaction_deltas_total src/util/doris_metrics.h /^ static IntCounter cumulative_compaction_deltas_total;$/;" m class:doris::DorisMetrics +cumulative_compaction_request_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::cumulative_compaction_request_failed;$/;" m class:doris::DorisMetrics file: +cumulative_compaction_request_failed src/util/doris_metrics.h /^ static IntCounter cumulative_compaction_request_failed;$/;" m class:doris::DorisMetrics +cumulative_compaction_request_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::cumulative_compaction_request_total;$/;" m class:doris::DorisMetrics file: +cumulative_compaction_request_total src/util/doris_metrics.h /^ static IntCounter cumulative_compaction_request_total;$/;" m class:doris::DorisMetrics +cumulative_layer_point src/olap/tablet.h /^inline const int64_t Tablet::cumulative_layer_point() const {$/;" f class:doris::Tablet +cumulative_layer_point src/olap/tablet_meta.h /^inline int64_t TabletMeta::cumulative_layer_point() const {$/;" f class:doris::TabletMeta +cur_expr_values src/exec/new_partitioned_hash_table.h /^ uint8_t* ALWAYS_INLINE cur_expr_values() const { return cur_expr_values_; }$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +cur_expr_values_ src/exec/new_partitioned_hash_table.h /^ uint8_t* cur_expr_values_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +cur_expr_values_hash_ src/exec/new_partitioned_hash_table.h /^ uint32_t* cur_expr_values_hash_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +cur_expr_values_hash_end_ src/exec/new_partitioned_hash_table.h /^ uint32_t* cur_expr_values_hash_end_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +cur_expr_values_null src/exec/new_partitioned_hash_table.h /^ uint8_t* ALWAYS_INLINE cur_expr_values_null() const { return cur_expr_values_null_; }$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +cur_expr_values_null_ src/exec/new_partitioned_hash_table.h /^ uint8_t* cur_expr_values_null_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +cur_file_info src/util/minizip/unzip.c /^ unz_file_info64 cur_file_info; \/* public info about the current file in minizip*\/$/;" m struct:__anon34 file: +cur_file_info_internal src/util/minizip/unzip.c /^ unz_file_info64_internal cur_file_info_internal; \/* private info about it*\/$/;" m struct:__anon34 file: +curdate src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::curdate(FunctionContext* context) {$/;" f class:doris::TimestampFunctions +curr src/olap/skiplist.h /^ Node* curr;$/;" m struct:doris::SkipList::Hint +curr_piece_ src/gutil/strings/split_internal.h /^ StringPiece curr_piece_;$/;" m class:strings::internal::SplitIterator +current_batch src/runtime/data_stream_recvr.cc /^ RowBatch* current_batch() const { {$/;" f class:doris::DataStreamRecvr::SenderQueue +current_bloom_filter_index src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ size_t current_bloom_filter_index() const {$/;" f class:doris::segment_v2::BloomFilterIndexIterator +current_chunk_idx_ src/runtime/mem_pool.h /^ int current_chunk_idx_;$/;" m class:doris::MemPool +current_file_ok src/util/minizip/unzip.c /^ ZPOS64_T current_file_ok; \/* flag about the usability of the current file*\/$/;" m struct:__anon34 file: +current_index src/util/frame_of_reference_coding.h /^ uint32_t current_index() const {$/;" f class:doris::ForDecoder +current_key src/olap/rowset/segment_v2/index_page.h /^ const Slice& current_key() const {$/;" f class:doris::segment_v2::IndexPageIterator +current_mem_size src/exec/partitioned_hash_table.cc /^int64_t PartitionedHashTable::current_mem_size() const {$/;" f class:doris::PartitionedHashTable +current_num_rows_per_row_block src/olap/rowset/segment_group.cpp /^size_t SegmentGroup::current_num_rows_per_row_block() const {$/;" f class:doris::SegmentGroup +current_ordinal src/olap/rowset/segment_v2/bitmap_index_reader.h /^ inline rowid_t current_ordinal() const {$/;" f class:doris::segment_v2::BitmapIndexIterator +current_page_pointer src/olap/rowset/segment_v2/index_page.h /^ const PagePointer& current_page_pointer() const {$/;" f class:doris::segment_v2::IndexPageIterator +current_row src/exec/merge_join_node.h /^ TupleRow* current_row;$/;" m struct:doris::MergeJoinNode::ChildReaderContext +current_row src/olap/generic_iterators.cpp /^ RowBlockRow current_row() const {$/;" f class:doris::MergeIteratorContext +current_row src/olap/reader.cpp /^ const RowCursor* current_row() const {$/;" f class:doris::CollectIterator::ChildCtx +current_row src/olap/reader.cpp /^ const RowCursor* current_row(bool* delete_flag) const {$/;" f class:doris::CollectIterator::ChildCtx +current_row src/olap/reader.cpp /^ const RowCursor* current_row(bool* delete_flag) const {$/;" f class:doris::CollectIterator +current_row src/runtime/sorted_run_merger.cc /^ TupleRow* current_row() const {$/;" f class:doris::SortedRunMerger::BatchedRowSupplier +current_sample_count_ src/util/streaming_sampler.h /^ int current_sample_count_;$/;" m class:doris::StreamingSampler +current_sample_sum_ src/util/streaming_sampler.h /^ T current_sample_sum_;$/;" m class:doris::StreamingSampler +current_sample_total_time_ src/util/streaming_sampler.h /^ int current_sample_total_time_;$/;" m class:doris::StreamingSampler +current_thread src/util/thread.cpp /^Thread* Thread::current_thread() {$/;" f class:doris::Thread +current_thread_id src/util/thread.cpp /^int64_t Thread::current_thread_id() {$/;" f class:doris::Thread +current_user_ident src/exec/schema_scanner.h /^ const TUserIdentity* current_user_ident; \/\/ to replace the user and user ip$/;" m struct:doris::SchemaScannerParam +current_value src/util/runtime_profile.h /^ int64_t current_value() const { return current_value_.load(); }$/;" f class:doris::RuntimeProfile::HighWaterMarkCounter +current_value_ src/util/rle_encoding.h /^ uint64_t current_value_;$/;" m class:doris::RleDecoder +current_value_ src/util/rle_encoding.h /^ uint64_t current_value_;$/;" m class:doris::RleEncoder +current_value_ src/util/runtime_profile.h /^ AtomicInt64 current_value_;$/;" m class:doris::RuntimeProfile::HighWaterMarkCounter +curtime src/exprs/timestamp_functions.cpp /^DoubleVal TimestampFunctions::curtime(FunctionContext* context) {$/;" f class:doris::TimestampFunctions +cv src/olap/data_dir.h /^ std::condition_variable cv;$/;" m class:doris::DataDir +cycles_per_ms src/util/cpu_info.h /^ static int64_t cycles_per_ms() {$/;" f class:doris::CpuInfo +cycles_per_ms_ src/util/cpu_info.cpp /^int64_t CpuInfo::cycles_per_ms_;$/;" m class:doris::CpuInfo file: +cycles_per_ms_ src/util/cpu_info.h /^ static int64_t cycles_per_ms_;$/;" m class:doris::CpuInfo +d1 test/util/radix_sort_test.cpp /^ float d1;$/;" m struct:doris::TestObject file: +d2 test/util/radix_sort_test.cpp /^ float d2;$/;" m struct:doris::TestObject file: +d_ino src/gutil/linux_syscall_support.h /^ long d_ino;$/;" m struct:kernel_dirent +d_ino src/gutil/linux_syscall_support.h /^ unsigned long long d_ino;$/;" m struct:kernel_dirent64 +d_name src/gutil/linux_syscall_support.h /^ char d_name[256];$/;" m struct:kernel_dirent +d_name src/gutil/linux_syscall_support.h /^ char d_name[256];$/;" m struct:kernel_dirent64 +d_off src/gutil/linux_syscall_support.h /^ long d_off;$/;" m struct:kernel_dirent +d_off src/gutil/linux_syscall_support.h /^ long long d_off;$/;" m struct:kernel_dirent64 +d_reclen src/gutil/linux_syscall_support.h /^ unsigned short d_reclen;$/;" m struct:kernel_dirent +d_reclen src/gutil/linux_syscall_support.h /^ unsigned short d_reclen;$/;" m struct:kernel_dirent64 +d_type src/gutil/linux_syscall_support.h /^ unsigned char d_type;$/;" m struct:kernel_dirent64 +data src/exec/hash_table.h /^ TupleRow* data() {$/;" f struct:doris::HashTable::Node +data src/gutil/ref_counted.h /^ T data;$/;" m class:doris::RefCountedData +data src/gutil/stl_util.h /^ Data data;$/;" m struct:STLEmptyBaseHandle +data src/gutil/strings/strcat.h /^ const char *data() const { return piece.data(); }$/;" f struct:AlphaNum +data src/gutil/strings/stringpiece.h /^ const char* data() const { return ptr_; }$/;" f class:StringPiece +data src/gutil/strings/substitute.h /^ inline const char* data() const { return text_; }$/;" f class:strings::internal::SubstituteArg +data src/olap/bloom_filter.hpp /^ uint64_t* data() const {$/;" f class:doris::BitSet +data src/olap/column_block.h /^ uint8_t* data() const { return _block->mutable_cell_ptr(_row_offset); }$/;" f class:doris::ColumnBlockView +data src/olap/column_block.h /^ uint8_t* data() const { return _data; }$/;" f class:doris::ColumnBlock +data src/olap/lru_cache.h /^ const char* data() const {$/;" f class:doris::CacheKey +data src/olap/olap_index.h /^ char* data;$/;" m struct:doris::EntrySlice +data src/olap/page_cache.h /^ Slice data() const { return _cache->value_slice(_handle); }$/;" f class:doris::PageCacheHandle +data src/olap/rowset/segment_v2/bloom_filter.h /^ char* data() const { return _data; }$/;" f class:doris::segment_v2::BloomFilter +data src/olap/rowset/segment_v2/column_writer.h /^ std::vector data;$/;" m struct:doris::segment_v2::ColumnWriter::Page +data src/olap/rowset/segment_v2/page_handle.h /^ Slice data() const {$/;" f class:doris::segment_v2::PageHandle +data src/olap/uint24.h /^ uint8_t data[3];$/;" m struct:doris::uint24_t +data src/runtime/buffered_tuple_stream.h /^ uint64_t data;$/;" m struct:doris::BufferedTupleStream::RowIdx +data src/runtime/buffered_tuple_stream2.h /^ uint64_t data;$/;" m struct:doris::BufferedTupleStream2::RowIdx +data src/runtime/bufferpool/buffer_pool.h /^ uint8_t* data() const {$/;" f class:doris::BufferPool::BufferHandle +data src/runtime/bufferpool/suballocator.h /^ uint8_t* data() const { return data_; }$/;" f class:doris::Suballocation +data src/runtime/memory/chunk.h /^ uint8_t* data = nullptr;$/;" m struct:doris::Chunk +data src/util/faststring.h /^ const uint8_t *data() const {$/;" f class:doris::faststring +data src/util/faststring.h /^ uint8_t *data() {$/;" f class:doris::faststring +data src/util/mem_range.h /^ uint8_t* data() const { return data_; }$/;" f class:doris::MemRange +data src/util/slice.h /^ char* data;$/;" m struct:doris::Slice +data test/util/utf8_check_test.cpp /^ const char *data;$/;" m struct:doris::test file: +data_ src/gutil/gscoped_ptr.h /^ Data data_;$/;" m class:doris::internal::gscoped_ptr_impl +data_ src/runtime/bufferpool/buffer_pool.h /^ uint8_t* data_;$/;" m class:doris::BufferPool::BufferHandle +data_ src/runtime/bufferpool/suballocator.h /^ uint8_t* data_;$/;" m class:doris::Suballocation +data_ src/util/faststring.h /^ uint8_t* data_;$/;" m class:doris::faststring +data_ src/util/mem_range.h /^ uint8_t* data_;$/;" m class:doris::MemRange +data_decoder src/olap/rowset/segment_v2/parsed_page.h /^ PageDecoder* data_decoder = nullptr;$/;" m struct:doris::segment_v2::ParsedPage +data_dir src/olap/tablet.h /^inline DataDir* Tablet::data_dir() const {$/;" f class:doris::Tablet +data_disk_size src/olap/rowset/rowset.h /^ size_t data_disk_size() const { return rowset_meta()->total_disk_size(); }$/;" f class:doris::Rowset +data_disk_size src/olap/rowset/rowset_meta.h /^ size_t data_disk_size() const {$/;" f class:doris::RowsetMeta +data_file_offset_t src/olap/olap_index.h /^typedef uint32_t data_file_offset_t;$/;" t namespace:doris +data_len src/olap/bloom_filter.hpp /^ uint32_t data_len() const {$/;" f class:doris::BitSet +data_length src/olap/olap_index.h /^ uint32_t data_length;$/;" m struct:doris::OLAPIndexFixedHeader +data_offset src/olap/olap_index.h /^ uint32_t data_offset; \/\/ offset in data file$/;" m struct:doris::RowBlockPosition +data_page_size src/olap/rowset/segment_v2/column_writer.h /^ size_t data_page_size = 64 * 1024;$/;" m struct:doris::segment_v2::ColumnWriterOptions +data_page_size src/olap/rowset/segment_v2/options.h /^ size_t data_page_size = DEFAULT_PAGE_SIZE;$/;" m struct:doris::segment_v2::PageBuilderOptions +data_pages_ src/exec/new_partitioned_hash_table.h /^ std::vector> data_pages_;$/;" m class:doris::NewPartitionedHashTable +data_saved_dir src/http/action/mini_load.cpp /^Status MiniLoadAction::data_saved_dir(const LoadHandle& desc,$/;" f class:doris::MiniLoadAction +data_size src/olap/olap_index.h /^ const size_t data_size() const {$/;" f class:doris::MemIndex +data_size src/olap/rowset/segment_group.h /^ size_t data_size() const {$/;" f class:doris::SegmentGroup +data_used_capacity src/olap/olap_common.h /^ int64_t data_used_capacity;$/;" m struct:doris::DataDirInfo +data_values test/runtime/data_stream_test.cpp /^ multiset data_values;$/;" m struct:doris::DataStreamTest::ReceiverInfo file: +database src/runtime/descriptors.h /^ const std::string& database() const {$/;" f class:doris::TableDescriptor +date_add_interval src/runtime/datetime_value.cpp /^bool DateTimeValue::date_add_interval(const TimeInterval& interval, TimeUnit unit) {$/;" f class:doris::DateTimeValue +date_diff src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::date_diff($/;" f class:doris::TimestampFunctions +date_format src/exprs/timestamp_functions.cpp /^StringVal TimestampFunctions::date_format($/;" f class:doris::TimestampFunctions +date_val test/runtime/dpp_writer_test.cpp /^ std::string date_val;$/;" m struct:doris::TestDataTuple file: +date_val test/runtime/mysql_table_writer_test.cpp /^ std::string date_val;$/;" m struct:doris::TestDataTuple file: +datetime test/olap/comparison_predicate_test.cpp /^namespace datetime {$/;" n namespace:doris file: +datetime test/olap/in_list_predicate_test.cpp /^namespace datetime {$/;" n namespace:doris file: +datetime test/olap/null_predicate_test.cpp /^namespace datetime {$/;" n namespace:doris file: +datetime_max_value src/runtime/datetime_value.h /^ static DateTimeValue datetime_max_value() {$/;" f class:doris::DateTimeValue +datetime_min_value src/runtime/datetime_value.h /^ static DateTimeValue datetime_min_value() {$/;" f class:doris::DateTimeValue +datetime_val src/exprs/expr_value.h /^ DateTimeValue datetime_val;$/;" m struct:doris::ExprValue +datetime_val test/runtime/dpp_writer_test.cpp /^ std::string datetime_val;$/;" m struct:doris::TestDataTuple file: +datetime_val test/runtime/mysql_table_writer_test.cpp /^ std::string datetime_val;$/;" m struct:doris::TestDataTuple file: +day src/runtime/datetime_value.h /^ int day() const {$/;" f class:doris::DateTimeValue +day src/runtime/datetime_value.h /^ int32_t day;$/;" m struct:doris::TimeInterval +day_name src/exprs/timestamp_functions.cpp /^StringVal TimestampFunctions::day_name($/;" f class:doris::TimestampFunctions +day_name src/runtime/datetime_value.cpp /^const char* DateTimeValue::day_name() const {$/;" f class:doris::DateTimeValue +day_of_month src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::day_of_month($/;" f class:doris::TimestampFunctions +day_of_week src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::day_of_week($/;" f class:doris::TimestampFunctions +day_of_year src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::day_of_year($/;" f class:doris::TimestampFunctions +day_of_year src/runtime/datetime_value.h /^ int day_of_year() const {$/;" f class:doris::DateTimeValue +daynr src/runtime/datetime_value.h /^ inline uint64_t daynr() const {$/;" f class:doris::DateTimeValue +days_add src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::days_add($/;" f class:doris::TimestampFunctions +days_diff src/exprs/timestamp_functions.cpp /^BigIntVal TimestampFunctions::days_diff($/;" f class:doris::TimestampFunctions +days_sub src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::days_sub($/;" f class:doris::TimestampFunctions +db src/exec/mysql_scanner.h /^ std::string db;$/;" m struct:doris::MysqlScannerParam +db src/exec/schema_scanner.h /^ const std::string* db;$/;" m struct:doris::SchemaScannerParam +db src/http/action/mini_load.h /^ std::string db;$/;" m struct:doris::LoadHandle +db src/runtime/mysql_table_writer.h /^ std::string db;$/;" m struct:doris::MysqlConnInfo +db src/runtime/stream_load/stream_load_context.h /^ std::string db;$/;" m class:doris::StreamLoadContext +db src/util/mysql_load_error_hub.h /^ std::string db;$/;" m struct:doris::MysqlLoadErrorHub::MysqlInfo +db_id src/exec/tablet_info.h /^ int64_t db_id() const { return _db_id; }$/;" f class:doris::OlapTableSchemaParam +db_id src/exec/tablet_info.h /^ int64_t db_id() const { return _t_param.db_id; }$/;" f class:doris::OlapTableLocationParam +db_id src/exec/tablet_info.h /^ int64_t db_id() const { return _t_param.db_id; }$/;" f class:doris::OlapTablePartitionParam +db_name src/runtime/runtime_state.h /^ const std::string& db_name() {$/;" f class:doris::RuntimeState +db_num test/exec/schema_scanner/schema_columns_scanner_test.cpp /^int db_num = 0;$/;" m namespace:doris file: +db_num test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^int db_num = 0;$/;" m namespace:doris file: +db_num test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^int db_num = 0;$/;" m namespace:doris file: +db_num test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^int db_num = 0;$/;" m namespace:doris file: +db_num test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^int db_num = 0;$/;" m namespace:doris file: +db_num test/exec/schema_scanner/schema_tables_scanner_test.cpp /^int db_num = 0;$/;" m namespace:doris file: +dcheck_locked src/util/spinlock.h /^ void dcheck_locked() {$/;" f class:doris::SpinLock +dcopy_with_new src/runtime/tuple.cpp /^Tuple* Tuple::dcopy_with_new(const TupleDescriptor& desc, MemPool* pool, int64_t* bytes) {$/;" f class:doris::Tuple +dcopy_with_new src/runtime/tuple.cpp /^int64_t Tuple::dcopy_with_new(Tuple* dst, const TupleDescriptor& desc) {$/;" f class:doris::Tuple +dcopy_with_new src/runtime/tuple_row.h /^ TupleRow* dcopy_with_new(const std::vector& descs, $/;" f class:doris::TupleRow +dcopy_with_new src/runtime/tuple_row.h /^ int64_t dcopy_with_new($/;" f class:doris::TupleRow +deallocate src/gutil/stl_util.h /^ void deallocate(pointer p, size_type n) {$/;" f class:STLCountingAllocator +deallocate src/util/radix_sort.h /^ void deallocate(void * ptr, size_t \/*size*\/) {$/;" f struct:doris::RadixSortMallocAllocator +debug src/runtime/etl_job_mgr.cpp /^void EtlJobMgr::debug(std::stringstream& ss) {$/;" f class:doris::EtlJobMgr +debug src/runtime/fragment_mgr.cpp /^void FragmentMgr::debug(std::stringstream& ss) {$/;" f class:doris::FragmentMgr +debug test/runtime/etl_job_mgr_test.cpp /^void FragmentMgr::debug(std::stringstream& ss) {$/;" f class:doris::FragmentMgr +debug test/runtime/export_task_mgr_test.cpp /^void FragmentMgr::debug(std::stringstream& ss) {$/;" f class:doris::FragmentMgr +debug_info src/exec/decompressor.cpp /^std::string Bzip2Decompressor::debug_info() {$/;" f class:doris::Bzip2Decompressor +debug_info src/exec/decompressor.cpp /^std::string Decompressor::debug_info() {$/;" f class:doris::Decompressor +debug_info src/exec/decompressor.cpp /^std::string GzipDecompressor::debug_info() {$/;" f class:doris::GzipDecompressor +debug_info src/exec/decompressor.cpp /^std::string Lz4FrameDecompressor::debug_info() {$/;" f class:doris::Lz4FrameDecompressor +debug_info src/exec/lzo_decompressor.cpp /^std::string LzopDecompressor::debug_info() {$/;" f class:doris::LzopDecompressor +debug_internal src/runtime/buffered_block_mgr2.cc /^string BufferedBlockMgr2::debug_internal() const {$/;" f class:doris::BufferedBlockMgr2 +debug_path src/runtime/etl_job_mgr.h /^ std::string debug_path;$/;" m struct:doris::EtlJobResult +debug_state_string src/exec/analytic_eval_node.cpp /^std::string AnalyticEvalNode::debug_state_string(bool detailed) const {$/;" f class:doris::AnalyticEvalNode +debug_string src/exec/aggregation_node.cpp /^void AggregationNode::debug_string(int indentation_level, std::stringstream* out) const {$/;" f class:doris::AggregationNode +debug_string src/exec/analytic_eval_node.cpp /^void AnalyticEvalNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::AnalyticEvalNode +debug_string src/exec/blocking_join_node.cpp /^void BlockingJoinNode::debug_string(int indentation_level, std::stringstream* out) const {$/;" f class:doris::BlockingJoinNode +debug_string src/exec/broker_scan_node.cpp /^void BrokerScanNode::debug_string(int ident_level, std::stringstream* out) const {$/;" f class:doris::BrokerScanNode +debug_string src/exec/csv_scan_node.cpp /^void CsvScanNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::CsvScanNode +debug_string src/exec/es_http_scan_node.cpp /^void EsHttpScanNode::debug_string(int ident_level, std::stringstream* out) const {$/;" f class:doris::EsHttpScanNode +debug_string src/exec/es_scan_node.cpp /^void EsScanNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::EsScanNode +debug_string src/exec/exchange_node.cpp /^void ExchangeNode::debug_string(int indentation_level, std::stringstream* out) const {$/;" f class:doris::ExchangeNode +debug_string src/exec/exec_node.cpp /^std::string ExecNode::debug_string() const {$/;" f class:doris::ExecNode +debug_string src/exec/exec_node.cpp /^void ExecNode::debug_string(int indentation_level, std::stringstream* out) const {$/;" f class:doris::ExecNode +debug_string src/exec/hash_join_node.cpp /^void HashJoinNode::debug_string(int indentation_level, std::stringstream* out) const {$/;" f class:doris::HashJoinNode +debug_string src/exec/hash_table.cpp /^std::string HashTable::debug_string(bool skip_empty, const RowDescriptor* desc) {$/;" f class:doris::HashTable +debug_string src/exec/kudu_scan_node.cpp /^void KuduScanNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::KuduScanNode +debug_string src/exec/merge_join_node.cpp /^void MergeJoinNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::MergeJoinNode +debug_string src/exec/mysql_scan_node.cpp /^void MysqlScanNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::MysqlScanNode +debug_string src/exec/olap_common.h /^ std::string debug_string() {$/;" f class:doris::OlapScanKeys +debug_string src/exec/olap_scan_node.cpp /^void OlapScanNode::debug_string($/;" f class:doris::OlapScanNode +debug_string src/exec/partitioned_aggregation_node.cc /^void PartitionedAggregationNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::PartitionedAggregationNode +debug_string src/exec/partitioned_hash_table.cc /^string PartitionedHashTable::debug_string($/;" f class:doris::PartitionedHashTable +debug_string src/exec/pl_task_root.cpp /^void ExchangeNode::debug_string(int indentation_level, std::stringstream* out) const {$/;" f class:doris::ExchangeNode +debug_string src/exec/pre_aggregation_node.cpp /^void PreAggregationNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::PreAggregationNode +debug_string src/exec/repeat_node.cpp /^void RepeatNode::debug_string(int indentation_level, std::stringstream* out) const {$/;" f class:doris::RepeatNode +debug_string src/exec/row_batch_list.h /^ std::string debug_string(const RowDescriptor& desc) {$/;" f class:doris::RowBatchList +debug_string src/exec/schema_scan_node.cpp /^void SchemaScanNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::SchemaScanNode +debug_string src/exec/sort_node.cpp /^void SortNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::SortNode +debug_string src/exec/spill_sort_node.cc /^void SpillSortNode::debug_string(int indentation_level, stringstream* out) const {$/;" f class:doris::SpillSortNode +debug_string src/exec/tablet_info.cpp /^std::string OlapTablePartition::debug_string(TupleDescriptor* tuple_desc) const {$/;" f class:doris::OlapTablePartition +debug_string src/exec/tablet_info.cpp /^std::string OlapTablePartitionParam::debug_string() const {$/;" f class:doris::OlapTablePartitionParam +debug_string src/exec/tablet_info.cpp /^std::string OlapTableSchemaParam::debug_string() const {$/;" f class:doris::OlapTableSchemaParam +debug_string src/exec/topn_node.cpp /^void TopNNode::debug_string(int indentation_level, std::stringstream* out) const {$/;" f class:doris::TopNNode +debug_string src/exprs/agg_fn_evaluator.cpp /^std::string AggFnEvaluator::debug_string() const {$/;" f class:doris::AggFnEvaluator +debug_string src/exprs/agg_fn_evaluator.cpp /^std::string AggFnEvaluator::debug_string(const std::vector& exprs) {$/;" f class:doris::AggFnEvaluator +debug_string src/exprs/binary_predicate.cpp /^std::string BinaryPredicate::debug_string() const {$/;" f class:doris::BinaryPredicate +debug_string src/exprs/case_expr.cpp /^std::string CaseExpr::debug_string() const {$/;" f class:doris::CaseExpr +debug_string src/exprs/compound_predicate.cpp /^std::string CompoundPredicate::debug_string() const {$/;" f class:doris::CompoundPredicate +debug_string src/exprs/compound_predicate.h /^ virtual std::string debug_string() const {$/;" f class:doris::AndPredicate +debug_string src/exprs/compound_predicate.h /^ virtual std::string debug_string() const {$/;" f class:doris::NotPredicate +debug_string src/exprs/compound_predicate.h /^ virtual std::string debug_string() const {$/;" f class:doris::OrPredicate +debug_string src/exprs/conditional_functions.h /^ virtual std::string debug_string() const { $/;" f class:doris::IfExpr +debug_string src/exprs/conditional_functions.h /^ virtual std::string debug_string() const { $/;" f class:doris::IfNullExpr +debug_string src/exprs/conditional_functions.h /^ virtual std::string debug_string() const { $/;" f class:doris::NullIfExpr +debug_string src/exprs/conditional_functions.h /^ virtual std::string debug_string() const { return Expr::debug_string("CoalesceExpr"); }$/;" f class:doris::CoalesceExpr +debug_string src/exprs/expr.cpp /^std::string Expr::debug_string() const {$/;" f class:doris::Expr +debug_string src/exprs/expr.cpp /^std::string Expr::debug_string(const std::vector& exprs) {$/;" f class:doris::Expr +debug_string src/exprs/expr.cpp /^std::string Expr::debug_string(const std::vector& ctxs) {$/;" f class:doris::Expr +debug_string src/exprs/expr.h /^ std::string debug_string(const std::string& expr_name) const {$/;" f class:doris::Expr +debug_string src/exprs/in_predicate.cpp /^std::string InPredicate::debug_string() const {$/;" f class:doris::InPredicate +debug_string src/exprs/info_func.cpp /^std::string InfoFunc::debug_string() const {$/;" f class:doris::InfoFunc +debug_string src/exprs/json_functions.h /^ std::string debug_string() {$/;" f struct:doris::JsonPath +debug_string src/exprs/scalar_fn_call.cpp /^std::string ScalarFnCall::debug_string() const {$/;" f class:doris::ScalarFnCall +debug_string src/exprs/slot_ref.cpp /^std::string SlotRef::debug_string() const {$/;" f class:doris::SlotRef +debug_string src/exprs/tuple_is_null_predicate.cpp /^std::string TupleIsNullPredicate::debug_string() const {$/;" f class:doris::TupleIsNullPredicate +debug_string src/http/http_request.cpp /^std::string HttpRequest::debug_string() const {$/;" f class:doris::HttpRequest +debug_string src/olap/field.h /^ std::string debug_string(const CellType& cell) const {$/;" f class:doris::Field +debug_string src/olap/row_block2.cpp /^std::string RowBlockRow::debug_string() const {$/;" f class:doris::RowBlockRow +debug_string src/runtime/buffered_block_mgr.h /^ std::string debug_string() const {$/;" f class:doris::BufferedBlockMgr::Block +debug_string src/runtime/buffered_block_mgr2.cc /^ string debug_string() const {$/;" f class:doris::BufferedBlockMgr2::Client +debug_string src/runtime/buffered_block_mgr2.cc /^string BufferedBlockMgr2::Block::debug_string() const {$/;" f class:doris::BufferedBlockMgr2::Block +debug_string src/runtime/buffered_block_mgr2.cc /^string BufferedBlockMgr2::debug_string(Client* client) {$/;" f class:doris::BufferedBlockMgr2 +debug_string src/runtime/buffered_tuple_stream.cpp /^std::string BufferedTupleStream::RowIdx::debug_string() const {$/;" f class:doris::BufferedTupleStream::RowIdx +debug_string src/runtime/buffered_tuple_stream.cpp /^std::string BufferedTupleStream::debug_string() const {$/;" f class:doris::BufferedTupleStream +debug_string src/runtime/buffered_tuple_stream2.cc /^string BufferedTupleStream2::RowIdx::debug_string() const {$/;" f class:doris::BufferedTupleStream2::RowIdx +debug_string src/runtime/buffered_tuple_stream2.cc /^string BufferedTupleStream2::debug_string() const {$/;" f class:doris::BufferedTupleStream2 +debug_string src/runtime/client_cache.cpp /^std::string ClientCacheHelper::debug_string() {$/;" f class:doris::ClientCacheHelper +debug_string src/runtime/client_cache.h /^ std::string debug_string() {$/;" f class:doris::ClientCache +debug_string src/runtime/datetime_value.h /^ std::string debug_string() const {$/;" f class:doris::DateTimeValue +debug_string src/runtime/descriptors.cpp /^std::string BrokerTableDescriptor::debug_string() const {$/;" f class:doris::BrokerTableDescriptor +debug_string src/runtime/descriptors.cpp /^std::string DescriptorTbl::debug_string() const {$/;" f class:doris::DescriptorTbl +debug_string src/runtime/descriptors.cpp /^std::string EsTableDescriptor::debug_string() const {$/;" f class:doris::EsTableDescriptor +debug_string src/runtime/descriptors.cpp /^std::string MySQLTableDescriptor::debug_string() const {$/;" f class:doris::MySQLTableDescriptor +debug_string src/runtime/descriptors.cpp /^std::string NullIndicatorOffset::debug_string() const {$/;" f class:doris::NullIndicatorOffset +debug_string src/runtime/descriptors.cpp /^std::string OlapTableDescriptor::debug_string() const {$/;" f class:doris::OlapTableDescriptor +debug_string src/runtime/descriptors.cpp /^std::string RowDescriptor::debug_string() const {$/;" f class:doris::RowDescriptor +debug_string src/runtime/descriptors.cpp /^std::string SchemaTableDescriptor::debug_string() const {$/;" f class:doris::SchemaTableDescriptor +debug_string src/runtime/descriptors.cpp /^std::string SlotDescriptor::debug_string() const {$/;" f class:doris::SlotDescriptor +debug_string src/runtime/descriptors.cpp /^std::string TableDescriptor::debug_string() const {$/;" f class:doris::TableDescriptor +debug_string src/runtime/descriptors.cpp /^std::string TupleDescriptor::debug_string() const {$/;" f class:doris::TupleDescriptor +debug_string src/runtime/disk_io_mgr.cc /^string DiskIoMgr::RequestContextCache::debug_string() {$/;" f class:doris::DiskIoMgr::RequestContextCache +debug_string src/runtime/disk_io_mgr.cc /^string DiskIoMgr::debug_string() {$/;" f class:doris::DiskIoMgr +debug_string src/runtime/disk_io_mgr_reader_context.cc /^string DiskIoMgr::RequestContext::debug_string() const {$/;" f class:doris::DiskIoMgr::RequestContext +debug_string src/runtime/disk_io_mgr_scan_range.cc /^string DiskIoMgr::ScanRange::debug_string() const {$/;" f class:doris::DiskIoMgr::ScanRange +debug_string src/runtime/dpp_sink_internal.h /^ std::string debug_string() const {$/;" f class:doris::PartRangeKey +debug_string src/runtime/dpp_sink_internal.h /^ std::string debug_string() const{$/;" f class:doris::PartRange +debug_string src/runtime/mem_pool.cpp /^string MemPool::debug_string() {$/;" f class:doris::MemPool +debug_string src/runtime/mem_tracker.h /^ std::string debug_string() {$/;" f class:doris::MemTracker +debug_string src/runtime/mysql_table_writer.cpp /^std::string MysqlConnInfo::debug_string() const {$/;" f class:doris::MysqlConnInfo +debug_string src/runtime/string_value.cpp /^std::string StringValue::debug_string() const {$/;" f class:doris::StringValue +debug_string src/runtime/types.cpp /^std::string TypeDescriptor::debug_string() const {$/;" f class:doris::TypeDescriptor +debug_string src/udf/udf_debug.h /^std::string debug_string(const StringVal& val) {$/;" f namespace:doris_udf +debug_string src/udf/udf_debug.h /^std::string debug_string(const T& val) {$/;" f namespace:doris_udf +debug_string src/util/broker_load_error_hub.cpp /^std::string BrokerLoadErrorHub::debug_string() const {$/;" f class:doris::BrokerLoadErrorHub +debug_string src/util/cgroup_util.cpp /^std::string CGroupUtil::debug_string() {$/;" f class:doris::CGroupUtil +debug_string src/util/cpu_info.cpp /^string CpuInfo::debug_string() {$/;" f class:doris::CpuInfo +debug_string src/util/disk_info.cpp /^std::string DiskInfo::debug_string() {$/;" f class:doris::DiskInfo +debug_string src/util/mem_info.cpp /^std::string MemInfo::debug_string() {$/;" f class:doris::MemInfo +debug_string src/util/mysql_load_error_hub.cpp /^std::string MysqlLoadErrorHub::debug_string() const {$/;" f class:doris::MysqlLoadErrorHub +debug_string src/util/null_load_error_hub.cpp /^std::string NullLoadErrorHub::debug_string() const {$/;" f class:doris::NullLoadErrorHub +debug_string_tuple src/exec/partitioned_hash_table.cc /^void PartitionedHashTable::debug_string_tuple($/;" f class:doris::PartitionedHashTable +debug_window_bound_string src/exec/analytic_eval_node.cpp /^string debug_window_bound_string(const TAnalyticWindowBoundary& b) {$/;" f namespace:doris +debug_window_string src/exec/analytic_eval_node.cpp /^std::string AnalyticEvalNode::debug_window_string() const {$/;" f class:doris::AnalyticEvalNode +debug_write_delay_ms_ src/runtime/bufferpool/buffer_pool_internal.h /^ int debug_write_delay_ms_;$/;" m class:doris::BufferPool::Client +dec src/util/counter_cond_variable.hpp /^ void dec(int dec = 1) {$/;" f class:doris::CounterCondVariable +decimal12_t src/olap/decimal12.h /^ decimal12_t() : integer(0), fraction(0) {}$/;" f struct:doris::decimal12_t +decimal12_t src/olap/decimal12.h /^ decimal12_t(const decimal12_t& value) {$/;" f struct:doris::decimal12_t +decimal12_t src/olap/decimal12.h /^ decimal12_t(int64_t int_part, int32_t frac_part) {$/;" f struct:doris::decimal12_t +decimal12_t src/olap/decimal12.h /^struct decimal12_t {$/;" s namespace:doris +decimal_abs src/exprs/udf_builtins.cpp /^DecimalV2Val UdfBuiltins::decimal_abs(FunctionContext* context, const DecimalV2Val& v) {$/;" f class:doris::UdfBuiltins +decimal_abs src/exprs/udf_builtins.cpp /^DecimalVal UdfBuiltins::decimal_abs(FunctionContext* context, const DecimalVal& v) {$/;" f class:doris::UdfBuiltins +decimal_avg_finalize src/exprs/aggregate_functions.cpp /^DecimalVal AggregateFunctions::decimal_avg_finalize(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::AggregateFunctions +decimal_avg_get_value src/exprs/aggregate_functions.cpp /^DecimalVal AggregateFunctions::decimal_avg_get_value(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::AggregateFunctions +decimal_avg_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::decimal_avg_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +decimal_avg_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::decimal_avg_merge(FunctionContext* ctx, const StringVal& src,$/;" f class:doris::AggregateFunctions +decimal_avg_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::decimal_avg_remove(doris_udf::FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +decimal_avg_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::decimal_avg_update(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +decimal_in_base_to_decimal src/exprs/math_functions.cpp /^bool MathFunctions::decimal_in_base_to_decimal($/;" f class:doris::MathFunctions +decimal_to_base src/exprs/math_functions.cpp /^StringVal MathFunctions::decimal_to_base($/;" f class:doris::MathFunctions +decimal_type src/runtime/descriptor_helper.h /^ TSlotDescriptorBuilder& decimal_type(int precision, int scale) {$/;" f class:doris::TSlotDescriptorBuilder +decimal_val src/exprs/expr_value.h /^ DecimalValue decimal_val;$/;" m struct:doris::ExprValue +decimalv2_avg_finalize src/exprs/aggregate_functions.cpp /^DecimalV2Val AggregateFunctions::decimalv2_avg_finalize(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::AggregateFunctions +decimalv2_avg_get_value src/exprs/aggregate_functions.cpp /^DecimalV2Val AggregateFunctions::decimalv2_avg_get_value(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::AggregateFunctions +decimalv2_avg_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::decimalv2_avg_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +decimalv2_avg_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::decimalv2_avg_merge(FunctionContext* ctx, const StringVal& src,$/;" f class:doris::AggregateFunctions +decimalv2_avg_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::decimalv2_avg_remove(doris_udf::FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +decimalv2_avg_serialize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::decimalv2_avg_serialize($/;" f class:doris::AggregateFunctions +decimalv2_avg_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::decimalv2_avg_update(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +decimalv2_val src/exprs/expr_value.h /^ DecimalV2Value decimalv2_val;$/;" m struct:doris::ExprValue +declare_plugin src/plugin/plugin.h 60;" d +declare_plugin_end src/plugin/plugin.h 71;" d +declare_tuple src/testutil/desc_tbl_builder.cc /^TupleDescBuilder& DescriptorTblBuilder::declare_tuple() {$/;" f class:doris::DescriptorTblBuilder +decode src/geo/geo_types.cpp /^bool GeoCircle::decode(const void* data, size_t size) {$/;" f class:doris::GeoCircle +decode src/geo/geo_types.cpp /^bool GeoLine::decode(const void* data, size_t size) {$/;" f class:doris::GeoLine +decode src/geo/geo_types.cpp /^bool GeoPoint::decode(const void* data, size_t size) {$/;" f class:doris::GeoPoint +decode src/geo/geo_types.cpp /^bool GeoPolygon::decode(const void* data, size_t size) {$/;" f class:doris::GeoPolygon +decode_address src/util/bfd_parser.cpp /^int BfdParser::decode_address(const char* str, const char** end, $/;" f class:doris::BfdParser +decode_ascending src/olap/field.h /^ Status decode_ascending(Slice* encoded_key, uint8_t* cell_ptr, MemPool* pool) const {$/;" f class:doris::Field +decode_ascending src/olap/key_coder.h /^ Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr, MemPool* pool) const {$/;" f class:doris::KeyCoder +decode_ascending src/olap/key_coder.h /^ static Status decode_ascending(Slice* encoded_key, size_t index_size,$/;" f class:doris::KeyCoderTraits +decode_bit_width src/olap/serialize.cpp /^uint32_t decode_bit_width(uint32_t n) {$/;" f namespace:doris::ser +decode_current_frame src/util/frame_of_reference_coding.cpp /^void ForDecoder::decode_current_frame(T* output) {$/;" f class:doris::ForDecoder +decode_fixed128_le src/util/coding.h /^inline uint128_t decode_fixed128_le(const uint8_t* buf) {$/;" f namespace:doris +decode_fixed16_le src/util/coding.h /^inline uint16_t decode_fixed16_le(const uint8_t* buf) {$/;" f namespace:doris +decode_fixed32_le src/util/coding.h /^inline uint32_t decode_fixed32_le(const uint8_t* buf) {$/;" f namespace:doris +decode_fixed64_le src/util/coding.h /^inline uint64_t decode_fixed64_le(const uint8_t* buf) {$/;" f namespace:doris +decode_fixed8 src/util/coding.h /^inline uint8_t decode_fixed8(const uint8_t* buf) {$/;" f namespace:doris +decode_frame_min_value src/util/frame_of_reference_coding.cpp /^T ForDecoder::decode_frame_min_value(uint32_t frame_index) {$/;" f class:doris::ForDecoder +decode_from src/geo/geo_types.cpp /^bool GeoShape::decode_from(const void* data, size_t size) {$/;" f class:doris::GeoShape +decode_from src/olap/rowset/segment_v2/page_pointer.h /^ bool decode_from(Slice* input) {$/;" f struct:doris::segment_v2::PagePointer +decode_from src/olap/rowset/segment_v2/page_pointer.h /^ const uint8_t* decode_from(const uint8_t* data, const uint8_t* limit) {$/;" f struct:doris::segment_v2::PagePointer +decode_rows_into_row_batch src/exec/kudu_scanner.cpp /^Status KuduScanner::decode_rows_into_row_batch(RowBatch* row_batch, Tuple** tuple_mem,$/;" f class:doris::KuduScanner +decode_varint32_ptr src/util/coding.h /^inline const uint8_t* decode_varint32_ptr($/;" f namespace:doris +decode_varint32_ptr_fallback src/util/coding.cpp /^const uint8_t* decode_varint32_ptr_fallback(const uint8_t* p, const uint8_t* limit,$/;" f namespace:doris +decode_varint64_ptr src/util/coding.cpp /^const uint8_t* decode_varint64_ptr(const uint8_t* p, const uint8_t* limit, uint64_t* value) {$/;" f namespace:doris +decode_vint_size src/exec/read_write_util.h /^inline int ReadWriteUtil::decode_vint_size(int8_t byte) {$/;" f class:doris::ReadWriteUtil +decoding_table src/util/url_coding.cpp /^static short decoding_table[256] = {$/;" m namespace:doris file: +decompress src/exec/decompressor.cpp /^Status Bzip2Decompressor::decompress($/;" f class:doris::Bzip2Decompressor +decompress src/exec/decompressor.cpp /^Status GzipDecompressor::decompress($/;" f class:doris::GzipDecompressor +decompress src/exec/decompressor.cpp /^Status Lz4FrameDecompressor::decompress($/;" f class:doris::Lz4FrameDecompressor +decompress src/exec/lzo_decompressor.cpp /^Status LzopDecompressor::decompress($/;" f class:doris::LzopDecompressor +decompress_lz4 src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp /^int64_t decompress_lz4(void* in, void* out, size_t size,$/;" f namespace:doris::bitshuffle +decompress_ns src/olap/olap_common.h /^ int64_t decompress_ns = 0;$/;" m struct:doris::OlapReaderStatistics +decrement_disk_ref_count src/runtime/disk_io_mgr_internal.h /^ void decrement_disk_ref_count() {$/;" f class:doris::DiskIoMgr::RequestContext +decrement_latch test/util/countdown_latch_test.cpp /^static void decrement_latch(CountDownLatch* latch, int amount) {$/;" f namespace:doris +decrement_request_thread src/runtime/disk_io_mgr_internal.h /^ void decrement_request_thread() {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +decrement_request_thread_and_check_done src/runtime/disk_io_mgr_internal.h /^ void decrement_request_thread_and_check_done(RequestContext* context) {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +decrement_senders src/runtime/data_stream_recvr.cc /^void DataStreamRecvr::SenderQueue::decrement_senders(int be_number) {$/;" f class:doris::DataStreamRecvr::SenderQueue +decrypt src/util/aes_util.cpp /^int AesUtil::decrypt(AesMode mode, const unsigned char* encrypt, uint32_t encrypt_length,$/;" f class:doris::AesUtil +deep_copy src/olap/field.h /^ void deep_copy(DstCellType* dst,$/;" f class:doris::Field +deep_copy src/olap/types.h /^ inline void deep_copy(void* dest, const void* src, MemPool* mem_pool) const {$/;" f class:doris::TypeInfo +deep_copy src/olap/types.h /^ static inline void deep_copy(void* dest, const void* src, MemPool* mem_pool) {$/;" f struct:doris::BaseFieldtypeTraits +deep_copy src/olap/types.h /^ static void deep_copy(void* dest, const void* src, MemPool* mem_pool) {$/;" f struct:doris::FieldTypeTraits +deep_copy src/runtime/buffered_tuple_stream2_ir.cc /^bool BufferedTupleStream2::deep_copy(TupleRow* row) {$/;" f class:doris::BufferedTupleStream2 +deep_copy src/runtime/buffered_tuple_stream_ir.cpp /^bool BufferedTupleStream::deep_copy(TupleRow* row, uint8_t** dst) {$/;" f class:doris::BufferedTupleStream +deep_copy src/runtime/tuple.cpp /^Tuple* Tuple::deep_copy(const TupleDescriptor& desc, MemPool* pool, bool convert_ptrs) {$/;" f class:doris::Tuple +deep_copy src/runtime/tuple.cpp /^void Tuple::deep_copy($/;" f class:doris::Tuple +deep_copy src/runtime/tuple.cpp /^void Tuple::deep_copy(Tuple* dst, const TupleDescriptor& desc, MemPool* pool,$/;" f class:doris::Tuple +deep_copy src/runtime/tuple.h /^ Tuple* deep_copy(const TupleDescriptor& desc, MemPool* pool) {$/;" f class:doris::Tuple +deep_copy src/runtime/tuple.h /^ void deep_copy(Tuple* dst, const TupleDescriptor& desc, MemPool* pool) {$/;" f class:doris::Tuple +deep_copy src/runtime/tuple.h /^ void deep_copy(const TupleDescriptor& desc, char** data, int* offset) {$/;" f class:doris::Tuple +deep_copy src/runtime/tuple_row.h /^ TupleRow* deep_copy(const std::vector& descs, MemPool* pool) {$/;" f class:doris::TupleRow +deep_copy src/runtime/tuple_row.h /^ void deep_copy(TupleRow* dst, const std::vector& descs, MemPool* pool,$/;" f class:doris::TupleRow +deep_copy_content src/olap/field.h /^ inline void deep_copy_content(char* dst, const char* src, MemPool* mem_pool) const {$/;" f class:doris::Field +deep_copy_internal src/runtime/buffered_tuple_stream2_ir.cc /^bool BufferedTupleStream2::deep_copy_internal(TupleRow* row) {$/;" f class:doris::BufferedTupleStream2 +deep_copy_internal src/runtime/buffered_tuple_stream_ir.cpp /^bool BufferedTupleStream::deep_copy_internal(TupleRow* row, uint8_t** dst) {$/;" f class:doris::BufferedTupleStream +defalut_reason src/http/http_status.cpp /^std::string defalut_reason(const HttpStatus& status) {$/;" f namespace:doris +default_collation src/exec/schema_scanner/schema_charsets_scanner.h /^ const char *default_collation;$/;" m struct:doris::SchemaCharsetsScanner::CharsetStruct +default_page_len_ src/runtime/buffered_tuple_stream3.h /^ const int64_t default_page_len_;$/;" m class:doris::BufferedTupleStream3 +default_rowset_type src/olap/storage_engine.h /^ RowsetTypePB default_rowset_type() const {$/;" f class:doris::StorageEngine +default_time_zone src/exprs/timezone_db.cpp /^const std::string TimezoneDatabase::default_time_zone = "+08:00";$/;" m class:doris::TimezoneDatabase file: +default_time_zone src/exprs/timezone_db.h /^ static const std::string default_time_zone;$/;" m class:doris::TimezoneDatabase +default_value src/olap/column_mapping.h /^ WrapperField* default_value;$/;" m struct:doris::ColumnMapping +default_value src/olap/tablet_schema.h /^ std::string default_value() const { return _default_value; }$/;" f class:doris::TabletColumn +defval src/common/configbase.h /^ const char* defval;$/;" m struct:doris::config::Register::Field +degrees src/exprs/math_functions.cpp /^DoubleVal MathFunctions::degrees($/;" f class:doris::MathFunctions +del src/runtime/buffered_block_mgr2.cc /^void BufferedBlockMgr2::Block::del() {$/;" f class:doris::BufferedBlockMgr2::Block +del_cond src/olap/delete_handler.h /^ Conditions* del_cond; \/\/ 删除条件$/;" m struct:doris::DeleteConditions +del_eval src/olap/olap_cond.cpp /^int Cond::del_eval(const std::pair& stat) const {$/;" f class:doris::Cond +del_eval src/olap/olap_cond.cpp /^int CondColumn::del_eval(const std::pair& statistic) const {$/;" f class:doris::CondColumn +deleteContext src/util/thrift_server.cpp /^void ThriftServer::ThriftServerEventProcessor::deleteContext($/;" f class:doris::ThriftServer::ThriftServerEventProcessor +delete_all_blocks src/runtime/spill_sorter.cc /^void SpillSorter::Run::delete_all_blocks() {$/;" f class:doris::SpillSorter::Run +delete_all_files src/olap/rowset/segment_group.cpp /^bool SegmentGroup::delete_all_files() {$/;" f class:doris::SegmentGroup +delete_all_files src/olap/tablet.cpp /^void Tablet::delete_all_files() {$/;" f class:doris::Tablet +delete_alter_task src/olap/tablet.cpp /^void Tablet::delete_alter_task() {$/;" f class:doris::Tablet +delete_alter_task src/olap/tablet_meta.cpp /^void TabletMeta::delete_alter_task() {$/;" f class:doris::TabletMeta +delete_block src/runtime/buffered_block_mgr.h /^ Status delete_block() {$/;" f class:doris::BufferedBlockMgr::Block +delete_block src/runtime/buffered_block_mgr2.cc /^void BufferedBlockMgr2::delete_block(Block* block) {$/;" f class:doris::BufferedBlockMgr2 +delete_conditions src/olap/iterators.h /^ std::vector delete_conditions;$/;" m class:doris::StorageReadOptions +delete_conditions_eval src/olap/olap_cond.cpp /^bool Conditions::delete_conditions_eval(const RowCursor& row) const {$/;" f class:doris::Conditions +delete_etl_files src/agent/agent_server.cpp /^void AgentServer::delete_etl_files(TAgentResult& t_agent_result,$/;" f class:doris::AgentServer +delete_etl_files src/service/backend_service.h /^ virtual void delete_etl_files(TAgentResult& result,$/;" f class:doris::BackendService +delete_etl_files test/runtime/data_stream_test.cpp /^ virtual void delete_etl_files($/;" f class:doris::DorisTestBackend +delete_expired_inc_rowsets src/olap/tablet.cpp /^void Tablet::delete_expired_inc_rowsets() {$/;" f class:doris::Tablet +delete_flag src/olap/olap_index.h /^ bool delete_flag() const {$/;" f class:doris::MemIndex +delete_flag src/olap/rowset/alpha_rowset_reader.cpp /^bool AlphaRowsetReader::delete_flag() {$/;" f class:doris::AlphaRowsetReader +delete_flag src/olap/rowset/column_data.h /^ bool delete_flag() const {$/;" f class:doris::ColumnData +delete_flag src/olap/rowset/rowset.h /^ bool delete_flag() const { return rowset_meta()->delete_flag(); }$/;" f class:doris::Rowset +delete_flag src/olap/rowset/rowset_meta.h /^ bool delete_flag() const {$/;" f class:doris::RowsetMeta +delete_flag src/olap/rowset/segment_group.h /^ inline bool delete_flag() const { return _delete_flag; }$/;" f class:doris::SegmentGroup +delete_handler src/olap/rowset/rowset_reader_context.h /^ const DeleteHandler* delete_handler = nullptr;$/;" m struct:doris::RowsetReaderContext +delete_handler src/olap/schema_change.h /^ DeleteHandler delete_handler;$/;" m struct:doris::SchemaChangeHandler::SchemaChangeParams +delete_inc_rs_meta_by_version src/olap/tablet_meta.cpp /^void TabletMeta::delete_inc_rs_meta_by_version(const Version& version) {$/;" f class:doris::TabletMeta +delete_meta src/tools/meta_tool.cpp /^void delete_meta(DataDir* data_dir) {$/;" f +delete_on_read_ src/runtime/buffered_tuple_stream3.h /^ bool delete_on_read_;$/;" m class:doris::BufferedTupleStream3 +delete_or_unpin_block src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::delete_or_unpin_block(Block* block, bool unpin) {$/;" f class:doris::BufferedBlockMgr2 +delete_predicate src/olap/rowset/rowset_meta.h /^ const DeletePredicatePB& delete_predicate() const {$/;" f class:doris::RowsetMeta +delete_predicates src/olap/tablet.h /^ DelPredicateArray delete_predicates() { return _tablet_meta->delete_predicates(); }$/;" f class:doris::Tablet +delete_predicates src/olap/tablet_meta.cpp /^DelPredicateArray TabletMeta::delete_predicates() const {$/;" f class:doris::TabletMeta +delete_pruning_filter src/olap/olap_cond.cpp /^int Conditions::delete_pruning_filter(const std::vector& zone_maps) const {$/;" f class:doris::Conditions +delete_pruning_filter src/olap/rowset/column_data.cpp /^int ColumnData::delete_pruning_filter() {$/;" f class:doris::ColumnData +delete_requests_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::delete_requests_failed;$/;" m class:doris::DorisMetrics file: +delete_requests_failed src/util/doris_metrics.h /^ static IntCounter delete_requests_failed;$/;" m class:doris::DorisMetrics +delete_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::delete_requests_total;$/;" m class:doris::DorisMetrics file: +delete_requests_total src/util/doris_metrics.h /^ static IntCounter delete_requests_total;$/;" m class:doris::DorisMetrics +delete_rowset_meta src/olap/tablet_sync_service.cpp /^std::future TabletSyncService::delete_rowset_meta(RowsetMetaPB& rowset_meta) {$/;" f class:doris::TabletSyncService +delete_rs_meta_by_version src/olap/tablet_meta.cpp /^void TabletMeta::delete_rs_meta_by_version(const Version& version,$/;" f class:doris::TabletMeta +delete_state src/olap/column_block.h /^ DelCondSatisfied delete_state() const { return _delete_state; }$/;" f class:doris::ColumnBlock +delete_state src/olap/row_block2.h /^ DelCondSatisfied delete_state() const { return _delete_state; }$/;" f class:doris::RowBlockV2 +delete_txn src/olap/txn_manager.cpp /^OLAPStatus TxnManager::delete_txn(OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id,$/;" f class:doris::TxnManager +delete_txn src/olap/txn_manager.cpp /^OLAPStatus TxnManager::delete_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, TTransactionId transaction_id) {$/;" f class:doris::TxnManager +delete_user_cgroups src/agent/cgroups_mgr.cpp /^AgentStatus CgroupsMgr::delete_user_cgroups(const string& user_name) {$/;" f class:doris::CgroupsMgr +delete_version_from_graph src/olap/rowset_graph.cpp /^OLAPStatus RowsetGraph::delete_version_from_graph(const Version& version) {$/;" f class:doris::RowsetGraph +deleter src/olap/lru_cache.h /^ void (*deleter)(const CacheKey&, void* value);$/;" m struct:doris::CachePriority::LRUHandle +deleter test/olap/lru_cache_test.cpp /^static void deleter(const CacheKey& key, void* v) {$/;" f namespace:doris +deleter_ src/gutil/stl_util.h /^ BaseDeleter *deleter_;$/;" m class:ElementDeleter +deleter_ src/gutil/stl_util.h /^ BaseDeleter *deleter_;$/;" m class:ValueDeleter +deleter_type src/gutil/gscoped_ptr.h /^ typedef D deleter_type;$/;" t class:gscoped_ptr +delimiter src/gutil/strings/split.cc /^namespace delimiter {$/;" n namespace:strings file: +delimiter src/gutil/strings/split.h /^namespace delimiter {$/;" n namespace:strings +delimiter_ src/gutil/strings/split.h /^ Delimiter delimiter_;$/;" m class:strings::delimiter::LimitImpl +delimiter_ src/gutil/strings/split.h /^ const string delimiter_;$/;" m class:strings::delimiter::Literal +delimiter_ src/gutil/strings/split_internal.h /^ Delimiter delimiter_;$/;" m class:strings::internal::SplitIterator +delimiters_ src/gutil/strings/split.h /^ const string delimiters_;$/;" m class:strings::delimiter::AnyOf +demangle src/util/symbols_util.cpp /^std::string SymbolsUtil::demangle(const std::string& name) {$/;" f class:doris::SymbolsUtil +demangle_name_only src/util/symbols_util.cpp /^std::string SymbolsUtil::demangle_name_only(const std::string& symbol) {$/;" f class:doris::SymbolsUtil +demangle_no_args src/util/symbols_util.cpp /^std::string SymbolsUtil::demangle_no_args(const std::string& symbol) {$/;" f class:doris::SymbolsUtil +dense_rank_get_value src/exprs/aggregate_functions.cpp /^BigIntVal AggregateFunctions::dense_rank_get_value(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +dense_rank_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::dense_rank_update(FunctionContext* ctx, StringVal* dst) { }$/;" f class:doris::AggregateFunctions +dequeue src/runtime/bufferpool/buffer_pool_internal.h /^ Page* dequeue() {$/;" f class:doris::BufferPool::PageList +dequeue src/util/internal_queue.h /^ T* dequeue() {$/;" f class:doris::InternalQueueBase +deregister_hook src/util/metrics.cpp /^void MetricRegistry::deregister_hook(const std::string& name) {$/;" f class:doris::MetricRegistry +deregister_metric src/util/metrics.h /^ void deregister_metric(Metric* metric) {$/;" f class:doris::MetricRegistry +deregister_pull_load_task test/runtime/data_stream_test.cpp /^ virtual void deregister_pull_load_task($/;" f class:doris::DorisTestBackend +deregister_recvr src/runtime/data_stream_mgr.cpp /^Status DataStreamMgr::deregister_recvr($/;" f class:doris::DataStreamMgr +deregister_tablet src/olap/data_dir.cpp /^OLAPStatus DataDir::deregister_tablet(Tablet* tablet) {$/;" f class:doris::DataDir +deregister_tablet_from_dir src/olap/tablet.h /^inline OLAPStatus Tablet::deregister_tablet_from_dir() {$/;" f class:doris::Tablet +desc_ src/runtime/buffered_tuple_stream3.h /^ const RowDescriptor* desc_;$/;" m class:doris::BufferedTupleStream3 +desc_num test/exec/schema_scanner/schema_columns_scanner_test.cpp /^int desc_num = 0;$/;" m namespace:doris file: +desc_num test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^int desc_num = 0;$/;" m namespace:doris file: +desc_num test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^int desc_num = 0;$/;" m namespace:doris file: +desc_num test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^int desc_num = 0;$/;" m namespace:doris file: +desc_num test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^int desc_num = 0;$/;" m namespace:doris file: +desc_num test/exec/schema_scanner/schema_tables_scanner_test.cpp /^int desc_num = 0;$/;" m namespace:doris file: +desc_tbl src/runtime/descriptor_helper.h /^ TDescriptorTable desc_tbl() { return _desc_tbl; }$/;" f class:doris::TDescriptorTableBuilder +desc_tbl src/runtime/plan_fragment_executor.h /^ const DescriptorTbl& desc_tbl() {$/;" f class:doris::PlanFragmentExecutor +desc_tbl src/runtime/runtime_state.h /^ const DescriptorTbl& desc_tbl() const {$/;" f class:doris::RuntimeState +describe_table src/exec/schema_scanner/schema_helper.cpp /^Status SchemaHelper::describe_table($/;" f class:doris::SchemaHelper +describe_table test/exec/schema_scanner/schema_columns_scanner_test.cpp /^Status SchemaJniHelper::describe_table(const TDescribeTableParams &desc_params,$/;" f class:doris::SchemaJniHelper +describe_table test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^Status SchemaJniHelper::describe_table(const TDescribeTableParams &desc_params,$/;" f class:doris::SchemaJniHelper +describe_table test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^Status SchemaJniHelper::describe_table(const TDescribeTableParams &desc_params,$/;" f class:doris::SchemaJniHelper +describe_table test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^Status SchemaJniHelper::describe_table(const TDescribeTableParams &desc_params,$/;" f class:doris::SchemaJniHelper +describe_table test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^Status SchemaJniHelper::describe_table(const TDescribeTableParams &desc_params,$/;" f class:doris::SchemaJniHelper +describe_table test/exec/schema_scanner/schema_tables_scanner_test.cpp /^Status SchemaJniHelper::describe_table(const TDescribeTableParams &desc_params,$/;" f class:doris::SchemaJniHelper +description src/exec/schema_scanner/schema_charsets_scanner.h /^ const char *description;$/;" m struct:doris::SchemaCharsetsScanner::CharsetStruct +deserialize src/exprs/bitmap_function.cpp /^ void deserialize(const char* src) {$/;" f struct:doris::BitmapIntersect +deserialize src/olap/hll.cpp /^bool HyperLogLog::deserialize(const Slice& slice) {$/;" f class:doris::HyperLogLog +deserialize src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::deserialize(const string& meta_binary) {$/;" f class:doris::TabletMeta +deserialize src/util/bitmap_value.h /^ bool deserialize(const char* src) {$/;" f class:doris::BitmapValue +deserialize_thrift_msg src/util/thrift_util.h /^Status deserialize_thrift_msg($/;" f namespace:doris +dest_node_id src/runtime/data_stream_recvr.h /^ PlanNodeId dest_node_id() const { return _dest_node_id; }$/;" f class:doris::DataStreamRecvr +destory src/exprs/aggregate_functions.cpp /^ static void destory(const StringVal& dst) {$/;" f class:doris::MultiDistinctCountDateState +destory src/exprs/aggregate_functions.cpp /^ static void destory(const StringVal& dst) {$/;" f class:doris::MultiDistinctDecimalState +destory src/exprs/aggregate_functions.cpp /^ static void destory(const StringVal& dst) {$/;" f class:doris::MultiDistinctDecimalV2State +destory src/exprs/aggregate_functions.cpp /^ static void destory(const StringVal& dst) {$/;" f class:doris::MultiDistinctNumericState +destory src/exprs/aggregate_functions.cpp /^ static void destory(const StringVal& dst) {$/;" f class:doris::MultiDistinctStringCountState +destroy src/runtime/exec_env_init.cpp /^void ExecEnv::destroy(ExecEnv* env) {$/;" f class:doris::ExecEnv +detail src/exprs/bitmap_function.cpp /^namespace detail {$/;" n namespace:doris file: +detail src/util/bitmap_value.h /^namespace detail {$/;" n namespace:doris +device_name src/util/disk_info.h /^ static const std::string& device_name(int disk_id) {$/;" f class:doris::DiskInfo +dict_page_size src/olap/rowset/segment_v2/options.h /^ size_t dict_page_size = DEFAULT_PAGE_SIZE;$/;" m struct:doris::segment_v2::PageBuilderOptions +difference_type src/gutil/strings/stringpiece.h /^ typedef ptrdiff_t difference_type;$/;" t class:StringPiece +difference_type src/util/bitmap_value.h /^ typedef int64_t difference_type;$/;" t class:doris::detail::final +digest src/exprs/aggregate_functions.cpp /^ TDigest *digest = nullptr;$/;" m struct:doris::PercentileApproxState file: +digest src/util/md5.cpp /^void Md5Digest::digest() {$/;" f class:doris::Md5Digest +digits src/gutil/strings/strcat.h /^ char digits[kFastToBufferSize];$/;" m struct:AlphaNum +dir_name src/util/path_util.cpp /^string dir_name(const string& path) {$/;" f namespace:doris::path_util +direct_copy src/olap/field.h /^ void direct_copy(DstCellType* dst, const SrcCellType& src) const {$/;" f class:doris::Field +direct_copy src/olap/types.h /^ inline void direct_copy(void* dest, const void* src) const {$/;" f class:doris::TypeInfo +direct_copy src/olap/types.h /^ static inline void direct_copy(void* dest, const void* src) {$/;" f struct:doris::BaseFieldtypeTraits +direct_copy src/olap/types.h /^ static void direct_copy(void* dest, const void* src) {$/;" f struct:doris::FieldTypeTraits +direct_copy_row src/olap/row.h /^void direct_copy_row(DstRowType* dst, const SrcRowType& src) {$/;" f namespace:doris +dirty_unpinned_pages_ src/runtime/bufferpool/buffer_pool_internal.h /^ PageList dirty_unpinned_pages_;$/;" m class:doris::BufferPool::Client +disable_stream_preaggregations src/runtime/runtime_state.h /^ bool disable_stream_preaggregations() {$/;" f class:doris::RuntimeState +disk_capacity src/olap/olap_common.h /^ int64_t disk_capacity; \/\/ actual disk capacity$/;" m struct:doris::DataDirInfo +disk_id src/runtime/disk_io_mgr.h /^ int disk_id() const { return _disk_id; }$/;" f class:doris::DiskIoMgr::RequestRange +disk_id src/runtime/disk_io_mgr_internal.h /^ int disk_id;$/;" m struct:doris::DiskIoMgr::DiskQueue +disk_id src/runtime/tmp_file_mgr.h /^ int disk_id() const {$/;" f class:doris::TmpFileMgr::File +disk_id src/util/disk_info.cpp /^int DiskInfo::disk_id(const char* path) {$/;" f class:doris::DiskInfo +disk_index src/olap/storage_engine.h /^ const uint32_t disk_index;$/;" m struct:doris::StorageEngine::CompactionDiskStat +disk_index src/olap/storage_engine.h /^ uint32_t disk_index = -1;$/;" m struct:doris::StorageEngine::CompactionCandidate +disk_io_mgr src/runtime/exec_env.h /^ DiskIoMgr* disk_io_mgr() { return _disk_io_mgr; }$/;" f class:doris::ExecEnv +disk_num_start src/util/minizip/unzip.h /^ uLong disk_num_start; \/* disk number start 2 bytes *\/$/;" m struct:unz_file_info64_s +disk_num_start src/util/minizip/unzip.h /^ uLong disk_num_start; \/* disk number start 2 bytes *\/$/;" m struct:unz_file_info_s +disk_sync_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::disk_sync_total;$/;" m class:doris::DorisMetrics file: +disk_sync_total src/util/doris_metrics.h /^ static IntCounter disk_sync_total;$/;" m class:doris::DorisMetrics +disks_avail_capacity src/util/doris_metrics.cpp /^IntGaugeMetricsMap DorisMetrics::disks_avail_capacity;$/;" m class:doris::DorisMetrics file: +disks_avail_capacity src/util/doris_metrics.h /^ static IntGaugeMetricsMap disks_avail_capacity;$/;" m class:doris::DorisMetrics +disks_data_used_capacity src/util/doris_metrics.cpp /^IntGaugeMetricsMap DorisMetrics::disks_data_used_capacity;$/;" m class:doris::DorisMetrics file: +disks_data_used_capacity src/util/doris_metrics.h /^ static IntGaugeMetricsMap disks_data_used_capacity;$/;" m class:doris::DorisMetrics +disks_state src/util/doris_metrics.cpp /^IntGaugeMetricsMap DorisMetrics::disks_state;$/;" m class:doris::DorisMetrics file: +disks_state src/util/doris_metrics.h /^ static IntGaugeMetricsMap disks_state;$/;" m class:doris::DorisMetrics +disks_total_capacity src/util/doris_metrics.cpp /^IntGaugeMetricsMap DorisMetrics::disks_total_capacity;$/;" m class:doris::DorisMetrics file: +disks_total_capacity src/util/doris_metrics.h /^ static IntGaugeMetricsMap disks_total_capacity;$/;" m class:doris::DorisMetrics +dispatch_thread src/util/threadpool.cpp /^void ThreadPool::dispatch_thread() {$/;" f class:doris::ThreadPool +display_heap src/exec/olap_scan_node.h /^ void display_heap(Heap& heap) {$/;" f class:doris::OlapScanNode +distince_estimate_finalize src/exprs/aggregate_functions.cpp /^double distince_estimate_finalize(const StringVal& src) {$/;" f namespace:doris +distinct_estimate_bitmap_to_string src/exprs/aggregate_functions.cpp /^std::string distinct_estimate_bitmap_to_string(uint8_t* v) {$/;" f namespace:doris +distributed_bucket src/runtime/dpp_sink_internal.h /^ int distributed_bucket() const {$/;" f class:doris::PartitionInfo +distributed_expr_ctxs src/runtime/dpp_sink_internal.h /^ const std::vector& distributed_expr_ctxs() const {$/;" f class:doris::PartitionInfo +divide src/util/runtime_profile.cpp /^void RuntimeProfile::divide(int n) {$/;" f class:doris::RuntimeProfile +dname test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^const std::string dname = ".\/ut_dir\/bloom_filter_index_reader_writer_test";$/;" m namespace:doris::segment_v2 file: +do_add src/runtime/decimal_value.cpp /^int32_t do_add($/;" f namespace:doris +do_add src/runtime/decimalv2_value.cpp /^static int do_add(int128_t x, int128_t y, int128_t* result) {$/;" f namespace:doris +do_aes_test test/util/aes_util_test.cpp /^void do_aes_test(const std::string& source, const std::string& key) {$/;" f namespace:doris +do_check test/olap/rowset/rowset_meta_test.cpp /^void do_check(RowsetMeta rowset_meta) {$/;" f namespace:doris +do_check_for_alpha test/olap/rowset/rowset_meta_test.cpp /^void do_check_for_alpha(AlphaRowsetMeta alpha_rowset_meta) {$/;" f namespace:doris +do_checksum src/http/action/checksum_action.cpp /^int64_t ChecksumAction::do_checksum(int64_t tablet_id, int64_t version, int64_t version_hash,$/;" f class:doris::ChecksumAction +do_close src/olap/rowset/beta_rowset.cpp /^void BetaRowset::do_close() {$/;" f class:doris::BetaRowset +do_compaction src/olap/compaction.cpp /^OLAPStatus Compaction::do_compaction() {$/;" f class:doris::Compaction +do_compaction_impl src/olap/compaction.cpp /^OLAPStatus Compaction::do_compaction_impl() {$/;" f class:doris::Compaction +do_decrypt src/util/aes_util.cpp /^static int do_decrypt(EVP_CIPHER_CTX* aes_ctx, const EVP_CIPHER* cipher,$/;" f namespace:doris +do_dir_response src/http/download_action.cpp /^void DownloadAction::do_dir_response($/;" f class:doris::DownloadAction +do_div src/runtime/decimalv2_value.cpp /^static int do_div(int128_t x, int128_t y, int128_t* result) {$/;" f namespace:doris +do_div_mod src/runtime/decimal_value.cpp /^int do_div_mod($/;" f namespace:doris +do_encrypt src/util/aes_util.cpp /^static int do_encrypt(EVP_CIPHER_CTX* aes_ctx, const EVP_CIPHER* cipher,$/;" f namespace:doris +do_file_response src/http/download_action.cpp /^void DownloadAction::do_file_response(const std::string& file_path, HttpRequest *req) {$/;" f class:doris::DownloadAction +do_load src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::do_load(bool use_cache) {$/;" f class:doris::AlphaRowset +do_load src/olap/rowset/beta_rowset.cpp /^OLAPStatus BetaRowset::do_load(bool \/*use_cache*\/) {$/;" f class:doris::BetaRowset +do_mod src/runtime/decimalv2_value.cpp /^static int do_mod(int128_t x, int128_t y, int128_t* result) {$/;" f namespace:doris +do_money_format src/exprs/string_functions.h /^ static StringVal do_money_format(FunctionContext *context, const std::string& v) {$/;" f class:doris::StringFunctions +do_mul src/runtime/decimal_value.cpp /^int do_mul(const DecimalValue& value1, const DecimalValue& value2, DecimalValue* to) {$/;" f namespace:doris +do_mul src/runtime/decimalv2_value.cpp /^static int do_mul(int128_t x, int128_t y, int128_t* result) {$/;" f namespace:doris +do_not_use_array_with_size_as_type src/gutil/gscoped_ptr.h /^ COMPILE_ASSERT(sizeof(T) == -1, do_not_use_array_with_size_as_type);$/;" m struct:doris::DefaultDeleter +do_open src/env/env_posix.cpp /^static Status do_open(const string& filename, Env::OpenMode mode, int* fd) {$/;" f namespace:doris +do_readv_at src/env/env_posix.cpp /^static Status do_readv_at(int fd, const std::string& filename, uint64_t offset,$/;" f namespace:doris +do_sub src/runtime/decimal_value.cpp /^int do_sub(const DecimalValue& value1, const DecimalValue& value2, DecimalValue *to) {$/;" f namespace:doris +do_sub src/runtime/decimalv2_value.cpp /^static int do_sub(int128_t x, int128_t y, int128_t* result) {$/;" f namespace:doris +do_submit src/util/threadpool.cpp /^Status ThreadPool::do_submit(std::shared_ptr r, ThreadPoolToken* token) {$/;" f class:doris::ThreadPool +do_sync src/env/env_posix.cpp /^static Status do_sync(int fd, const string& filename) {$/;" f namespace:doris +do_tablet_meta_checkpoint src/olap/tablet.cpp /^OLAPStatus Tablet::do_tablet_meta_checkpoint() {$/;" f class:doris::Tablet +do_tablet_meta_checkpoint src/olap/tablet_manager.cpp /^void TabletManager::do_tablet_meta_checkpoint(DataDir* data_dir) {$/;" f class:doris::TabletManager +do_writev_at src/env/env_posix.cpp /^static Status do_writev_at(int fd, const string& filename, uint64_t offset,$/;" f namespace:doris +does_contain_number test/agent/cgroups_mgr_test.cpp /^ static bool does_contain_number(const std::string& file_path, int32_t number) {$/;" f class:doris::CgroupsMgrTest +done src/exec/plain_text_line_reader.h /^ inline bool done() {$/;" f class:doris::PlainTextLineReader +done src/gutil/spinlock_internal.h /^ bool done;$/;" m struct:base::internal::SpinLockWaitTransition +done src/runtime/buffer_control_block.h /^ google::protobuf::Closure* done = nullptr;$/;" m struct:doris::GetResultBatchCtx +done src/runtime/disk_io_mgr_internal.h /^ bool done() const { return _done; }$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +done src/util/bitmap.h /^ bool done() const {$/;" f class:doris::BitmapIterator +done src/util/progress_updater.h /^ bool done() const {$/;" f class:doris::ProgressUpdater +dont_modify_unbounded src/gutil/strings/numbers.h /^ bool dont_modify_unbounded;$/;" m struct:DoubleRangeOptions +doris output/udf/include/udf.h /^namespace doris {$/;" n +doris src/agent/agent_server.cpp /^namespace doris {$/;" n file: +doris src/agent/agent_server.h /^namespace doris {$/;" n +doris src/agent/cgroups_mgr.cpp /^namespace doris {$/;" n file: +doris src/agent/cgroups_mgr.h /^namespace doris {$/;" n +doris src/agent/heartbeat_server.cpp /^namespace doris {$/;" n file: +doris src/agent/heartbeat_server.h /^namespace doris {$/;" n +doris src/agent/pusher.cpp /^namespace doris {$/;" n file: +doris src/agent/status.h /^namespace doris {$/;" n +doris src/agent/task_worker_pool.cpp /^namespace doris {$/;" n file: +doris src/agent/task_worker_pool.h /^namespace doris {$/;" n +doris src/agent/topic_listener.h /^namespace doris {$/;" n +doris src/agent/topic_subscriber.cpp /^namespace doris {$/;" n file: +doris src/agent/topic_subscriber.h /^namespace doris {$/;" n +doris src/agent/user_resource_listener.cpp /^namespace doris {$/;" n file: +doris src/agent/user_resource_listener.h /^namespace doris {$/;" n +doris src/agent/utils.cpp /^namespace doris {$/;" n file: +doris src/agent/utils.h /^namespace doris {$/;" n +doris src/common/atomic.h /^namespace doris {$/;" n +doris src/common/config.h /^namespace doris {$/;" n +doris src/common/configbase.cpp /^namespace doris {$/;" n file: +doris src/common/configbase.h /^namespace doris {$/;" n +doris src/common/daemon.cpp /^namespace doris {$/;" n file: +doris src/common/daemon.h /^namespace doris {$/;" n +doris src/common/global_types.h /^namespace doris {$/;" n +doris src/common/logconfig.cpp /^namespace doris {$/;" n file: +doris src/common/object_pool.h /^namespace doris {$/;" n +doris src/common/resource_tls.cpp /^namespace doris {$/;" n file: +doris src/common/resource_tls.h /^namespace doris {$/;" n +doris src/common/status.cpp /^namespace doris {$/;" n file: +doris src/common/status.h /^namespace doris {$/;" n +doris src/common/utils.h /^namespace doris {$/;" n +doris src/env/env.h /^namespace doris {$/;" n +doris src/env/env_posix.cpp /^namespace doris {$/;" n file: +doris src/env/env_util.cpp /^namespace doris {$/;" n file: +doris src/env/env_util.h /^namespace doris {$/;" n +doris src/exec/aggregation_node.cpp /^namespace doris {$/;" n file: +doris src/exec/aggregation_node.h /^namespace doris {$/;" n +doris src/exec/aggregation_node_ir.cpp /^namespace doris {$/;" n file: +doris src/exec/analytic_eval_node.cpp /^namespace doris {$/;" n file: +doris src/exec/analytic_eval_node.h /^namespace doris {$/;" n +doris src/exec/assert_num_rows_node.cpp /^namespace doris {$/;" n file: +doris src/exec/assert_num_rows_node.h /^namespace doris {$/;" n +doris src/exec/base_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/base_scanner.h /^namespace doris {$/;" n +doris src/exec/blocking_join_node.cpp /^namespace doris {$/;" n file: +doris src/exec/blocking_join_node.h /^namespace doris {$/;" n +doris src/exec/broker_reader.cpp /^namespace doris {$/;" n file: +doris src/exec/broker_reader.h /^namespace doris {$/;" n +doris src/exec/broker_scan_node.cpp /^namespace doris {$/;" n file: +doris src/exec/broker_scan_node.h /^namespace doris {$/;" n +doris src/exec/broker_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/broker_scanner.h /^namespace doris {$/;" n +doris src/exec/broker_writer.cpp /^namespace doris {$/;" n file: +doris src/exec/broker_writer.h /^namespace doris {$/;" n +doris src/exec/cross_join_node.cpp /^namespace doris {$/;" n file: +doris src/exec/cross_join_node.h /^namespace doris {$/;" n +doris src/exec/csv_scan_node.cpp /^namespace doris {$/;" n file: +doris src/exec/csv_scan_node.h /^namespace doris {$/;" n +doris src/exec/csv_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/csv_scanner.h /^namespace doris {$/;" n +doris src/exec/data_sink.cpp /^namespace doris {$/;" n file: +doris src/exec/data_sink.h /^namespace doris {$/;" n +doris src/exec/decompressor.cpp /^namespace doris {$/;" n file: +doris src/exec/decompressor.h /^namespace doris {$/;" n +doris src/exec/empty_set_node.cpp /^namespace doris {$/;" n file: +doris src/exec/empty_set_node.h /^namespace doris {$/;" n +doris src/exec/es/es_predicate.cpp /^namespace doris {$/;" n file: +doris src/exec/es/es_predicate.h /^namespace doris {$/;" n +doris src/exec/es/es_query_builder.cpp /^namespace doris {$/;" n file: +doris src/exec/es/es_query_builder.h /^namespace doris {$/;" n +doris src/exec/es/es_scan_reader.cpp /^namespace doris {$/;" n file: +doris src/exec/es/es_scan_reader.h /^namespace doris {$/;" n +doris src/exec/es/es_scroll_parser.cpp /^namespace doris {$/;" n file: +doris src/exec/es/es_scroll_parser.h /^namespace doris {$/;" n +doris src/exec/es/es_scroll_query.cpp /^namespace doris {$/;" n file: +doris src/exec/es/es_scroll_query.h /^namespace doris {$/;" n +doris src/exec/es_http_scan_node.cpp /^namespace doris {$/;" n file: +doris src/exec/es_http_scan_node.h /^namespace doris {$/;" n +doris src/exec/es_http_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/es_http_scanner.h /^namespace doris {$/;" n +doris src/exec/es_scan_node.cpp /^namespace doris {$/;" n file: +doris src/exec/es_scan_node.h /^namespace doris {$/;" n +doris src/exec/except_node.cpp /^namespace doris {$/;" n file: +doris src/exec/except_node.h /^namespace doris {$/;" n +doris src/exec/exchange_node.cpp /^namespace doris {$/;" n file: +doris src/exec/exchange_node.h /^namespace doris {$/;" n +doris src/exec/exec_node.cpp /^namespace doris {$/;" n file: +doris src/exec/exec_node.h /^namespace doris {$/;" n +doris src/exec/file_reader.h /^namespace doris {$/;" n +doris src/exec/file_writer.h /^namespace doris {$/;" n +doris src/exec/hash_join_node.cpp /^namespace doris {$/;" n file: +doris src/exec/hash_join_node.h /^namespace doris {$/;" n +doris src/exec/hash_join_node_ir.cpp /^namespace doris {$/;" n file: +doris src/exec/hash_table.cpp /^namespace doris {$/;" n file: +doris src/exec/hash_table.h /^namespace doris {$/;" n +doris src/exec/hash_table.hpp /^namespace doris {$/;" n +doris src/exec/intersect_node.cpp /^namespace doris {$/;" n file: +doris src/exec/intersect_node.h /^namespace doris {$/;" n +doris src/exec/kudu_scan_node.cpp /^namespace doris {$/;" n file: +doris src/exec/kudu_scan_node.h /^namespace doris {$/;" n +doris src/exec/kudu_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/kudu_scanner.h /^namespace doris {$/;" n +doris src/exec/kudu_util.cpp /^namespace doris {$/;" n file: +doris src/exec/kudu_util.h /^namespace doris {$/;" n +doris src/exec/line_reader.h /^namespace doris {$/;" n +doris src/exec/local_file_reader.cpp /^namespace doris {$/;" n file: +doris src/exec/local_file_reader.h /^namespace doris {$/;" n +doris src/exec/local_file_writer.cpp /^namespace doris {$/;" n file: +doris src/exec/local_file_writer.h /^namespace doris {$/;" n +doris src/exec/lzo_decompressor.cpp /^namespace doris {$/;" n file: +doris src/exec/merge_join_node.cpp /^namespace doris {$/;" n file: +doris src/exec/merge_join_node.h /^namespace doris {$/;" n +doris src/exec/merge_node.cpp /^namespace doris {$/;" n file: +doris src/exec/merge_node.h /^namespace doris {$/;" n +doris src/exec/mysql_scan_node.cpp /^namespace doris {$/;" n file: +doris src/exec/mysql_scan_node.h /^namespace doris {$/;" n +doris src/exec/mysql_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/mysql_scanner.h /^namespace doris {$/;" n +doris src/exec/new_partitioned_aggregation_node.cc /^namespace doris {$/;" n file: +doris src/exec/new_partitioned_aggregation_node.h /^namespace doris {$/;" n +doris src/exec/new_partitioned_hash_table.h /^namespace doris {$/;" n +doris src/exec/new_partitioned_hash_table.inline.h /^namespace doris {$/;" n +doris src/exec/olap_common.cpp /^namespace doris {$/;" n file: +doris src/exec/olap_common.h /^namespace doris {$/;" n +doris src/exec/olap_rewrite_node.cpp /^namespace doris {$/;" n file: +doris src/exec/olap_rewrite_node.h /^namespace doris {$/;" n +doris src/exec/olap_scan_node.cpp /^namespace doris {$/;" n file: +doris src/exec/olap_scan_node.h /^namespace doris {$/;" n +doris src/exec/olap_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/olap_scanner.h /^namespace doris {$/;" n +doris src/exec/olap_utils.h /^namespace doris {$/;" n +doris src/exec/orc_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/orc_scanner.h /^namespace doris {$/;" n +doris src/exec/parquet_reader.cpp /^namespace doris {$/;" n file: +doris src/exec/parquet_reader.h /^namespace doris {$/;" n +doris src/exec/parquet_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/parquet_scanner.h /^namespace doris {$/;" n +doris src/exec/partitioned_aggregation_node.cc /^namespace doris {$/;" n file: +doris src/exec/partitioned_aggregation_node.h /^namespace doris {$/;" n +doris src/exec/partitioned_aggregation_node_ir.cc /^namespace doris {$/;" n file: +doris src/exec/partitioned_hash_table.cc /^namespace doris {$/;" n file: +doris src/exec/partitioned_hash_table.h /^namespace doris {$/;" n +doris src/exec/partitioned_hash_table.inline.h /^namespace doris {$/;" n +doris src/exec/partitioned_hash_table_ir.cc /^namespace doris {$/;" n file: +doris src/exec/pl_task_root.cpp /^namespace doris {$/;" n file: +doris src/exec/pl_task_root.h /^namespace doris {$/;" n +doris src/exec/plain_text_line_reader.cpp /^namespace doris {$/;" n file: +doris src/exec/plain_text_line_reader.h /^namespace doris {$/;" n +doris src/exec/pre_aggregation_node.cpp /^namespace doris {$/;" n file: +doris src/exec/pre_aggregation_node.h /^namespace doris {$/;" n +doris src/exec/read_write_util.cpp /^namespace doris {$/;" n file: +doris src/exec/read_write_util.h /^namespace doris {$/;" n +doris src/exec/repeat_node.cpp /^namespace doris {$/;" n file: +doris src/exec/repeat_node.h /^namespace doris {$/;" n +doris src/exec/row_batch_list.h /^namespace doris {$/;" n +doris src/exec/scan_node.cpp /^namespace doris {$/;" n file: +doris src/exec/scan_node.h /^namespace doris {$/;" n +doris src/exec/schema_scan_node.cpp /^namespace doris {$/;" n file: +doris src/exec/schema_scan_node.h /^namespace doris {$/;" n +doris src/exec/schema_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/schema_scanner.h /^namespace doris {$/;" n +doris src/exec/schema_scanner/schema_charsets_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/schema_scanner/schema_charsets_scanner.h /^namespace doris {$/;" n +doris src/exec/schema_scanner/schema_collations_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/schema_scanner/schema_collations_scanner.h /^namespace doris {$/;" n +doris src/exec/schema_scanner/schema_columns_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/schema_scanner/schema_columns_scanner.h /^namespace doris {$/;" n +doris src/exec/schema_scanner/schema_dummy_scanner.cpp /^namespace doris $/;" n file: +doris src/exec/schema_scanner/schema_dummy_scanner.h /^namespace doris$/;" n +doris src/exec/schema_scanner/schema_helper.cpp /^namespace doris {$/;" n file: +doris src/exec/schema_scanner/schema_helper.h /^namespace doris {$/;" n +doris src/exec/schema_scanner/schema_schemata_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/schema_scanner/schema_schemata_scanner.h /^namespace doris {$/;" n +doris src/exec/schema_scanner/schema_tables_scanner.cpp /^namespace doris$/;" n file: +doris src/exec/schema_scanner/schema_tables_scanner.h /^namespace doris {$/;" n +doris src/exec/schema_scanner/schema_variables_scanner.cpp /^namespace doris {$/;" n file: +doris src/exec/schema_scanner/schema_variables_scanner.h /^namespace doris {$/;" n +doris src/exec/select_node.cpp /^namespace doris {$/;" n file: +doris src/exec/select_node.h /^namespace doris {$/;" n +doris src/exec/sort_exec_exprs.cpp /^namespace doris {$/;" n file: +doris src/exec/sort_exec_exprs.h /^namespace doris {$/;" n +doris src/exec/sort_node.cpp /^namespace doris {$/;" n file: +doris src/exec/sort_node.h /^namespace doris {$/;" n +doris src/exec/spill_sort_node.cc /^namespace doris {$/;" n file: +doris src/exec/spill_sort_node.h /^namespace doris {$/;" n +doris src/exec/tablet_info.cpp /^namespace doris {$/;" n file: +doris src/exec/tablet_info.h /^namespace doris {$/;" n +doris src/exec/tablet_sink.cpp /^namespace doris {$/;" n file: +doris src/exec/tablet_sink.h /^namespace doris {$/;" n +doris src/exec/text_converter.cpp /^namespace doris {$/;" n file: +doris src/exec/text_converter.h /^namespace doris {$/;" n +doris src/exec/text_converter.hpp /^namespace doris {$/;" n +doris src/exec/topn_node.cpp /^namespace doris {$/;" n file: +doris src/exec/topn_node.h /^namespace doris {$/;" n +doris src/exec/union_node.cpp /^namespace doris {$/;" n file: +doris src/exec/union_node.h /^namespace doris {$/;" n +doris src/exec/union_node_ir.cpp /^namespace doris {$/;" n file: +doris src/exprs/agg_fn.cc /^namespace doris {$/;" n file: +doris src/exprs/agg_fn.h /^namespace doris {$/;" n +doris src/exprs/agg_fn_evaluator.cpp /^namespace doris {$/;" n file: +doris src/exprs/agg_fn_evaluator.h /^namespace doris {$/;" n +doris src/exprs/aggregate_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/aggregate_functions.h /^namespace doris {$/;" n +doris src/exprs/anyval_util.cpp /^namespace doris {$/;" n file: +doris src/exprs/anyval_util.h /^namespace doris {$/;" n +doris src/exprs/arithmetic_expr.cpp /^namespace doris {$/;" n file: +doris src/exprs/arithmetic_expr.h /^namespace doris {$/;" n +doris src/exprs/base64.cpp /^namespace doris {$/;" n file: +doris src/exprs/base64.h /^namespace doris {$/;" n +doris src/exprs/binary_predicate.cpp /^namespace doris {$/;" n file: +doris src/exprs/binary_predicate.h /^namespace doris {$/;" n +doris src/exprs/bitmap_function.cpp /^namespace doris {$/;" n file: +doris src/exprs/bitmap_function.h /^namespace doris {$/;" n +doris src/exprs/case_expr.cpp /^namespace doris {$/;" n file: +doris src/exprs/case_expr.h /^namespace doris {$/;" n +doris src/exprs/cast_expr.cpp /^namespace doris {$/;" n file: +doris src/exprs/cast_expr.h /^namespace doris {$/;" n +doris src/exprs/cast_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/cast_functions.h /^namespace doris {$/;" n +doris src/exprs/compound_predicate.cpp /^namespace doris {$/;" n file: +doris src/exprs/compound_predicate.h /^namespace doris {$/;" n +doris src/exprs/conditional_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/conditional_functions.h /^namespace doris {$/;" n +doris src/exprs/conditional_functions_ir.cpp /^namespace doris {$/;" n file: +doris src/exprs/decimal_operators.cpp /^namespace doris {$/;" n file: +doris src/exprs/decimal_operators.h /^namespace doris {$/;" n +doris src/exprs/decimalv2_operators.cpp /^namespace doris {$/;" n file: +doris src/exprs/decimalv2_operators.h /^namespace doris {$/;" n +doris src/exprs/encryption_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/encryption_functions.h /^namespace doris {$/;" n +doris src/exprs/es_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/es_functions.h /^namespace doris {$/;" n +doris src/exprs/expr.cpp /^namespace doris {$/;" n file: +doris src/exprs/expr.h /^namespace doris {$/;" n +doris src/exprs/expr_context.cpp /^namespace doris {$/;" n file: +doris src/exprs/expr_context.h /^namespace doris {$/;" n +doris src/exprs/expr_ir.cpp /^namespace doris {$/;" n file: +doris src/exprs/expr_value.h /^namespace doris {$/;" n +doris src/exprs/grouping_sets_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/grouping_sets_functions.h /^namespace doris {$/;" n +doris src/exprs/hash_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/hash_functions.h /^namespace doris {$/;" n +doris src/exprs/hll_function.cpp /^namespace doris {$/;" n file: +doris src/exprs/hll_function.h /^namespace doris {$/;" n +doris src/exprs/hll_hash_function.cpp /^namespace doris {$/;" n file: +doris src/exprs/hll_hash_function.h /^namespace doris {$/;" n +doris src/exprs/hybird_map.h /^namespace doris {$/;" n +doris src/exprs/hybird_set.cpp /^namespace doris {$/;" n file: +doris src/exprs/hybird_set.h /^namespace doris {$/;" n +doris src/exprs/in_predicate.cpp /^namespace doris {$/;" n file: +doris src/exprs/in_predicate.h /^namespace doris {$/;" n +doris src/exprs/info_func.cpp /^namespace doris {$/;" n file: +doris src/exprs/info_func.h /^namespace doris {$/;" n +doris src/exprs/is_null_predicate.cpp /^namespace doris {$/;" n file: +doris src/exprs/is_null_predicate.h /^namespace doris {$/;" n +doris src/exprs/json_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/json_functions.h /^namespace doris {$/;" n +doris src/exprs/like_predicate.cpp /^namespace doris {$/;" n file: +doris src/exprs/like_predicate.h /^namespace doris {$/;" n +doris src/exprs/literal.cpp /^namespace doris {$/;" n file: +doris src/exprs/literal.h /^namespace doris {$/;" n +doris src/exprs/math_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/math_functions.h /^namespace doris {$/;" n +doris src/exprs/new_agg_fn_evaluator.h /^namespace doris {$/;" n +doris src/exprs/new_in_predicate.cpp /^namespace doris {$/;" n file: +doris src/exprs/new_in_predicate.h /^namespace doris {$/;" n +doris src/exprs/null_literal.cpp /^namespace doris {$/;" n file: +doris src/exprs/null_literal.h /^namespace doris {$/;" n +doris src/exprs/operators.cpp /^namespace doris {$/;" n file: +doris src/exprs/operators.h /^namespace doris {$/;" n +doris src/exprs/predicate.h /^namespace doris {$/;" n +doris src/exprs/scalar_fn_call.cpp /^namespace doris {$/;" n file: +doris src/exprs/scalar_fn_call.h /^namespace doris {$/;" n +doris src/exprs/slot_ref.cpp /^namespace doris {$/;" n file: +doris src/exprs/slot_ref.h /^namespace doris {$/;" n +doris src/exprs/string_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/string_functions.h /^namespace doris {$/;" n +doris src/exprs/time_operators.cpp /^namespace doris {$/;" n file: +doris src/exprs/time_operators.h /^namespace doris {$/;" n +doris src/exprs/timestamp_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/timestamp_functions.h /^namespace doris {$/;" n +doris src/exprs/timezone_db.cpp /^namespace doris {$/;" n file: +doris src/exprs/timezone_db.h /^namespace doris {$/;" n +doris src/exprs/tuple_is_null_predicate.cpp /^namespace doris {$/;" n file: +doris src/exprs/tuple_is_null_predicate.h /^namespace doris {$/;" n +doris src/exprs/udf_builtins.cpp /^namespace doris {$/;" n file: +doris src/exprs/udf_builtins.h /^namespace doris {$/;" n +doris src/exprs/utility_functions.cpp /^namespace doris {$/;" n file: +doris src/exprs/utility_functions.h /^namespace doris {$/;" n +doris src/geo/geo_common.cpp /^namespace doris {$/;" n file: +doris src/geo/geo_common.h /^namespace doris {$/;" n +doris src/geo/geo_functions.cpp /^namespace doris {$/;" n file: +doris src/geo/geo_functions.h /^namespace doris {$/;" n +doris src/geo/geo_types.cpp /^namespace doris {$/;" n file: +doris src/geo/geo_types.h /^namespace doris {$/;" n +doris src/geo/wkt_parse.cpp /^namespace doris {$/;" n file: +doris src/geo/wkt_parse.h /^namespace doris {$/;" n +doris src/geo/wkt_parse_ctx.h /^namespace doris {$/;" n +doris src/geo/wkt_parse_type.h /^namespace doris {$/;" n +doris src/gutil/gscoped_ptr.h /^namespace doris {$/;" n +doris src/gutil/ref_counted.cc /^namespace doris {$/;" n file: +doris src/gutil/ref_counted.h /^namespace doris {$/;" n +doris src/gutil/sysinfo-test.cc /^namespace doris {$/;" n file: +doris src/http/action/checksum_action.cpp /^namespace doris {$/;" n file: +doris src/http/action/checksum_action.h /^namespace doris {$/;" n +doris src/http/action/compaction_action.cpp /^namespace doris {$/;" n file: +doris src/http/action/compaction_action.h /^namespace doris {$/;" n +doris src/http/action/health_action.cpp /^namespace doris {$/;" n file: +doris src/http/action/health_action.h /^namespace doris {$/;" n +doris src/http/action/meta_action.cpp /^namespace doris {$/;" n file: +doris src/http/action/meta_action.h /^namespace doris {$/;" n +doris src/http/action/metrics_action.cpp /^namespace doris {$/;" n file: +doris src/http/action/metrics_action.h /^namespace doris {$/;" n +doris src/http/action/mini_load.cpp /^namespace doris {$/;" n file: +doris src/http/action/mini_load.h /^namespace doris {$/;" n +doris src/http/action/pprof_actions.cpp /^namespace doris {$/;" n file: +doris src/http/action/pprof_actions.h /^namespace doris {$/;" n +doris src/http/action/reload_tablet_action.cpp /^namespace doris {$/;" n file: +doris src/http/action/reload_tablet_action.h /^namespace doris {$/;" n +doris src/http/action/restore_tablet_action.cpp /^namespace doris {$/;" n file: +doris src/http/action/restore_tablet_action.h /^namespace doris {$/;" n +doris src/http/action/snapshot_action.cpp /^namespace doris {$/;" n file: +doris src/http/action/snapshot_action.h /^namespace doris {$/;" n +doris src/http/action/stream_load.cpp /^namespace doris {$/;" n file: +doris src/http/action/stream_load.h /^namespace doris {$/;" n +doris src/http/default_path_handlers.cpp /^namespace doris {$/;" n file: +doris src/http/default_path_handlers.h /^namespace doris {$/;" n +doris src/http/download_action.cpp /^namespace doris {$/;" n file: +doris src/http/download_action.h /^namespace doris {$/;" n +doris src/http/ev_http_server.cpp /^namespace doris {$/;" n file: +doris src/http/ev_http_server.h /^namespace doris {$/;" n +doris src/http/http_channel.cpp /^namespace doris {$/;" n file: +doris src/http/http_channel.h /^namespace doris {$/;" n +doris src/http/http_client.cpp /^namespace doris {$/;" n file: +doris src/http/http_client.h /^namespace doris {$/;" n +doris src/http/http_common.h /^namespace doris {$/;" n +doris src/http/http_handler.h /^namespace doris {$/;" n +doris src/http/http_headers.cpp /^namespace doris {$/;" n file: +doris src/http/http_headers.h /^namespace doris {$/;" n +doris src/http/http_method.cpp /^namespace doris {$/;" n file: +doris src/http/http_method.h /^namespace doris {$/;" n +doris src/http/http_parser.cpp /^namespace doris {$/;" n file: +doris src/http/http_parser.h /^namespace doris {$/;" n +doris src/http/http_request.cpp /^namespace doris {$/;" n file: +doris src/http/http_request.h /^namespace doris {$/;" n +doris src/http/http_response.cpp /^namespace doris {$/;" n file: +doris src/http/http_response.h /^namespace doris {$/;" n +doris src/http/http_status.cpp /^namespace doris {$/;" n file: +doris src/http/http_status.h /^namespace doris {$/;" n +doris src/http/monitor_action.cpp /^namespace doris {$/;" n file: +doris src/http/monitor_action.h /^namespace doris {$/;" n +doris src/http/rest_monitor_iface.h /^namespace doris {$/;" n +doris src/http/utils.cpp /^namespace doris {$/;" n file: +doris src/http/utils.h /^namespace doris {$/;" n +doris src/http/web_page_handler.cpp /^namespace doris {$/;" n file: +doris src/http/web_page_handler.h /^namespace doris {$/;" n +doris src/olap/aggregate_func.cpp /^namespace doris {$/;" n file: +doris src/olap/aggregate_func.h /^namespace doris {$/;" n +doris src/olap/base_compaction.cpp /^namespace doris {$/;" n file: +doris src/olap/base_compaction.h /^namespace doris {$/;" n +doris src/olap/bhp_lib.h /^namespace doris {$/;" n +doris src/olap/bloom_filter.hpp /^namespace doris {$/;" n +doris src/olap/bloom_filter_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/bloom_filter_reader.h /^namespace doris {$/;" n +doris src/olap/bloom_filter_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/bloom_filter_writer.h /^namespace doris {$/;" n +doris src/olap/byte_buffer.cpp /^namespace doris {$/;" n file: +doris src/olap/byte_buffer.h /^namespace doris {$/;" n +doris src/olap/column_block.h /^namespace doris {$/;" n +doris src/olap/column_mapping.h /^namespace doris {$/;" n +doris src/olap/column_predicate.h /^namespace doris {$/;" n +doris src/olap/compaction.cpp /^namespace doris {$/;" n file: +doris src/olap/compaction.h /^namespace doris {$/;" n +doris src/olap/comparison_predicate.cpp /^namespace doris {$/;" n file: +doris src/olap/comparison_predicate.h /^namespace doris {$/;" n +doris src/olap/compress.cpp /^namespace doris {$/;" n file: +doris src/olap/compress.h /^namespace doris {$/;" n +doris src/olap/cumulative_compaction.cpp /^namespace doris {$/;" n file: +doris src/olap/cumulative_compaction.h /^namespace doris {$/;" n +doris src/olap/data_dir.cpp /^namespace doris {$/;" n file: +doris src/olap/data_dir.h /^namespace doris {$/;" n +doris src/olap/decimal12.h /^namespace doris {$/;" n +doris src/olap/delete_handler.cpp /^namespace doris {$/;" n file: +doris src/olap/delete_handler.h /^namespace doris {$/;" n +doris src/olap/delta_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/delta_writer.h /^namespace doris {$/;" n +doris src/olap/field.h /^namespace doris {$/;" n +doris src/olap/file_helper.cpp /^namespace doris {$/;" n file: +doris src/olap/file_helper.h /^namespace doris {$/;" n +doris src/olap/file_stream.cpp /^namespace doris {$/;" n file: +doris src/olap/file_stream.h /^namespace doris {$/;" n +doris src/olap/fs/block_id.cpp /^namespace doris {$/;" n file: +doris src/olap/fs/block_id.h /^namespace doris {$/;" n +doris src/olap/fs/block_manager.cpp /^namespace doris {$/;" n file: +doris src/olap/fs/block_manager.h /^namespace doris {$/;" n +doris src/olap/fs/block_manager_metrics.cpp /^namespace doris {$/;" n file: +doris src/olap/fs/block_manager_metrics.h /^namespace doris {$/;" n +doris src/olap/fs/file_block_manager.cpp /^namespace doris {$/;" n file: +doris src/olap/fs/file_block_manager.h /^namespace doris {$/;" n +doris src/olap/fs/fs_util.cpp /^namespace doris {$/;" n file: +doris src/olap/fs/fs_util.h /^namespace doris {$/;" n +doris src/olap/generic_iterators.cpp /^namespace doris {$/;" n file: +doris src/olap/generic_iterators.h /^namespace doris {$/;" n +doris src/olap/hll.cpp /^namespace doris {$/;" n file: +doris src/olap/hll.h /^namespace doris {$/;" n +doris src/olap/in_list_predicate.cpp /^namespace doris {$/;" n file: +doris src/olap/in_list_predicate.h /^namespace doris {$/;" n +doris src/olap/in_stream.cpp /^namespace doris {$/;" n file: +doris src/olap/in_stream.h /^namespace doris {$/;" n +doris src/olap/iterators.h /^namespace doris {$/;" n +doris src/olap/key_coder.cpp /^namespace doris {$/;" n file: +doris src/olap/key_coder.h /^namespace doris {$/;" n +doris src/olap/lru_cache.cpp /^namespace doris {$/;" n file: +doris src/olap/lru_cache.h /^namespace doris {$/;" n +doris src/olap/memtable.cpp /^namespace doris {$/;" n file: +doris src/olap/memtable.h /^namespace doris {$/;" n +doris src/olap/memtable_flush_executor.cpp /^namespace doris {$/;" n file: +doris src/olap/memtable_flush_executor.h /^namespace doris {$/;" n +doris src/olap/merger.cpp /^namespace doris {$/;" n file: +doris src/olap/merger.h /^namespace doris {$/;" n +doris src/olap/null_predicate.cpp /^namespace doris {$/;" n file: +doris src/olap/null_predicate.h /^namespace doris {$/;" n +doris src/olap/olap_common.h /^namespace doris {$/;" n +doris src/olap/olap_cond.cpp /^namespace doris {$/;" n file: +doris src/olap/olap_cond.h /^namespace doris {$/;" n +doris src/olap/olap_define.h /^namespace doris {$/;" n +doris src/olap/olap_index.cpp /^namespace doris {$/;" n file: +doris src/olap/olap_index.h /^namespace doris {$/;" n +doris src/olap/olap_meta.cpp /^namespace doris {$/;" n file: +doris src/olap/olap_meta.h /^namespace doris {$/;" n +doris src/olap/olap_server.cpp /^namespace doris {$/;" n file: +doris src/olap/olap_snapshot_converter.cpp /^namespace doris {$/;" n file: +doris src/olap/olap_snapshot_converter.h /^namespace doris {$/;" n +doris src/olap/options.cpp /^namespace doris {$/;" n file: +doris src/olap/options.h /^namespace doris {$/;" n +doris src/olap/out_stream.cpp /^namespace doris {$/;" n file: +doris src/olap/out_stream.h /^namespace doris {$/;" n +doris src/olap/page_cache.cpp /^namespace doris {$/;" n file: +doris src/olap/page_cache.h /^namespace doris {$/;" n +doris src/olap/push_handler.cpp /^namespace doris {$/;" n file: +doris src/olap/push_handler.h /^namespace doris {$/;" n +doris src/olap/reader.cpp /^namespace doris {$/;" n file: +doris src/olap/reader.h /^namespace doris {$/;" n +doris src/olap/row.h /^namespace doris {$/;" n +doris src/olap/row_block.cpp /^namespace doris {$/;" n file: +doris src/olap/row_block.h /^namespace doris {$/;" n +doris src/olap/row_block2.cpp /^namespace doris {$/;" n file: +doris src/olap/row_block2.h /^namespace doris {$/;" n +doris src/olap/row_cursor.cpp /^namespace doris {$/;" n file: +doris src/olap/row_cursor.h /^namespace doris {$/;" n +doris src/olap/row_cursor_cell.h /^namespace doris {$/;" n +doris src/olap/rowset/alpha_rowset.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/alpha_rowset.h /^namespace doris {$/;" n +doris src/olap/rowset/alpha_rowset_meta.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/alpha_rowset_meta.h /^namespace doris {$/;" n +doris src/olap/rowset/alpha_rowset_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/alpha_rowset_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/alpha_rowset_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/alpha_rowset_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/beta_rowset.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/beta_rowset.h /^namespace doris {$/;" n +doris src/olap/rowset/beta_rowset_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/beta_rowset_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/beta_rowset_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/beta_rowset_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/bit_field_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/bit_field_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/bit_field_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/bit_field_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/column_data.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/column_data.h /^namespace doris {$/;" n +doris src/olap/rowset/column_data_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/column_data_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/column_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/column_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/column_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/column_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/rowset.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/rowset.h /^namespace doris {$/;" n +doris src/olap/rowset/rowset_converter.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/rowset_converter.h /^namespace doris {$/;" n +doris src/olap/rowset/rowset_factory.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/rowset_factory.h /^namespace doris {$/;" n +doris src/olap/rowset/rowset_id_generator.h /^namespace doris {$/;" n +doris src/olap/rowset/rowset_meta.h /^namespace doris {$/;" n +doris src/olap/rowset/rowset_meta_manager.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/rowset_meta_manager.h /^namespace doris {$/;" n +doris src/olap/rowset/rowset_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/rowset_reader_context.h /^namespace doris {$/;" n +doris src/olap/rowset/rowset_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/rowset_writer_context.h /^namespace doris {$/;" n +doris src/olap/rowset/run_length_byte_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/run_length_byte_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/run_length_byte_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/run_length_byte_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/run_length_integer_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/run_length_integer_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/run_length_integer_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/run_length_integer_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_group.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_group.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/binary_dict_page.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/binary_dict_page.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/binary_plain_page.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/binary_prefix_page.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/binary_prefix_page.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/bitmap_index_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/bitmap_index_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/bitmap_index_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/bitshuffle_page.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/bitshuffle_page.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/bitshuffle_wrapper.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/block_split_bloom_filter.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/block_split_bloom_filter.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/bloom_filter.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/bloom_filter.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/bloom_filter_index_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/column_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/column_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/column_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/column_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/common.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/empty_segment_iterator.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/empty_segment_iterator.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/encoding_info.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/encoding_info.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/frame_of_reference_page.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/index_page.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/index_page.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/indexed_column_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/indexed_column_reader.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/indexed_column_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/indexed_column_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/options.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/ordinal_page_index.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/ordinal_page_index.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/page_builder.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/page_decoder.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/page_handle.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/page_io.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/page_io.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/page_pointer.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/parsed_page.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/plain_page.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/rle_page.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/row_ranges.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/segment.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/segment.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/segment_iterator.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/segment_iterator.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/segment_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/segment_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_v2/zone_map_index.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_v2/zone_map_index.h /^namespace doris {$/;" n +doris src/olap/rowset/segment_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/segment_writer.h /^namespace doris {$/;" n +doris src/olap/rowset/unique_rowset_id_generator.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset/unique_rowset_id_generator.h /^namespace doris {$/;" n +doris src/olap/rowset_graph.cpp /^namespace doris {$/;" n file: +doris src/olap/rowset_graph.h /^namespace doris {$/;" n +doris src/olap/schema.cpp /^namespace doris {$/;" n file: +doris src/olap/schema.h /^namespace doris {$/;" n +doris src/olap/schema_change.cpp /^namespace doris {$/;" n file: +doris src/olap/schema_change.h /^namespace doris {$/;" n +doris src/olap/selection_vector.h /^namespace doris {$/;" n +doris src/olap/serialize.cpp /^namespace doris {$/;" n file: +doris src/olap/serialize.h /^namespace doris {$/;" n +doris src/olap/short_key_index.cpp /^namespace doris {$/;" n file: +doris src/olap/short_key_index.h /^namespace doris {$/;" n +doris src/olap/skiplist.h /^namespace doris {$/;" n +doris src/olap/snapshot_manager.cpp /^namespace doris {$/;" n file: +doris src/olap/snapshot_manager.h /^namespace doris {$/;" n +doris src/olap/storage_engine.cpp /^namespace doris {$/;" n file: +doris src/olap/storage_engine.h /^namespace doris {$/;" n +doris src/olap/stream_index_common.cpp /^namespace doris {$/;" n file: +doris src/olap/stream_index_common.h /^namespace doris {$/;" n +doris src/olap/stream_index_reader.cpp /^namespace doris {$/;" n file: +doris src/olap/stream_index_reader.h /^namespace doris {$/;" n +doris src/olap/stream_index_writer.cpp /^namespace doris {$/;" n file: +doris src/olap/stream_index_writer.h /^namespace doris {$/;" n +doris src/olap/stream_name.cpp /^namespace doris {$/;" n file: +doris src/olap/stream_name.h /^namespace doris {$/;" n +doris src/olap/tablet.cpp /^namespace doris {$/;" n file: +doris src/olap/tablet.h /^namespace doris {$/;" n +doris src/olap/tablet_manager.cpp /^namespace doris {$/;" n file: +doris src/olap/tablet_manager.h /^namespace doris {$/;" n +doris src/olap/tablet_meta.cpp /^namespace doris {$/;" n file: +doris src/olap/tablet_meta.h /^namespace doris {$/;" n +doris src/olap/tablet_meta_manager.cpp /^namespace doris {$/;" n file: +doris src/olap/tablet_meta_manager.h /^namespace doris {$/;" n +doris src/olap/tablet_schema.cpp /^namespace doris {$/;" n file: +doris src/olap/tablet_schema.h /^namespace doris {$/;" n +doris src/olap/tablet_sync_service.cpp /^namespace doris {$/;" n file: +doris src/olap/tablet_sync_service.h /^namespace doris {$/;" n +doris src/olap/task/engine_alter_tablet_task.cpp /^namespace doris {$/;" n file: +doris src/olap/task/engine_alter_tablet_task.h /^namespace doris {$/;" n +doris src/olap/task/engine_batch_load_task.cpp /^namespace doris {$/;" n file: +doris src/olap/task/engine_batch_load_task.h /^namespace doris {$/;" n +doris src/olap/task/engine_checksum_task.cpp /^namespace doris {$/;" n file: +doris src/olap/task/engine_checksum_task.h /^namespace doris {$/;" n +doris src/olap/task/engine_clone_task.cpp /^namespace doris {$/;" n file: +doris src/olap/task/engine_clone_task.h /^namespace doris {$/;" n +doris src/olap/task/engine_publish_version_task.cpp /^namespace doris {$/;" n file: +doris src/olap/task/engine_publish_version_task.h /^namespace doris {$/;" n +doris src/olap/task/engine_storage_migration_task.cpp /^namespace doris {$/;" n file: +doris src/olap/task/engine_storage_migration_task.h /^namespace doris {$/;" n +doris src/olap/task/engine_task.h /^namespace doris {$/;" n +doris src/olap/tuple.h /^namespace doris {$/;" n +doris src/olap/txn_manager.cpp /^namespace doris {$/;" n file: +doris src/olap/txn_manager.h /^namespace doris {$/;" n +doris src/olap/types.cpp /^namespace doris {$/;" n file: +doris src/olap/types.h /^namespace doris {$/;" n +doris src/olap/uint24.h /^namespace doris {$/;" n +doris src/olap/utils.cpp /^namespace doris {$/;" n file: +doris src/olap/utils.h /^namespace doris {$/;" n +doris src/olap/wrapper_field.cpp /^namespace doris {$/;" n file: +doris src/olap/wrapper_field.h /^namespace doris {$/;" n +doris src/plugin/plugin.h /^namespace doris {$/;" n +doris src/plugin/plugin_manager.h /^namespace doris {$/;" n +doris src/runtime/broker_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/broker_mgr.h /^namespace doris {$/;" n +doris src/runtime/buffer_control_block.cpp /^namespace doris {$/;" n file: +doris src/runtime/buffer_control_block.h /^namespace doris {$/;" n +doris src/runtime/buffered_block_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/buffered_block_mgr.h /^namespace doris {$/;" n +doris src/runtime/buffered_block_mgr2.cc /^namespace doris {$/;" n file: +doris src/runtime/buffered_block_mgr2.h /^namespace doris {$/;" n +doris src/runtime/buffered_tuple_stream.cpp /^namespace doris {$/;" n file: +doris src/runtime/buffered_tuple_stream.h /^namespace doris {$/;" n +doris src/runtime/buffered_tuple_stream2.cc /^namespace doris {$/;" n file: +doris src/runtime/buffered_tuple_stream2.h /^namespace doris {$/;" n +doris src/runtime/buffered_tuple_stream2.inline.h /^namespace doris {$/;" n +doris src/runtime/buffered_tuple_stream2_ir.cc /^namespace doris {$/;" n file: +doris src/runtime/buffered_tuple_stream3.h /^namespace doris {$/;" n +doris src/runtime/buffered_tuple_stream3.inline.h /^namespace doris {$/;" n +doris src/runtime/buffered_tuple_stream_ir.cpp /^namespace doris {$/;" n file: +doris src/runtime/bufferpool/buffer_allocator.cc /^namespace doris {$/;" n file: +doris src/runtime/bufferpool/buffer_allocator.h /^namespace doris {$/;" n +doris src/runtime/bufferpool/buffer_pool.cc /^namespace doris {$/;" n file: +doris src/runtime/bufferpool/buffer_pool.h /^namespace doris {$/;" n +doris src/runtime/bufferpool/buffer_pool_counters.h /^namespace doris {$/;" n +doris src/runtime/bufferpool/buffer_pool_internal.h /^namespace doris {$/;" n +doris src/runtime/bufferpool/free_list.h /^namespace doris {$/;" n +doris src/runtime/bufferpool/reservation_tracker.cc /^namespace doris {$/;" n file: +doris src/runtime/bufferpool/reservation_tracker.h /^namespace doris {$/;" n +doris src/runtime/bufferpool/reservation_tracker_counters.h /^namespace doris {$/;" n +doris src/runtime/bufferpool/reservation_util.cc /^namespace doris {$/;" n file: +doris src/runtime/bufferpool/reservation_util.h /^namespace doris {$/;" n +doris src/runtime/bufferpool/suballocator.cc /^namespace doris {$/;" n file: +doris src/runtime/bufferpool/suballocator.h /^namespace doris {$/;" n +doris src/runtime/bufferpool/system_allocator.cc /^namespace doris {$/;" n file: +doris src/runtime/bufferpool/system_allocator.h /^namespace doris {$/;" n +doris src/runtime/client_cache.cpp /^namespace doris {$/;" n file: +doris src/runtime/client_cache.h /^namespace doris {$/;" n +doris src/runtime/data_spliter.cpp /^namespace doris {$/;" n file: +doris src/runtime/data_spliter.h /^namespace doris {$/;" n +doris src/runtime/data_stream_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/data_stream_mgr.h /^namespace doris {$/;" n +doris src/runtime/data_stream_recvr.cc /^namespace doris {$/;" n file: +doris src/runtime/data_stream_recvr.h /^namespace doris {$/;" n +doris src/runtime/data_stream_recvr.hpp /^namespace doris {$/;" n +doris src/runtime/data_stream_sender.cpp /^namespace doris {$/;" n file: +doris src/runtime/data_stream_sender.h /^namespace doris {$/;" n +doris src/runtime/datetime_value.cpp /^namespace doris {$/;" n file: +doris src/runtime/datetime_value.h /^namespace doris {$/;" n +doris src/runtime/decimal_value.cpp /^namespace doris {$/;" n file: +doris src/runtime/decimal_value.h /^namespace doris {$/;" n +doris src/runtime/decimalv2_value.cpp /^namespace doris {$/;" n file: +doris src/runtime/decimalv2_value.h /^namespace doris {$/;" n +doris src/runtime/descriptor_helper.h /^namespace doris {$/;" n +doris src/runtime/descriptors.cpp /^namespace doris {$/;" n file: +doris src/runtime/descriptors.h /^namespace doris {$/;" n +doris src/runtime/disk_io_mgr.cc /^namespace doris {$/;" n file: +doris src/runtime/disk_io_mgr.h /^namespace doris {$/;" n +doris src/runtime/disk_io_mgr_internal.h /^namespace doris {$/;" n +doris src/runtime/disk_io_mgr_reader_context.cc /^namespace doris {$/;" n file: +doris src/runtime/disk_io_mgr_scan_range.cc /^namespace doris {$/;" n file: +doris src/runtime/dpp_sink.cpp /^namespace doris {$/;" n file: +doris src/runtime/dpp_sink.h /^namespace doris {$/;" n +doris src/runtime/dpp_sink_internal.cpp /^namespace doris {$/;" n file: +doris src/runtime/dpp_sink_internal.h /^namespace doris {$/;" n +doris src/runtime/dpp_writer.cpp /^namespace doris {$/;" n file: +doris src/runtime/dpp_writer.h /^namespace doris {$/;" n +doris src/runtime/etl_job_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/etl_job_mgr.h /^namespace doris {$/;" n +doris src/runtime/exec_env.cpp /^namespace doris {$/;" n file: +doris src/runtime/exec_env.h /^namespace doris {$/;" n +doris src/runtime/exec_env_init.cpp /^namespace doris {$/;" n file: +doris src/runtime/export_sink.cpp /^namespace doris {$/;" n file: +doris src/runtime/export_sink.h /^namespace doris {$/;" n +doris src/runtime/export_task_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/export_task_mgr.h /^namespace doris {$/;" n +doris src/runtime/external_scan_context_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/external_scan_context_mgr.h /^namespace doris {$/;" n +doris src/runtime/fragment_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/fragment_mgr.h /^namespace doris {$/;" n +doris src/runtime/free_list.hpp /^namespace doris {$/;" n +doris src/runtime/free_pool.hpp /^namespace doris {$/;" n +doris src/runtime/heartbeat_flags.h /^namespace doris {$/;" n +doris src/runtime/initial_reservations.cc /^namespace doris {$/;" n file: +doris src/runtime/initial_reservations.h /^namespace doris {$/;" n +doris src/runtime/large_int_value.cpp /^namespace doris {$/;" n file: +doris src/runtime/large_int_value.h /^namespace doris {$/;" n +doris src/runtime/load_channel.cpp /^namespace doris {$/;" n file: +doris src/runtime/load_channel.h /^namespace doris {$/;" n +doris src/runtime/load_channel_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/load_channel_mgr.h /^namespace doris {$/;" n +doris src/runtime/load_path_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/load_path_mgr.h /^namespace doris {$/;" n +doris src/runtime/mem_pool.cpp /^namespace doris {$/;" n file: +doris src/runtime/mem_pool.h /^namespace doris {$/;" n +doris src/runtime/mem_tracker.cpp /^namespace doris {$/;" n file: +doris src/runtime/mem_tracker.h /^namespace doris {$/;" n +doris src/runtime/memory/chunk.h /^namespace doris {$/;" n +doris src/runtime/memory/chunk_allocator.cpp /^namespace doris {$/;" n file: +doris src/runtime/memory/chunk_allocator.h /^namespace doris {$/;" n +doris src/runtime/memory/system_allocator.cpp /^namespace doris {$/;" n file: +doris src/runtime/memory/system_allocator.h /^namespace doris {$/;" n +doris src/runtime/memory_scratch_sink.cpp /^namespace doris {$/;" n file: +doris src/runtime/memory_scratch_sink.h /^namespace doris {$/;" n +doris src/runtime/merge_sorter.cpp /^namespace doris {$/;" n file: +doris src/runtime/merge_sorter.h /^namespace doris {$/;" n +doris src/runtime/message_body_sink.cpp /^namespace doris {$/;" n file: +doris src/runtime/message_body_sink.h /^namespace doris {$/;" n +doris src/runtime/multi_precision.h /^namespace doris {$/;" n +doris src/runtime/mysql_table_sink.cpp /^namespace doris {$/;" n file: +doris src/runtime/mysql_table_sink.h /^namespace doris {$/;" n +doris src/runtime/mysql_table_writer.cpp /^namespace doris {$/;" n file: +doris src/runtime/mysql_table_writer.h /^namespace doris {$/;" n +doris src/runtime/plan_fragment_executor.cpp /^namespace doris {$/;" n file: +doris src/runtime/plan_fragment_executor.h /^namespace doris {$/;" n +doris src/runtime/primitive_type.cpp /^namespace doris {$/;" n file: +doris src/runtime/primitive_type.h /^namespace doris {$/;" n +doris src/runtime/qsorter.cpp /^namespace doris {$/;" n file: +doris src/runtime/qsorter.h /^namespace doris {$/;" n +doris src/runtime/query_statistics.cpp /^namespace doris {$/;" n file: +doris src/runtime/query_statistics.h /^namespace doris {$/;" n +doris src/runtime/raw_value.cpp /^namespace doris {$/;" n file: +doris src/runtime/raw_value.h /^namespace doris {$/;" n +doris src/runtime/raw_value_ir.cpp /^namespace doris {$/;" n file: +doris src/runtime/record_batch_queue.cpp /^namespace doris {$/;" n file: +doris src/runtime/record_batch_queue.h /^namespace doris {$/;" n +doris src/runtime/result_buffer_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/result_buffer_mgr.h /^namespace doris {$/;" n +doris src/runtime/result_queue_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/result_queue_mgr.h /^namespace doris {$/;" n +doris src/runtime/result_sink.cpp /^namespace doris {$/;" n file: +doris src/runtime/result_sink.h /^namespace doris {$/;" n +doris src/runtime/result_writer.cpp /^namespace doris {$/;" n file: +doris src/runtime/result_writer.h /^namespace doris {$/;" n +doris src/runtime/routine_load/data_consumer.cpp /^namespace doris {$/;" n file: +doris src/runtime/routine_load/data_consumer.h /^namespace doris {$/;" n +doris src/runtime/routine_load/data_consumer_group.cpp /^namespace doris {$/;" n file: +doris src/runtime/routine_load/data_consumer_group.h /^namespace doris {$/;" n +doris src/runtime/routine_load/data_consumer_pool.cpp /^namespace doris {$/;" n file: +doris src/runtime/routine_load/data_consumer_pool.h /^namespace doris {$/;" n +doris src/runtime/routine_load/kafka_consumer_pipe.h /^namespace doris {$/;" n +doris src/runtime/routine_load/routine_load_task_executor.cpp /^namespace doris {$/;" n file: +doris src/runtime/routine_load/routine_load_task_executor.h /^namespace doris {$/;" n +doris src/runtime/row_batch.cpp /^namespace doris {$/;" n file: +doris src/runtime/row_batch.h /^namespace doris {$/;" n +doris src/runtime/row_batch_interface.hpp /^namespace doris {$/;" n +doris src/runtime/runtime_state.cpp /^namespace doris {$/;" n file: +doris src/runtime/runtime_state.h /^namespace doris {$/;" n +doris src/runtime/small_file_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/small_file_mgr.h /^namespace doris {$/;" n +doris src/runtime/snapshot_loader.cpp /^namespace doris {$/;" n file: +doris src/runtime/snapshot_loader.h /^namespace doris {$/;" n +doris src/runtime/sorted_run_merger.cc /^namespace doris {$/;" n file: +doris src/runtime/sorted_run_merger.h /^namespace doris {$/;" n +doris src/runtime/sorter.h /^namespace doris {$/;" n +doris src/runtime/spill_sorter.cc /^namespace doris {$/;" n file: +doris src/runtime/spill_sorter.h /^namespace doris {$/;" n +doris src/runtime/stream_load/load_stream_mgr.h /^namespace doris {$/;" n +doris src/runtime/stream_load/stream_load_context.cpp /^namespace doris {$/;" n file: +doris src/runtime/stream_load/stream_load_context.h /^namespace doris {$/;" n +doris src/runtime/stream_load/stream_load_executor.cpp /^namespace doris {$/;" n file: +doris src/runtime/stream_load/stream_load_executor.h /^namespace doris {$/;" n +doris src/runtime/stream_load/stream_load_pipe.h /^namespace doris {$/;" n +doris src/runtime/string_buffer.hpp /^namespace doris {$/;" n +doris src/runtime/string_search.hpp /^namespace doris {$/;" n +doris src/runtime/string_value.cpp /^namespace doris {$/;" n file: +doris src/runtime/string_value.h /^namespace doris {$/;" n +doris src/runtime/string_value.hpp /^namespace doris {$/;" n +doris src/runtime/string_value_ir.cpp /^namespace doris {$/;" n file: +doris src/runtime/tablets_channel.cpp /^namespace doris {$/;" n file: +doris src/runtime/tablets_channel.h /^namespace doris {$/;" n +doris src/runtime/test_env.cc /^namespace doris {$/;" n file: +doris src/runtime/test_env.h /^namespace doris {$/;" n +doris src/runtime/thread_resource_mgr.cpp /^namespace doris {$/;" n file: +doris src/runtime/thread_resource_mgr.h /^namespace doris {$/;" n +doris src/runtime/tmp_file_mgr.cc /^namespace doris {$/;" n file: +doris src/runtime/tmp_file_mgr.h /^namespace doris {$/;" n +doris src/runtime/tuple.cpp /^namespace doris {$/;" n file: +doris src/runtime/tuple.h /^namespace doris {$/;" n +doris src/runtime/tuple_row.cpp /^namespace doris {$/;" n file: +doris src/runtime/tuple_row.h /^namespace doris {$/;" n +doris src/runtime/types.cpp /^namespace doris {$/;" n file: +doris src/runtime/types.h /^namespace doris {$/;" n +doris src/runtime/user_function_cache.cpp /^namespace doris {$/;" n file: +doris src/runtime/user_function_cache.h /^namespace doris {$/;" n +doris src/runtime/vectorized_row_batch.cpp /^namespace doris {$/;" n file: +doris src/runtime/vectorized_row_batch.h /^namespace doris {$/;" n +doris src/service/backend_options.cpp /^namespace doris {$/;" n file: +doris src/service/backend_options.h /^namespace doris {$/;" n +doris src/service/backend_service.cpp /^namespace doris {$/;" n file: +doris src/service/backend_service.h /^namespace doris {$/;" n +doris src/service/brpc_service.cpp /^namespace doris {$/;" n file: +doris src/service/brpc_service.h /^namespace doris {$/;" n +doris src/service/doris_main.cpp /^namespace doris {$/;" n file: +doris src/service/http_service.cpp /^namespace doris {$/;" n file: +doris src/service/http_service.h /^namespace doris {$/;" n +doris src/service/internal_service.cpp /^namespace doris {$/;" n file: +doris src/service/internal_service.h /^namespace doris {$/;" n +doris src/testutil/desc_tbl_builder.cc /^namespace doris {$/;" n file: +doris src/testutil/desc_tbl_builder.h /^namespace doris {$/;" n +doris src/testutil/function_utils.cpp /^namespace doris {$/;" n file: +doris src/testutil/function_utils.h /^namespace doris {$/;" n +doris src/udf/udf.cpp /^namespace doris {$/;" n file: +doris src/udf/udf.h /^namespace doris {$/;" n +doris src/udf/udf_internal.h /^namespace doris {$/;" n +doris src/util/aes_util.cpp /^namespace doris {$/;" n file: +doris src/util/aes_util.h /^namespace doris {$/;" n +doris src/util/aligned_new.h /^namespace doris {$/;" n +doris src/util/arrow/row_batch.cpp /^namespace doris {$/;" n file: +doris src/util/arrow/row_batch.h /^namespace doris {$/;" n +doris src/util/arrow/row_block.cpp /^namespace doris {$/;" n file: +doris src/util/arrow/row_block.h /^namespace doris {$/;" n +doris src/util/arrow/utils.cpp /^namespace doris {$/;" n file: +doris src/util/arrow/utils.h /^namespace doris {$/;" n +doris src/util/barrier.h /^namespace doris {$/;" n +doris src/util/batch_process_thread_pool.hpp /^namespace doris {$/;" n +doris src/util/bfd_parser.cpp /^namespace doris {$/;" n file: +doris src/util/bfd_parser.h /^namespace doris {$/;" n +doris src/util/bit_stream_utils.h /^namespace doris {$/;" n +doris src/util/bit_stream_utils.inline.h /^namespace doris {$/;" n +doris src/util/bit_util.h /^namespace doris {$/;" n +doris src/util/bitmap.cpp /^namespace doris {$/;" n file: +doris src/util/bitmap.h /^namespace doris {$/;" n +doris src/util/bitmap_value.h /^namespace doris {$/;" n +doris src/util/block_compression.cpp /^namespace doris {$/;" n file: +doris src/util/block_compression.h /^namespace doris {$/;" n +doris src/util/blocking_priority_queue.hpp /^namespace doris {$/;" n +doris src/util/blocking_queue.hpp /^namespace doris {$/;" n +doris src/util/broker_load_error_hub.cpp /^namespace doris {$/;" n file: +doris src/util/broker_load_error_hub.h /^namespace doris {$/;" n +doris src/util/brpc_stub_cache.h /^namespace doris {$/;" n +doris src/util/buffer_builder.hpp /^namespace doris {$/;" n +doris src/util/byte_buffer.h /^namespace doris {$/;" n +doris src/util/cgroup_util.cpp /^namespace doris {$/;" n file: +doris src/util/cgroup_util.h /^namespace doris {$/;" n +doris src/util/cidr.cpp /^namespace doris {$/;" n file: +doris src/util/cidr.h /^namespace doris {$/;" n +doris src/util/coding.cpp /^namespace doris {$/;" n file: +doris src/util/coding.h /^namespace doris {$/;" n +doris src/util/condition_variable.cpp /^namespace doris {$/;" n file: +doris src/util/condition_variable.h /^namespace doris {$/;" n +doris src/util/container_util.hpp /^namespace doris {$/;" n +doris src/util/core_local.cpp /^namespace doris {$/;" n file: +doris src/util/core_local.h /^namespace doris {$/;" n +doris src/util/countdown_latch.h /^namespace doris {$/;" n +doris src/util/counter_cond_variable.hpp /^namespace doris {$/;" n +doris src/util/cpu_info.cpp /^namespace doris {$/;" n file: +doris src/util/cpu_info.h /^namespace doris {$/;" n +doris src/util/crc32c.cpp /^namespace doris {$/;" n file: +doris src/util/crc32c.h /^namespace doris {$/;" n +doris src/util/date_func.cpp /^namespace doris {$/;" n file: +doris src/util/date_func.h /^namespace doris {$/;" n +doris src/util/debug_counters.h /^namespace doris {$/;" n +doris src/util/debug_util.cpp /^namespace doris {$/;" n file: +doris src/util/debug_util.h /^namespace doris {$/;" n +doris src/util/defer_op.h /^namespace doris {$/;" n +doris src/util/disk_info.cpp /^namespace doris {$/;" n file: +doris src/util/disk_info.h /^namespace doris {$/;" n +doris src/util/doris_metrics.cpp /^namespace doris {$/;" n file: +doris src/util/doris_metrics.h /^namespace doris {$/;" n +doris src/util/dummy_runtime_profile.h /^namespace doris {$/;" n +doris src/util/dynamic_util.cpp /^namespace doris {$/;" n file: +doris src/util/dynamic_util.h /^namespace doris {$/;" n +doris src/util/errno.cpp /^namespace doris {$/;" n file: +doris src/util/errno.h /^namespace doris {$/;" n +doris src/util/error_util.cc /^namespace doris {$/;" n file: +doris src/util/error_util.h /^namespace doris {$/;" n +doris src/util/fake_lock.h /^namespace doris {$/;" n +doris src/util/faststring.cc /^namespace doris {$/;" n file: +doris src/util/faststring.h /^namespace doris {$/;" n +doris src/util/file_cache.cpp /^namespace doris {$/;" n file: +doris src/util/file_cache.h /^namespace doris {$/;" n +doris src/util/file_manager.cpp /^namespace doris {$/;" n file: +doris src/util/file_manager.h /^namespace doris {$/;" n +doris src/util/file_utils.cpp /^namespace doris {$/;" n file: +doris src/util/file_utils.h /^namespace doris {$/;" n +doris src/util/filesystem_util.cc /^namespace doris {$/;" n file: +doris src/util/filesystem_util.h /^namespace doris {$/;" n +doris src/util/frame_of_reference_coding.cpp /^namespace doris {$/;" n file: +doris src/util/frame_of_reference_coding.h /^namespace doris {$/;" n +doris src/util/hash_util.hpp /^namespace doris {$/;" n +doris src/util/hash_util_ir.cpp /^namespace doris {$/;" n file: +doris src/util/internal_queue.h /^namespace doris {$/;" n +doris src/util/json_util.cpp /^namespace doris {$/;" n file: +doris src/util/json_util.h /^namespace doris {$/;" n +doris src/util/load_error_hub.cpp /^namespace doris {$/;" n file: +doris src/util/load_error_hub.h /^namespace doris {$/;" n +doris src/util/logging.h /^namespace doris {$/;" n +doris src/util/lru_cache.hpp /^namespace doris {$/;" n +doris src/util/md5.cpp /^namespace doris {$/;" n file: +doris src/util/md5.h /^namespace doris {$/;" n +doris src/util/mem_info.cpp /^namespace doris {$/;" n file: +doris src/util/mem_info.h /^namespace doris {$/;" n +doris src/util/mem_range.h /^namespace doris {$/;" n +doris src/util/mem_util.hpp /^namespace doris {$/;" n +doris src/util/metrics.cpp /^namespace doris {$/;" n file: +doris src/util/metrics.h /^namespace doris {$/;" n +doris src/util/monotime.cpp /^namespace doris {$/;" n file: +doris src/util/monotime.h /^namespace doris {$/;" n +doris src/util/mutex.cpp /^namespace doris {$/;" n file: +doris src/util/mutex.h /^namespace doris {$/;" n +doris src/util/mysql_global.h /^namespace doris {$/;" n +doris src/util/mysql_load_error_hub.cpp /^namespace doris {$/;" n file: +doris src/util/mysql_load_error_hub.h /^namespace doris {$/;" n +doris src/util/mysql_row_buffer.cpp /^namespace doris {$/;" n file: +doris src/util/mysql_row_buffer.h /^namespace doris {$/;" n +doris src/util/network_util.cpp /^namespace doris {$/;" n file: +doris src/util/network_util.h /^namespace doris {$/;" n +doris src/util/null_load_error_hub.cpp /^namespace doris {$/;" n file: +doris src/util/null_load_error_hub.h /^namespace doris {$/;" n +doris src/util/once.h /^namespace doris {$/;" n +doris src/util/os_info.cpp /^namespace doris {$/;" n file: +doris src/util/os_info.h /^namespace doris {$/;" n +doris src/util/parse_util.cpp /^namespace doris {$/;" n file: +doris src/util/parse_util.h /^namespace doris {$/;" n +doris src/util/path_builder.cpp /^namespace doris {$/;" n file: +doris src/util/path_builder.h /^namespace doris {$/;" n +doris src/util/path_trie.hpp /^namespace doris {$/;" n +doris src/util/path_util.cpp /^namespace doris {$/;" n file: +doris src/util/path_util.h /^namespace doris {$/;" n +doris src/util/perf_counters.cpp /^namespace doris {$/;" n file: +doris src/util/perf_counters.h /^namespace doris {$/;" n +doris src/util/pretty_printer.h /^namespace doris {$/;" n +doris src/util/priority_thread_pool.hpp /^namespace doris {$/;" n +doris src/util/progress_updater.cpp /^namespace doris {$/;" n file: +doris src/util/progress_updater.h /^namespace doris {$/;" n +doris src/util/radix_sort.h /^namespace doris {$/;" n +doris src/util/random.h /^namespace doris {$/;" n +doris src/util/ref_count_closure.h /^namespace doris {$/;" n +doris src/util/rle_encoding.h /^namespace doris {$/;" n +doris src/util/runtime_profile.cpp /^namespace doris {$/;" n file: +doris src/util/runtime_profile.h /^namespace doris {$/;" n +doris src/util/scoped_cleanup.h /^namespace doris {$/;" n +doris src/util/slice.cpp /^namespace doris {$/;" n file: +doris src/util/slice.h /^namespace doris {$/;" n +doris src/util/spinlock.cc /^namespace doris {$/;" n file: +doris src/util/spinlock.h /^namespace doris {$/;" n +doris src/util/sse_util.hpp /^namespace doris {$/;" n +doris src/util/stack_util.cpp /^namespace doris {$/;" n file: +doris src/util/stack_util.h /^namespace doris {$/;" n +doris src/util/stat_util.hpp /^namespace doris {$/;" n +doris src/util/static_asserts.cpp /^namespace doris {$/;" n file: +doris src/util/stopwatch.hpp /^namespace doris {$/;" n +doris src/util/streaming_sampler.h /^namespace doris {$/;" n +doris src/util/string_parser.cpp /^namespace doris {$/;" n file: +doris src/util/string_parser.hpp /^namespace doris {$/;" n +doris src/util/string_util.cpp /^namespace doris {$/;" n file: +doris src/util/string_util.h /^namespace doris {$/;" n +doris src/util/symbols_util.cpp /^namespace doris {$/;" n file: +doris src/util/symbols_util.h /^namespace doris {$/;" n +doris src/util/system_metrics.cpp /^namespace doris {$/;" n file: +doris src/util/system_metrics.h /^namespace doris {$/;" n +doris src/util/tdigest.h /^namespace doris {$/;" n +doris src/util/thread.cpp /^namespace doris {$/;" n file: +doris src/util/thread.h /^namespace doris {$/;" n +doris src/util/threadpool.cpp /^namespace doris {$/;" n file: +doris src/util/threadpool.h /^namespace doris {$/;" n +doris src/util/thrift_client.cpp /^namespace doris {$/;" n file: +doris src/util/thrift_client.h /^namespace doris {$/;" n +doris src/util/thrift_rpc_helper.cpp /^namespace doris {$/;" n file: +doris src/util/thrift_rpc_helper.h /^namespace doris {$/;" n +doris src/util/thrift_server.cpp /^namespace doris {$/;" n file: +doris src/util/thrift_server.h /^namespace doris {$/;" n +doris src/util/thrift_util.cpp /^namespace doris {$/;" n file: +doris src/util/thrift_util.h /^namespace doris {$/;" n +doris src/util/time.h /^namespace doris {$/;" n +doris src/util/tuple_row_compare.cpp /^namespace doris {$/;" n file: +doris src/util/tuple_row_compare.h /^namespace doris {$/;" n +doris src/util/types.h /^namespace doris {$/;" n +doris src/util/uid_util.cpp /^namespace doris {$/;" n file: +doris src/util/uid_util.h /^namespace doris {$/;" n +doris src/util/url_coding.cpp /^namespace doris {$/;" n file: +doris src/util/url_coding.h /^namespace doris {$/;" n +doris src/util/url_parser.cpp /^namespace doris {$/;" n file: +doris src/util/url_parser.h /^namespace doris {$/;" n +doris src/util/utf8_check.cpp /^namespace doris {$/;" n file: +doris src/util/utf8_check.h /^namespace doris {$/;" n +doris src/util/uuid_generator.h /^namespace doris {$/;" n +doris src/util/zip_util.cpp /^namespace doris {$/;" n file: +doris src/util/zip_util.h /^namespace doris {$/;" n +doris test/agent/agent_server_test.cpp /^namespace doris {$/;" n file: +doris test/agent/cgroups_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/agent/heartbeat_server_test.cpp /^namespace doris {$/;" n file: +doris test/agent/mock_pusher.h /^namespace doris {$/;" n +doris test/agent/mock_task_worker_pool.h /^namespace doris {$/;" n +doris test/agent/mock_utils.h /^namespace doris {$/;" n +doris test/agent/utils_test.cpp /^namespace doris {$/;" n file: +doris test/common/resource_tls_test.cpp /^namespace doris {$/;" n file: +doris test/common/status_test.cpp /^namespace doris {$/;" n file: +doris test/env/env_posix_test.cpp /^namespace doris {$/;" n file: +doris test/exec/broker_reader_test.cpp /^namespace doris {$/;" n file: +doris test/exec/broker_scan_node_test.cpp /^namespace doris {$/;" n file: +doris test/exec/broker_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/csv_scan_bench_test.cpp /^namespace doris {$/;" n file: +doris test/exec/csv_scan_node_test.cpp /^namespace doris {$/;" n file: +doris test/exec/csv_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/es_http_scan_node_test.cpp /^namespace doris {$/;" n file: +doris test/exec/es_predicate_test.cpp /^namespace doris {$/;" n file: +doris test/exec/es_query_builder_test.cpp /^namespace doris {$/;" n file: +doris test/exec/es_scan_node_test.cpp /^namespace doris {$/;" n file: +doris test/exec/es_scan_reader_test.cpp /^namespace doris {$/;" n file: +doris test/exec/hash_table_test.cpp /^namespace doris {$/;" n file: +doris test/exec/mysql_scan_node_test.cpp /^namespace doris {$/;" n file: +doris test/exec/mysql_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/new_olap_scan_node_test.cpp /^namespace doris {$/;" n file: +doris test/exec/olap_common_test.cpp /^namespace doris {$/;" n file: +doris test/exec/olap_scan_node_test.cpp /^namespace doris {$/;" n file: +doris test/exec/olap_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/orc_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/parquet_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/partitioned_hash_table_test.cpp /^namespace doris {$/;" n file: +doris test/exec/plain_text_line_reader_bzip_test.cpp /^namespace doris {$/;" n file: +doris test/exec/plain_text_line_reader_gzip_test.cpp /^namespace doris {$/;" n file: +doris test/exec/plain_text_line_reader_lz4frame_test.cpp /^namespace doris {$/;" n file: +doris test/exec/plain_text_line_reader_lzop_test.cpp /^namespace doris {$/;" n file: +doris test/exec/plain_text_line_reader_uncompressed_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scan_node_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_authors_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_collations_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_columns_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_engines_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_tables_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner/schema_variables_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/schema_scanner_test.cpp /^namespace doris {$/;" n file: +doris test/exec/set_executor_test.cpp /^namespace doris {$/;" n file: +doris test/exec/tablet_info_test.cpp /^namespace doris {$/;" n file: +doris test/exec/tablet_sink_test.cpp /^namespace doris {$/;" n file: +doris test/exprs/binary_predicate_test.cpp /^namespace doris {$/;" n file: +doris test/exprs/bitmap_function_test.cpp /^namespace doris {$/;" n file: +doris test/exprs/hll_function_test.cpp /^namespace doris {$/;" n file: +doris test/exprs/hybird_set_test.cpp /^namespace doris {$/;" n file: +doris test/exprs/in_op_test.cpp /^namespace doris {$/;" n file: +doris test/exprs/in_predicate_test.cpp /^namespace doris {$/;" n file: +doris test/exprs/json_function_test.cpp /^namespace doris {$/;" n file: +doris test/exprs/percentile_approx_test.cpp /^namespace doris {$/;" n file: +doris test/exprs/string_functions_test.cpp /^namespace doris {$/;" n file: +doris test/exprs/timestamp_functions_test.cpp /^namespace doris {$/;" n file: +doris test/geo/geo_functions_test.cpp /^namespace doris {$/;" n file: +doris test/geo/geo_types_test.cpp /^namespace doris {$/;" n file: +doris test/geo/wkt_parse_test.cpp /^namespace doris {$/;" n file: +doris test/http/http_client_test.cpp /^namespace doris {$/;" n file: +doris test/http/http_utils_test.cpp /^namespace doris {$/;" n file: +doris test/http/message_body_sink_test.cpp /^namespace doris {$/;" n file: +doris test/http/metrics_action_test.cpp /^namespace doris {$/;" n file: +doris test/http/stream_load_test.cpp /^namespace doris {$/;" n file: +doris test/olap/aggregate_func_test.cpp /^namespace doris {$/;" n file: +doris test/olap/bit_field_test.cpp /^namespace doris {$/;" n file: +doris test/olap/bloom_filter_index_test.cpp /^namespace doris {$/;" n file: +doris test/olap/bloom_filter_test.cpp /^namespace doris {$/;" n file: +doris test/olap/byte_buffer_test.cpp /^namespace doris {$/;" n file: +doris test/olap/column_reader_test.cpp /^namespace doris {$/;" n file: +doris test/olap/comparison_predicate_test.cpp /^namespace doris {$/;" n file: +doris test/olap/decimal12_test.cpp /^namespace doris {$/;" n file: +doris test/olap/delete_handler_test.cpp /^namespace doris {$/;" n file: +doris test/olap/delta_writer_test.cpp /^namespace doris {$/;" n file: +doris test/olap/file_helper_test.cpp /^namespace doris {$/;" n file: +doris test/olap/file_utils_test.cpp /^namespace doris {$/;" n file: +doris test/olap/fs/file_block_manager_test.cpp /^namespace doris {$/;" n file: +doris test/olap/generic_iterators_test.cpp /^namespace doris {$/;" n file: +doris test/olap/hll_test.cpp /^namespace doris {$/;" n file: +doris test/olap/in_list_predicate_test.cpp /^namespace doris {$/;" n file: +doris test/olap/key_coder_test.cpp /^namespace doris {$/;" n file: +doris test/olap/lru_cache_test.cpp /^namespace doris {$/;" n file: +doris test/olap/memtable_flush_executor_test.cpp /^namespace doris {$/;" n file: +doris test/olap/mock_command_executor.h /^namespace doris {$/;" n +doris test/olap/null_predicate_test.cpp /^namespace doris {$/;" n file: +doris test/olap/olap_meta_test.cpp /^namespace doris {$/;" n file: +doris test/olap/olap_reader_test.cpp /^namespace doris {$/;" n file: +doris test/olap/olap_snapshot_converter_test.cpp /^namespace doris {$/;" n file: +doris test/olap/options_test.cpp /^namespace doris {$/;" n file: +doris test/olap/page_cache_test.cpp /^namespace doris {$/;" n file: +doris test/olap/row_block_test.cpp /^namespace doris {$/;" n file: +doris test/olap/row_block_v2_test.cpp /^namespace doris {$/;" n file: +doris test/olap/row_cursor_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/alpha_rowset_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/beta_rowset_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/rowset_converter_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/rowset_meta_manager_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/rowset_meta_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/binary_dict_page_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/binary_plain_page_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/binary_prefix_page_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/bitmap_index_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/block_bloom_filter_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/bloom_filter_page_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/encoding_info_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/ordinal_page_index_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/plain_page_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/rle_page_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/row_ranges_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/segment_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/segment_v2/zone_map_index_test.cpp /^namespace doris {$/;" n file: +doris test/olap/rowset/unique_rowset_id_generator_test.cpp /^namespace doris {$/;" n file: +doris test/olap/run_length_byte_test.cpp /^namespace doris {$/;" n file: +doris test/olap/run_length_integer_test.cpp /^namespace doris {$/;" n file: +doris test/olap/schema_change_test.cpp /^namespace doris {$/;" n file: +doris test/olap/selection_vector_test.cpp /^namespace doris {$/;" n file: +doris test/olap/serialize_test.cpp /^namespace doris {$/;" n file: +doris test/olap/short_key_index_test.cpp /^namespace doris {$/;" n file: +doris test/olap/skiplist_test.cpp /^namespace doris {$/;" n file: +doris test/olap/storage_types_test.cpp /^namespace doris {$/;" n file: +doris test/olap/stream_index_test.cpp /^namespace doris {$/;" n file: +doris test/olap/tablet_meta_manager_test.cpp /^namespace doris {$/;" n file: +doris test/olap/tablet_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/olap/tablet_schema_helper.h /^namespace doris {$/;" n +doris test/olap/txn_manager_test.cpp /^namespace doris {$/;" n file: +doris test/olap/vectorized_olap_reader_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/buffer_control_block_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/buffered_block_mgr2_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/buffered_tuple_stream2_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/buffered_tuple_stream_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/data_spliter_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/data_stream_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/datetime_value_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/decimal_value_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/decimalv2_value_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/disk_io_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/dpp_sink_internal_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/dpp_sink_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/dpp_writer_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/etl_job_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/export_task_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/external_scan_context_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/fragment_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/free_list_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/heartbeat_flags_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/kafka_consumer_pipe_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/large_int_value_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/load_channel_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/mem_limit_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/mem_pool_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/memory/chunk_allocator_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/memory/system_allocator_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/memory_scratch_sink_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/mysql_table_writer_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/qsorter_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/raw_value_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/result_buffer_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/result_queue_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/result_sink_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/result_writer_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/routine_load_task_executor_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/small_file_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/snapshot_loader_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/sorter_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/stream_load_pipe_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/string_buffer_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/string_value_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/thread_resource_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/tmp_file_mgr_test.cpp /^namespace doris {$/;" n file: +doris test/runtime/user_function_cache_test.cpp /^namespace doris {$/;" n file: +doris test/util/aes_util_test.cpp /^namespace doris {$/;" n file: +doris test/util/arrow/arrow_row_batch_test.cpp /^namespace doris {$/;" n file: +doris test/util/arrow/arrow_row_block_test.cpp /^namespace doris {$/;" n file: +doris test/util/arrow/arrow_work_flow_test.cpp /^namespace doris {$/;" n file: +doris test/util/bit_stream_utils_test.cpp /^namespace doris {$/;" n file: +doris test/util/bit_util_test.cpp /^namespace doris {$/;" n file: +doris test/util/bitmap_test.cpp /^namespace doris {$/;" n file: +doris test/util/bitmap_value_test.cpp /^namespace doris {$/;" n file: +doris test/util/block_compression_test.cpp /^namespace doris {$/;" n file: +doris test/util/blocking_queue_test.cpp /^namespace doris {$/;" n file: +doris test/util/brpc_stub_cache_test.cpp /^namespace doris {$/;" n file: +doris test/util/byte_buffer_test2.cpp /^namespace doris {$/;" n file: +doris test/util/cgroup_util_test.cpp /^namespace doris {$/;" n file: +doris test/util/cidr_test.cpp /^namespace doris {$/;" n file: +doris test/util/coding_test.cpp /^namespace doris {$/;" n file: +doris test/util/core_local_test.cpp /^namespace doris {$/;" n file: +doris test/util/countdown_latch_test.cpp /^namespace doris {$/;" n file: +doris test/util/counter_cond_variable_test.cpp /^namespace doris {$/;" n file: +doris test/util/crc32c_test.cpp /^namespace doris {$/;" n file: +doris test/util/decompress_test.cpp /^namespace doris {$/;" n file: +doris test/util/doris_metrics_test.cpp /^namespace doris {$/;" n file: +doris test/util/faststring_test.cpp /^namespace doris {$/;" n file: +doris test/util/file_cache_test.cpp /^namespace doris {$/;" n file: +doris test/util/file_manager_test.cpp /^namespace doris {$/;" n file: +doris test/util/filesystem_util_test.cpp /^namespace doris {$/;" n file: +doris test/util/frame_of_reference_coding_test.cpp /^namespace doris {$/;" n file: +doris test/util/internal_queue_test.cpp /^namespace doris {$/;" n file: +doris test/util/json_util_test.cpp /^namespace doris {$/;" n file: +doris test/util/lru_cache_util_test.cpp /^namespace doris {$/;" n file: +doris test/util/md5_test.cpp /^namespace doris {$/;" n file: +doris test/util/monotime_test.cpp /^namespace doris {$/;" n file: +doris test/util/new_metrics_test.cpp /^namespace doris {$/;" n file: +doris test/util/parse_util_test.cpp /^namespace doris {$/;" n file: +doris test/util/path_trie_test.cpp /^namespace doris {$/;" n file: +doris test/util/path_util_test.cpp /^namespace doris {$/;" n file: +doris test/util/radix_sort_test.cpp /^namespace doris {$/;" n file: +doris test/util/rle_encoding_test.cpp /^namespace doris {$/;" n file: +doris test/util/scoped_cleanup_test.cpp /^namespace doris {$/;" n file: +doris test/util/string_parser_test.cpp /^namespace doris {$/;" n file: +doris test/util/string_util_test.cpp /^namespace doris {$/;" n file: +doris test/util/system_metrics_test.cpp /^namespace doris {$/;" n file: +doris test/util/tdigest_test.cpp /^namespace doris {$/;" n file: +doris test/util/thread_pool_test.cpp /^namespace doris {$/;" n file: +doris test/util/thread_test.cpp /^namespace doris {$/;" n file: +doris test/util/threadpool_test.cpp /^namespace doris {$/;" n file: +doris test/util/types_test.cpp /^namespace doris {$/;" n file: +doris test/util/uid_util_test.cpp /^namespace doris {$/;" n file: +doris test/util/url_coding_test.cpp /^namespace doris {$/;" n file: +doris test/util/utf8_check_test.cpp /^namespace doris {$/;" n file: +doris test/util/zip_util_test.cpp /^namespace doris {$/;" n file: +doris_udf output/udf/include/uda_test_harness.h /^namespace doris_udf {$/;" n +doris_udf output/udf/include/udf.h /^namespace doris_udf {$/;" n +doris_udf src/exprs/hash_functions.h /^namespace doris_udf {$/;" n +doris_udf src/testutil/function_utils.h /^namespace doris_udf {$/;" n +doris_udf src/udf/uda_test_harness.h /^namespace doris_udf {$/;" n +doris_udf src/udf/udf.cpp /^namespace doris_udf {$/;" n file: +doris_udf src/udf/udf.h /^namespace doris_udf {$/;" n +doris_udf src/udf/udf_debug.h /^namespace doris_udf {$/;" n +doris_udf src/udf/udf_ir.cpp /^namespace doris_udf {$/;" n file: +doris_udf src/udf_samples/uda_sample.cpp /^namespace doris_udf {$/;" n file: +doris_udf src/udf_samples/uda_sample.h /^namespace doris_udf {$/;" n +doris_udf src/udf_samples/udf_sample.cpp /^namespace doris_udf {$/;" n file: +doris_udf src/udf_samples/udf_sample.h /^namespace doris_udf {$/;" n +doris_udf test/udf/uda_test.cpp /^namespace doris_udf {$/;" n file: +doris_udf test/udf/udf_test.cpp /^namespace doris_udf {$/;" n file: +dosDate src/util/minizip/unzip.h /^ uLong dosDate; \/* last mod file date in Dos fmt 4 bytes *\/$/;" m struct:unz_file_info64_s +dosDate src/util/minizip/unzip.h /^ uLong dosDate; \/* last mod file date in Dos fmt 4 bytes *\/$/;" m struct:unz_file_info_s +double_val src/exprs/expr_value.h /^ double double_val;$/;" m struct:doris::ExprValue +double_val test/runtime/dpp_writer_test.cpp /^ double double_val;$/;" m struct:doris::TestDataTuple file: +double_val test/runtime/mysql_table_writer_test.cpp /^ double double_val;$/;" m struct:doris::TestDataTuple file: +double_value src/util/runtime_profile.h /^ virtual double double_value() const {$/;" f class:doris::RuntimeProfile::Counter +down_cast src/gutil/casts.h /^inline To down_cast(From& f) {$/;" f +down_cast src/gutil/casts.h /^inline To down_cast(From* f) { \/\/ so we only accept pointers$/;" f +download src/http/http_client.cpp /^Status HttpClient::download(const std::string& local_path) {$/;" f class:doris::HttpClient +download src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::download($/;" f class:doris::SnapshotLoader +drain_and_shutdown src/util/batch_process_thread_pool.hpp /^ void drain_and_shutdown() {$/;" f class:doris::BatchProcessThreadPool +drain_and_shutdown src/util/priority_thread_pool.hpp /^ void drain_and_shutdown() {$/;" f class:doris::PriorityThreadPool +drop_cgroups src/agent/cgroups_mgr.cpp /^AgentStatus CgroupsMgr::drop_cgroups(const string& deleted_cgroups_path) {$/;" f class:doris::CgroupsMgr +drop_tablet src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::drop_tablet($/;" f class:doris::TabletManager +drop_tablet_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::drop_tablet_requests_total;$/;" m class:doris::DorisMetrics file: +drop_tablet_requests_total src/util/doris_metrics.h /^ static IntCounter drop_tablet_requests_total;$/;" m class:doris::DorisMetrics +drop_tablets_on_error_root_path src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::drop_tablets_on_error_root_path($/;" f class:doris::TabletManager +dummy src/exprs/expr_ir.cpp /^void dummy(doris_udf::FunctionContext*, doris_udf::BooleanVal*, doris_udf::TinyIntVal*,$/;" f +dummy src/gutil/template_util.h /^ char dummy[2];$/;" m struct:base::big_ +dummy_deleter src/runtime/load_channel_mgr.cpp /^static void dummy_deleter(const CacheKey& key, void* value) {$/;" f namespace:doris +dummy_profile_ src/runtime/bufferpool/reservation_tracker.h /^ boost::scoped_ptr dummy_profile_;$/;" m class:doris::ReservationTracker +dump_to_row_block src/runtime/vectorized_row_batch.cpp /^void VectorizedRowBatch::dump_to_row_block(RowBlock* row_block) {$/;" f class:doris::VectorizedRowBatch +duplicates src/exec/new_partitioned_hash_table.h /^ DuplicateNode* duplicates;$/;" m union:doris::NewPartitionedHashTable::Bucket::__anon41 +duplicates src/exec/partitioned_hash_table.h /^ DuplicateNode* duplicates;$/;" m union:doris::PartitionedHashTable::Bucket::__anon42 +dynamic_close src/util/dynamic_util.cpp /^void dynamic_close(void* handle) {$/;" f namespace:doris +dynamic_lookup src/util/dynamic_util.cpp /^Status dynamic_lookup(void* handle, const char* symbol, void** fn_ptr) {$/;" f namespace:doris +dynamic_open src/util/dynamic_util.cpp /^Status dynamic_open(const char* library, void** handle) {$/;" f namespace:doris +e src/exprs/math_functions.cpp /^DoubleVal MathFunctions::e(FunctionContext* ctx) {$/;" f class:doris::MathFunctions +edges src/olap/olap_common.h /^ std::list* edges;$/;" m struct:doris::Vertex +effective_cluster_id src/olap/storage_engine.h /^ int32_t effective_cluster_id() const {$/;" f class:doris::StorageEngine +elapsed_ms src/util/runtime_profile.h /^ int64_t elapsed_ms;$/;" m struct:doris::RuntimeProfile::RateCounterInfo +elapsed_time src/util/runtime_profile.h /^ int64_t elapsed_time() {$/;" f class:doris::RuntimeProfile::EventSequence +elapsed_time src/util/stopwatch.hpp /^ uint64_t elapsed_time() const {$/;" f class:doris::MonotonicStopWatch +element_type src/gutil/gscoped_ptr.h /^ typedef C element_type;$/;" t class:gscoped_array +element_type src/gutil/gscoped_ptr.h /^ typedef C element_type;$/;" t class:gscoped_ptr_malloc +element_type src/gutil/gscoped_ptr.h /^ typedef T element_type;$/;" t class:gscoped_ptr +element_type src/gutil/ref_counted.h /^ typedef T element_type;$/;" t class:scoped_refptr +emplaceOrInsert src/util/bitmap_value.h /^ void emplaceOrInsert(const uint32_t key, const Roaring& value) {$/;" f class:doris::detail::Roaring64Map +empty src/gutil/strings/stringpiece.h /^ bool empty() const { return length_ == 0; }$/;" f class:StringPiece +empty src/olap/delete_handler.h /^ bool empty() const {$/;" f class:doris::DeleteHandler +empty src/olap/hll.h /^ static std::string empty() {$/;" f class:doris::HyperLogLog +empty src/olap/lru_cache.h /^ bool empty() const {$/;" f class:doris::CacheKey +empty src/olap/rowset/column_data.h /^ bool empty() const { return _segment_group->empty(); }$/;" f class:doris::ColumnData +empty src/olap/rowset/rowset.h /^ bool empty() const { return rowset_meta()->empty(); }$/;" f class:doris::Rowset +empty src/olap/rowset/rowset_meta.h /^ bool empty() const {$/;" f class:doris::RowsetMeta +empty src/olap/rowset/segment_group.h /^ bool empty() const {$/;" f class:doris::SegmentGroup +empty src/runtime/bufferpool/buffer_pool_internal.h /^ bool empty() const { return list_.empty(); }$/;" f class:doris::BufferPool::PageList +empty src/runtime/string_buffer.hpp /^ bool empty() const {$/;" f class:doris::StringBuffer +empty src/util/internal_queue.h /^ bool empty() const { return head_ == NULL; }$/;" f class:doris::InternalQueueBase +empty src/util/metrics.h /^ bool empty() const {$/;" f class:doris::MetricCollector +empty src/util/metrics.h /^ bool empty() const {$/;" f struct:doris::MetricLabels +empty src/util/slice.h /^ bool empty() const { return size == 0; }$/;" f struct:doris::Slice +empty_buckets src/exec/partitioned_hash_table.h /^ int64_t empty_buckets() const { return _num_buckets - _num_filled_buckets; }$/;" f class:doris::PartitionedHashTable +empty_function src/runtime/fragment_mgr.cpp /^static void empty_function(PlanFragmentExecutor* exec) {$/;" f namespace:doris +enable src/util/cgroup_util.cpp /^bool CGroupUtil::enable() {$/;" f class:doris::CGroupUtil +enable_feature src/util/cpu_info.cpp /^void CpuInfo::enable_feature(long flag, bool enable) {$/;" f class:doris::CpuInfo +enable_if src/gutil/type_traits.h /^template struct enable_if { typedef T type; };$/;" s namespace:base +enable_if src/gutil/type_traits.h /^template struct enable_if {};$/;" s namespace:base +enable_metric src/olap/fs/block_manager.h /^ bool enable_metric;$/;" m struct:doris::fs::BlockManagerOptions +enable_reservation_reporting src/runtime/mem_tracker.cpp /^void MemTracker::enable_reservation_reporting(const ReservationTrackerCounters& counters) {$/;" f class:doris::MemTracker +encode src/geo/geo_types.cpp /^void GeoCircle::encode(std::string* buf) {$/;" f class:doris::GeoCircle +encode src/geo/geo_types.cpp /^void GeoLine::encode(std::string* buf) {$/;" f class:doris::GeoLine +encode src/geo/geo_types.cpp /^void GeoPoint::encode(std::string* buf) {$/;" f class:doris::GeoPoint +encode src/geo/geo_types.cpp /^void GeoPolygon::encode(std::string* buf) {$/;" f class:doris::GeoPolygon +encode src/olap/page_cache.h /^ std::string encode() const {$/;" f struct:doris::StoragePageCache::CacheKey +encode_ascending src/olap/field.h /^ void encode_ascending(const void* value, std::string* buf) const {$/;" f class:doris::Field +encode_ascending src/olap/key_coder.h /^ static void encode_ascending(const void* value, size_t index_size, std::string* buf) {$/;" f class:doris::KeyCoderTraits +encode_ascending src/olap/key_coder.h /^ void encode_ascending(const void* value, size_t index_size, std::string* buf) const {$/;" f class:doris::KeyCoder +encode_base64_internal src/util/url_coding.cpp /^static void encode_base64_internal(const std::string& in, std::string* out,$/;" f namespace:doris +encode_basic_auth src/http/utils.cpp /^std::string encode_basic_auth(const std::string& user, const std::string& passwd) {$/;" f namespace:doris +encode_bit_width src/olap/serialize.cpp /^uint32_t encode_bit_width(uint32_t n) {$/;" f namespace:doris::ser +encode_fixed128_le src/util/coding.h /^inline void encode_fixed128_le(uint8_t* buf, uint128_t val) {$/;" f namespace:doris +encode_fixed16_le src/util/coding.h /^inline void encode_fixed16_le(uint8_t* buf, uint16_t val) {$/;" f namespace:doris +encode_fixed32_le src/util/coding.h /^inline void encode_fixed32_le(uint8_t* buf, uint32_t val) {$/;" f namespace:doris +encode_fixed64_le src/util/coding.h /^inline void encode_fixed64_le(uint8_t* buf, uint64_t val) {$/;" f namespace:doris +encode_fixed8 src/util/coding.h /^inline void encode_fixed8(uint8_t* buf, uint8_t val) {$/;" f namespace:doris +encode_key src/olap/short_key_index.h /^void encode_key(std::string* buf, const RowType& row, size_t num_keys) {$/;" f namespace:doris +encode_key_with_padding src/olap/short_key_index.h /^void encode_key_with_padding(std::string* buf, const RowType& row,$/;" f namespace:doris +encode_to src/geo/geo_types.cpp /^void GeoShape::encode_to(std::string* buf) {$/;" f class:doris::GeoShape +encode_to src/olap/rowset/segment_v2/page_pointer.h /^ void encode_to(faststring* dst) const {$/;" f struct:doris::segment_v2::PagePointer +encode_to src/olap/rowset/segment_v2/page_pointer.h /^ void encode_to(std::string* dst) const {$/;" f struct:doris::segment_v2::PagePointer +encode_varint32 src/util/coding.cpp /^uint8_t* encode_varint32(uint8_t* dst, uint32_t v) {$/;" f namespace:doris +encode_varint64 src/util/coding.h /^inline uint8_t* encode_varint64(uint8_t* dst, uint64_t v) {$/;" f namespace:doris +encoded_buf src/geo/geo_functions.cpp /^ std::string encoded_buf;$/;" m struct:doris::StConstructState file: +encoding src/olap/rowset/segment_v2/encoding_info.cpp /^ static const EncodingTypePB encoding = encoding_type;$/;" m struct:doris::segment_v2::EncodingTraits file: +encoding src/olap/rowset/segment_v2/encoding_info.h /^ EncodingTypePB encoding() const { return _encoding; }$/;" f class:doris::segment_v2::EncodingInfo +encoding src/olap/rowset/segment_v2/indexed_column_writer.h /^ EncodingTypePB encoding = DEFAULT_ENCODING;$/;" m struct:doris::segment_v2::IndexedColumnWriterOptions +encoding_info src/olap/rowset/segment_v2/column_reader.h /^ const EncodingInfo* encoding_info() const { return _encoding_info; }$/;" f class:doris::segment_v2::ColumnReader +encoding_info src/olap/rowset/segment_v2/indexed_column_reader.h /^ const EncodingInfo* encoding_info() const { return _encoding_info; }$/;" f class:doris::segment_v2::IndexedColumnReader +encoding_table src/exec/olap_utils.h /^static char encoding_table[] = {$/;" m namespace:doris +encoding_table src/util/url_coding.cpp /^static char encoding_table[] = {$/;" m namespace:doris file: +encrypt src/util/aes_util.cpp /^int AesUtil::encrypt(AesMode mode, const unsigned char* source, uint32_t source_length,$/;" f class:doris::AesUtil +encrypted src/util/minizip/unzip.c /^ int encrypted;$/;" m struct:__anon34 file: +end src/exec/csv_scan_node.cpp /^ char const* end() const {$/;" f class:doris::StringRef +end src/exec/hash_table.h /^ Iterator end() {$/;" f class:doris::HashTable +end src/gutil/strings/split_internal.h /^ const Iterator& end() const { return end_; }$/;" f class:strings::internal::Splitter +end src/gutil/strings/stringpiece.h /^ iterator end() const { return ptr_ + length_; }$/;" f class:StringPiece +end src/olap/olap_index.h /^ const OLAPIndexOffset end() const {$/;" f class:doris::MemIndex +end src/olap/rowset/segment_reader.h /^ int64_t end;$/;" m struct:doris::SegmentReader::DiskRange +end src/olap/rowset/segment_v2/ordinal_page_index.h /^OrdinalPageIndexIterator OrdinalIndexReader::end() {$/;" f class:doris::segment_v2::OrdinalIndexReader +end src/olap/short_key_index.h /^ ShortKeyIndexIterator end() const {$/;" f class:doris::ShortKeyIndexDecoder +end src/util/bitmap_value.h /^inline Roaring64MapSetBitForwardIterator Roaring64Map::end() const {$/;" f class:doris::detail::Roaring64Map +end src/util/lru_cache.hpp /^ Iterator end() {$/;" f class:doris::LruCache +end src/util/tdigest.h /^ std::vector::const_iterator end;$/;" m struct:doris::CentroidList +end_ src/gutil/strings/split_internal.h /^ const Iterator end_;$/;" m class:strings::internal::Splitter +end_include src/exec/olap_common.h /^ bool end_include() const {$/;" f class:doris::OlapScanKeys +end_include src/exec/olap_utils.h /^ bool end_include;$/;" m struct:doris::OlapScanRange +end_key src/exec/tablet_info.h /^ Tuple* end_key = nullptr;$/;" m struct:doris::OlapTablePartition +end_key src/olap/reader.h /^ std::vector end_key;$/;" m struct:doris::ReaderParams +end_keys src/olap/reader.h /^ std::vector end_keys;$/;" m struct:doris::Reader::KeysParam +end_range src/olap/reader.h /^ std::string end_range;$/;" m struct:doris::Reader::KeysParam +end_range src/olap/reader.h /^ std::string end_range;$/;" m struct:doris::ReaderParams +end_scan_range src/exec/olap_utils.h /^ OlapTuple end_scan_range;$/;" m struct:doris::OlapScanRange +end_version src/olap/rowset/rowset.h /^ int64_t end_version() const { return rowset_meta()->version().second; }$/;" f class:doris::Rowset +end_version src/olap/rowset/rowset_meta.h /^ int64_t end_version() const {$/;" f class:doris::RowsetMeta +ends_with src/exprs/string_functions.cpp /^BooleanVal StringFunctions::ends_with($/;" f class:doris::StringFunctions +ends_with src/gutil/strings/stringpiece.h /^ bool ends_with(StringPiece x) const {$/;" f class:StringPiece +enqueue src/runtime/bufferpool/buffer_pool_internal.h /^ void enqueue(Page* page) {$/;" f class:doris::BufferPool::PageList +enqueue src/util/internal_queue.h /^ void enqueue(T* n) {$/;" f class:doris::InternalQueueBase +enqueue_buffer src/runtime/disk_io_mgr_scan_range.cc /^bool DiskIoMgr::ScanRange::enqueue_buffer(BufferDescriptor* buffer) {$/;" f class:doris::DiskIoMgr::ScanRange +enqueue_context src/runtime/disk_io_mgr_internal.h /^ inline void enqueue_context(RequestContext* worker) {$/;" f struct:doris::DiskIoMgr::DiskQueue +entry src/olap/bloom_filter_reader.cpp /^const BloomFilter& BloomFilterIndexReader::entry(uint64_t entry_id) {$/;" f class:doris::BloomFilterIndexReader +entry src/olap/stream_index_reader.cpp /^const PositionEntryReader& StreamIndexReader::entry(uint64_t entry_id) {$/;" f class:doris::StreamIndexReader +entry_count src/olap/bloom_filter_reader.cpp /^size_t BloomFilterIndexReader::entry_count() {$/;" f class:doris::BloomFilterIndexReader +entry_count src/olap/stream_index_reader.cpp /^size_t StreamIndexReader::entry_count() {$/;" f class:doris::StreamIndexReader +entry_length src/olap/olap_index.h /^ const size_t entry_length() const {$/;" f class:doris::MemIndex +entry_size src/olap/stream_index_reader.cpp /^size_t PositionEntryReader::entry_size() const {$/;" f class:doris::PositionEntryReader +entry_size src/olap/stream_index_writer.cpp /^size_t StreamIndexWriter::entry_size() {$/;" f class:doris::StreamIndexWriter +enum_limits src/gutil/casts.h /^class enum_limits {$/;" c +env src/olap/fs/file_block_manager.h /^ Env* env() const { return _env; }$/;" f class:doris::fs::FileBlockManager +env_util src/env/env_util.cpp /^namespace env_util {$/;" n namespace:doris file: +env_util src/env/env_util.h /^namespace env_util {$/;" n namespace:doris +eof src/olap/file_stream.h /^ inline bool eof() {$/;" f class:doris::ReadOnlyFileStream::FileCursor +eof src/olap/file_stream.h /^ bool eof() {$/;" f class:doris::ReadOnlyFileStream +eof src/olap/in_stream.h /^ bool eof() {$/;" f class:doris::InStream +eof src/olap/push_handler.h /^ virtual bool eof() {$/;" f class:doris::BinaryReader +eof src/olap/push_handler.h /^ virtual bool eof() {$/;" f class:doris::LzoBinaryReader +eof src/olap/rowset/column_data.h /^ bool eof() { return _eof; }$/;" f class:doris::ColumnData +eof src/olap/rowset/column_reader.h /^ bool eof() {$/;" f class:doris::IntegerColumnReader +eof src/olap/rowset/segment_reader.h /^ bool eof() const {$/;" f class:doris::SegmentReader +eof_counters test/exec/tablet_sink_test.cpp /^ int64_t eof_counters = 0;$/;" m class:doris::stream_load::TestInternalService file: +eof_ptr src/olap/rowset/column_data.h /^ bool* eof_ptr() { return &_eof; }$/;" f class:doris::ColumnData +eosr src/runtime/disk_io_mgr.h /^ bool eosr() { return _eosr; }$/;" f class:doris::DiskIoMgr::BufferDescriptor +eq src/runtime/raw_value.h /^inline bool RawValue::eq(const void* v1, const void* v2, const TypeDescriptor& type) {$/;" f class:doris::RawValue +eq src/runtime/string_value.hpp /^inline bool StringValue::eq(const StringValue& other) const {$/;" f class:doris::StringValue +eq_tuple_row src/runtime/dpp_sink.cpp /^bool Translator::eq_tuple_row(TupleRow* last, TupleRow* cur) {$/;" f class:doris::Translator +equal src/olap/field.h /^ bool equal(const LhsCellType& lhs,$/;" f class:doris::Field +equal src/olap/tablet.h /^inline bool Tablet::equal(int64_t id, int32_t hash) {$/;" f class:doris::Tablet +equal src/olap/types.h /^ inline bool equal(const void* left, const void* right) const {$/;" f class:doris::TypeInfo +equal src/olap/types.h /^ static bool equal(const void* left, const void* right) {$/;" f struct:doris::FieldTypeTraits +equal src/olap/types.h /^ static inline bool equal(const void* left, const void* right) {$/;" f struct:doris::BaseFieldtypeTraits +equal src/runtime/decimal_value.h /^ bool equal(const DecimalValue& other) const {$/;" f class:doris::DecimalValue +equal_row src/olap/row.h /^bool equal_row(const std::vector& ids,$/;" f namespace:doris +equals src/exec/hash_table.cpp /^bool HashTable::equals(TupleRow* build_row) {$/;" f class:doris::HashTable +equals src/exec/partitioned_hash_table.cc /^bool PartitionedHashTableCtx::equals(TupleRow* build_row) {$/;" f class:doris::PartitionedHashTableCtx +equals src/exprs/anyval_util.h /^ static inline bool equals(const FunctionContext::TypeDesc& type, const T& x, const T& y) {$/;" f class:doris::AnyValUtil +equals src/exprs/anyval_util.h /^ static inline bool equals(const PrimitiveType& type, const T& x, const T& y) {$/;" f class:doris::AnyValUtil +equals src/exprs/anyval_util.h /^ static inline bool equals(const T& x, const T& y) {$/;" f class:doris::AnyValUtil +equals src/exprs/anyval_util.h /^ static inline bool equals(const TypeDescriptor& type, const T& x, const T& y) {$/;" f class:doris::AnyValUtil +equals src/runtime/descriptors.cpp /^bool RowDescriptor::equals(const RowDescriptor& other_desc) const {$/;" f class:doris::RowDescriptor +equals src/runtime/descriptors.h /^ bool equals(const NullIndicatorOffset& o) const {$/;" f struct:doris::NullIndicatorOffset +equals_intenal src/exprs/anyval_util.h /^inline bool AnyValUtil::equals_intenal(const DateTimeVal& x, const DateTimeVal& y) {$/;" f class:doris::AnyValUtil +equals_intenal src/exprs/anyval_util.h /^inline bool AnyValUtil::equals_intenal(const DecimalV2Val& x, const DecimalV2Val& y) {$/;" f class:doris::AnyValUtil +equals_intenal src/exprs/anyval_util.h /^inline bool AnyValUtil::equals_intenal(const DecimalVal& x, const DecimalVal& y) {$/;" f class:doris::AnyValUtil +equals_intenal src/exprs/anyval_util.h /^inline bool AnyValUtil::equals_intenal(const StringVal& x, const StringVal& y) {$/;" f class:doris::AnyValUtil +equals_intenal src/exprs/anyval_util.h /^inline bool AnyValUtil::equals_intenal(const T& x, const T& y) {$/;" f class:doris::AnyValUtil +erase src/olap/lru_cache.cpp /^void LRUCache::erase(const CacheKey& key, uint32_t hash) {$/;" f class:doris::LRUCache +erase src/olap/lru_cache.cpp /^void ShardedLRUCache::erase(const CacheKey& key) {$/;" f class:doris::ShardedLRUCache +erase src/util/lru_cache.hpp /^ void erase(const Key& key) {$/;" f class:doris::LruCache +erase_export_task src/service/backend_service.cpp /^void BackendService::erase_export_task(TStatus& t_status, const TUniqueId& task_id) {$/;" f class:doris::BackendService +erase_handle src/http/action/mini_load.cpp /^void MiniLoadAction::erase_handle(const LoadHandle& desc) {$/;" f class:doris::MiniLoadAction +erase_job src/runtime/etl_job_mgr.cpp /^Status EtlJobMgr::erase_job(const TDeleteEtlFilesRequest& req) {$/;" f class:doris::EtlJobMgr +erase_task src/runtime/export_task_mgr.cpp /^Status ExportTaskMgr::erase_task(const TUniqueId& id) {$/;" f class:doris::ExportTaskMgr +err_handler src/runtime/routine_load/routine_load_task_executor.cpp /^void RoutineLoadTaskExecutor::err_handler($/;" f class:doris::RoutineLoadTaskExecutor +errno_to_cstring src/util/errno.cpp /^void errno_to_cstring(int err, char *buf, size_t buf_len) {$/;" f namespace:doris +errno_to_string src/util/errno.h /^inline static std::string errno_to_string(int err) {$/;" f namespace:doris +error_log src/runtime/runtime_state.cpp /^std::string RuntimeState::error_log() {$/;" f class:doris::RuntimeState +error_log src/runtime/runtime_state.h /^ const std::vector& error_log() const {$/;" f class:doris::RuntimeState +error_log_is_empty src/runtime/runtime_state.cpp /^bool RuntimeState::error_log_is_empty() {$/;" f class:doris::RuntimeState +error_msg src/udf/udf.cpp /^const char* FunctionContext::error_msg() const {$/;" f class:doris_udf::FunctionContext +error_status src/util/mysql_load_error_hub.cpp /^Status MysqlLoadErrorHub::error_status(const std::string& prefix, MYSQL* my_conn) {$/;" f class:doris::MysqlLoadErrorHub +error_url src/runtime/stream_load/stream_load_context.h /^ std::string error_url = "";$/;" m class:doris::StreamLoadContext +escape_char src/exprs/like_predicate.h /^ char escape_char;$/;" m struct:doris::LikePredicate::LikePredicateState +escape_for_html src/util/url_coding.cpp /^void escape_for_html(const std::string& in, std::stringstream* out) {$/;" f namespace:doris +estimate_buffer_size src/olap/rowset/segment_v2/column_writer.cpp /^uint64_t ColumnWriter::estimate_buffer_size() {$/;" f class:doris::segment_v2::ColumnWriter +estimate_buffered_memory src/olap/bloom_filter_writer.cpp /^uint64_t BloomFilterIndexWriter::estimate_buffered_memory() {$/;" f class:doris::BloomFilterIndexWriter +estimate_buffered_memory src/olap/rowset/column_writer.cpp /^uint64_t ColumnWriter::estimate_buffered_memory() {$/;" f class:doris::ColumnWriter +estimate_buffered_memory src/olap/rowset/column_writer.cpp /^uint64_t VarStringColumnWriter::estimate_buffered_memory() {$/;" f class:doris::VarStringColumnWriter +estimate_cardinality src/olap/hll.cpp /^int64_t HyperLogLog::estimate_cardinality() const {$/;" f class:doris::HyperLogLog +estimate_merge_mem src/runtime/spill_sorter.cc /^uint64_t SpillSorter::estimate_merge_mem($/;" f class:doris::SpillSorter +estimate_segment_size src/olap/rowset/segment_v2/segment_writer.cpp /^uint64_t SegmentWriter::estimate_segment_size() {$/;" f class:doris::segment_v2::SegmentWriter +estimate_segment_size src/olap/rowset/segment_writer.cpp /^uint64_t SegmentWriter::estimate_segment_size() {$/;" f class:doris::SegmentWriter +estimate_uncompressed_length src/olap/in_stream.h /^ uint64_t estimate_uncompressed_length() {$/;" f class:doris::InStream +estimated_input_cardinality_ src/exec/new_partitioned_aggregation_node.h /^ int64_t estimated_input_cardinality_;$/;" m class:doris::NewPartitionedAggregationNode +etl_job_mgr src/runtime/exec_env.h /^ EtlJobMgr* etl_job_mgr() { return _etl_job_mgr; }$/;" f class:doris::ExecEnv +etl_thread_pool src/runtime/exec_env.h /^ PriorityThreadPool* etl_thread_pool() { return _etl_thread_pool; }$/;" f class:doris::ExecEnv +eval src/olap/olap_cond.cpp /^bool Cond::eval(const BloomFilter& bf) const {$/;" f class:doris::Cond +eval src/olap/olap_cond.cpp /^bool Cond::eval(const RowCursorCell& cell) const {$/;" f class:doris::Cond +eval src/olap/olap_cond.cpp /^bool Cond::eval(const segment_v2::BloomFilter* bf) const {$/;" f class:doris::Cond +eval src/olap/olap_cond.cpp /^bool Cond::eval(const std::pair& statistic) const {$/;" f class:doris::Cond +eval src/olap/olap_cond.cpp /^bool CondColumn::eval(const BloomFilter& bf) const {$/;" f class:doris::CondColumn +eval src/olap/olap_cond.cpp /^bool CondColumn::eval(const RowCursor& row) const {$/;" f class:doris::CondColumn +eval src/olap/olap_cond.cpp /^bool CondColumn::eval(const segment_v2::BloomFilter* bf) const {$/;" f class:doris::CondColumn +eval src/olap/olap_cond.cpp /^bool CondColumn::eval(const std::pair &statistic) const {$/;" f class:doris::CondColumn +eval_and_hash_build src/exec/partitioned_hash_table.inline.h /^inline bool PartitionedHashTableCtx::eval_and_hash_build(TupleRow* row, uint32_t* hash) {$/;" f class:doris::PartitionedHashTableCtx +eval_and_hash_probe src/exec/partitioned_hash_table.inline.h /^inline bool PartitionedHashTableCtx::eval_and_hash_probe(TupleRow* row, uint32_t* hash) {$/;" f class:doris::PartitionedHashTableCtx +eval_and_materialize_exprs src/exec/merge_node.cpp /^bool MergeNode::eval_and_materialize_exprs($/;" f class:doris::MergeNode +eval_build_row src/exec/hash_table.h /^ bool IR_NO_INLINE eval_build_row(TupleRow* row) {$/;" f class:doris::HashTable +eval_conjuncts src/exec/exec_node.cpp /^bool ExecNode::eval_conjuncts(ExprContext* const* ctxs, int num_ctxs, TupleRow* row) {$/;" f class:doris::ExecNode +eval_other_join_conjuncts src/exec/hash_join_node_ir.cpp /^bool IR_NO_INLINE eval_other_join_conjuncts($/;" f namespace:doris +eval_probe_row src/exec/hash_table.h /^ bool IR_NO_INLINE eval_probe_row(TupleRow* row) {$/;" f class:doris::HashTable +eval_row src/exec/hash_table.cpp /^bool HashTable::eval_row(TupleRow* row, const vector& ctxs) {$/;" f class:doris::HashTable +eval_row src/exec/partitioned_hash_table.cc /^bool PartitionedHashTableCtx::eval_row(TupleRow* row, const vector& ctxs) {$/;" f class:doris::PartitionedHashTableCtx +evaluate src/exprs/expr.h /^inline bool Expr::evaluate(VectorizedRowBatch* batch) {$/;" f class:doris::Expr +evaluate src/olap/null_predicate.cpp /^Status NullPredicate::evaluate(const Schema& schema, const vector& iterators,$/;" f class:doris::NullPredicate +evaluate src/olap/null_predicate.cpp /^void NullPredicate::evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const {$/;" f class:doris::NullPredicate +evaluate src/olap/null_predicate.cpp /^void NullPredicate::evaluate(VectorizedRowBatch* batch) const {$/;" f class:doris::NullPredicate +evaluate_children src/exprs/scalar_fn_call.cpp /^void ScalarFnCall::evaluate_children($/;" f class:doris::ScalarFnCall +event_cb src/runtime/routine_load/data_consumer.h /^ void event_cb(RdKafka::Event &event) {$/;" f class:doris::KafkaEventCb +events src/util/runtime_profile.h /^ const EventList& events() const {$/;" f class:doris::RuntimeProfile::EventSequence +exceeded_limit src/exec/hash_table.h /^ bool exceeded_limit() const {$/;" f class:doris::HashTable +exceeds_limit src/runtime/data_stream_recvr.h /^ bool exceeds_limit(int batch_size) {$/;" f class:doris::DataStreamRecvr +exchange_data src/runtime/mem_pool.cpp /^void MemPool::exchange_data(MemPool* other) {$/;" f class:doris::MemPool +exec_actual src/runtime/fragment_mgr.cpp /^void FragmentMgr::exec_actual($/;" f class:doris::FragmentMgr +exec_cmd src/agent/utils.cpp /^bool AgentUtils::exec_cmd(const string& command, string* errmsg) {$/;" f class:doris::AgentUtils +exec_debug_action src/exec/exec_node.cpp /^Status ExecNode::exec_debug_action(TExecNodePhase::type phase) {$/;" f class:doris::ExecNode +exec_env src/runtime/runtime_state.h /^ ExecEnv* exec_env() {$/;" f class:doris::RuntimeState +exec_env src/runtime/stream_load/stream_load_context.h /^ ExecEnv* exec_env() { return _exec_env; }$/;" f class:doris::StreamLoadContext +exec_env src/runtime/test_env.h /^ ExecEnv* exec_env() {$/;" f class:doris::TestEnv +exec_external_plan_fragment src/runtime/fragment_mgr.cpp /^Status FragmentMgr::exec_external_plan_fragment(const TScanOpenParams& params, const TUniqueId& fragment_instance_id, std::vector* selected_columns) {$/;" f class:doris::FragmentMgr +exec_plan_fragment src/runtime/fragment_mgr.cpp /^Status FragmentMgr::exec_plan_fragment($/;" f class:doris::FragmentMgr +exec_plan_fragment src/service/backend_service.cpp /^void BackendService::exec_plan_fragment(TExecPlanFragmentResult& return_val,$/;" f class:doris::BackendService +exec_plan_fragment src/service/internal_service.cpp /^void PInternalServiceImpl::exec_plan_fragment($/;" f class:doris::PInternalServiceImpl +exec_plan_fragment test/runtime/data_stream_test.cpp /^ virtual void exec_plan_fragment($/;" f class:doris::DorisTestBackend +exec_plan_fragment test/runtime/etl_job_mgr_test.cpp /^Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params,$/;" f class:doris::FragmentMgr +exec_plan_fragment test/runtime/export_task_mgr_test.cpp /^Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params, FinishCallback cb) {$/;" f class:doris::FragmentMgr +exec_task src/runtime/routine_load/routine_load_task_executor.cpp /^void RoutineLoadTaskExecutor::exec_task($/;" f class:doris::RoutineLoadTaskExecutor +execute output/udf/include/uda_test_harness.h /^ bool execute(const std::vector& values1, const std::vector& values2, $/;" f class:doris_udf::UdaTestHarness2 +execute output/udf/include/uda_test_harness.h /^ bool execute(const std::vector& values1, const std::vector& values2,$/;" f class:doris_udf::UdaTestHarness3 +execute output/udf/include/uda_test_harness.h /^ bool execute(const std::vector& values1, const std::vector& values2,$/;" f class:doris_udf::UdaTestHarness4 +execute output/udf/include/uda_test_harness.h /^ bool execute(const std::vector& values, const RESULT& expected) {$/;" f class:doris_udf::UdaTestHarness +execute output/udf/include/uda_test_harness.h /^ bool execute(const std::vector& values, const RESULT& expected) {$/;" f class:doris_udf::UdaTestHarness +execute output/udf/include/uda_test_harness.h /^ bool execute(const std::vector& values, const RESULT& expected, $/;" f class:doris_udf::UdaTestHarness +execute src/http/http_client.cpp /^Status HttpClient::execute(const std::function& callback) {$/;" f class:doris::HttpClient +execute src/http/http_client.cpp /^Status HttpClient::execute(std::string* response) {$/;" f class:doris::HttpClient +execute src/olap/task/engine_alter_tablet_task.cpp /^OLAPStatus EngineAlterTabletTask::execute() {$/;" f class:doris::EngineAlterTabletTask +execute src/olap/task/engine_batch_load_task.cpp /^OLAPStatus EngineBatchLoadTask::execute() {$/;" f class:doris::EngineBatchLoadTask +execute src/olap/task/engine_checksum_task.cpp /^OLAPStatus EngineChecksumTask::execute() {$/;" f class:doris::EngineChecksumTask +execute src/olap/task/engine_clone_task.cpp /^OLAPStatus EngineCloneTask::execute() {$/;" f class:doris::EngineCloneTask +execute src/olap/task/engine_storage_migration_task.cpp /^OLAPStatus EngineStorageMigrationTask::execute() {$/;" f class:doris::EngineStorageMigrationTask +execute src/olap/task/engine_task.h /^ virtual OLAPStatus execute() { return OLAP_SUCCESS; }$/;" f class:doris::EngineTask +execute src/runtime/fragment_mgr.cpp /^Status FragmentExecState::execute() {$/;" f class:doris::FragmentExecState +execute src/udf/uda_test_harness.h /^ bool execute(const std::vector& values1, const std::vector& values2, $/;" f class:doris_udf::UdaTestHarness2 +execute src/udf/uda_test_harness.h /^ bool execute(const std::vector& values1, const std::vector& values2,$/;" f class:doris_udf::UdaTestHarness3 +execute src/udf/uda_test_harness.h /^ bool execute(const std::vector& values1, const std::vector& values2,$/;" f class:doris_udf::UdaTestHarness4 +execute src/udf/uda_test_harness.h /^ bool execute(const std::vector& values, const RESULT& expected) {$/;" f class:doris_udf::UdaTestHarness +execute src/udf/uda_test_harness.h /^ bool execute(const std::vector& values, const RESULT& expected) {$/;" f class:doris_udf::UdaTestHarness +execute src/udf/uda_test_harness.h /^ bool execute(const std::vector& values, const RESULT& expected, $/;" f class:doris_udf::UdaTestHarness +executeLSD src/util/radix_sort.h /^ static void executeLSD(Element * arr, size_t size) {$/;" f struct:doris::RadixSort +execute_delete_request src/http/http_client.cpp /^Status HttpClient::execute_delete_request(const std::string& payload, std::string* response) {$/;" f class:doris::HttpClient +execute_plan_fragment src/runtime/stream_load/stream_load_executor.cpp /^Status StreamLoadExecutor::execute_plan_fragment(StreamLoadContext* ctx) {$/;" f class:doris::StreamLoadExecutor +execute_post_request src/http/http_client.cpp /^Status HttpClient::execute_post_request(const std::string& payload, std::string* response) {$/;" f class:doris::HttpClient +execute_task src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::execute_task(EngineTask* task) {$/;" f class:doris::StorageEngine +execute_with_retry src/http/http_client.cpp /^Status HttpClient::execute_with_retry(int retry_times, int sleep_time,$/;" f class:doris::HttpClient +executor src/runtime/fragment_mgr.cpp /^ PlanFragmentExecutor* executor() {$/;" f class:doris::FragmentExecState +existing_job_status src/runtime/stream_load/stream_load_context.h /^ std::string existing_job_status = "";$/;" m class:doris::StreamLoadContext +exists src/util/lru_cache.hpp /^ bool exists(const Key& key) const {$/;" f class:doris::LruCache +expected_build_rows test/exec/hash_table_test.cpp /^ vector expected_build_rows;$/;" m struct:doris::HashTableTest::ProbeTestData file: +expected_build_rows test/exec/partitioned_hash_table_test.cpp /^ vector expected_build_rows;$/;" m struct:doris::PartitionedHashTableTest::ProbeTestData file: +expected_local src/runtime/disk_io_mgr.h /^ bool expected_local() const { return _expected_local; }$/;" f class:doris::DiskIoMgr::ScanRange +export_error src/util/broker_load_error_hub.cpp /^Status BrokerLoadErrorHub::export_error(const ErrorMsg& error_msg) {$/;" f class:doris::BrokerLoadErrorHub +export_error src/util/mysql_load_error_hub.cpp /^Status MysqlLoadErrorHub::export_error(const ErrorMsg& error_msg) {$/;" f class:doris::MysqlLoadErrorHub +export_error src/util/null_load_error_hub.cpp /^Status NullLoadErrorHub::export_error(const ErrorMsg& error_msg) {$/;" f class:doris::NullLoadErrorHub +export_load_error src/runtime/runtime_state.cpp /^void RuntimeState::export_load_error(const std::string& err_msg) {$/;" f class:doris::RuntimeState +export_output_files src/runtime/runtime_state.h /^ const std::vector& export_output_files() const {$/;" f class:doris::RuntimeState +expr_idx src/exprs/expr.cpp /^ int expr_idx;$/;" m struct:doris::MemLayoutData file: +expr_mem_pool src/exec/exec_node.h /^ MemPool* expr_mem_pool() { $/;" f class:doris::ExecNode +expr_mem_tracker src/exec/exec_node.h /^ MemTracker* expr_mem_tracker() const {$/;" f class:doris::ExecNode +expr_results_pool_ src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr expr_results_pool_;$/;" m class:doris::NewPartitionedAggregationNode +expr_results_pool_ src/exec/new_partitioned_hash_table.h /^ MemPool* expr_results_pool_;$/;" m class:doris::NewPartitionedHashTableCtx +expr_values_array_ src/exec/new_partitioned_hash_table.h /^ boost::scoped_array expr_values_array_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +expr_values_bytes_per_row src/exec/new_partitioned_hash_table.h /^ int ALWAYS_INLINE expr_values_bytes_per_row() const {$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +expr_values_bytes_per_row_ src/exec/new_partitioned_hash_table.h /^ int expr_values_bytes_per_row_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +expr_values_cache src/exec/new_partitioned_hash_table.h /^ ExprValuesCache* ALWAYS_INLINE expr_values_cache() { return &expr_values_cache_; }$/;" f class:doris::NewPartitionedHashTableCtx +expr_values_cache_ src/exec/new_partitioned_hash_table.h /^ ExprValuesCache expr_values_cache_;$/;" m class:doris::NewPartitionedHashTableCtx +expr_values_hash_array_ src/exec/new_partitioned_hash_table.h /^ boost::scoped_array expr_values_hash_array_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +expr_values_null_array_ src/exec/new_partitioned_hash_table.h /^ boost::scoped_array expr_values_null_array_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +expr_values_offsets src/exec/new_partitioned_hash_table.h /^ int ALWAYS_INLINE expr_values_offsets(int expr_idx) const {$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +expr_values_offsets_ src/exec/new_partitioned_hash_table.h /^ std::vector expr_values_offsets_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +expr_without_cast src/exprs/expr.cpp /^const Expr* Expr::expr_without_cast(const Expr* expr) {$/;" f class:doris::Expr +ext_family_ src/gutil/cpu.h /^ int ext_family_;$/;" m class:base::final +ext_model_ src/gutil/cpu.h /^ int ext_model_;$/;" m class:base::final +extdatasource_client_cache src/runtime/exec_env.h /^ ClientCache* extdatasource_client_cache() { return _extdatasource_client_cache; }$/;" f class:doris::ExecEnv +extend_input_buf src/exec/plain_text_line_reader.cpp /^void PlainTextLineReader::extend_input_buf() {$/;" f class:doris::PlainTextLineReader +extend_output_buf src/exec/plain_text_line_reader.cpp /^void PlainTextLineReader::extend_output_buf() {$/;" f class:doris::PlainTextLineReader +extend_scan_key src/exec/olap_common.h /^Status OlapScanKeys::extend_scan_key(ColumnValueRange& range) {$/;" f class:doris::OlapScanKeys +extended_family src/gutil/cpu.h /^ int extended_family() const { return ext_family_; }$/;" f class:base::final +extended_model src/gutil/cpu.h /^ int extended_model() const { return ext_model_; }$/;" f class:base::final +external_fa src/util/minizip/unzip.h /^ uLong external_fa; \/* external file attributes 4 bytes *\/$/;" m struct:unz_file_info64_s +external_fa src/util/minizip/unzip.h /^ uLong external_fa; \/* external file attributes 4 bytes *\/$/;" m struct:unz_file_info_s +external_scan_context_mgr src/runtime/exec_env.h /^ ExternalScanContextMgr* external_scan_context_mgr() {return _external_scan_context_mgr;}$/;" f class:doris::ExecEnv +extra src/olap/file_helper.h /^ const ExtraType& extra() const {$/;" f class:doris::FileHeader +extract src/util/zip_util.cpp /^Status ZipFile::extract(const std::string& target_path, const std::string& dir_name) {$/;" f class:doris::ZipFile +extractKey src/util/radix_sort.h /^ static Key & extractKey(Element & elem) { return elem; }$/;" f struct:doris::RadixSortFloatTraits +extractKey src/util/radix_sort.h /^ static Key & extractKey(Element & elem) { return elem; }$/;" f struct:doris::RadixSortIntTraits +extractKey src/util/radix_sort.h /^ static Key & extractKey(Element & elem) { return elem; }$/;" f struct:doris::RadixSortUIntTraits +extractKey src/util/tdigest.h /^ static Key & extractKey(Element& elem) { return elem.mean(); }$/;" f struct:doris::TDigest::TDigestRadixSortTraits +extractKey test/util/radix_sort_test.cpp /^ static Key & extractKey(Element& elem) { return elem.d1; }$/;" f struct:doris::RadixSortTestTraits +extract_db_name src/exec/schema_scanner/schema_helper.cpp /^std::string SchemaHelper::extract_db_name(const std::string& full_name) {$/;" f class:doris::SchemaHelper +extract_file src/util/zip_util.cpp /^Status ZipFile::extract_file(const std::string& target_path) {$/;" f class:doris::ZipFile +extract_stream src/olap/rowset/column_reader.h /^inline ReadOnlyFileStream* extract_stream(uint32_t column_unique_id,$/;" f namespace:doris +extract_template src/util/path_trie.hpp /^ std::string extract_template(const std::string& key) {$/;" f class:doris::PathTrie::TrieNode +f_ src/gutil/stl_util.h /^ BinaryOp f_;$/;" m class:BinaryOperateOnFirst +f_ src/gutil/stl_util.h /^ BinaryOp f_;$/;" m class:BinaryOperateOnSecond +f_ src/gutil/stl_util.h /^ F f_;$/;" m class:BinaryComposeBinary +f_ src/gutil/stl_util.h /^ UnaryOp f_;$/;" m class:UnaryOperateOnFirst +f_ src/gutil/stl_util.h /^ UnaryOp f_;$/;" m class:UnaryOperateOnSecond +f_ src/util/scoped_cleanup.h /^ F f_;$/;" m class:doris::ScopedCleanup +fake_update src/runtime/dpp_sink.cpp /^static void fake_update(SlotRef* ref, TupleRow* agg_row, TupleRow *row) {$/;" f namespace:doris +false_ src/gutil/template_util.h /^typedef false_type false_;$/;" t namespace:base +false_type src/gutil/template_util.h /^typedef integral_constant false_type;$/;" t namespace:base +family src/gutil/cpu.h /^ int family() const { return family_; }$/;" f class:base::final +family_ src/gutil/cpu.h /^ int family_; \/\/ family of the processor$/;" m class:base::final +fast_clock src/util/os_info.h /^ static clockid_t fast_clock() {$/;" f class:doris::OsInfo +fast_clock_ src/util/os_info.cpp /^clockid_t OsInfo::fast_clock_ = CLOCK_MONOTONIC;$/;" m class:doris::OsInfo file: +fast_clock_ src/util/os_info.h /^ static clockid_t fast_clock_;$/;" m class:doris::OsInfo +fastmemcmp_inlined src/gutil/strings/fastmem.h /^inline int fastmemcmp_inlined(const void *a_void, const void *b_void, size_t n) {$/;" f namespace:strings +faststring src/util/faststring.h /^ explicit faststring(size_t capacity)$/;" f class:doris::faststring +faststring src/util/faststring.h /^ faststring() :$/;" f class:doris::faststring +faststring src/util/faststring.h /^class faststring {$/;" c namespace:doris +fastunion src/util/bitmap_value.h /^ static Roaring64Map fastunion(size_t n, const Roaring64Map** inputs) {$/;" f class:doris::detail::Roaring64Map +fclose_file_func src/util/minizip/ioapi.c /^static int ZCALLBACK fclose_file_func (voidpf opaque, voidpf stream)$/;" f file: +fd src/http/action/mini_load.cpp /^ int fd = -1;$/;" m struct:doris::MiniLoadAsyncCtx file: +fd src/olap/file_helper.h /^ int fd() { return ::fileno(_fp); }$/;" f class:doris::FileHandlerWithBuf +fd src/olap/file_helper.h /^ int fd() {$/;" f class:doris::FileHandler +fd src/olap/file_helper.h /^ int fd;$/;" m struct:doris::FileDescriptor +fd src/util/perf_counters.h /^ int fd;$/;" m union:doris::PerfCounters::CounterData::__anon32 +fd_ src/env/env_posix.cpp /^ const int fd_;$/;" m class:doris::ScopedFdCloser file: +fd_num_limit src/util/system_metrics.cpp /^ IntGauge fd_num_limit;$/;" m struct:doris::FileDescriptorMetrics file: +fd_num_used src/util/system_metrics.cpp /^ IntGauge fd_num_used;$/;" m struct:doris::FileDescriptorMetrics file: +feature_ src/util/cpu_info.h /^ int64_t feature_;$/;" m struct:doris::CpuInfo::TempDisable +features build/CMakeFiles/feature_tests.c /^ const char features[] = {"\\n"$/;" v +features build/CMakeFiles/feature_tests.cxx /^ const char features[] = {"\\n"$/;" v +ferror_file_func src/util/minizip/ioapi.c /^static int ZCALLBACK ferror_file_func (voidpf opaque, voidpf stream)$/;" f file: +fetch_all_pull_load_task_infos test/runtime/data_stream_test.cpp /^ virtual void fetch_all_pull_load_task_infos($/;" f class:doris::DorisTestBackend +fetch_and_update src/common/atomic.h /^ T fetch_and_update(T delta) {$/;" f class:doris::AtomicInt +fetch_data src/runtime/result_buffer_mgr.cpp /^Status ResultBufferMgr::fetch_data($/;" f class:doris::ResultBufferMgr +fetch_data src/runtime/result_buffer_mgr.cpp /^void ResultBufferMgr::fetch_data(const PUniqueId& finst_id, GetResultBatchCtx* ctx) {$/;" f class:doris::ResultBufferMgr +fetch_data src/service/backend_service.cpp /^void BackendService::fetch_data(TFetchDataResult& return_val,$/;" f class:doris::BackendService +fetch_data src/service/internal_service.cpp /^void PInternalServiceImpl::fetch_data($/;" f class:doris::PInternalServiceImpl +fetch_data test/runtime/data_stream_test.cpp /^ virtual void fetch_data(TFetchDataResult& return_val, const TFetchDataParams& params) {}$/;" f class:doris::DorisTestBackend +fetch_pull_load_task_info test/runtime/data_stream_test.cpp /^ virtual void fetch_pull_load_task_info($/;" f class:doris::DorisTestBackend +fetch_result src/runtime/result_queue_mgr.cpp /^Status ResultQueueMgr::fetch_result(const TUniqueId& fragment_instance_id, std::shared_ptr* result, bool *eos) {$/;" f class:doris::ResultQueueMgr +fetch_rowset src/olap/tablet_sync_service.cpp /^std::future TabletSyncService::fetch_rowset(TabletSharedPtr tablet, Version& version, bool load_data) {$/;" f class:doris::TabletSyncService +fetch_rowset src/olap/tablet_sync_service.cpp /^std::future TabletSyncService::fetch_rowset(TabletSharedPtr tablet, int64_t txn_id, bool load_data) {$/;" f class:doris::TabletSyncService +fetch_tablet_meta src/olap/tablet_sync_service.cpp /^std::future TabletSyncService::fetch_tablet_meta(TabletSharedPtr tablet, bool load_data) {$/;" f class:doris::TabletSyncService +field src/olap/wrapper_field.h /^ const Field* field() const { return _rep; }$/;" f class:doris::WrapperField +field_count src/olap/row_cursor.h /^ size_t field_count() const {$/;" f class:doris::RowCursor +field_idx src/runtime/descriptors.h /^ int field_idx() const {$/;" f class:doris::SlotDescriptor +field_index src/olap/tablet.h /^inline size_t Tablet::field_index(const string& field_name) const {$/;" f class:doris::Tablet +field_index src/olap/tablet_schema.cpp /^size_t TabletSchema::field_index(const std::string& field_name) const {$/;" f class:doris::TabletSchema +field_names src/runtime/types.h /^ std::vector field_names;$/;" m struct:doris::TypeDescriptor +field_num src/exec/mysql_scanner.h /^ int field_num() const {$/;" f class:doris::MysqlScanner +field_ptr src/olap/row_block.h /^ inline char* field_ptr(size_t row, size_t col) const {$/;" f class:doris::RowBlock +field_size src/olap/field.h /^ inline size_t field_size() const { return size() + 1; }$/;" f class:doris::Field +field_size src/olap/wrapper_field.h /^ size_t field_size() const { return _rep->field_size(); }$/;" f class:doris::WrapperField +file src/olap/rowset/segment_v2/column_reader.h /^ RandomAccessFile* file = nullptr;$/;" m struct:doris::segment_v2::ColumnIteratorOptions +file src/olap/rowset/segment_v2/page_io.h /^ RandomAccessFile* file = nullptr;$/;" m struct:doris::segment_v2::PageReadOptions +file src/runtime/disk_io_mgr.h /^ hdfsFile file() const { return _hdfs_file; }$/;" f class:doris::DiskIoMgr::HdfsCachedFileHandle +file src/runtime/disk_io_mgr.h /^ const char* file() const { return _file.c_str(); }$/;" f class:doris::DiskIoMgr::RequestRange +file src/util/file_cache.h /^ FileType* file() const {$/;" f class:doris::OpenedFileHandle +file_block_mgr src/olap/fs/fs_util.cpp /^BlockManager* file_block_mgr(Env* env, BlockManagerOptions opts) {$/;" f namespace:doris::fs::fs_util +file_extension src/util/path_util.cpp /^string file_extension(const string& path) {$/;" f namespace:doris::path_util +file_header src/olap/olap_index.h /^ FileHeader file_header;$/;" m struct:doris::SegmentMetaInfo +file_in_zip64_read_info_s src/util/minizip/unzip.c /^} file_in_zip64_read_info_s;$/;" t typeref:struct:__anon33 file: +file_length src/olap/file_helper.h /^ uint32_t file_length;$/;" m struct:doris::_FixedFileHeader +file_length src/olap/file_helper.h /^ uint64_t file_length() const {$/;" f class:doris::FileHeader +file_length src/olap/file_helper.h /^ uint64_t file_length;$/;" m struct:doris::_FixedFileHeaderV2 +file_length src/olap/push_handler.h /^ size_t file_length() const {$/;" f class:doris::BinaryFile +file_map src/runtime/etl_job_mgr.h /^ std::map file_map;$/;" m struct:doris::EtlJobResult +file_name src/olap/file_helper.h /^ const std::string& file_name() { return _file_name; }$/;" f class:doris::FileHandlerWithBuf +file_name src/olap/file_helper.h /^ const std::string& file_name() {$/;" f class:doris::FileHandler +file_name src/olap/file_stream.h /^ const std::string& file_name() const { return _file_handler->file_name(); }$/;" f class:doris::ReadOnlyFileStream::FileCursor +file_name src/util/bfd_parser.cpp /^ const char* file_name;$/;" m struct:doris::BfdFindCtx file: +file_path src/http/action/mini_load.cpp /^ std::string file_path;$/;" m struct:doris::MiniLoadAsyncCtx file: +filename src/exec/decompressor.h /^ std::string filename;$/;" m struct:doris::LzopDecompressor::HeaderInfo +files src/runtime/export_task_mgr.h /^ std::vector files;$/;" m struct:doris::ExportTaskResult +filestream src/util/minizip/unzip.c /^ voidpf filestream; \/* io structore of the zipfile *\/$/;" m struct:__anon33 file: +filestream src/util/minizip/unzip.c /^ voidpf filestream; \/* io structore of the zipfile *\/$/;" m struct:__anon34 file: +fill_dest_tuple src/exec/base_scanner.cpp /^bool BaseScanner::fill_dest_tuple(const Slice& line, Tuple* dest_tuple, MemPool* mem_pool) {$/;" f class:doris::BaseScanner +fill_fix_length_string src/exec/broker_scanner.cpp /^void BrokerScanner::fill_fix_length_string($/;" f class:doris::BrokerScanner +fill_fix_length_string src/exec/csv_scan_node.cpp /^void CsvScanNode::fill_fix_length_string($/;" f class:doris::CsvScanNode +fill_fopen64_filefunc src/util/minizip/ioapi.c /^void fill_fopen64_filefunc (zlib_filefunc64_def* pzlib_filefunc_def)$/;" f +fill_fopen_filefunc src/util/minizip/ioapi.c /^void fill_fopen_filefunc (pzlib_filefunc_def)$/;" f +fill_input_row_batch src/exec/exchange_node.cpp /^Status ExchangeNode::fill_input_row_batch(RuntimeState* state) {$/;" f class:doris::ExchangeNode +fill_one_row src/exec/schema_scanner/schema_charsets_scanner.cpp /^Status SchemaCharsetsScanner::fill_one_row(Tuple *tuple, MemPool *pool) {$/;" f class:doris::SchemaCharsetsScanner +fill_one_row src/exec/schema_scanner/schema_collations_scanner.cpp /^Status SchemaCollationsScanner::fill_one_row(Tuple *tuple, MemPool *pool) {$/;" f class:doris::SchemaCollationsScanner +fill_one_row src/exec/schema_scanner/schema_columns_scanner.cpp /^Status SchemaColumnsScanner::fill_one_row(Tuple *tuple, MemPool *pool) {$/;" f class:doris::SchemaColumnsScanner +fill_one_row src/exec/schema_scanner/schema_schemata_scanner.cpp /^Status SchemaSchemataScanner::fill_one_row(Tuple *tuple, MemPool *pool) {$/;" f class:doris::SchemaSchemataScanner +fill_one_row src/exec/schema_scanner/schema_tables_scanner.cpp /^Status SchemaTablesScanner::fill_one_row(Tuple *tuple, MemPool *pool) {$/;" f class:doris::SchemaTablesScanner +fill_one_row src/exec/schema_scanner/schema_variables_scanner.cpp /^Status SchemaVariablesScanner::fill_one_row(Tuple *tuple, MemPool *pool) {$/;" f class:doris::SchemaVariablesScanner +fill_slot src/exec/parquet_reader.cpp /^inline void ParquetReaderWrap::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {$/;" f class:doris::ParquetReaderWrap +fill_slots_of_columns_from_path src/exec/base_scanner.cpp /^void BaseScanner::fill_slots_of_columns_from_path(int start, const std::vector& columns_from_path) {$/;" f class:doris::BaseScanner +fill_tuple src/exec/es/es_scroll_parser.cpp /^Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, $/;" f class:doris::ScrollParser +fill_zlib_filefunc64_32_def_from_filefunc32 src/util/minizip/ioapi.c /^void fill_zlib_filefunc64_32_def_from_filefunc32(zlib_filefunc64_32_def* p_filefunc64_32,const zlib_filefunc_def* p_filefunc32)$/;" f +filled src/exec/new_partitioned_hash_table.h /^ bool filled;$/;" m struct:doris::NewPartitionedHashTable::Bucket +filled src/exec/partitioned_hash_table.h /^ bool filled;$/;" m struct:doris::PartitionedHashTable::Bucket +filter src/exec/es/es_query_builder.cpp /^void BooleanQueryBuilder::filter(QueryBuilder* filter) {$/;" f class:doris::BooleanQueryBuilder +filter_version src/olap/delete_handler.h /^ int32_t filter_version; \/\/ 删除条件版本号$/;" m struct:doris::DeleteConditions +filtered_rows src/olap/merger.h /^ int64_t filtered_rows = 0;$/;" m struct:doris::Merger::Statistics +filtered_rows src/olap/reader.h /^ uint64_t filtered_rows() const {$/;" f class:doris::Reader +filtered_rows src/olap/rowset/alpha_rowset_reader.cpp /^int64_t AlphaRowsetReader::filtered_rows() {$/;" f class:doris::AlphaRowsetReader +filtered_rows src/olap/schema_change.h /^ uint64_t filtered_rows() const {$/;" f class:doris::SchemaChange +final src/gutil/cpu.h /^class CPU final {$/;" c namespace:base +final src/util/bitmap_value.h /^class Roaring64MapSetBitForwardIterator final {$/;" c namespace:doris::detail +finalize src/exprs/agg_fn_evaluator.h /^inline void AggFnEvaluator::finalize($/;" f class:doris::AggFnEvaluator +finalize src/exprs/agg_fn_evaluator.h /^inline void AggFnEvaluator::finalize(const std::vector& evaluators,$/;" f class:doris::AggFnEvaluator +finalize src/exprs/aggregate_functions.cpp /^ BigIntVal finalize() {$/;" f class:doris::MultiDistinctStringCountState +finalize src/olap/aggregate_func.h /^ inline void finalize(RowCursorCell* src, MemPool* mem_pool) const {$/;" f class:doris::AggregateInfo +finalize src/olap/aggregate_func.h /^ static void finalize(RowCursorCell* src, MemPool* mem_pool) {$/;" f struct:doris::AggregateFuncTraits +finalize src/olap/aggregate_func.h /^ static void finalize(RowCursorCell* src, MemPool* mem_pool) {$/;" f struct:doris::BaseAggregateFuncs +finalize src/olap/delete_handler.cpp /^void DeleteHandler::finalize() {$/;" f class:doris::DeleteHandler +finalize src/olap/fs/file_block_manager.cpp /^Status FileWritableBlock::finalize() {$/;" f class:doris::fs::internal::FileWritableBlock +finalize src/olap/olap_cond.h /^ void finalize() {$/;" f class:doris::Conditions +finalize src/olap/push_handler.cpp /^OLAPStatus BinaryReader::finalize() {$/;" f class:doris::BinaryReader +finalize src/olap/push_handler.cpp /^OLAPStatus LzoBinaryReader::finalize() {$/;" f class:doris::LzoBinaryReader +finalize src/olap/row_block.cpp /^OLAPStatus RowBlock::finalize(uint32_t row_num) {$/;" f class:doris::RowBlock +finalize src/olap/rowset/column_data_writer.cpp /^OLAPStatus ColumnDataWriter::finalize() {$/;" f class:doris::ColumnDataWriter +finalize src/olap/rowset/column_writer.cpp /^OLAPStatus ByteColumnWriter::finalize(ColumnDataHeaderMessage* header) {$/;" f class:doris::ByteColumnWriter +finalize src/olap/rowset/column_writer.cpp /^OLAPStatus ColumnWriter::finalize(ColumnDataHeaderMessage* header) {$/;" f class:doris::ColumnWriter +finalize src/olap/rowset/column_writer.cpp /^OLAPStatus DecimalColumnWriter::finalize(ColumnDataHeaderMessage* header) {$/;" f class:doris::DecimalColumnWriter +finalize src/olap/rowset/column_writer.cpp /^OLAPStatus LargeIntColumnWriter::finalize(ColumnDataHeaderMessage* header) {$/;" f class:doris::LargeIntColumnWriter +finalize src/olap/rowset/column_writer.cpp /^OLAPStatus VarStringColumnWriter::finalize(ColumnDataHeaderMessage* header) {$/;" f class:doris::VarStringColumnWriter +finalize src/olap/rowset/column_writer.h /^ OLAPStatus finalize(ColumnDataHeaderMessage* header) {$/;" f class:doris::IntegerColumnWriter +finalize src/olap/rowset/column_writer.h /^ virtual OLAPStatus finalize(ColumnDataHeaderMessage* header) {$/;" f class:doris::DoubleColumnWriterBase +finalize src/olap/rowset/column_writer.h /^ virtual OLAPStatus finalize(ColumnDataHeaderMessage* header) {$/;" f class:doris::IntegerColumnWriterWrapper +finalize src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::finalize(uint64_t* segment_file_size, uint64_t* index_size) {$/;" f class:doris::segment_v2::SegmentWriter +finalize src/olap/rowset/segment_writer.cpp /^OLAPStatus SegmentWriter::finalize(uint32_t* segment_file_size) {$/;" f class:doris::SegmentWriter +finalize src/olap/short_key_index.cpp /^Status ShortKeyIndexBuilder::finalize(uint32_t num_segment_rows,$/;" f class:doris::ShortKeyIndexBuilder +finalize_fn src/exprs/agg_fn.h /^ void* finalize_fn() const { return finalize_fn_; }$/;" f class:doris::AggFn +finalize_fn_ src/exprs/agg_fn.h /^ void* finalize_fn_ = nullptr;$/;" m class:doris::AggFn +finalize_job src/runtime/etl_job_mgr.cpp /^void EtlJobMgr::finalize_job(PlanFragmentExecutor* executor) {$/;" f class:doris::EtlJobMgr +finalize_one_merge src/runtime/dpp_sink.cpp /^void HllDppSinkMerge::finalize_one_merge(TupleRow* agg_row, MemPool* pool, $/;" f class:doris::HllDppSinkMerge +finalize_segment src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::finalize_segment(uint32_t data_segment_size, int64_t num_rows) {$/;" f class:doris::SegmentGroup +finalize_task src/runtime/export_task_mgr.cpp /^void ExportTaskMgr::finalize_task(PlanFragmentExecutor* executor) {$/;" f class:doris::ExportTaskMgr +finalize_tuple src/exec/aggregation_node.cpp /^Tuple* AggregationNode::finalize_tuple(Tuple* tuple, MemPool* pool) {$/;" f class:doris::AggregationNode +find src/exec/hash_table.hpp /^inline HashTable::Iterator HashTable::find(TupleRow* probe_row) {$/;" f class:doris::HashTable +find src/exec/partitioned_hash_table.inline.h /^inline PartitionedHashTable::Iterator PartitionedHashTable::find($/;" f class:doris::PartitionedHashTable +find src/exprs/hybird_set.h /^ virtual bool find(void* data) {$/;" f class:doris::HybirdSet +find src/exprs/hybird_set.h /^ virtual bool find(void* data) {$/;" f class:doris::StringValueSet +find src/gutil/strings/stringpiece.cc /^int StringPiece::find(StringPiece s, size_type pos) const {$/;" f class:StringPiece +find src/gutil/strings/stringpiece.cc /^int StringPiece::find(char c, size_type pos) const {$/;" f class:StringPiece +find src/olap/olap_index.cpp /^const OLAPIndexOffset MemIndex::find(const RowCursor& k,$/;" f class:doris::MemIndex +find_abs_cgroup_path src/util/cgroup_util.cpp /^Status CGroupUtil::find_abs_cgroup_path(const string& subsystem, string* path) {$/;" f class:doris::CGroupUtil +find_addr_in_section src/util/bfd_parser.cpp /^static void find_addr_in_section(bfd* abfd, asection* sec, void* arg) {$/;" f namespace:doris +find_best_tablet_to_compaction src/olap/tablet_manager.cpp /^TabletSharedPtr TabletManager::find_best_tablet_to_compaction(CompactionType compaction_type,$/;" f class:doris::TabletManager +find_bucket src/exec/partitioned_hash_table.inline.h /^inline PartitionedHashTable::Iterator PartitionedHashTable::find_bucket($/;" f class:doris::PartitionedHashTable +find_buffer src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::find_buffer($/;" f class:doris::BufferedBlockMgr2 +find_buffer_for_block src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::find_buffer_for_block(Block* block, bool* in_mem) {$/;" f class:doris::BufferedBlockMgr2 +find_cgroup_cpu_limit src/util/cgroup_util.cpp /^Status CGroupUtil::find_cgroup_cpu_limit(float* cpu_count) {$/;" f class:doris::CGroupUtil +find_cgroup_mem_limit src/util/cgroup_util.cpp /^Status CGroupUtil::find_cgroup_mem_limit(int64_t* bytes) {$/;" f class:doris::CGroupUtil +find_cgroup_mounts src/util/cgroup_util.cpp /^Status CGroupUtil::find_cgroup_mounts($/;" f class:doris::CGroupUtil +find_chr_from_mem src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline int find_chr_from_mem(const char* s, int c, int len) {$/;" f namespace:doris +find_chr_from_str src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline int find_chr_from_str(const char* s, int c, int len) {$/;" f namespace:doris +find_chunk src/runtime/mem_pool.cpp /^bool MemPool::find_chunk(size_t min_size, bool check_limits) {$/;" f class:doris::MemPool +find_closet_num_bits src/olap/serialize.cpp /^uint32_t find_closet_num_bits(int64_t value) {$/;" f namespace:doris::ser +find_control_block src/runtime/result_buffer_mgr.cpp /^boost::shared_ptr ResultBufferMgr::find_control_block($/;" f class:doris::ResultBufferMgr +find_first src/olap/olap_index.h /^ const OLAPIndexOffset find_first() const {$/;" f class:doris::MemIndex +find_first_non_localhost src/util/network_util.cpp /^bool find_first_non_localhost(const std::vector& addresses, std::string* addr) {$/;" f namespace:doris +find_first_not_of src/gutil/strings/stringpiece.cc /^int StringPiece::find_first_not_of(StringPiece s, size_type pos) const {$/;" f class:StringPiece +find_first_not_of src/gutil/strings/stringpiece.cc /^int StringPiece::find_first_not_of(char c, size_type pos) const {$/;" f class:StringPiece +find_first_of src/gutil/strings/stringpiece.cc /^int StringPiece::find_first_of(StringPiece s, size_type pos) const {$/;" f class:StringPiece +find_first_of src/gutil/strings/stringpiece.h /^ int find_first_of(char c, size_type pos = 0) const { return find(c, pos); }$/;" f class:StringPiece +find_first_row_block src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::find_first_row_block(RowBlockPosition* position) const {$/;" f class:doris::SegmentGroup +find_global_cgroup src/util/cgroup_util.cpp /^Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {$/;" f class:doris::CGroupUtil +find_in_lib src/runtime/datetime_value.cpp /^static int find_in_lib(const char* lib[], const char* str, const char* end) {$/;" f namespace:doris +find_in_set src/exprs/string_functions.cpp /^IntVal StringFunctions::find_in_set($/;" f class:doris::StringFunctions +find_last src/olap/olap_index.h /^ const OLAPIndexOffset find_last() const {$/;" f class:doris::MemIndex +find_last_not_of src/gutil/strings/stringpiece.cc /^int StringPiece::find_last_not_of(StringPiece s, size_type pos) const {$/;" f class:StringPiece +find_last_not_of src/gutil/strings/stringpiece.cc /^int StringPiece::find_last_not_of(char c, size_type pos) const {$/;" f class:StringPiece +find_last_of src/gutil/strings/stringpiece.cc /^int StringPiece::find_last_of(StringPiece s, size_type pos) const {$/;" f class:StringPiece +find_last_of src/gutil/strings/stringpiece.h /^ int find_last_of(char c, size_type pos = npos) const { return rfind(c, pos); }$/;" f class:StringPiece +find_last_row_block src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::find_last_row_block(RowBlockPosition* position) const {$/;" f class:doris::SegmentGroup +find_limit_exceeded_tracker src/runtime/mem_tracker.h /^ MemTracker* find_limit_exceeded_tracker() {$/;" f class:doris::MemTracker +find_mid_point src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::find_mid_point(const RowBlockPosition& low,$/;" f class:doris::SegmentGroup +find_next_row_block src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::find_next_row_block(RowBlockPosition* pos, bool* eof) const {$/;" f class:doris::SegmentGroup +find_node src/exec/tablet_info.h /^ const NodeInfo* find_node(int64_t id) const {$/;" f class:doris::DorisNodesInfo +find_or_insert src/util/container_util.hpp /^V* find_or_insert(boost::unordered_map* m, const K& key, const V& default_val) {$/;" f namespace:doris +find_or_insert src/util/container_util.hpp /^V* find_or_insert(std::map* m, const K& key, const V& default_val) {$/;" f namespace:doris +find_or_insert_set src/exprs/hybird_map.h /^ virtual HybirdSetBase* find_or_insert_set(uint64_t dst, bool* is_add_buckets) {$/;" f class:doris::HybirdMap +find_partition src/runtime/data_stream_sender.cpp /^Status DataStreamSender::find_partition($/;" f class:doris::DataStreamSender +find_prev_point src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::find_prev_point($/;" f class:doris::SegmentGroup +find_recvr src/runtime/data_stream_mgr.cpp /^shared_ptr DataStreamMgr::find_recvr($/;" f class:doris::DataStreamMgr +find_short_key src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::find_short_key(const RowCursor& key,$/;" f class:doris::SegmentGroup +find_tablet src/exec/tablet_info.cpp /^bool OlapTablePartitionParam::find_tablet(Tuple* tuple,$/;" f class:doris::OlapTablePartitionParam +find_tablet src/exec/tablet_info.h /^ TabletLocation* find_tablet(int64_t tablet_id) const {$/;" f class:doris::OlapTableLocationParam +find_tablet_in_trash src/olap/data_dir.cpp /^void DataDir::find_tablet_in_trash(int64_t tablet_id, std::vector* paths) {$/;" f class:doris::DataDir +find_timezone src/exprs/timezone_db.cpp /^boost::local_time::time_zone_ptr TimezoneDatabase::find_timezone(const std::string &tz) {$/;" f class:doris::TimezoneDatabase +find_with_default src/util/container_util.hpp /^const V& find_with_default(const boost::unordered_map& m, const K& key,$/;" f namespace:doris +find_with_default src/util/container_util.hpp /^const V& find_with_default(const std::map& m, const K& key, const V& default_val) {$/;" f namespace:doris +finds_nulls_ src/exec/new_partitioned_hash_table.h /^ const std::vector finds_nulls_;$/;" m class:doris::NewPartitionedHashTableCtx +finds_some_nulls src/exec/new_partitioned_hash_table.h /^ int finds_some_nulls;$/;" m struct:doris::NewPartitionedHashTableCtx::HashTableReplacedConstants +finds_some_nulls src/exec/new_partitioned_hash_table.h /^ bool IR_NO_INLINE finds_some_nulls() const { return finds_some_nulls_; }$/;" f class:doris::NewPartitionedHashTableCtx +finds_some_nulls_ src/exec/new_partitioned_hash_table.h /^ const bool finds_some_nulls_;$/;" m class:doris::NewPartitionedHashTableCtx +finish src/olap/rowset/segment_v2/binary_dict_page.cpp /^OwnedSlice BinaryDictPageBuilder::finish() {$/;" f class:doris::segment_v2::BinaryDictPageBuilder +finish src/olap/rowset/segment_v2/binary_prefix_page.cpp /^OwnedSlice BinaryPrefixPageBuilder::finish() {$/;" f class:doris::segment_v2::BinaryPrefixPageBuilder +finish src/olap/rowset/segment_v2/column_writer.cpp /^ OwnedSlice finish() {$/;" f class:doris::segment_v2::NullBitmapBuilder +finish src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::finish() {$/;" f class:doris::segment_v2::ColumnWriter +finish src/olap/rowset/segment_v2/index_page.cpp /^void IndexPageBuilder::finish(OwnedSlice* body, PageFooterPB* footer) {$/;" f class:doris::segment_v2::IndexPageBuilder +finish src/olap/rowset/segment_v2/indexed_column_writer.cpp /^Status IndexedColumnWriter::finish(IndexedColumnMetaPB* meta) {$/;" f class:doris::segment_v2::IndexedColumnWriter +finish src/olap/rowset/segment_v2/ordinal_page_index.cpp /^Status OrdinalIndexWriter::finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* meta) {$/;" f class:doris::segment_v2::OrdinalIndexWriter +finish src/olap/rowset/segment_v2/zone_map_index.cpp /^Status ZoneMapIndexWriter::finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* index_meta) {$/;" f class:doris::segment_v2::ZoneMapIndexWriter +finish src/olap/task/engine_publish_version_task.cpp /^OLAPStatus EnginePublishVersionTask::finish() {$/;" f class:doris::EnginePublishVersionTask +finish src/olap/task/engine_task.h /^ virtual OLAPStatus finish() { return OLAP_SUCCESS; }$/;" f class:doris::EngineTask +finish src/runtime/dpp_sink.cpp /^Status DppSink::finish(RuntimeState* state) {$/;" f class:doris::DppSink +finish src/runtime/message_body_sink.cpp /^Status MessageBodyFileSink::finish() {$/;" f class:doris::MessageBodyFileSink +finish src/runtime/message_body_sink.h /^ virtual Status finish() {$/;" f class:doris::MessageBodySink +finish_job src/runtime/etl_job_mgr.cpp /^Status EtlJobMgr::finish_job(const TUniqueId& id,$/;" f class:doris::EtlJobMgr +finish_status src/runtime/etl_job_mgr.h /^ Status finish_status;$/;" m struct:doris::EtlJobCtx +finish_tarns src/runtime/mysql_table_writer.h /^ Status finish_tarns() {$/;" f class:doris::MysqlTableWriter +finish_task src/agent/utils.cpp /^AgentStatus MasterServerClient::finish_task($/;" f class:doris::MasterServerClient +finish_task src/runtime/export_task_mgr.cpp /^Status ExportTaskMgr::finish_task(const TUniqueId& id,$/;" f class:doris::ExportTaskMgr +finish_task_requests_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::finish_task_requests_failed;$/;" m class:doris::DorisMetrics file: +finish_task_requests_failed src/util/doris_metrics.h /^ static IntCounter finish_task_requests_failed;$/;" m class:doris::DorisMetrics +finish_task_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::finish_task_requests_total;$/;" m class:doris::DorisMetrics file: +finish_task_requests_total src/util/doris_metrics.h /^ static IntCounter finish_task_requests_total;$/;" m class:doris::DorisMetrics +finish_thread src/util/thread.cpp /^void Thread::finish_thread(void* arg) {$/;" f class:doris::Thread +first src/olap/olap_common.h /^ int64_t first;$/;" m struct:doris::Version +first src/olap/olap_index.h /^ uint32_t first;$/;" m struct:doris::IDRange +first_ordinal src/olap/rowset/segment_v2/ordinal_page_index.h /^ ordinal_t first_ordinal() const { return _index->get_first_ordinal(_cur_idx); }$/;" f class:doris::segment_v2::OrdinalPageIndexIterator +first_ordinal src/olap/rowset/segment_v2/parsed_page.h /^ ordinal_t first_ordinal = 0;$/;" m struct:doris::segment_v2::ParsedPage +first_read_symbol src/olap/rowset/alpha_rowset_reader.h /^ bool first_read_symbol = true;$/;" m struct:doris::AlphaMergeContext +first_row_index src/olap/column_block.h /^ size_t first_row_index() const { return _row_offset; }$/;" f class:doris::ColumnBlockView +first_unmatched src/exec/partitioned_hash_table.inline.h /^inline PartitionedHashTable::Iterator PartitionedHashTable::first_unmatched($/;" f class:doris::PartitionedHashTable +first_val_rewrite_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::first_val_rewrite_update(FunctionContext* ctx, const T& src,$/;" f class:doris::AggregateFunctions +first_val_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::first_val_update(FunctionContext* ctx, const IntVal& src, IntVal* dst) {$/;" f class:doris::AggregateFunctions +first_val_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::first_val_update(FunctionContext* ctx, const StringVal& src,$/;" f class:doris::AggregateFunctions +first_val_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::first_val_update(FunctionContext* ctx, const T& src, T* dst) {$/;" f class:doris::AggregateFunctions +fix_intg_frac_error src/runtime/decimal_value.cpp /^inline void fix_intg_frac_error($/;" f namespace:doris +fixed_size_memory_copy src/util/mem_util.hpp /^inline void fixed_size_memory_copy(void* dst, const void* src) {$/;" f namespace:doris +fixed_size_memory_copy src/util/mem_util.hpp /^template<> inline void fixed_size_memory_copy<0>(void*, const void*) {}$/;" f namespace:doris +fixed_size_memory_copy src/util/mem_util.hpp /^template<> inline void fixed_size_memory_copy<1>(void* dst, const void* src) {$/;" f namespace:doris +fixed_size_memory_copy src/util/mem_util.hpp /^template<> inline void fixed_size_memory_copy<2>(void* dst, const void* src) {$/;" f namespace:doris +fixed_size_memory_copy src/util/mem_util.hpp /^template<> inline void fixed_size_memory_copy<4>(void* dst, const void* src) {$/;" f namespace:doris +fixed_size_memory_copy src/util/mem_util.hpp /^template<> inline void fixed_size_memory_copy<8>(void* dst, const void* src) {$/;" f namespace:doris +fixed_tuple_sizes_ src/runtime/buffered_tuple_stream3.h /^ std::vector fixed_tuple_sizes_;$/;" m class:doris::BufferedTupleStream3 +flag src/util/cpu_info.cpp /^ int64_t flag;$/;" m struct:doris::__anon36 file: +flag src/util/minizip/unzip.h /^ uLong flag; \/* general purpose bit flag 2 bytes *\/$/;" m struct:unz_file_info64_s +flag src/util/minizip/unzip.h /^ uLong flag; \/* general purpose bit flag 2 bytes *\/$/;" m struct:unz_file_info_s +flag_mappings src/util/cpu_info.cpp /^} flag_mappings[] =$/;" m namespace:doris typeref:struct:doris::__anon36 file: +flags src/plugin/plugin.h /^ uint64_t flags;$/;" m struct:doris::Plugin +flat_row src/exec/new_partitioned_hash_table.h /^ BufferedTupleStream3::FlatRowPtr flat_row;$/;" m union:doris::NewPartitionedHashTable::HtData +flip src/olap/byte_buffer.h /^ void flip() {$/;" f class:doris::StorageByteBuffer +flip src/util/bitmap_value.h /^ void flip(uint64_t range_start, uint64_t range_end) {$/;" f class:doris::detail::Roaring64Map +flip src/util/byte_buffer.h /^ void flip() {$/;" f struct:doris::ByteBuffer +float_to_string src/exprs/cast_functions.cpp /^int float_to_string(double value, char* buf) {$/;" f namespace:doris +float_to_string src/exprs/cast_functions.cpp /^int float_to_string(float value, char* buf) {$/;" f namespace:doris +float_val src/exprs/expr_value.h /^ float float_val;$/;" m struct:doris::ExprValue +float_val test/runtime/dpp_writer_test.cpp /^ float float_val;$/;" m struct:doris::TestDataTuple file: +float_val test/runtime/mysql_table_writer_test.cpp /^ float float_val;$/;" m struct:doris::TestDataTuple file: +flush src/olap/memtable.cpp /^OLAPStatus MemTable::flush() {$/;" f class:doris::MemTable +flush src/olap/out_stream.cpp /^OLAPStatus OutStream::flush() {$/;" f class:doris::OutStream +flush src/olap/rowset/alpha_rowset_writer.cpp /^OLAPStatus AlphaRowsetWriter::flush() {$/;" f class:doris::AlphaRowsetWriter +flush src/olap/rowset/beta_rowset_writer.cpp /^OLAPStatus BetaRowsetWriter::flush() {$/;" f class:doris::BetaRowsetWriter +flush src/olap/rowset/bit_field_writer.cpp /^OLAPStatus BitFieldWriter::flush() {$/;" f class:doris::BitFieldWriter +flush src/olap/rowset/column_writer.cpp /^OLAPStatus ColumnWriter::flush() {$/;" f class:doris::ColumnWriter +flush src/olap/rowset/column_writer.h /^ OLAPStatus flush() {$/;" f class:doris::IntegerColumnWriter +flush src/olap/rowset/column_writer.h /^ virtual OLAPStatus flush() {$/;" f class:doris::ByteColumnWriter +flush src/olap/rowset/column_writer.h /^ virtual OLAPStatus flush() {$/;" f class:doris::DecimalColumnWriter +flush src/olap/rowset/column_writer.h /^ virtual OLAPStatus flush() {$/;" f class:doris::DoubleColumnWriterBase +flush src/olap/rowset/column_writer.h /^ virtual OLAPStatus flush() {$/;" f class:doris::FixLengthStringColumnWriter +flush src/olap/rowset/column_writer.h /^ virtual OLAPStatus flush() {$/;" f class:doris::IntegerColumnWriterWrapper +flush src/olap/rowset/column_writer.h /^ virtual OLAPStatus flush() {$/;" f class:doris::LargeIntColumnWriter +flush src/olap/rowset/column_writer.h /^ virtual OLAPStatus flush() {$/;" f class:doris::VarStringColumnWriter +flush src/olap/rowset/run_length_byte_writer.cpp /^OLAPStatus RunLengthByteWriter::flush() {$/;" f class:doris::RunLengthByteWriter +flush src/olap/rowset/run_length_integer_writer.cpp /^OLAPStatus RunLengthIntegerWriter::flush() {$/;" f class:doris::RunLengthIntegerWriter +flush src/olap/rowset/segment_v2/zone_map_index.cpp /^Status ZoneMapIndexWriter::flush() {$/;" f class:doris::segment_v2::ZoneMapIndexWriter +flush src/util/frame_of_reference_coding.cpp /^uint32_t ForEncoder::flush() {$/;" f class:doris::ForEncoder +flush_count src/olap/memtable_flush_executor.h /^ int64_t flush_count= 0;$/;" m struct:doris::FlushStatistic +flush_data_async src/olap/fs/file_block_manager.cpp /^Status FileWritableBlock::flush_data_async() {$/;" f class:doris::fs::internal::FileWritableBlock +flush_memtable_and_wait src/olap/delta_writer.cpp /^OLAPStatus DeltaWriter::flush_memtable_and_wait() {$/;" f class:doris::DeltaWriter +flush_memtable_and_wait test/runtime/load_channel_mgr_test.cpp /^OLAPStatus DeltaWriter::flush_memtable_and_wait() {$/;" f class:doris::DeltaWriter +flush_size_bytes src/olap/memtable_flush_executor.h /^ int64_t flush_size_bytes = 0;$/;" m struct:doris::FlushStatistic +flush_time_ns src/olap/memtable_flush_executor.h /^ int64_t flush_time_ns = 0;$/;" m struct:doris::FlushStatistic +fmix32 src/util/hash_util.hpp /^ ALWAYS_INLINE static uint32_t fmix32(uint32_t h) {$/;" f class:doris::HashUtil +fmix32 src/util/murmur_hash3.cpp /^FORCE_INLINE uint32_t fmix32(uint32_t h)$/;" f +fmix64 src/util/murmur_hash3.cpp /^FORCE_INLINE uint64_t fmix64(uint64_t k)$/;" f +fmod_double src/exprs/math_functions.cpp /^DoubleVal MathFunctions::fmod_double($/;" f class:doris::MathFunctions +fmod_float src/exprs/math_functions.cpp /^FloatVal MathFunctions::fmod_float($/;" f class:doris::MathFunctions +fn src/exprs/expr.h /^ const TFunction& fn() const {$/;" f class:doris::Expr +fn_context src/exprs/expr_context.h /^ FunctionContext* fn_context(int i) {$/;" f class:doris::ExprContext +fn_ctx_idx src/exprs/expr.h /^ int fn_ctx_idx() const { return _fn_ctx_idx; }$/;" f class:doris::Expr +fn_name src/exprs/agg_fn.h /^ const std::string& fn_name() const { return _fn.name.function_name; }$/;" f class:doris::AggFn +fn_name src/exprs/agg_fn_evaluator.h /^ const std::string& fn_name() const {$/;" f class:doris::AggFnEvaluator +fname src/olap/page_cache.h /^ std::string fname;$/;" m struct:doris::StoragePageCache::CacheKey +fnv_hash src/util/hash_util.hpp /^ static uint32_t fnv_hash(const void* data, int32_t bytes, uint32_t hash) {$/;" f class:doris::HashUtil +fnv_hash64 src/util/hash_util.hpp /^ static uint64_t fnv_hash64(const void* data, int32_t bytes, uint64_t hash) {$/;" f class:doris::HashUtil +footer src/olap/rowset/segment_v2/column_writer.h /^ PageFooterPB footer;$/;" m struct:doris::segment_v2::ColumnWriter::Page +footer src/olap/rowset/segment_v2/segment.h /^ const SegmentFooterPB& footer() const {$/;" f class:doris::segment_v2::Segment +fopen64 src/util/minizip/ioapi.h 49;" d +fopen64 src/util/minizip/ioapi.h 54;" d +fopen64 src/util/minizip/ioapi.h 59;" d +fopen64_file_func src/util/minizip/ioapi.c /^static voidpf ZCALLBACK fopen64_file_func (voidpf opaque, const void* filename, int mode)$/;" f file: +fopen_file_func src/util/minizip/ioapi.c /^static voidpf ZCALLBACK fopen_file_func (voidpf opaque, const char* filename, int mode)$/;" f file: +force_rollback_tablet_related_txns src/olap/txn_manager.cpp /^void TxnManager::force_rollback_tablet_related_txns(OlapMeta* meta, TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid) {$/;" f class:doris::TxnManager +format src/runtime/stream_load/stream_load_context.h /^ TFileFormatType::type format = TFileFormatType::FORMAT_CSV_PLAIN;$/;" m class:doris::StreamLoadContext +format_one test/runtime/dpp_writer_test.cpp /^void DppWriterTest::format_one(TestDataTuple& data, std::vector& output_expr) {$/;" f class:doris::DppWriterTest +format_one test/runtime/mysql_table_writer_test.cpp /^void MysqlTableWriterTest::format_one(TestDataTuple& data, std::vector& output_expr) {$/;" f class:doris::MysqlTableWriterTest +format_output_path src/runtime/dpp_sink.cpp /^void Translator::format_output_path(RuntimeState* state) {$/;" f class:doris::Translator +forward src/util/radix_sort.h /^ static KeyBits forward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); }$/;" f struct:doris::RadixSortSignedTransform +forward src/util/radix_sort.h /^ static KeyBits forward(KeyBits x) { return x; }$/;" f struct:doris::RadixSortIdentityTransform +forward src/util/radix_sort.h /^ static KeyBits forward(KeyBits x) {$/;" f struct:doris::RadixSortFloatTransform +found src/util/bfd_parser.cpp /^ bool found;$/;" m struct:doris::BfdFindCtx file: +fpclassify src/gutil/port.h 920;" d +fpclassify_double src/gutil/port.h /^inline int fpclassify_double(double x) {$/;" f +fpclassify_float src/gutil/port.h /^inline int fpclassify_float(float x) {$/;" f +fpp src/olap/rowset/segment_v2/bloom_filter.h /^ double fpp = 0.05;$/;" m struct:doris::segment_v2::BloomFilterOptions +fptr_map src/runtime/user_function_cache.cpp /^ std::unordered_map fptr_map;$/;" m struct:doris::UserFunctionCacheEntry file: +frac src/olap/tablet_schema.h /^ int frac() const { return _frac; }$/;" f class:doris::TabletColumn +frac_len output/udf/include/udf.h /^ int8_t frac_len;$/;" m struct:doris_udf::DecimalVal +frac_len src/udf/udf.h /^ int8_t frac_len;$/;" m struct:doris_udf::DecimalVal +frac_max src/runtime/decimal_value.h /^static const int32_t frac_max[DIG_PER_DEC1 - 1] =$/;" m namespace:doris +frac_value src/runtime/decimal_value.h /^ int32_t frac_value() const {$/;" f class:doris::DecimalValue +frac_value src/runtime/decimalv2_value.h /^ int32_t frac_value() const {$/;" f class:doris::DecimalV2Value +fraction src/olap/decimal12.h /^ int32_t fraction;$/;" m struct:doris::decimal12_t +fragment_executor src/runtime/fragment_mgr.cpp /^static void* fragment_executor(void* param) {$/;" f namespace:doris +fragment_hash_seed src/runtime/runtime_state.h /^ uint32_t fragment_hash_seed() const {$/;" f class:doris::RuntimeState +fragment_id src/runtime/buffer_control_block.h /^ const TUniqueId& fragment_id() const {$/;" f class:doris::BufferControlBlock +fragment_instance_id src/runtime/data_stream_recvr.h /^ const TUniqueId& fragment_instance_id() const { return _fragment_instance_id; }$/;" f class:doris::DataStreamRecvr +fragment_instance_id src/runtime/external_scan_context_mgr.h /^ TUniqueId fragment_instance_id;$/;" m struct:doris::ScanContext +fragment_instance_id src/runtime/fragment_mgr.cpp /^ TUniqueId fragment_instance_id() const {$/;" f class:doris::FragmentExecState +fragment_instance_id src/runtime/runtime_state.h /^ const TUniqueId& fragment_instance_id() const {$/;" f class:doris::RuntimeState +fragment_mem_tracker src/runtime/runtime_state.h /^ MemTracker* fragment_mem_tracker() {$/;" f class:doris::RuntimeState +fragment_mgr src/runtime/exec_env.h /^ FragmentMgr* fragment_mgr() { return _fragment_mgr; }$/;" f class:doris::ExecEnv +fragment_request_duration_us src/util/doris_metrics.cpp /^IntCounter DorisMetrics::fragment_request_duration_us;$/;" m class:doris::DorisMetrics file: +fragment_request_duration_us src/util/doris_metrics.h /^ static IntCounter fragment_request_duration_us;$/;" m class:doris::DorisMetrics +fragment_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::fragment_requests_total;$/;" m class:doris::DorisMetrics file: +fragment_requests_total src/util/doris_metrics.h /^ static IntCounter fragment_requests_total;$/;" m class:doris::DorisMetrics +frame_size src/util/frame_of_reference_coding.h /^ inline uint32_t frame_size(uint32_t frame_index) {$/;" f class:doris::ForDecoder +fread_file_func src/util/minizip/ioapi.c /^static uLong ZCALLBACK fread_file_func (voidpf opaque, voidpf stream, void* buf, uLong size)$/;" f file: +free src/olap/lru_cache.h /^ void free() {$/;" f struct:doris::CachePriority::LRUHandle +free src/runtime/free_pool.hpp /^ void free(uint8_t* ptr) {$/;" f class:doris::FreePool +free src/runtime/memory/chunk_allocator.cpp /^void ChunkAllocator::free(const Chunk& chunk) {$/;" f class:doris::ChunkAllocator +free src/runtime/memory/system_allocator.cpp /^void SystemAllocator::free(uint8_t* ptr, size_t length) {$/;" f class:doris::SystemAllocator +free src/udf/udf.cpp /^ void free(uint8_t* ptr) {$/;" f class:doris::FreePool +free src/udf/udf.cpp /^void FunctionContext::free(int64_t bytes) {$/;" f class:doris_udf::FunctionContext +free src/udf/udf.cpp /^void FunctionContext::free(uint8_t* buffer) { $/;" f class:doris_udf::FunctionContext +free_all src/runtime/mem_pool.cpp /^void MemPool::free_all() {$/;" f class:doris::MemPool +free_buffers src/runtime/bufferpool/buffer_allocator.cc /^ FreeList free_buffers;$/;" m struct:doris::BufferPool::FreeBufferArena::PerSizeLists file: +free_buffers_idx src/runtime/disk_io_mgr.cc /^int DiskIoMgr::free_buffers_idx(int64_t buffer_size) {$/;" f class:doris::DiskIoMgr +free_handler_ctx src/http/action/mini_load.cpp /^void MiniLoadAction::free_handler_ctx(void* param) {$/;" f class:doris::MiniLoadAction +free_handler_ctx src/http/action/stream_load.cpp /^void StreamLoadAction::free_handler_ctx(void* param) {$/;" f class:doris::StreamLoadAction +free_handler_ctx src/http/http_handler.h /^ virtual void free_handler_ctx(void* handler_ctx) { }$/;" f class:doris::HttpHandler +free_list_ src/runtime/bufferpool/free_list.h /^ std::vector free_list_;$/;" m class:doris::FreeList +free_lists_ src/runtime/bufferpool/suballocator.h /^ std::unique_ptr free_lists_[NUM_FREE_LISTS];$/;" m class:doris::Suballocator +free_local_allocations src/exprs/expr_context.cpp /^void ExprContext::free_local_allocations() {$/;" f class:doris::ExprContext +free_local_allocations src/exprs/expr_context.cpp /^void ExprContext::free_local_allocations(const std::vector& ctxs) {$/;" f class:doris::ExprContext +free_local_allocations src/exprs/expr_context.cpp /^void ExprContext::free_local_allocations(const std::vector& fn_ctxs) {$/;" f class:doris::ExprContext +free_local_allocations src/udf/udf.cpp /^void FunctionContextImpl::free_local_allocations() {$/;" f class:doris::FunctionContextImpl +from src/gutil/spinlock_internal.h /^ int32 from;$/;" m struct:base::internal::SpinLockWaitTransition +from src/olap/rowset/segment_v2/row_ranges.h /^ int64_t from() const {$/;" f class:doris::segment_v2::RowRange +from src/olap/rowset/segment_v2/row_ranges.h /^ int64_t from() {$/;" f class:doris::segment_v2::RowRanges +from_base64 src/exprs/encryption_functions.cpp /^StringVal EncryptionFunctions::from_base64(FunctionContext* ctx, const StringVal &src) {$/;" f class:doris::EncryptionFunctions +from_buffer src/exprs/anyval_util.h /^ static StringVal from_buffer(FunctionContext* ctx, const char* ptr, int len) {$/;" f class:doris::AnyValUtil +from_buffer_temp src/exprs/anyval_util.h /^ static StringVal from_buffer_temp(FunctionContext* ctx, const char* ptr, int len) {$/;" f class:doris::AnyValUtil +from_coord src/geo/geo_types.cpp /^GeoParseStatus GeoPoint::from_coord(const GeoCoordinate& coord) {$/;" f class:doris::GeoPoint +from_coord src/geo/geo_types.cpp /^GeoParseStatus GeoPoint::from_coord(double x, double y) {$/;" f class:doris::GeoPoint +from_coords src/geo/geo_types.cpp /^GeoParseStatus GeoLine::from_coords(const GeoCoordinateList& list) {$/;" f class:doris::GeoLine +from_coords src/geo/geo_types.cpp /^GeoParseStatus GeoPolygon::from_coords(const GeoCoordinateListList& list) {$/;" f class:doris::GeoPolygon +from_date_daynr src/runtime/datetime_value.cpp /^bool DateTimeValue::from_date_daynr(uint64_t daynr) {$/;" f class:doris::DateTimeValue +from_date_format_str src/runtime/datetime_value.cpp /^bool DateTimeValue::from_date_format_str($/;" f class:doris::DateTimeValue +from_date_format_str src/runtime/datetime_value.h /^ bool from_date_format_str(const char* format, int format_len,$/;" f class:doris::DateTimeValue +from_date_int64 src/runtime/datetime_value.cpp /^bool DateTimeValue::from_date_int64(int64_t value) {$/;" f class:doris::DateTimeValue +from_date_str src/runtime/datetime_value.cpp /^bool DateTimeValue::from_date_str(const char* date_str, int len) {$/;" f class:doris::DateTimeValue +from_datetime_val src/runtime/datetime_value.h /^ static DateTimeValue from_datetime_val(const doris_udf::DateTimeVal& tv) {$/;" f class:doris::DateTimeValue +from_days src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::from_days($/;" f class:doris::TimestampFunctions +from_decimal_val src/runtime/decimal_value.h /^ static DecimalValue from_decimal_val(const doris_udf::DecimalVal& val) {$/;" f class:doris::DecimalValue +from_decimal_val src/runtime/decimalv2_value.h /^ static DecimalV2Value from_decimal_val(const DecimalV2Val& val) {$/;" f class:doris::DecimalV2Value +from_encoded src/geo/geo_types.cpp /^GeoShape* GeoShape::from_encoded(const void* ptr, size_t size) {$/;" f class:doris::GeoShape +from_hex src/util/uid_util.h /^inline void from_hex(T* ret, const std::string& buf) {$/;" f namespace:doris +from_olap_date src/runtime/datetime_value.h /^ bool from_olap_date(uint64_t date) { $/;" f class:doris::DateTimeValue +from_olap_datetime src/runtime/datetime_value.h /^ bool from_olap_datetime(uint64_t datetime) {$/;" f class:doris::DateTimeValue +from_olap_decimal src/runtime/decimalv2_value.h /^ inline bool from_olap_decimal(int64_t int_value, int64_t frac_value) {$/;" f class:doris::DecimalV2Value +from_packed_time src/runtime/datetime_value.h /^ void from_packed_time(int64_t packed_time) {$/;" f class:doris::DateTimeValue +from_protobuf src/runtime/types.h /^ static TypeDescriptor from_protobuf(const PTypeDesc& ptype) {$/;" f struct:doris::TypeDescriptor +from_string src/exprs/anyval_util.h /^ static StringVal from_string(FunctionContext* ctx, const std::string& s) {$/;" f class:doris::AnyValUtil +from_string src/olap/decimal12.h /^ OLAPStatus from_string(const std::string& str) {$/;" f struct:doris::decimal12_t +from_string src/olap/field.h /^ inline OLAPStatus from_string(char* buf, const std::string& value_string) const {$/;" f class:doris::Field +from_string src/olap/types.h /^ OLAPStatus from_string(void* buf, const std::string& scan_key) const {$/;" f class:doris::TypeInfo +from_string src/olap/types.h /^ static OLAPStatus from_string(void* buf, const std::string& scan_key) {$/;" f struct:doris::BaseFieldtypeTraits +from_string src/olap/types.h /^ static OLAPStatus from_string(void* buf, const std::string& scan_key) {$/;" f struct:doris::FieldTypeTraits +from_string src/olap/wrapper_field.h /^ OLAPStatus from_string(const std::string& value_string) {$/;" f class:doris::WrapperField +from_string_temp src/exprs/anyval_util.h /^ static StringVal from_string_temp(FunctionContext* ctx, const std::string& s) {$/;" f class:doris::AnyValUtil +from_string_val src/runtime/string_value.h /^ static StringValue from_string_val(const doris_udf::StringVal& sv) {$/;" f struct:doris::StringValue +from_thrift src/exprs/arithmetic_expr.cpp /^Expr* ArithmeticExpr::from_thrift(const TExprNode& node) {$/;" f class:doris::ArithmeticExpr +from_thrift src/exprs/binary_predicate.cpp /^Expr* BinaryPredicate::from_thrift(const TExprNode& node) {$/;" f class:doris::BinaryPredicate +from_thrift src/exprs/cast_expr.cpp /^Expr* CastExpr::from_thrift(const TExprNode& node) {$/;" f class:doris::CastExpr +from_thrift src/runtime/data_spliter.cpp /^Status DataSpliter::from_thrift($/;" f class:doris::DataSpliter +from_thrift src/runtime/dpp_sink_internal.cpp /^Status PartRange::from_thrift($/;" f class:doris::PartRange +from_thrift src/runtime/dpp_sink_internal.cpp /^Status PartRangeKey::from_thrift($/;" f class:doris::PartRangeKey +from_thrift src/runtime/dpp_sink_internal.cpp /^Status PartitionInfo::from_thrift($/;" f class:doris::PartitionInfo +from_thrift src/runtime/dpp_sink_internal.cpp /^Status RollupSchema::from_thrift($/;" f class:doris::RollupSchema +from_thrift src/runtime/types.h /^ static TypeDescriptor from_thrift(const TTypeDesc& t) {$/;" f struct:doris::TypeDescriptor +from_time_int64 src/runtime/datetime_value.cpp /^bool DateTimeValue::from_time_int64(int64_t value) {$/;" f class:doris::DateTimeValue +from_tuple src/olap/row_cursor.cpp /^OLAPStatus RowCursor::from_tuple(const OlapTuple& tuple) {$/;" f class:doris::RowCursor +from_unix src/exprs/timestamp_functions.cpp /^StringVal TimestampFunctions::from_unix($/;" f class:doris::TimestampFunctions +from_unixtime src/runtime/datetime_value.cpp /^bool DateTimeValue::from_unixtime(int64_t timestamp, const std::string& timezone) {$/;" f class:doris::DateTimeValue +from_value src/runtime/dpp_sink_internal.cpp /^Status PartRangeKey::from_value($/;" f class:doris::PartRangeKey +from_wkt src/geo/geo_types.cpp /^GeoShape* GeoShape::from_wkt(const char* data, size_t size, GeoParseStatus* status) {$/;" f class:doris::GeoShape +frontend_client_cache src/runtime/exec_env.h /^ ClientCache* frontend_client_cache() { return _frontend_client_cache; }$/;" f class:doris::ExecEnv +fs src/olap/fs/block_manager.cpp /^namespace fs {$/;" n namespace:doris file: +fs src/olap/fs/block_manager.h /^namespace fs {$/;" n namespace:doris +fs src/olap/fs/block_manager_metrics.cpp /^namespace fs {$/;" n namespace:doris file: +fs src/olap/fs/block_manager_metrics.h /^namespace fs {$/;" n namespace:doris +fs src/olap/fs/file_block_manager.cpp /^namespace fs {$/;" n namespace:doris file: +fs src/olap/fs/file_block_manager.h /^namespace fs {$/;" n namespace:doris +fs src/olap/fs/fs_util.cpp /^namespace fs {$/;" n namespace:doris file: +fs src/olap/fs/fs_util.h /^namespace fs {$/;" n namespace:doris +fs src/olap/rowset/beta_rowset_writer.h /^namespace fs {$/;" n namespace:doris +fs src/olap/rowset/segment_v2/bitmap_index_writer.h /^namespace fs {$/;" n namespace:doris +fs src/olap/rowset/segment_v2/bloom_filter_index_writer.h /^namespace fs {$/;" n namespace:doris +fs src/olap/rowset/segment_v2/column_writer.h /^namespace fs {$/;" n namespace:doris +fs src/olap/rowset/segment_v2/indexed_column_writer.h /^namespace fs {$/;" n namespace:doris +fs src/olap/rowset/segment_v2/ordinal_page_index.h /^namespace fs {$/;" n namespace:doris +fs src/olap/rowset/segment_v2/page_io.h /^namespace fs {$/;" n namespace:doris +fs src/olap/rowset/segment_v2/segment_writer.h /^namespace fs {$/;" n namespace:doris +fs src/olap/rowset/segment_v2/zone_map_index.h /^namespace fs {$/;" n namespace:doris +fs_util src/olap/fs/fs_util.cpp /^namespace fs_util {$/;" n namespace:doris::fs file: +fs_util src/olap/fs/fs_util.h /^namespace fs_util {$/;" n namespace:doris::fs +fseek64_file_func src/util/minizip/ioapi.c /^static long ZCALLBACK fseek64_file_func (voidpf opaque, voidpf stream, ZPOS64_T offset, int origin)$/;" f file: +fseek_file_func src/util/minizip/ioapi.c /^static long ZCALLBACK fseek_file_func (voidpf opaque, voidpf stream, uLong offset, int origin)$/;" f file: +fseeko64 src/util/minizip/ioapi.h 51;" d +fseeko64 src/util/minizip/ioapi.h 56;" d +fseeko64 src/util/minizip/ioapi.h 62;" d +fseeko64 src/util/minizip/ioapi.h 65;" d +ftell64_file_func src/util/minizip/ioapi.c /^static ZPOS64_T ZCALLBACK ftell64_file_func (voidpf opaque, voidpf stream)$/;" f file: +ftell_file_func src/util/minizip/ioapi.c /^static long ZCALLBACK ftell_file_func (voidpf opaque, voidpf stream)$/;" f file: +ftello64 src/util/minizip/ioapi.h 50;" d +ftello64 src/util/minizip/ioapi.h 55;" d +ftello64 src/util/minizip/ioapi.h 61;" d +ftello64 src/util/minizip/ioapi.h 64;" d +full_encode_ascending src/olap/field.h /^ void full_encode_ascending(const void* value, std::string* buf) const {$/;" f class:doris::Field +full_encode_ascending src/olap/key_coder.h /^ static void full_encode_ascending(const void* value, std::string* buf) {$/;" f class:doris::KeyCoderTraits +full_encode_ascending src/olap/key_coder.h /^ void full_encode_ascending(const void* value, std::string* buf) const {$/;" f class:doris::KeyCoder +full_name src/olap/tablet.h /^inline const std::string Tablet::full_name() const {$/;" f class:doris::Tablet +full_name src/olap/tablet_meta.cpp /^string TabletMeta::full_name() const {$/;" f class:doris::TabletMeta +full_scan test/exec/hash_table_test.cpp /^ void full_scan(HashTable* table, int min, int max, bool all_unique,$/;" f class:doris::HashTableTest +fullrune src/gutil/utf/rune.c /^fullrune(const char *str, int n)$/;" f +func_name src/exec/es/es_predicate.h /^ const std::string& func_name;$/;" m struct:doris::ExtFunction +func_name src/util/bfd_parser.cpp /^ const char* func_name;$/;" m struct:doris::BfdFindCtx file: +function src/exprs/like_predicate.h /^ LikePredicateFunction function;$/;" m struct:doris::LikePredicate::LikePredicateState +function_id src/runtime/user_function_cache.cpp /^ int64_t function_id = 0;$/;" m struct:doris::UserFunctionCacheEntry file: +futex_private_flag src/gutil/spinlock_linux-inl.h /^static int futex_private_flag = FUTEX_PRIVATE_FLAG;$/;" v +future src/runtime/stream_load/stream_load_context.h /^ std::future future = promise.get_future();$/;" m class:doris::StreamLoadContext +fwrite_file_func src/util/minizip/ioapi.c /^static uLong ZCALLBACK fwrite_file_func (voidpf opaque, voidpf stream, const void* buf, uLong size)$/;" f file: +g1_ src/gutil/stl_util.h /^ G1 g1_;$/;" m class:BinaryComposeBinary +g2_ src/gutil/stl_util.h /^ G2 g2_;$/;" m class:BinaryComposeBinary +g_bshuf_compress_lz4 src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp /^decltype(&bshuf_compress_lz4) g_bshuf_compress_lz4;$/;" m namespace:doris::bitshuffle::__anon51 file: +g_bshuf_compress_lz4_bound src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp /^decltype(&bshuf_compress_lz4_bound) g_bshuf_compress_lz4_bound;$/;" m namespace:doris::bitshuffle::__anon51 file: +g_bshuf_decompress_lz4 src/olap/rowset/segment_v2/bitshuffle_wrapper.cpp /^decltype(&bshuf_decompress_lz4) g_bshuf_decompress_lz4;$/;" m namespace:doris::bitshuffle::__anon51 file: +g_cond test/util/counter_cond_variable_test.cpp /^CounterCondVariable g_cond;$/;" m namespace:doris file: +g_download_path test/runtime/small_file_mgr_test.cpp /^std::string g_download_path = ".\/be\/test\/runtime\/test_data\/small_file\/downloaded\/";$/;" m namespace:doris file: +g_file_12345 test/runtime/small_file_mgr_test.cpp /^int64_t g_file_12345 = 12345L; \/\/ ready to be downloaded file$/;" m namespace:doris file: +g_file_67890 test/runtime/small_file_mgr_test.cpp /^int64_t g_file_67890 = 67890L; \/\/ already exist file$/;" m namespace:doris file: +g_io_mu test/util/counter_cond_variable_test.cpp /^std::mutex g_io_mu;$/;" m namespace:doris file: +g_md5_12345 test/runtime/small_file_mgr_test.cpp /^std::string g_md5_12345 = "5dd39cab1c53c2c77cd352983f9641e1";$/;" m namespace:doris file: +g_md5_67890 test/runtime/small_file_mgr_test.cpp /^std::string g_md5_67890 = "a06e26ae5511b0acea8273cf180ca7bf";$/;" m namespace:doris file: +g_power_table src/olap/utils.h /^const static int32_t g_power_table[] = {$/;" m namespace:doris +g_schema_change_active_threads src/olap/olap_server.cpp /^volatile uint32_t g_schema_change_active_threads = 0;$/;" m namespace:doris file: +g_src_path test/runtime/small_file_mgr_test.cpp /^std::string g_src_path = ".\/be\/test\/runtime\/test_data\/small_file\/source\/";$/;" m namespace:doris file: +g_thread_counters test/util/thread_pool_test.cpp /^int g_thread_counters[NUM_THREADS];$/;" m namespace:doris file: +g_thread_mutexes test/util/thread_pool_test.cpp /^boost::mutex g_thread_mutexes[NUM_THREADS];$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_authors_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_collations_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_columns_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_engines_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_tables_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner/schema_variables_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +g_tuple_buf test/exec/schema_scanner_test.cpp /^char g_tuple_buf[10000];\/\/ enougth for tuple$/;" m namespace:doris file: +gap_width src/olap/rowset/run_length_integer_writer.h /^ gap_width: 3;$/;" m struct:doris::RunLengthIntegerWriter::PatchedBaseHead +gbswap_128 src/gutil/endian.h /^inline unsigned __int128 gbswap_128(unsigned __int128 host_int) {$/;" f +gbswap_64 src/gutil/endian.h /^inline uint64 gbswap_64(uint64 host_int) {$/;" f +gc_expired_context src/runtime/external_scan_context_mgr.cpp /^void ExternalScanContextMgr::gc_expired_context() {$/;" f class:doris::ExternalScanContextMgr +gc_io_buffers src/runtime/disk_io_mgr.cc /^void DiskIoMgr::gc_io_buffers() {$/;" f class:doris::DiskIoMgr +gc_unused_rowsets src/olap/compaction.cpp /^OLAPStatus Compaction::gc_unused_rowsets() {$/;" f class:doris::Compaction +ge src/runtime/string_value.h /^ bool ge(const StringValue& other) const {$/;" f struct:doris::StringValue +gen test/olap/skiplist_test.cpp /^ static uint64_t gen(Key key) { return (key >> 8) & 0xffffffffu; }$/;" f class:doris::ConcurrentTest file: +gen_file_name src/runtime/export_sink.cpp /^std::string ExportSink::gen_file_name() {$/;" f class:doris::ExportSink +gen_row_buffer src/runtime/export_sink.cpp /^Status ExportSink::gen_row_buffer(TupleRow* row, std::stringstream* ss) {$/;" f class:doris::ExportSink +gen_sql src/util/mysql_load_error_hub.cpp /^Status MysqlLoadErrorHub::gen_sql(MYSQL* my_conn,$/;" f class:doris::MysqlLoadErrorHub +gen_timestamp_string src/olap/utils.cpp /^OLAPStatus gen_timestamp_string(string* out_string) {$/;" f namespace:doris +gen_type_desc src/runtime/primitive_type.cpp /^TTypeDesc gen_type_desc(const TPrimitiveType::type val) {$/;" f namespace:doris +gen_type_desc src/runtime/primitive_type.cpp /^TTypeDesc gen_type_desc(const TPrimitiveType::type val, const std::string& name) {$/;" f namespace:doris +gen_uid src/util/uid_util.h /^ static UniqueId gen_uid() {$/;" f struct:doris::UniqueId +generate_check_load_req src/http/action/mini_load.cpp /^Status MiniLoadAction::generate_check_load_req($/;" f class:doris::MiniLoadAction +generate_delete_predicate src/olap/delete_handler.cpp /^OLAPStatus DeleteConditionHandler::generate_delete_predicate($/;" f class:doris::DeleteConditionHandler +generate_str test/util/block_compression_test.cpp /^static std::string generate_str(size_t len) {$/;" f namespace:doris +generate_tablet_meta_copy src/olap/tablet.cpp /^OLAPStatus Tablet::generate_tablet_meta_copy(TabletMetaSharedPtr new_tablet_meta) {$/;" f class:doris::Tablet +generate_uuid src/util/uid_util.h /^inline TUniqueId generate_uuid() {$/;" f namespace:doris +generate_uuid_string src/util/uid_util.h /^inline std::string generate_uuid_string() {$/;" f namespace:doris +generation test/olap/skiplist_test.cpp /^ std::atomic generation[K];$/;" m struct:doris::ConcurrentTest::State file: +get src/common/configbase.cpp /^bool Properties::get(const char* key, const char* defstr, T& retval) const {$/;" f class:doris::config::Properties +get src/gutil/gscoped_ptr.h /^ C* get() const {$/;" f class:gscoped_array +get src/gutil/gscoped_ptr.h /^ C* get() const {$/;" f class:gscoped_ptr_malloc +get src/gutil/gscoped_ptr.h /^ T* get() const { return data_.ptr; }$/;" f class:doris::internal::gscoped_ptr_impl +get src/gutil/gscoped_ptr.h /^ element_type* get() const { return impl_.get(); }$/;" f class:gscoped_ptr +get src/gutil/ref_counted.h /^ T* get() const { return ptr_; }$/;" f class:scoped_refptr +get src/olap/bloom_filter.hpp /^ bool get(uint32_t index) const {$/;" f class:doris::BitSet +get src/olap/byte_buffer.h /^ inline OLAPStatus get(char* dst, uint64_t dst_size) {$/;" f class:doris::StorageByteBuffer +get src/olap/byte_buffer.h /^ inline OLAPStatus get(char* dst, uint64_t dst_size, uint64_t length) {$/;" f class:doris::StorageByteBuffer +get src/olap/byte_buffer.h /^ inline OLAPStatus get(char* result) {$/;" f class:doris::StorageByteBuffer +get src/olap/byte_buffer.h /^ inline OLAPStatus get(uint64_t index, char* result) {$/;" f class:doris::StorageByteBuffer +get src/olap/olap_meta.cpp /^OLAPStatus OlapMeta::get(int column_family_index, const std::string& key, std::string* value) {$/;" f class:doris::OlapMeta +get src/olap/rowset/column_writer.h /^ const std::string& get() const {$/;" f class:doris::VarStringColumnWriter::DictKey +get src/olap/rowset/segment_v2/encoding_info.cpp /^Status EncodingInfo::get(const TypeInfo* type_info,$/;" f class:doris::segment_v2::EncodingInfo +get src/olap/rowset/segment_v2/encoding_info.cpp /^Status EncodingInfoResolver::get($/;" f class:doris::segment_v2::EncodingInfoResolver +get src/runtime/row_batch.h /^ TupleRow* IR_ALWAYS_INLINE get() { return reinterpret_cast(_row); }$/;" f class:doris::RowBatch::Iterator +get src/runtime/stream_load/load_stream_mgr.h /^ std::shared_ptr get(const UniqueId& id) {$/;" f class:doris::LoadStreamMgr +get src/util/frame_of_reference_coding.h /^ bool get(T* val) {$/;" f class:doris::ForDecoder +get src/util/lru_cache.hpp /^ bool get(const Key& key, Value* value) {$/;" f class:doris::LruCache +get test/olap/skiplist_test.cpp /^ int get(int k) { return generation[k].load(std::memory_order_acquire); }$/;" f struct:doris::ConcurrentTest::State +getCopyOnWrite src/util/bitmap_value.h /^ bool getCopyOnWrite() const { return copyOnWrite; }$/;" f class:doris::detail::Roaring64Map +getPart src/util/radix_sort.h /^ static ALWAYS_INLINE KeyBits getPart(size_t N, KeyBits x) {$/;" f struct:doris::RadixSort +getSizeInBytes src/util/bitmap_value.h /^ size_t getSizeInBytes() const {$/;" f class:doris::detail::Roaring64Map +getSizeInBytes src/util/bitmap_value.h /^ size_t getSizeInBytes() {$/;" f class:doris::BitmapValue +get_absolute_offset src/olap/olap_index.h /^ const iterator_offset_t get_absolute_offset(const OLAPIndexOffset& offset) const {$/;" f class:doris::MemIndex +get_absolute_shard_path src/olap/data_dir.cpp /^std::string DataDir::get_absolute_shard_path(const std::string& shard_string) {$/;" f class:doris::DataDir +get_absolute_tablet_path src/olap/data_dir.cpp /^std::string DataDir::get_absolute_tablet_path(const T& tablet_meta, bool with_schema_hash) {$/;" f class:doris::DataDir +get_aggregate_info src/olap/aggregate_func.cpp /^ const AggregateInfo* get_aggregate_info(const FieldAggregationMethod agg_method,$/;" f class:doris::AggregateFuncResolver +get_aggregate_info src/olap/aggregate_func.cpp /^const AggregateInfo* get_aggregate_info(const FieldAggregationMethod agg_method,$/;" f namespace:doris +get_aggregation_type_by_string src/olap/tablet_schema.cpp /^FieldAggregationMethod TabletColumn::get_aggregation_type_by_string(const std::string& str) {$/;" f class:doris::TabletColumn +get_all_children src/util/runtime_profile.cpp /^void RuntimeProfile::get_all_children(std::vector* children) {$/;" f class:doris::RuntimeProfile +get_all_data_dir_info src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::get_all_data_dir_info(vector* data_dir_infos,$/;" f class:doris::StorageEngine +get_all_related_tablets src/olap/txn_manager.cpp /^void TxnManager::get_all_related_tablets(std::set* tablet_infos) {$/;" f class:doris::TxnManager +get_allocator src/util/core_local.cpp /^CoreDataAllocator* CoreDataAllocatorFactory::get_allocator(size_t cpu_idx, size_t data_bytes) {$/;" f class:doris::CoreDataAllocatorFactory +get_arg_type src/udf/udf.cpp /^const FunctionContext::TypeDesc* FunctionContext::get_arg_type(int arg_idx) const {$/;" f class:doris_udf::FunctionContext +get_base_lock src/olap/tablet.h /^ inline Mutex* get_base_lock() { return &_base_lock; }$/;" f class:doris::Tablet +get_batch src/exec/olap_scanner.cpp /^Status OlapScanner::get_batch($/;" f class:doris::OlapScanner +get_batch src/runtime/buffer_control_block.cpp /^Status BufferControlBlock::get_batch(TFetchDataResult* result) {$/;" f class:doris::BufferControlBlock +get_batch src/runtime/buffer_control_block.cpp /^void BufferControlBlock::get_batch(GetResultBatchCtx* ctx) {$/;" f class:doris::BufferControlBlock +get_batch src/runtime/data_stream_recvr.cc /^Status DataStreamRecvr::SenderQueue::get_batch(RowBatch** next_batch) {$/;" f class:doris::DataStreamRecvr::SenderQueue +get_batch src/runtime/data_stream_recvr.cc /^Status DataStreamRecvr::get_batch(RowBatch** next_batch) {$/;" f class:doris::DataStreamRecvr +get_batch src/runtime/data_stream_recvr.hpp /^ RowBatch* get_batch(bool* is_cancelled) {$/;" f class:doris::DataStreamRecvr +get_batch src/util/frame_of_reference_coding.cpp /^bool ForDecoder::get_batch(T* val, size_t count) {$/;" f class:doris::ForDecoder +get_batch_size src/runtime/row_batch.cpp /^int RowBatch::get_batch_size(const PRowBatch& batch) {$/;" f class:doris::RowBatch +get_batch_size src/runtime/row_batch.cpp /^int RowBatch::get_batch_size(const TRowBatch& batch) {$/;" f class:doris::RowBatch +get_big_int_val src/exprs/expr.cpp /^BigIntVal Expr::get_big_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_big_int_val src/exprs/expr_context.cpp /^BigIntVal ExprContext::get_big_int_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_big_int_val src/exprs/expr_ir.cpp /^BigIntVal Expr::get_big_int_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_big_int_val src/exprs/info_func.cpp /^BigIntVal InfoFunc::get_big_int_val(ExprContext* context, TupleRow*) {$/;" f class:doris::InfoFunc +get_big_int_val src/exprs/literal.cpp /^BigIntVal Literal::get_big_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_big_int_val src/exprs/null_literal.cpp /^BigIntVal NullLiteral::get_big_int_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_big_int_val src/exprs/scalar_fn_call.cpp /^BigIntVal ScalarFnCall::get_big_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_big_int_val src/exprs/slot_ref.cpp /^BigIntVal SlotRef::get_big_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_bitmap_reader_iter test/olap/rowset/segment_v2/bitmap_index_test.cpp /^void get_bitmap_reader_iter(std::string& file_name, const ColumnIndexMetaPB& meta,$/;" f namespace:doris::segment_v2 +get_block src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::get_block($/;" f class:doris::SegmentReader +get_block_compression_codec src/util/block_compression.cpp /^Status get_block_compression_codec($/;" f namespace:doris +get_block_size src/exec/decompressor.cpp /^size_t Lz4FrameDecompressor::get_block_size(const LZ4F_frameInfo_t* info) {$/;" f class:doris::Lz4FrameDecompressor +get_bloom_filter_info src/olap/rowset/column_writer.cpp /^void ColumnWriter::get_bloom_filter_info(bool* has_bf_column,$/;" f class:doris::ColumnWriter +get_bloom_filter_reader_iter test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^void get_bloom_filter_reader_iter(const std::string& file_name, const ColumnIndexMetaPB& meta,$/;" f namespace:doris::segment_v2 +get_bool src/exec/es/es_predicate.cpp /^bool ExtLiteral::get_bool() {$/;" f class:doris::ExtLiteral +get_boolean_val src/exprs/binary_predicate.cpp /^BooleanVal EqForNullStringValPred::get_boolean_val(ExprContext* ctx, TupleRow* row) {$/;" f class:doris::EqForNullStringValPred +get_boolean_val src/exprs/binary_predicate.cpp /^BooleanVal EqStringValPred::get_boolean_val(ExprContext* ctx, TupleRow* row) {$/;" f class:doris::EqStringValPred +get_boolean_val src/exprs/compound_predicate.cpp /^BooleanVal AndPredicate::get_boolean_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::AndPredicate +get_boolean_val src/exprs/compound_predicate.cpp /^BooleanVal NotPredicate::get_boolean_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::NotPredicate +get_boolean_val src/exprs/compound_predicate.cpp /^BooleanVal OrPredicate::get_boolean_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::OrPredicate +get_boolean_val src/exprs/expr.cpp /^BooleanVal Expr::get_boolean_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_boolean_val src/exprs/expr_context.cpp /^BooleanVal ExprContext::get_boolean_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_boolean_val src/exprs/expr_ir.cpp /^BooleanVal Expr::get_boolean_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_boolean_val src/exprs/in_predicate.cpp /^BooleanVal InPredicate::get_boolean_val(ExprContext* ctx, TupleRow* row) {$/;" f class:doris::InPredicate +get_boolean_val src/exprs/literal.cpp /^BooleanVal Literal::get_boolean_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_boolean_val src/exprs/null_literal.cpp /^BooleanVal NullLiteral::get_boolean_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_boolean_val src/exprs/scalar_fn_call.cpp /^BooleanVal ScalarFnCall::get_boolean_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_boolean_val src/exprs/slot_ref.cpp /^BooleanVal SlotRef::get_boolean_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_boolean_val src/exprs/tuple_is_null_predicate.cpp /^BooleanVal TupleIsNullPredicate::get_boolean_val(ExprContext* ctx, TupleRow* row) {$/;" f class:doris::TupleIsNullPredicate +get_buf src/olap/file_stream.h /^ inline void get_buf(char** buf, uint32_t* remaining_bytes) {$/;" f class:doris::ReadOnlyFileStream +get_buf src/olap/row_cursor.h /^ inline char* get_buf() const { return _fixed_buf; }$/;" f class:doris::RowCursor +get_buffer src/util/thrift_util.h /^ void get_buffer(uint8_t** buffer, uint32_t* length) {$/;" f class:doris::ThriftSerializer +get_buffer_desc src/runtime/disk_io_mgr.cc /^DiskIoMgr::BufferDescriptor* DiskIoMgr::get_buffer_desc($/;" f class:doris::DiskIoMgr +get_buffer_size src/olap/file_stream.h /^ size_t get_buffer_size() {$/;" f class:doris::ReadOnlyFileStream +get_buffer_size src/olap/rowset/column_reader.h /^ size_t get_buffer_size() {$/;" f class:doris::StringColumnDictionaryReader +get_buffer_size src/olap/rowset/column_reader.h /^ size_t get_buffer_size() {$/;" f class:doris::StringColumnDirectReader +get_buffer_size src/olap/rowset/column_reader.h /^ virtual size_t get_buffer_size() {$/;" f class:doris::ColumnReader +get_buffer_size src/olap/rowset/column_reader.h /^ virtual size_t get_buffer_size() {$/;" f class:doris::DecimalColumnReader +get_buffer_size src/olap/rowset/column_reader.h /^ virtual size_t get_buffer_size() {$/;" f class:doris::FixLengthStringColumnReader +get_buffer_size src/olap/rowset/column_reader.h /^ virtual size_t get_buffer_size() {$/;" f class:doris::IntegerColumnReaderWrapper +get_buffer_size src/olap/rowset/column_reader.h /^ virtual size_t get_buffer_size() {$/;" f class:doris::LargeIntColumnReader +get_buffer_size src/olap/rowset/column_reader.h /^ virtual size_t get_buffer_size() {$/;" f class:doris::TinyColumnReader +get_buffer_size src/olap/rowset/column_reader.h /^ virtual size_t get_buffer_size() {$/;" f class:doris::VarStringColumnReader +get_build_version src/util/debug_util.cpp /^std::string get_build_version(bool compact) {$/;" f namespace:doris +get_byte src/exec/es/es_predicate.cpp /^int8_t ExtLiteral::get_byte() {$/;" f class:doris::ExtLiteral +get_byte_size src/runtime/primitive_type.h /^inline int get_byte_size(PrimitiveType type) {$/;" f namespace:doris +get_byte_size src/runtime/types.h /^ inline int get_byte_size() const {$/;" f struct:doris::TypeDescriptor +get_byte_unit src/util/pretty_printer.h /^ static double get_byte_unit(T value, std::string* unit) {$/;" f class:doris::PrettyPrinter +get_bytes src/util/byte_buffer.h /^ void get_bytes(char* data, size_t size) {$/;" f struct:doris::ByteBuffer +get_bytes_points_string src/olap/bloom_filter.hpp /^ std::string get_bytes_points_string(const char* buf, uint32_t len) const {$/;" f class:doris::BloomFilter +get_cache_status src/olap/lru_cache.cpp /^void ShardedLRUCache::get_cache_status(rapidjson::Document* document) {$/;" f class:doris::ShardedLRUCache +get_cache_status src/olap/storage_engine.cpp /^void StorageEngine::get_cache_status(rapidjson::Document* document) const {$/;" f class:doris::StorageEngine +get_capacity src/olap/lru_cache.h /^ size_t get_capacity() {$/;" f class:doris::CachePriority::LRUCache +get_cgroups_version src/agent/cgroups_mgr.h /^ int64_t get_cgroups_version() {$/;" f class:doris::CgroupsMgr +get_child src/exprs/expr.h /^ Expr* get_child(int i) const {$/;" f class:doris::Expr +get_child src/util/path_trie.hpp /^ TrieNode* get_child(const std::string& key) {$/;" f class:doris::PathTrie::TrieNode +get_child_val src/exprs/case_expr.cpp /^void CaseExpr::get_child_val(int child_idx, ExprContext* ctx, TupleRow* row, AnyVal* dst) {$/;" f class:doris::CaseExpr +get_children src/util/runtime_profile.cpp /^void RuntimeProfile::get_children(std::vector* children) {$/;" f class:doris::RuntimeProfile +get_children_count src/util/file_utils.cpp /^Status FileUtils::get_children_count(Env* env, const std::string& dir, int64_t* count) {$/;" f class:doris::FileUtils +get_client src/runtime/client_cache.cpp /^Status ClientCacheHelper::get_client($/;" f class:doris::ClientCacheHelper +get_client src/runtime/client_cache.h /^ Status get_client(const TNetworkAddress& hostport, T** iface, int timeout_ms) {$/;" f class:doris::ClientCache +get_client_cache src/runtime/exec_env.h /^ ClientCache* get_client_cache() { return nullptr; }$/;" f class:doris::ExecEnv +get_client_cache src/runtime/exec_env.h /^inline ClientCache* ExecEnv::get_client_cache() { return _backend_client_cache; }$/;" f class:doris::ExecEnv +get_client_cache src/runtime/exec_env.h /^inline ClientCache* ExecEnv::get_client_cache() { return _frontend_client_cache; }$/;" f class:doris::ExecEnv +get_client_cache src/runtime/exec_env.h /^inline ClientCache* ExecEnv::get_client_cache() { return _extdatasource_client_cache; }$/;" f class:doris::ExecEnv +get_client_cache src/runtime/exec_env.h /^inline ClientCache* ExecEnv::get_client_cache() { return _broker_client_cache; }$/;" f class:doris::ExecEnv +get_client_id src/runtime/broker_mgr.cpp /^const std::string& BrokerMgr::get_client_id(const TNetworkAddress& address) {$/;" f class:doris::BrokerMgr +get_closet_fixed_bits src/olap/serialize.h /^inline uint32_t get_closet_fixed_bits(uint32_t n) {$/;" f namespace:doris::ser +get_coder src/olap/key_coder.cpp /^ KeyCoder* get_coder(FieldType field_type) const {$/;" f class:doris::KeyCoderResolver +get_column src/olap/olap_cond.cpp /^CondColumn* Conditions::get_column(int32_t cid) const {$/;" f class:doris::Conditions +get_common_type src/runtime/descriptor_helper.h /^ TTypeDesc get_common_type(TPrimitiveType::type type) {$/;" f class:doris::TSlotDescriptorBuilder +get_compaction_score src/olap/rowset/rowset_meta.h /^ uint32_t get_compaction_score() const {$/;" f class:doris::RowsetMeta +get_compaction_status src/olap/tablet.cpp /^OLAPStatus Tablet::get_compaction_status(std::string* json_result) {$/;" f class:doris::Tablet +get_compare_func src/exec/olap_utils.h /^inline CompareLargeFunc get_compare_func(PrimitiveType type) {$/;" f namespace:doris +get_conds_version src/olap/delete_handler.cpp /^vector DeleteHandler::get_conds_version() {$/;" f class:doris::DeleteHandler +get_conjunct_ctxs src/exec/kudu_scan_node.cpp /^Status KuduScanNode::get_conjunct_ctxs(vector* ctxs) {$/;" f class:doris::KuduScanNode +get_const_val src/exprs/expr.cpp /^doris_udf::AnyVal* Expr::get_const_val(ExprContext* context) {$/;" f class:doris::Expr +get_const_value src/exprs/expr_context.cpp /^Status ExprContext::get_const_value(RuntimeState* state, Expr& expr,$/;" f class:doris::ExprContext +get_constant_arg src/udf/udf_ir.cpp /^AnyVal* FunctionContext::get_constant_arg(int i) const {$/;" f class:doris_udf::FunctionContext +get_consumer src/runtime/routine_load/data_consumer_pool.cpp /^Status DataConsumerPool::get_consumer($/;" f class:doris::DataConsumerPool +get_consumer_grp src/runtime/routine_load/data_consumer_pool.cpp /^Status DataConsumerPool::get_consumer_grp($/;" f class:doris::DataConsumerPool +get_content_length src/http/http_client.h /^ int64_t get_content_length() const {$/;" f class:doris::HttpClient +get_content_type src/http/download_action.cpp /^std::string DownloadAction::get_content_type(const std::string& file_name) {$/;" f class:doris::DownloadAction +get_convertible_fixed_value_size src/exec/olap_common.h /^size_t ColumnValueRange::get_convertible_fixed_value_size() const {$/;" f class:doris::ColumnValueRange +get_cores_of_numa_node src/util/cpu_info.h /^ static const std::vector& get_cores_of_numa_node(int node) {$/;" f class:doris::CpuInfo +get_cores_of_same_numa_node src/util/cpu_info.h /^ static const std::vector& get_cores_of_same_numa_node(int core) {$/;" f class:doris::CpuInfo +get_counter src/util/runtime_profile.cpp /^RuntimeProfile::Counter* RuntimeProfile::get_counter(const std::string& name) {$/;" f class:doris::RuntimeProfile +get_counter_name src/util/perf_counters.cpp /^static string get_counter_name(PerfCounters::Counter counter) {$/;" f namespace:doris +get_counters src/util/runtime_profile.cpp /^void RuntimeProfile::get_counters(const std::string& name, std::vector* counters) {$/;" f class:doris::RuntimeProfile +get_cumulative_lock src/olap/tablet.h /^ inline Mutex* get_cumulative_lock() { return &_cumulative_lock; }$/;" f class:doris::Tablet +get_current_core src/util/cpu_info.cpp /^int CpuInfo::get_current_core() {$/;" f class:doris::CpuInfo +get_current_ordinal src/olap/rowset/segment_v2/indexed_column_reader.h /^ ordinal_t get_current_ordinal() const {$/;" f class:doris::segment_v2::IndexedColumnIterator +get_data src/runtime/tuple.h /^ void* get_data() { return this; }$/;" f class:doris::Tuple +get_data src/util/slice.h /^ const char* get_data() const { return data; }$/;" f struct:doris::Slice +get_data_sink test/exec/tablet_sink_test.cpp /^TDataSink get_data_sink(TDescriptorTable* desc_tbl) {$/;" f namespace:doris::stream_load +get_date_from_daynr src/runtime/datetime_value.cpp /^bool DateTimeValue::get_date_from_daynr(uint64_t daynr) {$/;" f class:doris::DateTimeValue +get_date_string src/exec/es/es_predicate.cpp /^std::string ExtLiteral::get_date_string() {$/;" f class:doris::ExtLiteral +get_datetime_slot src/runtime/tuple.h /^ DateTimeValue* get_datetime_slot(int offset) {$/;" f class:doris::Tuple +get_datetime_val src/exprs/expr.cpp /^DateTimeVal Expr::get_datetime_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_datetime_val src/exprs/expr_context.cpp /^DateTimeVal ExprContext::get_datetime_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_datetime_val src/exprs/expr_ir.cpp /^DateTimeVal Expr::get_datetime_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_datetime_val src/exprs/literal.cpp /^DateTimeVal Literal::get_datetime_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_datetime_val src/exprs/null_literal.cpp /^DateTimeVal NullLiteral::get_datetime_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_datetime_val src/exprs/scalar_fn_call.cpp /^DateTimeVal ScalarFnCall::get_datetime_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_datetime_val src/exprs/slot_ref.cpp /^DateTimeVal SlotRef::get_datetime_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_db_names src/exec/schema_scanner/schema_helper.cpp /^Status SchemaHelper::get_db_names($/;" f class:doris::SchemaHelper +get_db_names test/exec/schema_scanner/schema_columns_scanner_test.cpp /^Status SchemaJniHelper::get_db_names(const TGetDbsParams &db_params,$/;" f class:doris::SchemaJniHelper +get_db_names test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^Status SchemaJniHelper::get_db_names(const TGetDbsParams &db_params,$/;" f class:doris::SchemaJniHelper +get_db_names test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^Status SchemaJniHelper::get_db_names(const TGetDbsParams &db_params,$/;" f class:doris::SchemaJniHelper +get_db_names test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^Status SchemaJniHelper::get_db_names(const TGetDbsParams &db_params,$/;" f class:doris::SchemaJniHelper +get_db_names test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^Status SchemaJniHelper::get_db_names(const TGetDbsParams &db_params,$/;" f class:doris::SchemaJniHelper +get_db_names test/exec/schema_scanner/schema_tables_scanner_test.cpp /^Status SchemaJniHelper::get_db_names(const TGetDbsParams &db_params,$/;" f class:doris::SchemaJniHelper +get_debug_info src/runtime/decimal_value.h /^ std::string get_debug_info() const {$/;" f class:doris::DecimalValue +get_debug_info src/runtime/decimalv2_value.h /^ std::string get_debug_info() const {$/;" f class:doris::DecimalV2Value +get_decimal_byte_size src/runtime/types.h /^ static inline int get_decimal_byte_size(int precision) {$/;" f struct:doris::TypeDescriptor +get_decimal_sink test/exec/tablet_sink_test.cpp /^TDataSink get_decimal_sink(TDescriptorTable* desc_tbl) {$/;" f namespace:doris::stream_load +get_decimal_slot src/runtime/tuple.h /^ DecimalValue* get_decimal_slot(int offset) {$/;" f class:doris::Tuple +get_decimal_string src/exec/es/es_predicate.cpp /^std::string ExtLiteral::get_decimal_string() {$/;" f class:doris::ExtLiteral +get_decimal_val src/exprs/expr.cpp /^DecimalVal Expr::get_decimal_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_decimal_val src/exprs/expr_context.cpp /^DecimalVal ExprContext::get_decimal_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_decimal_val src/exprs/expr_ir.cpp /^DecimalVal Expr::get_decimal_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_decimal_val src/exprs/literal.cpp /^DecimalVal Literal::get_decimal_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_decimal_val src/exprs/null_literal.cpp /^DecimalVal NullLiteral::get_decimal_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_decimal_val src/exprs/scalar_fn_call.cpp /^DecimalVal ScalarFnCall::get_decimal_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_decimal_val src/exprs/slot_ref.cpp /^DecimalVal SlotRef::get_decimal_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_decimalv2_slot src/runtime/tuple.h /^ DecimalV2Value* get_decimalv2_slot(int offset) {$/;" f class:doris::Tuple +get_decimalv2_string src/exec/es/es_predicate.cpp /^std::string ExtLiteral::get_decimalv2_string() {$/;" f class:doris::ExtLiteral +get_decimalv2_val src/exprs/expr.cpp /^DecimalV2Val Expr::get_decimalv2_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_decimalv2_val src/exprs/expr_context.cpp /^DecimalV2Val ExprContext::get_decimalv2_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_decimalv2_val src/exprs/expr_ir.cpp /^DecimalV2Val Expr::get_decimalv2_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_decimalv2_val src/exprs/literal.cpp /^DecimalV2Val Literal::get_decimalv2_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_decimalv2_val src/exprs/null_literal.cpp /^DecimalV2Val NullLiteral::get_decimalv2_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_decimalv2_val src/exprs/scalar_fn_call.cpp /^DecimalV2Val ScalarFnCall::get_decimalv2_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_decimalv2_val src/exprs/slot_ref.cpp /^DecimalV2Val SlotRef::get_decimalv2_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_default_encoding src/olap/rowset/segment_v2/encoding_info.cpp /^ EncodingTypePB get_default_encoding(FieldType type, bool optimize_value_seek) const {$/;" f class:doris::segment_v2::EncodingInfoResolver +get_default_encoding src/olap/rowset/segment_v2/encoding_info.cpp /^EncodingTypePB EncodingInfo::get_default_encoding(const TypeInfo* type_info, bool optimize_value_seek) {$/;" f class:doris::segment_v2::EncodingInfo +get_delete_conditions src/olap/delete_handler.h /^ const std::vector& get_delete_conditions() const {$/;" f class:doris::DeleteHandler +get_delete_conditions_after_version src/olap/delete_handler.cpp /^void DeleteHandler::get_delete_conditions_after_version(int32_t version,$/;" f class:doris::DeleteHandler +get_deleter src/gutil/gscoped_ptr.h /^ D& get_deleter() { return data_; }$/;" f class:doris::internal::gscoped_ptr_impl +get_deleter src/gutil/gscoped_ptr.h /^ const D& get_deleter() const { return data_; }$/;" f class:doris::internal::gscoped_ptr_impl +get_deleter src/gutil/gscoped_ptr.h /^ const deleter_type& get_deleter() const { return impl_.get_deleter(); }$/;" f class:gscoped_ptr +get_deleter src/gutil/gscoped_ptr.h /^ deleter_type& get_deleter() { return impl_.get_deleter(); }$/;" f class:gscoped_ptr +get_device_names src/util/disk_info.cpp /^void DiskInfo::get_device_names() {$/;" f class:doris::DiskInfo +get_dict_page_pointer src/olap/rowset/segment_v2/column_reader.h /^ PagePointer get_dict_page_pointer() const { return _meta.dict_page(); }$/;" f class:doris::segment_v2::ColumnReader +get_dictionary_page src/olap/rowset/segment_v2/binary_dict_page.cpp /^Status BinaryDictPageBuilder::get_dictionary_page(OwnedSlice* dictionary_page) {$/;" f class:doris::segment_v2::BinaryDictPageBuilder +get_dictionary_page src/olap/rowset/segment_v2/page_builder.h /^ virtual Status get_dictionary_page(OwnedSlice* dictionary_page) {$/;" f class:doris::segment_v2::PageBuilder +get_dir_info src/olap/data_dir.h /^ DataDirInfo get_dir_info() {$/;" f class:doris::DataDir +get_disjuncts src/exec/es_scan_node.cpp /^bool EsScanNode::get_disjuncts(ExprContext* context, Expr* conjunct,$/;" f class:doris::EsScanNode +get_disk_devices src/util/disk_info.cpp /^Status DiskInfo::get_disk_devices(const std::vector& paths,$/;" f class:doris::DiskInfo +get_disks_io_time src/util/system_metrics.cpp /^void SystemMetrics::get_disks_io_time(std::map* map) {$/;" f class:doris::SystemMetrics +get_distinct_estimate_bit src/exprs/aggregate_functions.cpp /^static inline bool get_distinct_estimate_bit(uint8_t* bitmap,$/;" f namespace:doris +get_double src/exec/es/es_predicate.cpp /^double ExtLiteral::get_double() {$/;" f class:doris::ExtLiteral +get_double_val src/exprs/arithmetic_expr.cpp /^DoubleVal ModExpr::get_double_val(ExprContext* context, TupleRow* row) { $/;" f class:doris::ModExpr +get_double_val src/exprs/expr.cpp /^DoubleVal Expr::get_double_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_double_val src/exprs/expr_context.cpp /^DoubleVal ExprContext::get_double_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_double_val src/exprs/expr_ir.cpp /^DoubleVal Expr::get_double_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_double_val src/exprs/literal.cpp /^DoubleVal Literal::get_double_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_double_val src/exprs/null_literal.cpp /^DoubleVal NullLiteral::get_double_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_double_val src/exprs/scalar_fn_call.cpp /^DoubleVal ScalarFnCall::get_double_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_double_val src/exprs/slot_ref.cpp /^DoubleVal SlotRef::get_double_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_elapse_second src/olap/utils.h /^ double get_elapse_second() {$/;" f class:doris::OlapStopWatch +get_elapse_time_us src/olap/utils.h /^ uint64_t get_elapse_time_us() {$/;" f class:doris::OlapStopWatch +get_entry src/olap/olap_index.cpp /^OLAPStatus MemIndex::get_entry(const OLAPIndexOffset& pos, EntrySlice* slice) const {$/;" f class:doris::MemIndex +get_error src/exprs/expr_context.cpp /^Status ExprContext::get_error(int start_idx, int end_idx) const {$/;" f class:doris::ExprContext +get_error_log_file_path src/runtime/runtime_state.h /^ const std::string get_error_log_file_path() const {$/;" f class:doris::RuntimeState +get_error_msg output/udf/include/uda_test_harness.h /^ const std::string& get_error_msg() const {$/;" f class:doris_udf::UdaTestHarnessBase +get_error_msg src/common/status.h /^ std::string get_error_msg() const {$/;" f class:doris::Status +get_error_msg src/exprs/expr_context.cpp /^std::string ExprContext::get_error_msg() const {$/;" f class:doris::ExprContext +get_error_msg src/udf/uda_test_harness.h /^ const std::string& get_error_msg() const {$/;" f class:doris_udf::UdaTestHarnessBase +get_error_row_number src/runtime/runtime_state.h /^ const int64_t get_error_row_number() const {$/;" f class:doris::RuntimeState +get_es_query_status src/exec/es/es_predicate.h /^ Status get_es_query_status() {$/;" f class:doris::EsPredicate +get_etl_status src/agent/agent_server.cpp /^void AgentServer::get_etl_status(TMiniLoadEtlStatusResult& t_agent_result,$/;" f class:doris::AgentServer +get_etl_status src/service/backend_service.h /^ virtual void get_etl_status(TMiniLoadEtlStatusResult& result,$/;" f class:doris::BackendService +get_etl_status test/runtime/data_stream_test.cpp /^ virtual void get_etl_status($/;" f class:doris::DorisTestBackend +get_evhttp_request src/http/http_request.h /^ struct evhttp_request* get_evhttp_request() const { return _ev_req; }$/;" f class:doris::HttpRequest +get_evp_type src/util/aes_util.cpp /^const EVP_CIPHER* get_evp_type(const AesMode mode) {$/;" f namespace:doris +get_explicit_count src/olap/hll.h /^ int get_explicit_count() {$/;" f class:doris::HllSetResolver +get_explicit_value src/olap/hll.h /^ uint64_t get_explicit_value(int index) {$/;" f class:doris::HllSetResolver +get_export_status src/service/backend_service.cpp /^void BackendService::get_export_status(TExportStatusResult& result, const TUniqueId& task_id) {$/;" f class:doris::BackendService +get_fd_cache src/olap/file_helper.h /^ static Cache* get_fd_cache() {$/;" f class:doris::FileHandler +get_field_length_by_type src/olap/tablet_schema.cpp /^uint32_t TabletColumn::get_field_length_by_type(TPrimitiveType::type type, uint32_t string_length) {$/;" f class:doris::TabletColumn +get_field_type_by_string src/olap/tablet_schema.cpp /^FieldType TabletColumn::get_field_type_by_string(const std::string& type_str) {$/;" f class:doris::TabletColumn +get_file src/runtime/small_file_mgr.cpp /^Status SmallFileMgr::get_file($/;" f class:doris::SmallFileMgr +get_file src/runtime/tmp_file_mgr.cc /^Status TmpFileMgr::get_file($/;" f class:doris::TmpFileMgr +get_filted_rows src/olap/rowset/column_data.cpp /^uint64_t ColumnData::get_filted_rows() {$/;" f class:doris::ColumnData +get_first_key src/olap/rowset/segment_v2/index_page.cpp /^Status IndexPageBuilder::get_first_key(Slice* key) const {$/;" f class:doris::segment_v2::IndexPageBuilder +get_first_no_zero_index src/runtime/decimal_value.h /^inline const int32_t* DecimalValue::get_first_no_zero_index($/;" f class:doris::DecimalValue +get_first_ordinal src/olap/rowset/segment_v2/ordinal_page_index.h /^ ordinal_t get_first_ordinal(int page_index) const {$/;" f class:doris::segment_v2::OrdinalIndexReader +get_first_row_block src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::get_first_row_block(RowBlock** row_block) {$/;" f class:doris::ColumnData +get_first_value src/olap/rowset/segment_v2/binary_dict_page.cpp /^Status BinaryDictPageBuilder::get_first_value(void* value) const {$/;" f class:doris::segment_v2::BinaryDictPageBuilder +get_fixed_len src/olap/row_cursor.h /^ inline size_t get_fixed_len() const { return _fixed_len; }$/;" f class:doris::RowCursor +get_fixed_value_set src/exec/olap_common.h /^ const std::set& get_fixed_value_set() const {$/;" f class:doris::ColumnValueRange +get_fixed_value_size src/exec/olap_common.h /^ size_t get_fixed_value_size() const {$/;" f class:doris::ColumnValueRange +get_float src/exec/es/es_predicate.cpp /^float ExtLiteral::get_float() {$/;" f class:doris::ExtLiteral +get_float_val src/exprs/arithmetic_expr.cpp /^FloatVal ModExpr::get_float_val(ExprContext* context, TupleRow* row) { $/;" f class:doris::ModExpr +get_float_val src/exprs/expr.cpp /^FloatVal Expr::get_float_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_float_val src/exprs/expr_context.cpp /^FloatVal ExprContext::get_float_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_float_val src/exprs/expr_ir.cpp /^FloatVal Expr::get_float_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_float_val src/exprs/literal.cpp /^FloatVal Literal::get_float_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_float_val src/exprs/null_literal.cpp /^FloatVal NullLiteral::get_float_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_float_val src/exprs/scalar_fn_call.cpp /^FloatVal ScalarFnCall::get_float_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_float_val src/exprs/slot_ref.cpp /^FloatVal SlotRef::get_float_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_float_value src/exec/es/es_scroll_parser.cpp /^static Status get_float_value(const rapidjson::Value &col, PrimitiveType type, void* slot, bool pure_doc_value) {$/;" f namespace:doris +get_fn_context_error src/exprs/expr.cpp /^Status Expr::get_fn_context_error(ExprContext* ctx) {$/;" f class:doris::Expr +get_fn_ctx src/testutil/function_utils.h /^ doris_udf::FunctionContext* get_fn_ctx() {$/;" f class:doris::FunctionUtils +get_free_buffer src/runtime/disk_io_mgr.cc /^char* DiskIoMgr::get_free_buffer(int64_t* buffer_size) {$/;" f class:doris::DiskIoMgr +get_free_offset src/runtime/mem_pool.h /^ int64_t get_free_offset() const {$/;" f class:doris::MemPool +get_full_build_path src/util/path_builder.cpp /^void PathBuilder::get_full_build_path(const std::string& path, std::string* full_path) {$/;" f class:doris::PathBuilder +get_full_path src/util/path_builder.cpp /^void PathBuilder::get_full_path(const std::string& path, std::string* full_path) {$/;" f class:doris::PathBuilder +get_full_value src/olap/hll.h /^ char* get_full_value() {$/;" f class:doris::HllSetResolver +get_function src/exprs/scalar_fn_call.cpp /^Status ScalarFnCall::get_function(RuntimeState* state, const std::string& symbol, void** fn) {$/;" f class:doris::ScalarFnCall +get_function_ptr src/runtime/user_function_cache.cpp /^Status UserFunctionCache::get_function_ptr($/;" f class:doris::UserFunctionCache +get_function_state src/udf/udf_ir.cpp /^void* FunctionContext::get_function_state(FunctionStateScope scope) const {$/;" f class:doris_udf::FunctionContext +get_hash_seed src/exec/partitioned_hash_table_ir.cc /^uint32_t PartitionedHashTableCtx::get_hash_seed() const {$/;" f class:doris::PartitionedHashTableCtx +get_hash_value src/runtime/data_stream_mgr.cpp /^inline uint32_t DataStreamMgr::get_hash_value($/;" f class:doris::DataStreamMgr +get_hash_value src/runtime/raw_value.h /^ static uint32_t get_hash_value(const void* value, const PrimitiveType& type) {$/;" f class:doris::RawValue +get_hash_value src/runtime/raw_value.h /^ static uint32_t get_hash_value(const void* value, const TypeDescriptor& type) {$/;" f class:doris::RawValue +get_hash_value src/runtime/raw_value.h /^ static uint32_t get_hash_value(const void* value, const TypeDescriptor& type, uint32_t seed) {$/;" f class:doris::RawValue +get_hash_value src/runtime/raw_value.h /^inline uint32_t RawValue::get_hash_value($/;" f class:doris::RawValue +get_hash_value_fvn src/runtime/raw_value.h /^ static uint32_t get_hash_value_fvn($/;" f class:doris::RawValue +get_hash_value_fvn src/runtime/raw_value.h /^inline uint32_t RawValue::get_hash_value_fvn($/;" f class:doris::RawValue +get_header_lock_ptr src/olap/tablet.h /^ inline RWMutex* get_header_lock_ptr() { return &_meta_lock; }$/;" f class:doris::Tablet +get_hints src/exec/olap_scan_node.cpp /^static Status get_hints($/;" f namespace:doris +get_hit_count src/olap/lru_cache.h /^ uint64_t get_hit_count() {$/;" f class:doris::CachePriority::LRUCache +get_hll_data_type src/olap/hll.h /^ HllDataType get_hll_data_type() {$/;" f class:doris::HllSetResolver +get_host_address_v4 src/util/network_util.cpp /^std::string InetAddress::get_host_address_v4() {$/;" f class:doris::InetAddress +get_host_port src/exec/es_http_scan_node.cpp /^static std::string get_host_port(const std::vector& es_hosts) {$/;" f namespace:doris +get_hostname src/util/network_util.cpp /^Status get_hostname(std::string* hostname) {$/;" f namespace:doris +get_hosts_v4 src/util/network_util.cpp /^Status get_hosts_v4(std::vector* hosts) {$/;" f namespace:doris +get_http_status src/http/http_client.h /^ long get_http_status() const {$/;" f class:doris::HttpClient +get_id src/util/core_local.h /^ int get_id() {$/;" f class:doris::CoreLocalValueController +get_inc_rowset_by_version src/olap/tablet.cpp /^const RowsetSharedPtr Tablet::get_inc_rowset_by_version(const Version& version) const {$/;" f class:doris::Tablet +get_index_size src/olap/row_cursor.h /^ const size_t get_index_size(size_t index) const {$/;" f class:doris::RowCursor +get_inet_interfaces src/util/network_util.cpp /^Status get_inet_interfaces(std::vector* interfaces, bool include_ipv6) {$/;" f namespace:doris +get_info src/service/internal_service.cpp /^void PInternalServiceImpl::get_info($/;" f class:doris::PInternalServiceImpl +get_info_string src/util/runtime_profile.cpp /^const std::string* RuntimeProfile::get_info_string(const std::string& key) {$/;" f class:doris::RuntimeProfile +get_init_status src/olap/delete_handler.h /^ bool get_init_status() const {$/;" f class:doris::DeleteHandler +get_input_row src/exec/merge_join_node.cpp /^Status MergeJoinNode::get_input_row(RuntimeState* state, int child_idx) {$/;" f class:doris::MergeJoinNode +get_int src/exec/es/es_predicate.cpp /^int32_t ExtLiteral::get_int() {$/;" f class:doris::ExtLiteral +get_int src/exec/read_write_util.h /^inline int32_t ReadWriteUtil::get_int(const uint8_t* buf) {$/;" f class:doris::ReadWriteUtil +get_int_val src/exprs/expr.cpp /^IntVal Expr::get_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_int_val src/exprs/expr_context.cpp /^IntVal ExprContext::get_int_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_int_val src/exprs/expr_ir.cpp /^IntVal Expr::get_int_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_int_val src/exprs/literal.cpp /^IntVal Literal::get_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_int_val src/exprs/null_literal.cpp /^IntVal NullLiteral::get_int_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_int_val src/exprs/scalar_fn_call.cpp /^IntVal ScalarFnCall::get_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_int_val src/exprs/slot_ref.cpp /^IntVal SlotRef::get_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_int_value src/exec/es/es_scroll_parser.cpp /^static Status get_int_value(const rapidjson::Value &col, PrimitiveType type, void* slot, bool pure_doc_value) {$/;" f namespace:doris +get_job_state src/runtime/etl_job_mgr.cpp /^Status EtlJobMgr::get_job_state(const TUniqueId& id,$/;" f class:doris::EtlJobMgr +get_json_double src/exprs/json_functions.cpp /^DoubleVal JsonFunctions::get_json_double($/;" f class:doris::JsonFunctions +get_json_int src/exprs/json_functions.cpp /^IntVal JsonFunctions::get_json_int($/;" f class:doris::JsonFunctions +get_json_meta src/olap/tablet_meta_manager.cpp /^OLAPStatus TabletMetaManager::get_json_meta(DataDir* store,$/;" f class:doris::TabletMetaManager +get_json_object src/exprs/json_functions.cpp /^rapidjson::Value* JsonFunctions::get_json_object($/;" f class:doris::JsonFunctions +get_json_rowset_meta src/olap/rowset/rowset_meta_manager.cpp /^OLAPStatus RowsetMetaManager::get_json_rowset_meta(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id, std::string* json_rowset_meta) {$/;" f class:doris::RowsetMetaManager +get_json_string src/exprs/json_functions.cpp /^StringVal JsonFunctions::get_json_string($/;" f class:doris::JsonFunctions +get_kafka_partition_meta src/runtime/routine_load/routine_load_task_executor.cpp /^Status RoutineLoadTaskExecutor::get_kafka_partition_meta($/;" f class:doris::RoutineLoadTaskExecutor +get_key src/olap/rowset/segment_v2/index_page.h /^ inline const Slice& get_key(int idx) const {$/;" f class:doris::segment_v2::IndexPageReader +get_key_coder src/olap/key_coder.cpp /^const KeyCoder* get_key_coder(FieldType type) {$/;" f namespace:doris +get_key_range src/exec/olap_common.cpp /^Status OlapScanKeys::get_key_range($/;" f class:doris::OlapScanKeys +get_large_int_val src/exprs/expr.cpp /^LargeIntVal Expr::get_large_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_large_int_val src/exprs/expr_ir.cpp /^LargeIntVal Expr::get_large_int_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_large_int_val src/exprs/literal.cpp /^LargeIntVal Literal::get_large_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_large_int_val src/exprs/scalar_fn_call.cpp /^LargeIntVal ScalarFnCall::get_large_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_large_int_val src/exprs/slot_ref.cpp /^LargeIntVal SlotRef::get_large_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_largeint_string src/exec/es/es_predicate.cpp /^std::string ExtLiteral::get_largeint_string() {$/;" f class:doris::ExtLiteral +get_last_ordinal src/olap/rowset/segment_v2/ordinal_page_index.h /^ ordinal_t get_last_ordinal(int page_index) const {$/;" f class:doris::segment_v2::OrdinalIndexReader +get_last_value src/olap/rowset/segment_v2/binary_dict_page.cpp /^Status BinaryDictPageBuilder::get_last_value(void* value) const {$/;" f class:doris::segment_v2::BinaryDictPageBuilder +get_left_child_row_string src/exec/blocking_join_node.cpp /^std::string BlockingJoinNode::get_left_child_row_string(TupleRow* row) {$/;" f class:doris::BlockingJoinNode +get_length_prefixed_slice src/util/coding.h /^inline bool get_length_prefixed_slice(Slice* input, Slice* val) {$/;" f namespace:doris +get_load_data_path src/runtime/load_path_mgr.cpp /^void LoadPathMgr::get_load_data_path(std::vector* data_paths) {$/;" f class:doris::LoadPathMgr +get_load_error_absolute_path src/runtime/load_path_mgr.cpp /^std::string LoadPathMgr::get_load_error_absolute_path(const std::string& file_path) {$/;" f class:doris::LoadPathMgr +get_load_error_file_dir src/runtime/load_path_mgr.h /^ const std::string& get_load_error_file_dir() const {$/;" f class:doris::LoadPathMgr +get_load_error_file_name src/runtime/load_path_mgr.cpp /^Status LoadPathMgr::get_load_error_file_name($/;" f class:doris::LoadPathMgr +get_load_mem_limit src/runtime/runtime_state.cpp /^int64_t RuntimeState::get_load_mem_limit() {$/;" f class:doris::RuntimeState +get_localhost src/service/backend_options.cpp /^std::string BackendOptions::get_localhost() {$/;" f class:doris::BackendOptions +get_long src/exec/es/es_predicate.cpp /^int64_t ExtLiteral::get_long() {$/;" f class:doris::ExtLiteral +get_long_int src/exec/read_write_util.h /^inline int64_t ReadWriteUtil::get_long_int(const uint8_t* buf) {$/;" f class:doris::ReadWriteUtil +get_lookup_count src/olap/lru_cache.h /^ uint64_t get_lookup_count() {$/;" f class:doris::CachePriority::LRUCache +get_max_decimal src/runtime/decimal_value.h /^ static DecimalValue get_max_decimal() {$/;" f class:doris::DecimalValue +get_max_decimal src/runtime/decimalv2_value.h /^ static DecimalV2Value get_max_decimal() {$/;" f class:doris::DecimalV2Value +get_max_io_util src/util/system_metrics.cpp /^int64_t SystemMetrics::get_max_io_util($/;" f class:doris::SystemMetrics +get_max_net_traffic src/util/system_metrics.cpp /^void SystemMetrics::get_max_net_traffic($/;" f class:doris::SystemMetrics +get_max_num_cores src/util/cpu_info.h /^ static int get_max_num_cores() { return max_num_cores_; }$/;" f class:doris::CpuInfo +get_max_num_numa_nodes src/util/cpu_info.h /^ static int get_max_num_numa_nodes() { return max_num_numa_nodes_; }$/;" f class:doris::CpuInfo +get_mem_alloc_failed_error_msg src/runtime/spill_sorter.cc /^static std::string get_mem_alloc_failed_error_msg(const std::string& block_type) {$/;" f namespace:doris +get_mem_pool src/runtime/qsorter.h /^ MemPool* get_mem_pool() { $/;" f class:doris::QSorter +get_memory_usage src/olap/lru_cache.cpp /^size_t ShardedLRUCache::get_memory_usage() {$/;" f class:doris::ShardedLRUCache +get_meta src/olap/data_dir.h /^ OlapMeta* get_meta() { return _meta; }$/;" f class:doris::DataDir +get_meta src/olap/tablet_meta_manager.cpp /^OLAPStatus TabletMetaManager::get_meta($/;" f class:doris::TabletMetaManager +get_meta src/tools/meta_tool.cpp /^void get_meta(DataDir* data_dir) {$/;" f +get_metric src/util/metrics.cpp /^Metric* MetricCollector::get_metric(const MetricLabels& labels) const {$/;" f class:doris::MetricCollector +get_metric src/util/metrics.cpp /^Metric* MetricRegistry::get_metric(const std::string& name, const MetricLabels& labels) const {$/;" f class:doris::MetricRegistry +get_metric src/util/metrics.h /^ Metric* get_metric(const std::string& name) const {$/;" f class:doris::MetricRegistry +get_metrics src/util/metrics.cpp /^void MetricCollector::get_metrics(std::vector* metrics) {$/;" f class:doris::MetricCollector +get_migration_lock_ptr src/olap/tablet.h /^ inline RWMutex* get_migration_lock_ptr() { return &_migration_lock; }$/;" f class:doris::Tablet +get_min_decimal src/runtime/decimal_value.h /^ static DecimalValue get_min_decimal() {$/;" f class:doris::DecimalValue +get_min_decimal src/runtime/decimalv2_value.h /^ static DecimalV2Value get_min_decimal() {$/;" f class:doris::DecimalV2Value +get_mutable_column_mapping src/olap/schema_change.cpp /^ColumnMapping* RowBlockChanger::get_mutable_column_mapping(size_t column_index) {$/;" f class:doris::RowBlockChanger +get_network_traffic src/util/system_metrics.cpp /^void SystemMetrics::get_network_traffic($/;" f class:doris::SystemMetrics +get_new_block src/runtime/buffered_block_mgr.cpp /^Status BufferedBlockMgr::get_new_block(Block** block) {$/;" f class:doris::BufferedBlockMgr +get_new_block src/runtime/buffered_block_mgr.cpp /^Status BufferedBlockMgr::get_new_block(Block** block, int64_t len) {$/;" f class:doris::BufferedBlockMgr +get_new_block src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::get_new_block($/;" f class:doris::BufferedBlockMgr2 +get_new_context src/runtime/disk_io_mgr.cc /^ RequestContext* get_new_context() {$/;" f class:doris::DiskIoMgr::RequestContextCache +get_new_desc src/exec/schema_scanner/schema_columns_scanner.cpp /^Status SchemaColumnsScanner::get_new_desc() {$/;" f class:doris::SchemaColumnsScanner +get_new_table src/exec/schema_scanner/schema_columns_scanner.cpp /^Status SchemaColumnsScanner::get_new_table() {$/;" f class:doris::SchemaColumnsScanner +get_new_table src/exec/schema_scanner/schema_tables_scanner.cpp /^Status SchemaTablesScanner::get_new_table() {$/;" f class:doris::SchemaTablesScanner +get_next src/exec/aggregation_node.cpp /^Status AggregationNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::AggregationNode +get_next src/exec/analytic_eval_node.cpp /^Status AnalyticEvalNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::AnalyticEvalNode +get_next src/exec/assert_num_rows_node.cpp /^Status AssertNumRowsNode::get_next(RuntimeState* state, RowBatch* output_batch, bool* eos) {$/;" f class:doris::AssertNumRowsNode +get_next src/exec/broker_scan_node.cpp /^Status BrokerScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::BrokerScanNode +get_next src/exec/broker_scanner.cpp /^Status BrokerScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {$/;" f class:doris::BrokerScanner +get_next src/exec/cross_join_node.cpp /^Status CrossJoinNode::get_next(RuntimeState* state, RowBatch* output_batch, bool* eos) {$/;" f class:doris::CrossJoinNode +get_next src/exec/csv_scan_node.cpp /^Status CsvScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::CsvScanNode +get_next src/exec/empty_set_node.cpp /^Status EmptySetNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::EmptySetNode +get_next src/exec/es/es_scan_reader.cpp /^Status ESScanReader::get_next(bool* scan_eos, std::unique_ptr& scroll_parser) {$/;" f class:doris::ESScanReader +get_next src/exec/es_http_scan_node.cpp /^Status EsHttpScanNode::get_next(RuntimeState* state, RowBatch* row_batch, $/;" f class:doris::EsHttpScanNode +get_next src/exec/es_http_scanner.cpp /^Status EsHttpScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof, const std::map& docvalue_context) {$/;" f class:doris::EsHttpScanner +get_next src/exec/es_scan_node.cpp /^Status EsScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::EsScanNode +get_next src/exec/exchange_node.cpp /^Status ExchangeNode::get_next(RuntimeState* state, RowBatch* output_batch, bool* eos) {$/;" f class:doris::ExchangeNode +get_next src/exec/hash_join_node.cpp /^Status HashJoinNode::get_next(RuntimeState* state, RowBatch* out_batch, bool* eos) {$/;" f class:doris::HashJoinNode +get_next src/exec/intersect_node.cpp /^Status IntersectNode::get_next(RuntimeState* state, RowBatch* out_batch, bool* eos) {$/;" f class:doris::IntersectNode +get_next src/exec/kudu_scan_node.cpp /^Status KuduScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::KuduScanNode +get_next src/exec/kudu_scanner.cpp /^Status KuduScanner::get_next(RowBatch* row_batch, bool* eos) {$/;" f class:doris::KuduScanner +get_next src/exec/merge_join_node.cpp /^Status MergeJoinNode::get_next(RuntimeState* state, RowBatch* out_batch, bool* eos) {$/;" f class:doris::MergeJoinNode +get_next src/exec/merge_node.cpp /^Status MergeNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::MergeNode +get_next src/exec/mysql_scan_node.cpp /^Status MysqlScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::MysqlScanNode +get_next src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::get_next(RuntimeState* state, RowBatch* row_batch,$/;" f class:doris::NewPartitionedAggregationNode +get_next src/exec/olap_rewrite_node.cpp /^Status OlapRewriteNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::OlapRewriteNode +get_next src/exec/olap_scan_node.cpp /^Status OlapScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::OlapScanNode +get_next src/exec/orc_scanner.cpp /^Status ORCScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {$/;" f class:doris::ORCScanner +get_next src/exec/parquet_scanner.cpp /^Status ParquetScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {$/;" f class:doris::ParquetScanner +get_next src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::PartitionedAggregationNode +get_next src/exec/pl_task_root.cpp /^Status ExchangeNode::get_next(RuntimeState* state, RowBatch* output_batch, bool* eos) {$/;" f class:doris::ExchangeNode +get_next src/exec/pre_aggregation_node.cpp /^Status PreAggregationNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::PreAggregationNode +get_next src/exec/repeat_node.cpp /^Status RepeatNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::RepeatNode +get_next src/exec/schema_scan_node.cpp /^Status SchemaScanNode::get_next(RuntimeState* state, RowBatch* row_batch,$/;" f class:doris::SchemaScanNode +get_next src/exec/select_node.cpp /^Status SelectNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::SelectNode +get_next src/exec/sort_node.cpp /^Status SortNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::SortNode +get_next src/exec/spill_sort_node.cc /^Status SpillSortNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::SpillSortNode +get_next src/exec/topn_node.cpp /^Status TopNNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::TopNNode +get_next src/exec/union_node.cpp /^Status UnionNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {$/;" f class:doris::UnionNode +get_next src/olap/stream_index_reader.h /^ int64_t get_next() {$/;" f class:doris::PositionProvider +get_next src/runtime/buffered_tuple_stream.cpp /^Status BufferedTupleStream::get_next(RowBatch* batch, bool* eos,$/;" f class:doris::BufferedTupleStream +get_next src/runtime/buffered_tuple_stream.h /^ Status get_next(RowBatch* batch, bool* eos) {$/;" f class:doris::BufferedTupleStream +get_next src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::get_next(RowBatch* batch, bool* eos,$/;" f class:doris::BufferedTupleStream2 +get_next src/runtime/data_stream_recvr.cc /^Status DataStreamRecvr::get_next(RowBatch* output_batch, bool* eos) {$/;" f class:doris::DataStreamRecvr +get_next src/runtime/disk_io_mgr_scan_range.cc /^Status DiskIoMgr::ScanRange::get_next(BufferDescriptor** buffer) {$/;" f class:doris::DiskIoMgr::ScanRange +get_next src/runtime/merge_sorter.cpp /^Status MergeSorter::Run::get_next(RowBatch* output_batch, bool* eos) {$/;" f class:doris::MergeSorter::Run +get_next src/runtime/merge_sorter.cpp /^Status MergeSorter::get_next(RowBatch* output_batch, bool* eos) {$/;" f class:doris::MergeSorter +get_next src/runtime/plan_fragment_executor.cpp /^Status PlanFragmentExecutor::get_next(RowBatch** batch) {$/;" f class:doris::PlanFragmentExecutor +get_next src/runtime/qsorter.cpp /^Status QSorter::get_next(RowBatch* batch, bool* eos) {$/;" f class:doris::QSorter +get_next src/runtime/sorted_run_merger.cc /^Status SortedRunMerger::get_next(RowBatch* output_batch, bool* eos) {$/;" f class:doris::SortedRunMerger +get_next src/runtime/spill_sorter.cc /^Status SpillSorter::Run::get_next(RowBatch* output_batch, bool* eos) {$/;" f class:doris::SpillSorter::Run +get_next src/runtime/spill_sorter.cc /^Status SpillSorter::get_next(RowBatch* output_batch, bool* eos) {$/;" f class:doris::SpillSorter +get_next src/service/backend_service.cpp /^void BackendService::get_next(TScanBatchResult& result_, const TScanNextBatchParams& params) {$/;" f class:doris::BackendService +get_next_batch src/runtime/spill_sorter.cc /^Status SpillSorter::Run::get_next_batch(RowBatch** output_batch) {$/;" f class:doris::SpillSorter::Run +get_next_batch test/runtime/data_stream_test.cpp /^ void get_next_batch(RowBatch* batch, int* next_val) {$/;" f class:doris::DataStreamTest +get_next_block src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::get_next_block(RowBlock** row_block) {$/;" f class:doris::ColumnData +get_next_const src/exec/union_node.cpp /^Status UnionNode::get_next_const(RuntimeState* state, RowBatch* row_batch) {$/;" f class:doris::UnionNode +get_next_from_es src/exec/es_scan_node.cpp /^Status EsScanNode::get_next_from_es(TExtGetNextResult& result) {$/;" f class:doris::EsScanNode +get_next_instance_id test/runtime/data_stream_test.cpp /^ void get_next_instance_id(TUniqueId* instance_id) {$/;" f class:doris::DataStreamTest +get_next_internal src/runtime/buffered_tuple_stream.cpp /^Status BufferedTupleStream::get_next_internal(RowBatch* batch, bool* eos,$/;" f class:doris::BufferedTupleStream +get_next_internal src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::get_next_internal($/;" f class:doris::BufferedTupleStream2 +get_next_internal src/runtime/plan_fragment_executor.cpp /^Status PlanFragmentExecutor::get_next_internal(RowBatch** batch) {$/;" f class:doris::PlanFragmentExecutor +get_next_materialized src/exec/union_node.cpp /^Status UnionNode::get_next_materialized(RuntimeState* state, RowBatch* row_batch) {$/;" f class:doris::UnionNode +get_next_merging src/exec/exchange_node.cpp /^Status ExchangeNode::get_next_merging(RuntimeState* state, RowBatch* output_batch, bool* eos) {$/;" f class:doris::ExchangeNode +get_next_output_batch src/exec/analytic_eval_node.cpp /^Status AnalyticEvalNode::get_next_output_batch(RuntimeState* state, RowBatch* output_batch,$/;" f class:doris::AnalyticEvalNode +get_next_pass_through src/exec/union_node.cpp /^Status UnionNode::get_next_pass_through(RuntimeState* state, RowBatch* row_batch) {$/;" f class:doris::UnionNode +get_next_range src/runtime/disk_io_mgr.cc /^Status DiskIoMgr::get_next_range(RequestContext* reader, ScanRange** range) {$/;" f class:doris::DiskIoMgr +get_next_request_range src/runtime/disk_io_mgr.cc /^bool DiskIoMgr::get_next_request_range(DiskQueue* disk_queue, RequestRange** range,$/;" f class:doris::DiskIoMgr +get_next_row src/exec/csv_scanner.cpp /^ Status CsvScanner::get_next_row(std::string* line_str, bool* eos) {$/;" f class:doris::CsvScanner +get_next_row src/exec/merge_join_node.cpp /^Status MergeJoinNode::get_next_row(RuntimeState* state, TupleRow* out_row, bool* eos) {$/;" f class:doris::MergeJoinNode +get_next_row src/exec/mysql_scanner.cpp /^Status MysqlScanner::get_next_row(char** *buf, unsigned long** lengths, bool* eos) {$/;" f class:doris::MysqlScanner +get_next_row src/exec/schema_scanner.cpp /^Status SchemaScanner::get_next_row(Tuple* tuple, MemPool* pool, bool* eos) {$/;" f class:doris::SchemaScanner +get_next_row src/exec/schema_scanner/schema_charsets_scanner.cpp /^Status SchemaCharsetsScanner::get_next_row(Tuple *tuple, MemPool *pool, bool *eos) {$/;" f class:doris::SchemaCharsetsScanner +get_next_row src/exec/schema_scanner/schema_collations_scanner.cpp /^Status SchemaCollationsScanner::get_next_row(Tuple *tuple, MemPool *pool, bool *eos) {$/;" f class:doris::SchemaCollationsScanner +get_next_row src/exec/schema_scanner/schema_columns_scanner.cpp /^Status SchemaColumnsScanner::get_next_row(Tuple *tuple, MemPool *pool, bool *eos) {$/;" f class:doris::SchemaColumnsScanner +get_next_row src/exec/schema_scanner/schema_dummy_scanner.cpp /^Status SchemaDummyScanner::get_next_row(Tuple *tuple, MemPool *pool, bool *eos) {$/;" f class:doris::SchemaDummyScanner +get_next_row src/exec/schema_scanner/schema_schemata_scanner.cpp /^Status SchemaSchemataScanner::get_next_row(Tuple *tuple, MemPool *pool, bool *eos) {$/;" f class:doris::SchemaSchemataScanner +get_next_row src/exec/schema_scanner/schema_tables_scanner.cpp /^Status SchemaTablesScanner::get_next_row(Tuple *tuple, MemPool *pool, bool *eos) {$/;" f class:doris::SchemaTablesScanner +get_next_row src/exec/schema_scanner/schema_variables_scanner.cpp /^Status SchemaVariablesScanner::get_next_row(Tuple *tuple, MemPool *pool, bool *eos) {$/;" f class:doris::SchemaVariablesScanner +get_next_scan_token src/exec/kudu_scan_node.cpp /^const string* KuduScanNode::get_next_scan_token() {$/;" f class:doris::KuduScanNode +get_next_scanner_batch src/exec/kudu_scanner.cpp /^Status KuduScanner::get_next_scanner_batch() {$/;" f class:doris::KuduScanner +get_node src/exec/hash_table.h /^ Node* get_node(int64_t idx) {$/;" f class:doris::HashTable +get_node_id_from_profile src/exec/exec_node.cpp /^int ExecNode::get_node_id_from_profile(RuntimeProfile* p) {$/;" f class:doris::ExecNode +get_normal_row_number src/runtime/runtime_state.h /^ const int64_t get_normal_row_number() const {$/;" f class:doris::RuntimeState +get_null_supported src/olap/olap_index.h /^ bool get_null_supported(uint32_t seg_id) {$/;" f class:doris::MemIndex +get_null_supported src/olap/rowset/segment_group.h /^ inline bool get_null_supported(uint32_t seg_id) {$/;" f class:doris::SegmentGroup +get_num_args src/udf/udf_ir.cpp /^int FunctionContext::get_num_args() const {$/;" f class:doris_udf::FunctionContext +get_num_children src/exprs/expr.h /^ int get_num_children() const {$/;" f class:doris::Expr +get_num_constant_args src/udf/udf_ir.cpp /^int FunctionContext::get_num_constant_args() const {$/;" f class:doris_udf::FunctionContext +get_num_data_bytes_sent src/runtime/data_stream_sender.cpp /^int64_t DataStreamSender::get_num_data_bytes_sent() const {$/;" f class:doris::DataStreamSender +get_num_key_columns src/olap/rowset/segment_group.cpp /^int SegmentGroup::get_num_key_columns() {$/;" f class:doris::SegmentGroup +get_num_rows_per_row_block src/olap/rowset/segment_group.cpp /^size_t SegmentGroup::get_num_rows_per_row_block() {$/;" f class:doris::SegmentGroup +get_num_short_key_columns src/olap/rowset/segment_group.cpp /^int SegmentGroup::get_num_short_key_columns() {$/;" f class:doris::SegmentGroup +get_numa_node_core_idx src/util/cpu_info.h /^ static int get_numa_node_core_idx(int core) {$/;" f class:doris::CpuInfo +get_numa_node_of_core src/util/cpu_info.h /^ static int get_numa_node_of_core(int core) {$/;" f class:doris::CpuInfo +get_object_pool test/runtime/sorter_test.cpp /^ ObjectPool* get_object_pool() {return _object_pool;}$/;" f class:doris::SorterTest +get_offset src/olap/olap_index.cpp /^const OLAPIndexOffset MemIndex::get_offset(const RowBlockPosition& pos) const {$/;" f class:doris::MemIndex +get_olap_size src/exec/olap_utils.h /^inline int get_olap_size(PrimitiveType type) {$/;" f namespace:doris +get_or_create_translator src/runtime/dpp_sink.cpp /^Status DppSink::get_or_create_translator($/;" f class:doris::DppSink +get_output_tuple src/exec/partitioned_aggregation_node.cc /^Tuple* PartitionedAggregationNode::get_output_tuple($/;" f class:doris::PartitionedAggregationNode +get_parsed_paths src/exprs/json_functions.cpp /^void JsonFunctions::get_parsed_paths($/;" f class:doris::JsonFunctions +get_partition_id src/exec/broker_scan_node.cpp /^int64_t BrokerScanNode::get_partition_id($/;" f class:doris::BrokerScanNode +get_partition_ids src/olap/txn_manager.cpp /^void TxnManager::get_partition_ids(TTransactionId transaction_id, std::vector* partition_ids) {$/;" f class:doris::TxnManager +get_partition_meta src/runtime/routine_load/data_consumer.cpp /^Status KafkaDataConsumer::get_partition_meta(std::vector* partition_ids) {$/;" f class:doris::KafkaDataConsumer +get_partition_related_tablets src/olap/tablet_manager.cpp /^void TabletManager::get_partition_related_tablets(int64_t partition_id,$/;" f class:doris::TabletManager +get_partitions src/exec/tablet_info.h /^ const std::vector& get_partitions() const {$/;" f class:doris::OlapTablePartitionParam +get_pin_failed_error_msg src/runtime/spill_sorter.cc /^static std::string get_pin_failed_error_msg(const std::string& block_type) {$/;" f namespace:doris +get_position src/olap/file_stream.h /^ inline void get_position(uint32_t* position) {$/;" f class:doris::ReadOnlyFileStream +get_position src/olap/out_stream.cpp /^void OutStream::get_position(PositionEntryWriter* index_entry) const {$/;" f class:doris::OutStream +get_position src/olap/rowset/bit_field_writer.cpp /^void BitFieldWriter::get_position(PositionEntryWriter* index_entry) const {$/;" f class:doris::BitFieldWriter +get_position src/olap/rowset/run_length_byte_writer.cpp /^void RunLengthByteWriter::get_position(PositionEntryWriter* index_entry) const {$/;" f class:doris::RunLengthByteWriter +get_position src/olap/rowset/run_length_integer_writer.cpp /^void RunLengthIntegerWriter::get_position(PositionEntryWriter* index_entry, bool print) const {$/;" f class:doris::RunLengthIntegerWriter +get_predicate_list src/exec/es/es_predicate.cpp /^const vector& EsPredicate::get_predicate_list(){$/;" f class:doris::EsPredicate +get_probe_row_output_string src/exec/hash_join_node.cpp /^string HashJoinNode::get_probe_row_output_string(TupleRow* probe_row) {$/;" f class:doris::HashJoinNode +get_proc_self_io_counters src/util/perf_counters.cpp /^bool PerfCounters::get_proc_self_io_counters(vector& buffer) {$/;" f class:doris::PerfCounters +get_proc_self_status_counters src/util/perf_counters.cpp /^bool PerfCounters::get_proc_self_status_counters(vector& buffer) {$/;" f class:doris::PerfCounters +get_push_lock src/olap/tablet.h /^ inline Mutex* get_push_lock() { return &_ingest_lock; }$/;" f class:doris::Tablet +get_queue_size src/util/batch_process_thread_pool.hpp /^ uint32_t get_queue_size() const {$/;" f class:doris::BatchProcessThreadPool +get_queue_size src/util/priority_thread_pool.hpp /^ uint32_t get_queue_size() const {$/;" f class:doris::PriorityThreadPool +get_range_count src/olap/rowset/segment_v2/row_ranges.h /^ size_t get_range_count(size_t range_index) {$/;" f class:doris::segment_v2::RowRanges +get_range_from src/olap/rowset/segment_v2/row_ranges.h /^ int64_t get_range_from(size_t range_index) {$/;" f class:doris::segment_v2::RowRanges +get_range_max_value src/exec/olap_common.h /^ T get_range_max_value() const {$/;" f class:doris::ColumnValueRange +get_range_min_value src/exec/olap_common.h /^ T get_range_min_value() const {$/;" f class:doris::ColumnValueRange +get_range_to src/olap/rowset/segment_v2/row_ranges.h /^ int64_t get_range_to(size_t range_index) {$/;" f class:doris::segment_v2::RowRanges +get_read_throughput src/runtime/disk_io_mgr.cc /^int64_t DiskIoMgr::get_read_throughput() {$/;" f class:doris::DiskIoMgr +get_real_byte_size src/runtime/primitive_type.h /^inline int get_real_byte_size(PrimitiveType type) {$/;" f namespace:doris +get_real_port src/http/ev_http_server.h /^ int get_real_port() {$/;" f class:doris::EvHttpServer +get_real_symbol src/runtime/user_function_cache.cpp /^std::string get_real_symbol(const std::string& symbol) {$/;" f namespace:doris +get_related_tablets src/olap/task/engine_task.h /^ virtual void get_related_tablets(std::vector* tablet_infos) {}$/;" f class:doris::EngineTask +get_relative_offset src/olap/olap_index.cpp /^const OLAPIndexOffset MemIndex::get_relative_offset(iterator_offset_t absolute_offset) const {$/;" f class:doris::MemIndex +get_repeated_batch src/exec/repeat_node.cpp /^Status RepeatNode::get_repeated_batch($/;" f class:doris::RepeatNode +get_request_body src/http/http_request.cpp /^std::string HttpRequest::get_request_body() {$/;" f class:doris::HttpRequest +get_resource_tls src/common/resource_tls.cpp /^TResourceInfo* ResourceTls::get_resource_tls() {$/;" f class:doris::ResourceTls +get_response_content_type src/http/http_client.h /^ std::string get_response_content_type() {$/;" f class:doris::HttpClient +get_results_timer_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* get_results_timer_;$/;" m class:doris::NewPartitionedAggregationNode +get_return_type src/udf/udf_ir.cpp /^const FunctionContext::TypeDesc& FunctionContext::get_return_type() const {$/;" f class:doris_udf::FunctionContext +get_root_path src/olap/olap_meta.cpp /^std::string OlapMeta::get_root_path() {$/;" f class:doris::OlapMeta +get_root_path_from_schema_hash_path_in_trash src/olap/data_dir.cpp /^std::string DataDir::get_root_path_from_schema_hash_path_in_trash($/;" f class:doris::DataDir +get_row src/exec/hash_table.h /^ TupleRow* get_row() {$/;" f class:doris::HashTable::Iterator +get_row src/exec/partitioned_hash_table.inline.h /^inline TupleRow* PartitionedHashTable::Iterator::get_row() const {$/;" f class:doris::PartitionedHashTable::Iterator +get_row src/exec/partitioned_hash_table.inline.h /^inline TupleRow* PartitionedHashTable::get_row(Bucket* bucket, TupleRow* row) const {$/;" f class:doris::PartitionedHashTable +get_row src/exec/partitioned_hash_table.inline.h /^inline TupleRow* PartitionedHashTable::get_row(HtData& htdata, TupleRow* row) const {$/;" f class:doris::PartitionedHashTable +get_row src/exec/row_batch_list.h /^ TupleRow* get_row() {$/;" f class:doris::RowBatchList::TupleRowIterator +get_row src/olap/row_block.h /^ inline void get_row(uint32_t row_index, RowCursor* cursor) const {$/;" f class:doris::RowBlock +get_row src/runtime/row_batch.h /^ TupleRow* get_row(int row_idx) const {$/;" f class:doris::RowBatch +get_row_block_entry src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::get_row_block_entry(const RowBlockPosition& pos, EntrySlice* entry) const {$/;" f class:doris::SegmentGroup +get_row_block_position src/olap/olap_index.cpp /^OLAPStatus MemIndex::get_row_block_position($/;" f class:doris::MemIndex +get_row_block_position src/olap/rowset/segment_group.h /^ OLAPStatus get_row_block_position(const OLAPIndexOffset& pos, RowBlockPosition* rbp) const {$/;" f class:doris::SegmentGroup +get_row_ranges_by_bloom_filter src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::get_row_ranges_by_bloom_filter(CondColumn* cond_column, RowRanges* row_ranges) {$/;" f class:doris::segment_v2::ColumnReader +get_row_ranges_by_bloom_filter src/olap/rowset/segment_v2/column_reader.cpp /^Status FileColumnIterator::get_row_ranges_by_bloom_filter(CondColumn* cond_column, RowRanges* row_ranges) {$/;" f class:doris::segment_v2::FileColumnIterator +get_row_ranges_by_bloom_filter src/olap/rowset/segment_v2/column_reader.h /^ virtual Status get_row_ranges_by_bloom_filter(CondColumn* cond_column, RowRanges* row_ranges) {$/;" f class:doris::segment_v2::ColumnIterator +get_row_ranges_by_zone_map src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::get_row_ranges_by_zone_map(CondColumn* cond_column,$/;" f class:doris::segment_v2::ColumnReader +get_row_ranges_by_zone_map src/olap/rowset/segment_v2/column_reader.cpp /^Status FileColumnIterator::get_row_ranges_by_zone_map(CondColumn* cond_column,$/;" f class:doris::segment_v2::FileColumnIterator +get_row_ranges_by_zone_map src/olap/rowset/segment_v2/column_reader.h /^ virtual Status get_row_ranges_by_zone_map(CondColumn* cond_column,$/;" f class:doris::segment_v2::ColumnIterator +get_row_size src/runtime/descriptors.cpp /^int RowDescriptor::get_row_size() const {$/;" f class:doris::RowDescriptor +get_rows src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::get_rows(scoped_ptr* batch, bool* got_rows) {$/;" f class:doris::BufferedTupleStream2 +get_rowset_by_version src/olap/tablet.cpp /^const RowsetSharedPtr Tablet::get_rowset_by_version(const Version& version) const {$/;" f class:doris::Tablet +get_rowset_id_from_path src/olap/tablet_manager.cpp /^bool TabletManager::get_rowset_id_from_path(const string& path, RowsetId* rowset_id) {$/;" f class:doris::TabletManager +get_rowset_meta src/olap/rowset/rowset_meta_manager.cpp /^OLAPStatus RowsetMetaManager::get_rowset_meta(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id, RowsetMetaSharedPtr rowset_meta) {$/;" f class:doris::RowsetMetaManager +get_runtime_state test/runtime/sorter_test.cpp /^ RuntimeState* get_runtime_state() {return _runtime_state;}$/;" f class:doris::SorterTest +get_scale_base src/runtime/decimalv2_value.h /^ inline static int128_t get_scale_base(int scale) {$/;" f class:doris::DecimalV2Value +get_scale_multiplier src/util/string_parser.hpp /^inline __int128 StringParser::get_scale_multiplier(int scale) {$/;" f class:doris::StringParser +get_scan_context src/runtime/external_scan_context_mgr.cpp /^Status ExternalScanContextMgr::get_scan_context(const std::string& context_id, std::shared_ptr* p_context) {$/;" f class:doris::ExternalScanContextMgr +get_schema test/exec/tablet_info_test.cpp /^TOlapTableSchemaParam get_schema(TDescriptorTable* desc_tbl) {$/;" f namespace:doris +get_schema_hash_full_path src/olap/snapshot_manager.cpp /^string SnapshotManager::get_schema_hash_full_path($/;" f class:doris::SnapshotManager +get_schema_mapping src/olap/schema_change.h /^ SchemaMapping get_schema_mapping() const {$/;" f class:doris::RowBlockChanger +get_scroll_id src/exec/es/es_scroll_parser.cpp /^const std::string& ScrollParser::get_scroll_id() {$/;" f class:doris::ScrollParser +get_seg_pb src/olap/rowset/segment_group.h /^ inline const FileHeader* get_seg_pb(uint32_t seg_id) const {$/;" f class:doris::SegmentGroup +get_segment_groups src/olap/rowset/alpha_rowset_meta.cpp /^void AlphaRowsetMeta::get_segment_groups(std::vector* segment_groups) {$/;" f class:doris::AlphaRowsetMeta +get_shard src/olap/data_dir.cpp /^OLAPStatus DataDir::get_shard(uint64_t* shard) {$/;" f class:doris::DataDir +get_short src/exec/es/es_predicate.cpp /^int16_t ExtLiteral::get_short() {$/;" f class:doris::ExtLiteral +get_sink src/runtime/plan_fragment_executor.h /^ DataSink* get_sink() {$/;" f class:doris::PlanFragmentExecutor +get_sink test/runtime/data_stream_test.cpp /^ const TDataStreamSink& get_sink(TPartitionType::type partition_type) {$/;" f class:doris::DataStreamTest +get_size src/exec/es/es_scroll_parser.cpp /^int ScrollParser::get_size() {$/;" f class:doris::ScrollParser +get_size src/util/blocking_priority_queue.hpp /^ uint32_t get_size() const {$/;" f class:doris::BlockingPriorityQueue +get_size src/util/blocking_queue.hpp /^ uint32_t get_size() const {$/;" f class:doris::BlockingQueue +get_size src/util/slice.h /^ size_t get_size() const { return size; }$/;" f struct:doris::Slice +get_slot src/exprs/slot_ref.h /^inline void* SlotRef::get_slot(TupleRow* row) {$/;" f class:doris::SlotRef +get_slot src/runtime/tuple.h /^ const void* get_slot(int offset) const {$/;" f class:doris::Tuple +get_slot src/runtime/tuple.h /^ void* get_slot(int offset) {$/;" f class:doris::Tuple +get_slot_desc src/exec/es/es_predicate.cpp /^const SlotDescriptor* EsPredicate::get_slot_desc(const SlotRef* slotRef) {$/;" f class:doris::EsPredicate +get_slot_desc src/exec/es_scan_node.cpp /^SlotDescriptor* EsScanNode::get_slot_desc(SlotRef* slotRef) {$/;" f class:doris::EsScanNode +get_slot_descriptor src/runtime/descriptors.cpp /^SlotDescriptor* DescriptorTbl::get_slot_descriptor(SlotId id) const {$/;" f class:doris::DescriptorTbl +get_slot_ids src/exprs/expr.cpp /^int Expr::get_slot_ids(std::vector* slot_ids) const {$/;" f class:doris::Expr +get_slot_ids src/exprs/slot_ref.cpp /^int SlotRef::get_slot_ids(std::vector* slot_ids) const {$/;" f class:doris::SlotRef +get_slot_size src/runtime/primitive_type.h /^inline int get_slot_size(PrimitiveType type) {$/;" f namespace:doris +get_slot_size src/runtime/types.h /^ inline int get_slot_size() const {$/;" f struct:doris::TypeDescriptor +get_small_int src/exec/read_write_util.h /^inline int16_t ReadWriteUtil::get_small_int(const uint8_t* buf) {$/;" f class:doris::ReadWriteUtil +get_small_int_val src/exprs/expr.cpp /^SmallIntVal Expr::get_small_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_small_int_val src/exprs/expr_context.cpp /^SmallIntVal ExprContext::get_small_int_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_small_int_val src/exprs/expr_ir.cpp /^SmallIntVal Expr::get_small_int_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_small_int_val src/exprs/literal.cpp /^SmallIntVal Literal::get_small_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_small_int_val src/exprs/null_literal.cpp /^SmallIntVal NullLiteral::get_small_int_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_small_int_val src/exprs/scalar_fn_call.cpp /^SmallIntVal ScalarFnCall::get_small_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_small_int_val src/exprs/slot_ref.cpp /^SmallIntVal SlotRef::get_small_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_space_available src/util/filesystem_util.cc /^Status FileSystemUtil::get_space_available(const string& directory_path,$/;" f class:doris::FileSystemUtil +get_sparse_map src/olap/hll.h /^ std::map& get_sparse_map() {$/;" f class:doris::HllSetResolver +get_stack_trace src/util/stack_util.cpp /^std::string get_stack_trace() {$/;" f namespace:doris +get_stats src/olap/memtable_flush_executor.h /^ const FlushStatistic& get_stats() const { return _stats; }$/;" f class:doris::FlushToken +get_store src/olap/storage_engine.cpp /^DataDir* StorageEngine::get_store(const std::string& path) {$/;" f class:doris::StorageEngine +get_stores src/olap/storage_engine.cpp /^std::vector StorageEngine::get_stores() {$/;" f class:doris::StorageEngine +get_stores_for_create_tablet src/olap/storage_engine.cpp /^std::vector StorageEngine::get_stores_for_create_tablet($/;" f class:doris::StorageEngine +get_str_err_msg src/util/error_util.cc /^string get_str_err_msg() {$/;" f namespace:doris +get_stream_length src/olap/out_stream.cpp /^uint64_t OutStream::get_stream_length() const {$/;" f class:doris::OutStream +get_string src/exec/es/es_predicate.cpp /^std::string ExtLiteral::get_string() {$/;" f class:doris::ExtLiteral +get_string_by_aggregation_type src/olap/tablet_schema.cpp /^std::string TabletColumn::get_string_by_aggregation_type(FieldAggregationMethod type) {$/;" f class:doris::TabletColumn +get_string_by_field_type src/olap/tablet_schema.cpp /^std::string TabletColumn::get_string_by_field_type(FieldType type) {$/;" f class:doris::TabletColumn +get_string_slot src/runtime/tuple.h /^ StringValue* get_string_slot(int offset) {$/;" f class:doris::Tuple +get_string_slot src/runtime/tuple.h /^ const StringValue* get_string_slot(int offset) const {$/;" f class:doris::Tuple +get_string_val src/exprs/expr.cpp /^StringVal Expr::get_string_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_string_val src/exprs/expr_context.cpp /^StringVal ExprContext::get_string_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_string_val src/exprs/expr_ir.cpp /^StringVal Expr::get_string_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_string_val src/exprs/info_func.cpp /^StringVal InfoFunc::get_string_val(ExprContext* context, TupleRow*) {$/;" f class:doris::InfoFunc +get_string_val src/exprs/literal.cpp /^StringVal Literal::get_string_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_string_val src/exprs/null_literal.cpp /^StringVal NullLiteral::get_string_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_string_val src/exprs/scalar_fn_call.cpp /^StringVal ScalarFnCall::get_string_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_string_val src/exprs/slot_ref.cpp /^StringVal SlotRef::get_string_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_stub src/util/brpc_stub_cache.h /^ palo::PInternalService_Stub* get_stub(const TNetworkAddress& taddr) {$/;" f class:doris::BrpcStubCache +get_stub src/util/brpc_stub_cache.h /^ palo::PInternalService_Stub* get_stub(const butil::EndPoint& endpoint) {$/;" f class:doris::BrpcStubCache +get_stub src/util/brpc_stub_cache.h /^ palo::PInternalService_Stub* get_stub(const std::string& host, int port) {$/;" f class:doris::BrpcStubCache +get_sys_counters src/util/perf_counters.cpp /^bool PerfCounters::get_sys_counters(vector& buffer) {$/;" f class:doris::PerfCounters +get_table_descriptor src/runtime/descriptors.cpp /^TableDescriptor* DescriptorTbl::get_table_descriptor(TableId id) const {$/;" f class:doris::DescriptorTbl +get_table_names src/exec/schema_scanner/schema_helper.cpp /^Status SchemaHelper::get_table_names($/;" f class:doris::SchemaHelper +get_table_names test/exec/schema_scanner/schema_columns_scanner_test.cpp /^Status SchemaJniHelper::get_table_names(const TGetTablesParams &table_params,$/;" f class:doris::SchemaJniHelper +get_table_names test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^Status SchemaJniHelper::get_table_names(const TGetTablesParams &table_params,$/;" f class:doris::SchemaJniHelper +get_table_names test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^Status SchemaJniHelper::get_table_names(const TGetTablesParams &table_params,$/;" f class:doris::SchemaJniHelper +get_table_names test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^Status SchemaJniHelper::get_table_names(const TGetTablesParams &table_params,$/;" f class:doris::SchemaJniHelper +get_table_names test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^Status SchemaJniHelper::get_table_names(const TGetTablesParams &table_params,$/;" f class:doris::SchemaJniHelper +get_table_names test/exec/schema_scanner/schema_tables_scanner_test.cpp /^Status SchemaJniHelper::get_table_names(const TGetTablesParams &table_params,$/;" f class:doris::SchemaJniHelper +get_tablet src/olap/tablet_manager.cpp /^TabletSharedPtr TabletManager::get_tablet(TTabletId tablet_id, SchemaHash schema_hash,$/;" f class:doris::TabletManager +get_tablet_convert_finished src/olap/olap_meta.cpp /^OLAPStatus OlapMeta::get_tablet_convert_finished(bool& flag) {$/;" f class:doris::OlapMeta +get_tablet_id src/olap/rowset/segment_group.cpp /^int64_t SegmentGroup::get_tablet_id() {$/;" f class:doris::SegmentGroup +get_tablet_id_and_schema_hash_from_path src/olap/tablet_manager.cpp /^bool TabletManager::get_tablet_id_and_schema_hash_from_path($/;" f class:doris::TabletManager +get_tablet_info src/olap/tablet.cpp /^TabletInfo Tablet::get_tablet_info() const {$/;" f class:doris::Tablet +get_tablet_related_txns src/olap/txn_manager.cpp /^void TxnManager::get_tablet_related_txns(TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid, $/;" f class:doris::TxnManager +get_tablet_schema src/olap/rowset/segment_group.cpp /^const TabletSchema& SegmentGroup::get_tablet_schema() {$/;" f class:doris::SegmentGroup +get_tablet_stat src/olap/tablet_manager.cpp /^void TabletManager::get_tablet_stat(TTabletStatResult* result) {$/;" f class:doris::TabletManager +get_tablet_stat src/service/backend_service.cpp /^void BackendService::get_tablet_stat(TTabletStatResult& result) {$/;" f class:doris::BackendService +get_task_state src/runtime/export_task_mgr.cpp /^Status ExportTaskMgr::get_task_state(const TUniqueId& id, TExportStatusResult* result) {$/;" f class:doris::ExportTaskMgr +get_thread_session_key src/util/thrift_server.cpp /^ThriftServer::SessionKey* ThriftServer::get_thread_session_key() {$/;" f class:doris::ThriftServer +get_timeout_second src/runtime/fragment_mgr.cpp /^ int get_timeout_second() const { return _timeout_second; } $/;" f class:doris::FragmentExecState +get_tiny_int_val src/exprs/expr.cpp /^TinyIntVal Expr::get_tiny_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_tiny_int_val src/exprs/expr_context.cpp /^TinyIntVal ExprContext::get_tiny_int_val(TupleRow* row) {$/;" f class:doris::ExprContext +get_tiny_int_val src/exprs/expr_ir.cpp /^TinyIntVal Expr::get_tiny_int_val(Expr* expr, ExprContext* context, TupleRow* row) {$/;" f class:doris::Expr +get_tiny_int_val src/exprs/literal.cpp /^TinyIntVal Literal::get_tiny_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::Literal +get_tiny_int_val src/exprs/null_literal.cpp /^TinyIntVal NullLiteral::get_tiny_int_val(ExprContext*, TupleRow*) {$/;" f class:doris::NullLiteral +get_tiny_int_val src/exprs/scalar_fn_call.cpp /^TinyIntVal ScalarFnCall::get_tiny_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +get_tiny_int_val src/exprs/slot_ref.cpp /^TinyIntVal SlotRef::get_tiny_int_val(ExprContext* context, TupleRow* row) {$/;" f class:doris::SlotRef +get_tmp_dir_path src/runtime/tmp_file_mgr.cc /^string TmpFileMgr::get_tmp_dir_path(DeviceId device_id) const {$/;" f class:doris::TmpFileMgr +get_total src/exec/es/es_scroll_parser.cpp /^int ScrollParser::get_total() {$/;" f class:doris::ScrollParser +get_total_buffer_size src/olap/out_stream.cpp /^uint64_t OutStream::get_total_buffer_size() const {$/;" f class:doris::OutStream +get_tracker src/runtime/buffered_block_mgr2.cc /^MemTracker* BufferedBlockMgr2::get_tracker(Client* client) const {$/;" f class:doris::BufferedBlockMgr2 +get_tuple src/exec/partitioned_hash_table.inline.h /^inline Tuple* PartitionedHashTable::Iterator::get_tuple() const {$/;" f class:doris::PartitionedHashTable::Iterator +get_tuple src/exprs/slot_ref.h /^inline Tuple* SlotRef::get_tuple(TupleRow* row) {$/;" f class:doris::SlotRef +get_tuple src/runtime/tuple_row.h /^ Tuple* get_tuple(int tuple_idx) {$/;" f class:doris::TupleRow +get_tuple_descriptor src/runtime/descriptors.cpp /^TupleDescriptor* DescriptorTbl::get_tuple_descriptor(TupleId id) const {$/;" f class:doris::DescriptorTbl +get_tuple_descs src/runtime/descriptors.cpp /^void DescriptorTbl::get_tuple_descs(std::vector* descs) const {$/;" f class:doris::DescriptorTbl +get_tuple_ids src/exec/exec_node.h /^ const std::vector& get_tuple_ids() const {$/;" f class:doris::ExecNode +get_tuple_idx src/runtime/descriptors.cpp /^int RowDescriptor::get_tuple_idx(TupleId id) const {$/;" f class:doris::RowDescriptor +get_tuple_row src/runtime/buffered_tuple_stream.cpp /^inline void BufferedTupleStream::get_tuple_row(const RowIdx& idx, TupleRow* row) const {$/;" f class:doris::BufferedTupleStream +get_tuple_row src/runtime/buffered_tuple_stream2.inline.h /^inline void BufferedTupleStream2::get_tuple_row(const RowIdx& idx, TupleRow* row) const {$/;" f class:doris::BufferedTupleStream2 +get_txn_related_tablets src/olap/txn_manager.cpp /^void TxnManager::get_txn_related_tablets(TTransactionId transaction_id,$/;" f class:doris::TxnManager +get_type src/exec/decompressor.h /^ CompressType get_type() { return _ctype; }$/;" f class:doris::Decompressor +get_type_info src/olap/types.cpp /^ TypeInfo* get_type_info(const FieldType t) {$/;" f class:doris::TypeInfoResolver +get_type_info src/olap/types.cpp /^TypeInfo* get_type_info(FieldType field_type) {$/;" f namespace:doris +get_uint16 src/exec/decompressor.h /^ inline uint8_t* get_uint16(uint8_t* ptr, uint16_t* value) {$/;" f class:doris::LzopDecompressor +get_uint32 src/exec/decompressor.h /^ inline uint8_t* get_uint32(uint8_t* ptr, uint32_t* value) {$/;" f class:doris::LzopDecompressor +get_uint8 src/exec/decompressor.h /^ inline uint8_t* get_uint8(uint8_t* ptr, uint8_t* value) {$/;" f class:doris::LzopDecompressor +get_unit src/util/pretty_printer.h /^ static double get_unit(T value, std::string* unit) {$/;" f class:doris::PrettyPrinter +get_unreported_errors src/runtime/runtime_state.cpp /^void RuntimeState::get_unreported_errors(std::vector* new_errors) {$/;" f class:doris::RuntimeState +get_unused_block src/runtime/buffered_block_mgr2.cc /^BufferedBlockMgr2::Block* BufferedBlockMgr2::get_unused_block(Client* client) {$/;" f class:doris::BufferedBlockMgr2 +get_url_part src/util/url_parser.cpp /^UrlParser::UrlPart UrlParser::get_url_part(const StringValue& part) {$/;" f class:doris::UrlParser +get_usage src/olap/lru_cache.h /^ size_t get_usage() {$/;" f class:doris::CachePriority::LRUCache +get_usage src/tools/meta_tool.cpp /^std::string get_usage(const std::string& progname) {$/;" f +get_val src/exprs/bitmap_function.cpp /^DateTimeValue get_val(const DateTimeVal& x) {$/;" f namespace:doris::detail +get_val src/exprs/bitmap_function.cpp /^DecimalV2Value get_val(const DecimalV2Val& x) {$/;" f namespace:doris::detail +get_val src/exprs/bitmap_function.cpp /^StringValue get_val(const StringVal& x) {$/;" f namespace:doris::detail +get_val src/exprs/bitmap_function.cpp /^T get_val(const ValType& x) {$/;" f namespace:doris::detail +get_val src/exprs/new_in_predicate.cpp /^DateTimeValue get_val($/;" f namespace:doris +get_val src/exprs/new_in_predicate.cpp /^DecimalV2Value get_val($/;" f namespace:doris +get_val src/exprs/new_in_predicate.cpp /^DecimalValue get_val($/;" f namespace:doris +get_val src/exprs/new_in_predicate.cpp /^SetType get_val(const FunctionContext::TypeDesc* type, const T& x) {$/;" f namespace:doris +get_val src/exprs/new_in_predicate.cpp /^StringValue get_val(const FunctionContext::TypeDesc* type, const StringVal& x) {$/;" f namespace:doris +get_value src/exprs/agg_fn_evaluator.h /^inline void AggFnEvaluator::get_value($/;" f class:doris::AggFnEvaluator +get_value src/exprs/agg_fn_evaluator.h /^inline void AggFnEvaluator::get_value(const std::vector& evaluators,$/;" f class:doris::AggFnEvaluator +get_value src/exprs/expr.h /^ void* get_value(TupleRow* row) {$/;" f class:doris::Expr +get_value src/exprs/expr_context.cpp /^void ExprContext::get_value(TupleRow* row, bool as_ascii, TColumnValue* col_val) {$/;" f class:doris::ExprContext +get_value src/exprs/expr_context.cpp /^void* ExprContext::get_value(Expr* e, TupleRow* row) {$/;" f class:doris::ExprContext +get_value src/exprs/expr_context.cpp /^void* ExprContext::get_value(TupleRow* row) {$/;" f class:doris::ExprContext +get_value src/exprs/hybird_set.h /^ virtual const void* get_value() {$/;" f class:doris::HybirdSet::Iterator +get_value src/exprs/hybird_set.h /^ virtual const void* get_value() {$/;" f class:doris::StringValueSet::Iterator +get_value src/exprs/slot_ref.h /^inline void* SlotRef::get_value(Expr* expr, TupleRow* row) {$/;" f class:doris::SlotRef +get_value src/olap/rowset/segment_v2/index_page.h /^ inline const PagePointer& get_value(int idx) const {$/;" f class:doris::segment_v2::IndexPageReader +get_value src/olap/tuple.h /^ const std::string& get_value(size_t i) const { return _values[i]; }$/;" f class:doris::OlapTuple +get_value_fn src/exprs/agg_fn.h /^ void* get_value_fn() const { return get_value_fn_; }$/;" f class:doris::AggFn +get_value_fn_ src/exprs/agg_fn.h /^ void* get_value_fn_ = nullptr;$/;" m class:doris::AggFn +get_vararg_start_idx src/exprs/agg_fn.h /^ const int get_vararg_start_idx() const {$/;" f class:doris::AggFn +get_variable_len src/olap/field.h /^ virtual size_t get_variable_len() const {$/;" f class:doris::Field +get_variable_len src/olap/row_cursor.h /^ inline size_t get_variable_len() const { return _variable_len; }$/;" f class:doris::RowCursor +get_varint32 src/util/coding.h /^inline bool get_varint32(Slice* input, uint32_t* val) {$/;" f namespace:doris +get_varint64 src/util/coding.h /^inline bool get_varint64(Slice* input, uint64_t* val) {$/;" f namespace:doris +get_version_string src/util/debug_util.cpp /^std::string get_version_string(bool compact) {$/;" f namespace:doris +get_vint src/exec/read_write_util.h /^inline int ReadWriteUtil::get_vint(uint8_t* buf, int32_t* vint) {$/;" f class:doris::ReadWriteUtil +get_vlong src/exec/read_write_util.h /^inline int ReadWriteUtil::get_vlong(uint8_t* buf, int64_t offset, int64_t* vlong) {$/;" f class:doris::ReadWriteUtil +get_vlong src/exec/read_write_util.h /^inline int ReadWriteUtil::get_vlong(uint8_t* buf, int64_t* vlong) {$/;" f class:doris::ReadWriteUtil +get_zone_maps src/olap/rowset/segment_group.h /^ const std::vector>& get_zone_maps() {$/;" f class:doris::SegmentGroup +getblock32 src/util/murmur_hash3.cpp /^FORCE_INLINE uint32_t getblock32(const uint32_t* p, int i)$/;" f +getblock64 src/util/murmur_hash3.cpp /^FORCE_INLINE uint64_t getblock64(const uint64_t* p, int i)$/;" f +getcwd src/gutil/port.h 845;" d +gethostbyname src/gutil/port.h 988;" d +getlock src/util/mutex.h /^ pthread_mutex_t* getlock() {$/;" f class:doris::Mutex +getmap src/common/configbase.cpp /^const std::map& Properties::getmap() const {$/;" f class:doris::config::Properties +gettid src/agent/cgroups_mgr.cpp 357;" d file: +gettid src/gutil/linux_syscall_support.h /^ LSS_INLINE pid_t LSS_NAME(gettid)() {$/;" f +ghtonl src/gutil/endian.h /^inline uint32 ghtonl(uint32 x) { return bswap_32(x); }$/;" f +ghtonl src/gutil/endian.h /^inline uint32 ghtonl(uint32 x) { return x; }$/;" f +ghtonll src/gutil/endian.h /^inline uint64 ghtonll(uint64 x) { return gbswap_64(x); }$/;" f +ghtonll src/gutil/endian.h /^inline uint64 ghtonll(uint64 x) { return x; }$/;" f +ghtons src/gutil/endian.h /^inline uint16 ghtons(uint16 x) { return bswap_16(x); }$/;" f +ghtons src/gutil/endian.h /^inline uint16 ghtons(uint16 x) { return x; }$/;" f +gi src/util/minizip/unzip.c /^ unz_global_info64 gi; \/* public global information *\/$/;" m struct:__anon34 file: +give_up_after_ms src/util/thread.cpp /^ThreadJoiner& ThreadJoiner::give_up_after_ms(int ms) {$/;" f class:doris::ThreadJoiner +glog_internal_namespace_ src/util/stack_util.cpp /^namespace glog_internal_namespace_ {$/;" n namespace:google file: +gntohl src/gutil/endian.h 96;" d +gntohll src/gutil/endian.h 98;" d +gntohs src/gutil/endian.h 97;" d +google src/runtime/buffer_control_block.h /^namespace google {$/;" n +google src/runtime/data_stream_mgr.h /^namespace google {$/;" n +google src/runtime/data_stream_recvr.h /^namespace google {$/;" n +google src/util/stack_util.cpp /^namespace google {$/;" n file: +greater_than_scale src/runtime/decimalv2_value.cpp /^bool DecimalV2Value::greater_than_scale(int scale) {$/;" f class:doris::DecimalV2Value +group_consume src/runtime/routine_load/data_consumer.cpp /^Status KafkaDataConsumer::group_consume($/;" f class:doris::KafkaDataConsumer +grouping src/exprs/grouping_sets_functions.cpp /^BigIntVal GroupingSetsFunctions::grouping($/;" f class:doris::GroupingSetsFunctions +grouping_expr_ctxs_ src/exec/new_partitioned_aggregation_node.h /^ std::vector grouping_expr_ctxs_;$/;" m class:doris::NewPartitionedAggregationNode +grouping_exprs_ src/exec/new_partitioned_aggregation_node.h /^ std::vector grouping_exprs_;$/;" m class:doris::NewPartitionedAggregationNode +grouping_id src/exprs/grouping_sets_functions.cpp /^doris_udf::BigIntVal GroupingSetsFunctions::grouping_id($/;" f class:doris::GroupingSetsFunctions +grow_buffer src/runtime/string_buffer.hpp /^ void grow_buffer(int new_len) {$/;" f class:doris::StringBuffer +grow_node_array src/exec/hash_table.cpp /^void HashTable::grow_node_array() {$/;" f class:doris::HashTable +grow_node_array src/exec/partitioned_hash_table.cc /^bool PartitionedHashTable::grow_node_array() {$/;" f class:doris::PartitionedHashTable +grp_id src/runtime/routine_load/data_consumer_group.h /^ const UniqueId& grp_id() { return _grp_id; }$/;" f class:doris::DataConsumerGroup +gscoped_array src/gutil/gscoped_ptr.h /^ explicit gscoped_array(C* p = NULL) : array_(p) { }$/;" f class:gscoped_array +gscoped_array src/gutil/gscoped_ptr.h /^ gscoped_array(RValue rvalue)$/;" f class:gscoped_array +gscoped_array src/gutil/gscoped_ptr.h /^class gscoped_array {$/;" c +gscoped_ptr src/gutil/gscoped_ptr.h /^ explicit gscoped_ptr(element_type* array) : impl_(array) { }$/;" f class:gscoped_ptr +gscoped_ptr src/gutil/gscoped_ptr.h /^ explicit gscoped_ptr(element_type* p) : impl_(p) { }$/;" f class:gscoped_ptr +gscoped_ptr src/gutil/gscoped_ptr.h /^ gscoped_ptr() : impl_(NULL) { }$/;" f class:gscoped_ptr +gscoped_ptr src/gutil/gscoped_ptr.h /^ gscoped_ptr(RValue rvalue) : impl_(&rvalue.object->impl_) { }$/;" f class:gscoped_ptr +gscoped_ptr src/gutil/gscoped_ptr.h /^ gscoped_ptr(element_type* p, const D& d) : impl_(p, d) { }$/;" f class:gscoped_ptr +gscoped_ptr src/gutil/gscoped_ptr.h /^ gscoped_ptr(gscoped_ptr other) : impl_(&other.impl_) {$/;" f class:gscoped_ptr +gscoped_ptr src/gutil/gscoped_ptr.h /^class gscoped_ptr {$/;" c +gscoped_ptr src/gutil/gscoped_ptr.h /^class gscoped_ptr {$/;" c +gscoped_ptr_impl src/gutil/gscoped_ptr.h /^ explicit gscoped_ptr_impl(T* p) : data_(p) { }$/;" f class:doris::internal::gscoped_ptr_impl +gscoped_ptr_impl src/gutil/gscoped_ptr.h /^ gscoped_ptr_impl(T* p, const D& d) : data_(p, d) {}$/;" f class:doris::internal::gscoped_ptr_impl +gscoped_ptr_impl src/gutil/gscoped_ptr.h /^ gscoped_ptr_impl(gscoped_ptr_impl* other)$/;" f class:doris::internal::gscoped_ptr_impl +gscoped_ptr_impl src/gutil/gscoped_ptr.h /^class gscoped_ptr_impl {$/;" c namespace:doris::internal +gscoped_ptr_malloc src/gutil/gscoped_ptr.h /^ explicit gscoped_ptr_malloc(C* p = NULL): ptr_(p) {}$/;" f class:gscoped_ptr_malloc +gscoped_ptr_malloc src/gutil/gscoped_ptr.h /^ gscoped_ptr_malloc(RValue rvalue)$/;" f class:gscoped_ptr_malloc +gscoped_ptr_malloc src/gutil/gscoped_ptr.h /^class gscoped_ptr_malloc {$/;" c +gstrcasestr src/gutil/strings/util.cc /^char *gstrcasestr(const char* haystack, const char* needle) {$/;" f +gstrncasestr src/gutil/strings/util.cc /^char *gstrncasestr(char* haystack, const char* needle, size_t len) {$/;" f +gstrncasestr src/gutil/strings/util.cc /^const char *gstrncasestr(const char* haystack, const char* needle, size_t len) {$/;" f +gstrncasestr_split src/gutil/strings/util.cc /^char *gstrncasestr_split(const char* str,$/;" f +gstrsep src/gutil/strings/util.cc /^char* gstrsep(char** stringp, const char* delim) {$/;" f +gt src/runtime/string_value.h /^ bool gt(const StringValue& other) const {$/;" f struct:doris::StringValue +h src/gutil/bits.h /^ static const T h = ~(l * 0x7f);$/;" m struct:BitPattern +half_ones src/gutil/bits.h /^ static const T half_ones = (static_cast(1) << (sizeof(T)*4)) - 1;$/;" m struct:BitPattern +handle src/http/action/checksum_action.cpp /^void ChecksumAction::handle(HttpRequest *req) {$/;" f class:doris::ChecksumAction +handle src/http/action/compaction_action.cpp /^void CompactionAction::handle(HttpRequest *req) {$/;" f class:doris::CompactionAction +handle src/http/action/health_action.cpp /^void HealthAction::handle(HttpRequest *req) {$/;" f class:doris::HealthAction +handle src/http/action/meta_action.cpp /^void MetaAction::handle(HttpRequest *req) {$/;" f class:doris::MetaAction +handle src/http/action/metrics_action.cpp /^void MetricsAction::handle(HttpRequest* req) {$/;" f class:doris::MetricsAction +handle src/http/action/mini_load.cpp /^void MiniLoadAction::handle(HttpRequest *http_req) {$/;" f class:doris::MiniLoadAction +handle src/http/action/pprof_actions.cpp /^void CmdlineAction::handle(HttpRequest* req) {$/;" f class:doris::CmdlineAction +handle src/http/action/pprof_actions.cpp /^void GrowthAction::handle(HttpRequest* req) {$/;" f class:doris::GrowthAction +handle src/http/action/pprof_actions.cpp /^void HeapAction::handle(HttpRequest* req) {$/;" f class:doris::HeapAction +handle src/http/action/pprof_actions.cpp /^void ProfileAction::handle(HttpRequest* req) {$/;" f class:doris::ProfileAction +handle src/http/action/pprof_actions.cpp /^void SymbolAction::handle(HttpRequest* req) {$/;" f class:doris::SymbolAction +handle src/http/action/reload_tablet_action.cpp /^void ReloadTabletAction::handle(HttpRequest *req) {$/;" f class:doris::ReloadTabletAction +handle src/http/action/restore_tablet_action.cpp /^void RestoreTabletAction::handle(HttpRequest *req) {$/;" f class:doris::RestoreTabletAction +handle src/http/action/snapshot_action.cpp /^void SnapshotAction::handle(HttpRequest *req) {$/;" f class:doris::SnapshotAction +handle src/http/action/stream_load.cpp /^void StreamLoadAction::handle(HttpRequest* req) {$/;" f class:doris::StreamLoadAction +handle src/http/download_action.cpp /^void DownloadAction::handle(HttpRequest *req) {$/;" f class:doris::DownloadAction +handle src/http/monitor_action.cpp /^void MonitorAction::handle(HttpRequest *req) {$/;" f class:doris::MonitorAction +handle src/http/web_page_handler.cpp /^void WebPageHandler::handle(HttpRequest *req) {$/;" f class:doris::WebPageHandler +handle src/runtime/buffered_tuple_stream3.h /^ BufferPool::PageHandle handle;$/;" m struct:doris::BufferedTupleStream3::Page +handle_empty_projection src/exec/kudu_scanner.cpp /^Status KuduScanner::handle_empty_projection(RowBatch* row_batch, bool* batch_done) {$/;" f class:doris::KuduScanner +handle_error_log src/http/download_action.cpp /^void DownloadAction::handle_error_log($/;" f class:doris::DownloadAction +handle_mem_exceed_limit src/runtime/load_channel.cpp /^void LoadChannel::handle_mem_exceed_limit(bool force) {$/;" f class:doris::LoadChannel +handle_normal src/http/download_action.cpp /^void DownloadAction::handle_normal($/;" f class:doris::DownloadAction +handle_parse_result src/exprs/math_functions.cpp /^bool MathFunctions::handle_parse_result($/;" f class:doris::MathFunctions +handle_read_finished src/runtime/disk_io_mgr.cc /^void DiskIoMgr::handle_read_finished(DiskQueue* disk_queue, RequestContext* reader,$/;" f class:doris::DiskIoMgr +handle_timestamp src/exec/parquet_reader.cpp /^Status ParquetReaderWrap::handle_timestamp(const std::shared_ptr& ts_array, uint8_t *buf, int32_t *wbytes) {$/;" f class:doris::ParquetReaderWrap +handle_update src/agent/user_resource_listener.cpp /^void UserResourceListener::handle_update(const TAgentServiceVersion::type& protocol_version, $/;" f class:doris::UserResourceListener +handle_updates src/agent/topic_subscriber.cpp /^void TopicSubscriber::handle_updates(const TAgentPublishRequest& agent_publish_request) {$/;" f class:doris::TopicSubscriber +handle_write_finished src/runtime/disk_io_mgr.cc /^void DiskIoMgr::handle_write_finished(RequestContext* writer, WriteRange* write_range,$/;" f class:doris::DiskIoMgr +handler src/http/action/mini_load.cpp /^ MiniLoadAction* handler;$/;" m struct:doris::MiniLoadAsyncCtx file: +handler src/http/http_request.h /^ HttpHandler* handler() const { return _handler; }$/;" f class:doris::HttpRequest +handler src/plugin/plugin.h /^ void* handler;$/;" m struct:doris::Plugin +handler_ctx src/http/http_request.h /^ void* handler_ctx() const { return _handler_ctx; }$/;" f class:doris::HttpRequest +hardware_flags src/util/cpu_info.h /^ static int64_t hardware_flags() {$/;" f class:doris::CpuInfo +hardware_flags_ src/util/cpu_info.cpp /^int64_t CpuInfo::hardware_flags_ = 0;$/;" m class:doris::CpuInfo file: +hardware_flags_ src/util/cpu_info.h /^ static int64_t hardware_flags_;$/;" m class:doris::CpuInfo +hasDuplicates src/exec/new_partitioned_hash_table.h /^ bool hasDuplicates;$/;" m struct:doris::NewPartitionedHashTable::Bucket +hasDuplicates src/exec/partitioned_hash_table.h /^ bool hasDuplicates;$/;" m struct:doris::PartitionedHashTable::Bucket +has_aesni src/gutil/cpu.h /^ bool has_aesni() const { return has_aesni_; }$/;" f class:base::final +has_aesni_ src/gutil/cpu.h /^ bool has_aesni_;$/;" m class:base::final +has_amd_lock_mb_bug src/gutil/atomicops-internals-tsan.h /^ bool has_amd_lock_mb_bug; \/\/ Processor has AMD memory-barrier bug; do lfence$/;" m struct:AtomicOps_x86CPUFeatureStruct +has_avx src/gutil/cpu.h /^ bool has_avx() const { return has_avx_; }$/;" f class:base::final +has_avx2 src/gutil/cpu.h /^ bool has_avx2() const { return has_avx2_; }$/;" f class:base::final +has_avx2_ src/gutil/cpu.h /^ bool has_avx2_;$/;" m class:base::final +has_avx_ src/gutil/cpu.h /^ bool has_avx_;$/;" m class:base::final +has_bitmap_index src/olap/rowset/segment_v2/column_reader.h /^ bool has_bitmap_index() const { return _bitmap_index_meta != nullptr; }$/;" f class:doris::segment_v2::ColumnReader +has_bitmap_index src/olap/tablet_schema.h /^ inline bool has_bitmap_index() const {return _has_bitmap_index; }$/;" f class:doris::TabletColumn +has_bloom_filter_index src/olap/rowset/segment_v2/column_reader.h /^ bool has_bloom_filter_index() const { return _bf_index_meta != nullptr; }$/;" f class:doris::segment_v2::ColumnReader +has_called src/util/once.h /^ bool has_called() const {$/;" f class:doris::DorisCallOnce +has_case_expr src/exprs/case_expr.h /^ bool has_case_expr() { $/;" f class:doris::CaseExpr +has_cmpxchg16b src/gutil/atomicops-internals-x86.h /^ bool has_cmpxchg16b; \/\/ Processor supports cmpxchg16b instruction.$/;" m struct:GutilAtomicOps_x86CPUFeatureStruct +has_default_value src/olap/tablet_schema.h /^ bool has_default_value() const { return _has_default_value; }$/;" f class:doris::TabletColumn +has_delete_predicate src/olap/rowset/rowset_meta.h /^ bool has_delete_predicate() const {$/;" f class:doris::RowsetMeta +has_else_expr src/exprs/case_expr.h /^ bool has_else_expr() { $/;" f class:doris::CaseExpr +has_error src/udf/udf.cpp /^bool FunctionContext::has_error() const {$/;" f class:doris_udf::FunctionContext +has_fn_ctx src/exprs/expr.h /^ virtual bool has_fn_ctx() const { $/;" f class:doris::Expr +has_intersection src/exec/olap_common.h /^bool ColumnValueRange::has_intersection(ColumnValueRange& range) {$/;" f class:doris::ColumnValueRange +has_less src/gutil/strings/escaping.cc 1979;" d file: +has_less src/gutil/strings/escaping.cc 1981;" d file: +has_limit src/runtime/mem_tracker.h /^ bool has_limit() const {$/;" f class:doris::MemTracker +has_matches_ src/exec/new_partitioned_hash_table.h /^ bool has_matches_;$/;" m class:doris::NewPartitionedHashTable +has_mmx src/gutil/cpu.h /^ bool has_mmx() const { return has_mmx_; }$/;" f class:base::final +has_mmx_ src/gutil/cpu.h /^ bool has_mmx_;$/;" m class:base::final +has_more_const src/exec/union_node.h /^ bool has_more_const(const RuntimeState* state) const {$/;" f class:doris::UnionNode +has_more_materialized src/exec/union_node.h /^ bool has_more_materialized() const {$/;" f class:doris::UnionNode +has_more_passthrough src/exec/union_node.h /^ bool has_more_passthrough() const {$/;" f class:doris::UnionNode +has_more_range src/olap/rowset/segment_v2/segment_iterator.cpp /^ bool has_more_range() const { return !_eof; }$/;" f class:doris::segment_v2::SegmentIterator::BitmapRangeIterator +has_next src/exec/hash_table.h /^ bool has_next() {$/;" f class:doris::HashTable::Iterator +has_next src/exprs/hybird_set.h /^ virtual bool has_next() const {$/;" f class:doris::HybirdSet::Iterator +has_next src/exprs/hybird_set.h /^ virtual bool has_next() const {$/;" f class:doris::StringValueSet::Iterator +has_next src/olap/rowset/run_length_byte_reader.cpp /^bool RunLengthByteReader::has_next() const {$/;" f class:doris::RunLengthByteReader +has_next src/olap/rowset/run_length_integer_reader.h /^ inline bool has_next() const {$/;" f class:doris::RunLengthIntegerReader +has_non_stop_time_stamp_counter src/gutil/cpu.h /^ bool has_non_stop_time_stamp_counter() const {$/;" f class:base::final +has_non_stop_time_stamp_counter_ src/gutil/cpu.h /^ bool has_non_stop_time_stamp_counter_;$/;" m class:base::final +has_not_null src/olap/rowset/segment_v2/zone_map_index.h /^ bool has_not_null = false;$/;" m struct:doris::segment_v2::ZoneMap +has_null src/olap/rowset/segment_v2/bloom_filter.h /^ bool has_null() const {$/;" f class:doris::segment_v2::BloomFilter +has_null src/olap/rowset/segment_v2/column_writer.cpp /^ bool has_null() const { return _has_null; }$/;" f class:doris::segment_v2::NullBitmapBuilder +has_null src/olap/rowset/segment_v2/parsed_page.h /^ bool has_null;$/;" m struct:doris::segment_v2::ParsedPage +has_null src/olap/rowset/segment_v2/zone_map_index.h /^ bool has_null = false;$/;" m struct:doris::segment_v2::ZoneMap +has_null_bitmap src/olap/rowset/segment_v2/bitmap_index_reader.h /^ bool has_null_bitmap() const { return _reader->_has_null; }$/;" f class:doris::segment_v2::BitmapIndexIterator +has_nullable_tuple_ src/runtime/buffered_tuple_stream3.h /^ const bool has_nullable_tuple_;$/;" m class:doris::BufferedTupleStream3 +has_nullbyte src/olap/row_block.h /^ bool has_nullbyte() {$/;" f class:doris::RowBlock +has_popcnt src/gutil/cpu.h /^ bool has_popcnt() const { return has_popcnt_; }$/;" f class:base::final +has_popcnt_ src/gutil/cpu.h /^ bool has_popcnt_;$/;" m class:base::final +has_range_value src/exec/olap_common.h /^ bool has_range_value() {$/;" f class:doris::OlapScanKeys +has_read_block src/runtime/buffered_tuple_stream.h /^ bool has_read_block() const {$/;" f class:doris::BufferedTupleStream +has_read_block src/runtime/buffered_tuple_stream2.h /^ bool has_read_block() const { return _read_block != _blocks.end(); }$/;" f class:doris::BufferedTupleStream2 +has_read_iterator src/runtime/buffered_tuple_stream3.h /^ bool has_read_iterator() const { return has_read_iterator_; }$/;" f class:doris::BufferedTupleStream3 +has_read_iterator_ src/runtime/buffered_tuple_stream3.h /^ bool has_read_iterator_;$/;" m class:doris::BufferedTupleStream3 +has_read_write_page src/runtime/buffered_tuple_stream3.h /^ bool has_read_write_page() const {$/;" f class:doris::BufferedTupleStream3 +has_reference_column src/olap/tablet_schema.h /^ bool has_reference_column() const { return _has_referenced_column; }$/;" f class:doris::TabletColumn +has_remaining src/olap/row_block.h /^ bool has_remaining() const { return _pos < _limit; }$/;" f class:doris::RowBlock +has_remaining src/olap/rowset/segment_v2/parsed_page.h /^ bool has_remaining() const { return offset_in_page < num_rows; }$/;" f struct:doris::segment_v2::ParsedPage +has_remaining src/util/byte_buffer.h /^ bool has_remaining() const { return limit > pos; }$/;" f struct:doris::ByteBuffer +has_sse src/gutil/cpu.h /^ bool has_sse() const { return has_sse_; }$/;" f class:base::final +has_sse2 src/gutil/atomicops-internals-tsan.h /^ bool has_sse2; \/\/ Processor has SSE2.$/;" m struct:AtomicOps_x86CPUFeatureStruct +has_sse2 src/gutil/atomicops-internals-x86.h /^ bool has_sse2; \/\/ Processor has SSE2.$/;" m struct:GutilAtomicOps_x86CPUFeatureStruct +has_sse2 src/gutil/cpu.h /^ bool has_sse2() const { return has_sse2_; }$/;" f class:base::final +has_sse2_ src/gutil/cpu.h /^ bool has_sse2_;$/;" m class:base::final +has_sse3 src/gutil/cpu.h /^ bool has_sse3() const { return has_sse3_; }$/;" f class:base::final +has_sse3_ src/gutil/cpu.h /^ bool has_sse3_;$/;" m class:base::final +has_sse41 src/gutil/cpu.h /^ bool has_sse41() const { return has_sse41_; }$/;" f class:base::final +has_sse41_ src/gutil/cpu.h /^ bool has_sse41_;$/;" m class:base::final +has_sse42 src/gutil/cpu.h /^ bool has_sse42() const { return has_sse42_; }$/;" f class:base::final +has_sse42_ src/gutil/cpu.h /^ bool has_sse42_;$/;" m class:base::final +has_sse_ src/gutil/cpu.h /^ bool has_sse_;$/;" m class:base::final +has_ssse3 src/gutil/cpu.h /^ bool has_ssse3() const { return has_ssse3_; }$/;" f class:base::final +has_ssse3_ src/gutil/cpu.h /^ bool has_ssse3_;$/;" m class:base::final +has_statistic src/olap/stream_index_writer.cpp /^bool PositionEntryWriter::has_statistic() const {$/;" f class:doris::PositionEntryWriter +has_trivial_assign src/gutil/type_traits.h /^template struct has_trivial_assign$/;" s namespace:base +has_trivial_assign src/gutil/type_traits.h /^template struct has_trivial_assign >$/;" s namespace:base +has_trivial_assign src/gutil/type_traits.h /^template struct has_trivial_assign : is_pod { };$/;" s namespace:base +has_trivial_constructor src/gutil/type_traits.h /^template struct has_trivial_constructor$/;" s namespace:base +has_trivial_constructor src/gutil/type_traits.h /^template struct has_trivial_constructor >$/;" s namespace:base +has_trivial_constructor src/gutil/type_traits.h /^template struct has_trivial_constructor : is_pod { };$/;" s namespace:base +has_trivial_constructor src/gutil/type_traits.h /^template struct has_trivial_constructor$/;" s namespace:base +has_trivial_copy src/gutil/type_traits.h /^template struct has_trivial_copy$/;" s namespace:base +has_trivial_copy src/gutil/type_traits.h /^template struct has_trivial_copy >$/;" s namespace:base +has_trivial_copy src/gutil/type_traits.h /^template struct has_trivial_copy : is_pod { };$/;" s namespace:base +has_trivial_copy src/gutil/type_traits.h /^template struct has_trivial_copy : has_trivial_copy { };$/;" s namespace:base +has_trivial_destructor src/gutil/type_traits.h /^template struct has_trivial_destructor$/;" s namespace:base +has_trivial_destructor src/gutil/type_traits.h /^template struct has_trivial_destructor >$/;" s namespace:base +has_trivial_destructor src/gutil/type_traits.h /^template struct has_trivial_destructor : is_pod { };$/;" s namespace:base +has_trivial_destructor src/gutil/type_traits.h /^template struct has_trivial_destructor$/;" s namespace:base +has_tuple_footprint src/runtime/buffered_tuple_stream2.h /^ bool has_tuple_footprint() const {$/;" f class:doris::BufferedTupleStream2 +has_txn src/olap/txn_manager.cpp /^bool TxnManager::has_txn(TPartitionId partition_id, TTransactionId transaction_id,$/;" f class:doris::TxnManager +has_unpinned_pages src/runtime/bufferpool/buffer_pool.cc /^bool BufferPool::ClientHandle::has_unpinned_pages() const {$/;" f class:doris::BufferPool::ClientHandle +has_unpinned_pages src/runtime/bufferpool/buffer_pool_internal.h /^ bool has_unpinned_pages() const {$/;" f class:doris::BufferPool::Client +has_var_len_blocks src/runtime/spill_sorter.cc /^ inline bool has_var_len_blocks() const {$/;" f class:doris::SpillSorter::Run file: +has_varlen_slots src/runtime/descriptors.h /^ bool has_varlen_slots() const { {$/;" f class:doris::TupleDescriptor +has_varlen_slots src/runtime/descriptors.h /^ bool has_varlen_slots() const {$/;" f class:doris::RowDescriptor +has_version src/olap/rowset/rowset_meta.h /^ bool has_version() const {$/;" f class:doris::RowsetMeta +has_write_block src/runtime/buffered_tuple_stream.h /^ bool has_write_block() const {$/;" f class:doris::BufferedTupleStream +has_write_block src/runtime/buffered_tuple_stream2.h /^ bool has_write_block() const { return _write_block != NULL; }$/;" f class:doris::BufferedTupleStream2 +has_write_iterator src/runtime/buffered_tuple_stream3.h /^ bool has_write_iterator() const { return has_write_iterator_; }$/;" f class:doris::BufferedTupleStream3 +has_write_iterator_ src/runtime/buffered_tuple_stream3.h /^ bool has_write_iterator_;$/;" m class:doris::BufferedTupleStream3 +has_zone_map src/olap/rowset/segment_v2/column_reader.h /^ bool has_zone_map() const { return _zone_map_index_meta != nullptr; }$/;" f class:doris::segment_v2::ColumnReader +has_zone_maps src/olap/rowset/segment_group.h /^ bool has_zone_maps() {$/;" f class:doris::SegmentGroup +hash src/exec/new_partitioned_hash_table.h /^ uint32_t hash;$/;" m struct:doris::NewPartitionedHashTable::Bucket +hash src/exec/partitioned_hash_table.h /^ uint32_t hash;$/;" m struct:doris::PartitionedHashTable::Bucket +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::BigIntVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::BooleanVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::DateTimeVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::DecimalV2Val& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::DecimalVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::DoubleVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::FloatVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::IntVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::LargeIntVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::SmallIntVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::StringVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/exprs/anyval_util.h /^ static uint32_t hash(const doris_udf::TinyIntVal& v, int seed) {$/;" f class:doris::AnyValUtil +hash src/gutil/hash/hash.h /^struct hash {$/;" s namespace:__gnu_cxx +hash src/gutil/hash/hash.h /^struct hash > {$/;" s namespace:__gnu_cxx +hash src/gutil/hash/hash.h /^struct hash > {$/;" s namespace:__gnu_cxx +hash src/gutil/hash/hash.h /^template<> struct hash {$/;" s namespace:__gnu_cxx +hash src/gutil/hash/hash.h /^template<> struct hash {$/;" s namespace:__gnu_cxx +hash src/gutil/hash/hash.h /^template<> struct hash {$/;" s namespace:__gnu_cxx +hash src/gutil/hash/hash.h /^template<> struct hash {$/;" s namespace:__gnu_cxx +hash src/gutil/hash/hash.h /^template<> struct hash {$/;" s namespace:__gnu_cxx +hash src/gutil/hash/hash.h /^template struct hash {$/;" s namespace:__gnu_cxx +hash src/gutil/strings/stringpiece.h /^template<> struct hash {$/;" s namespace:std +hash src/olap/lru_cache.cpp /^uint32_t CacheKey::hash(const char* data, size_t n, uint32_t seed) const {$/;" f class:doris::CacheKey +hash src/olap/lru_cache.h /^ uint32_t hash; \/\/ Hash of key(); used for fast sharding and comparisons$/;" m struct:doris::CachePriority::LRUHandle +hash src/olap/rowset/segment_v2/bloom_filter.h /^ uint64_t hash(char* buf, uint32_t size) const {$/;" f class:doris::segment_v2::BloomFilter +hash src/runtime/datetime_value.h /^ inline uint32_t hash(int seed) const {$/;" f class:doris::DateTimeValue +hash src/runtime/datetime_value.h /^ struct hash {$/;" s namespace:std +hash src/runtime/decimal_value.h /^ struct hash {$/;" s namespace:std +hash src/runtime/decimal_value.h /^ uint32_t hash(uint32_t seed) const {$/;" f class:doris::DecimalValue +hash src/runtime/decimalv2_value.h /^ struct hash {$/;" s namespace:std +hash src/runtime/decimalv2_value.h /^ uint32_t hash(uint32_t seed) const {$/;" f class:doris::DecimalV2Value +hash src/runtime/dpp_sink_internal.h /^struct hash {$/;" s namespace:std +hash src/util/hash_util.hpp /^ static uint32_t hash(const void* data, int32_t bytes, uint32_t seed) {$/;" f class:doris::HashUtil +hash src/util/hash_util.hpp /^struct hash<__int128> {$/;" s namespace:std +hash src/util/hash_util.hpp /^struct hash {$/;" s namespace:std +hash src/util/hash_util.hpp /^struct hash {$/;" s namespace:std +hash src/util/hash_util.hpp /^struct hash> {$/;" s namespace:std +hash src/util/uid_util.h /^ size_t hash(size_t seed = 0) const {$/;" f struct:doris::UniqueId +hash src/util/uid_util.h /^struct hash {$/;" s namespace:std +hash test/olap/hll_test.cpp /^static uint64_t hash(uint64_t value) {$/;" f namespace:doris +hash test/olap/skiplist_test.cpp /^ static uint64_t hash(Key key) { return key & 0xff; }$/;" f class:doris::ConcurrentTest file: +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::BigIntVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::BooleanVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::DateTimeVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::DecimalV2Val& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::DecimalVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::DoubleVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::FloatVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::IntVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::LargeIntVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::SmallIntVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::StringVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/exprs/anyval_util.h /^ static uint64_t hash64(const doris_udf::TinyIntVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64 src/util/hash_util.hpp /^ static uint64_t hash64(const void* data, int32_t bytes, uint64_t seed) {$/;" f class:doris::HashUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::BigIntVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::BooleanVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::DateTimeVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::DecimalV2Val& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::DecimalVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::DoubleVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::FloatVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::IntVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::LargeIntVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::SmallIntVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::StringVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash64_murmur src/exprs/anyval_util.h /^ static uint64_t hash64_murmur(const doris_udf::TinyIntVal& v, int64_t seed) {$/;" f class:doris::AnyValUtil +hash_code src/olap/field.h /^uint32_t Field::hash_code(const CellType& cell, uint32_t seed) const {$/;" f class:doris::Field +hash_code src/olap/types.h /^ inline uint32_t hash_code(const void* data, uint32_t seed) const { return _hash_code(data, seed); }$/;" f class:doris::TypeInfo +hash_code src/olap/types.h /^ static inline uint32_t hash_code(const void* data, uint32_t seed) {$/;" f struct:doris::BaseFieldtypeTraits +hash_code src/olap/types.h /^ static uint32_t hash_code(const void* data, uint32_t seed) {$/;" f struct:doris::FieldTypeTraits +hash_code src/olap/wrapper_field.h /^ uint32_t hash_code() const { return _rep->hash_code(*this, 0); }$/;" f class:doris::WrapperField +hash_current_row src/exec/hash_table.h /^ uint32_t IR_NO_INLINE hash_current_row() {$/;" f class:doris::HashTable +hash_function_num src/olap/bloom_filter.hpp /^ uint32_t hash_function_num() const {$/;" f class:doris::BloomFilter +hash_help src/exec/partitioned_hash_table.h /^ uint32_t inline hash_help(const void* input, int len, int32_t hash) {$/;" f class:doris::PartitionedHashTableCtx +hash_internal src/gutil/hash/string_hash.h /^namespace hash_internal {$/;" n +hash_numbers test/olap/skiplist_test.cpp /^ static uint64_t hash_numbers(uint64_t k, uint64_t g) {$/;" f class:doris::ConcurrentTest file: +hash_of_path src/util/string_util.cpp /^size_t hash_of_path(const std::string& identifier, const std::string& path) {$/;" f namespace:doris +hash_partitions_ src/exec/new_partitioned_aggregation_node.h /^ std::vector hash_partitions_;$/;" m class:doris::NewPartitionedAggregationNode +hash_row src/olap/row.h /^uint32_t hash_row(const RowType& row, uint32_t seed) {$/;" f namespace:doris +hash_set src/runtime/dpp_sink.cpp /^ std::set hash_set;$/;" m struct:doris::HllDppSinkMerge::HllMergeValue file: +hash_tbl src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr hash_tbl;$/;" m struct:doris::NewPartitionedAggregationNode::Partition +hash_tbl src/exec/partitioned_aggregation_node.h /^ boost::scoped_ptr hash_tbl;$/;" m struct:doris::PartitionedAggregationNode::Partition +hash_tbls_ src/exec/new_partitioned_aggregation_node.h /^ NewPartitionedHashTable* hash_tbls_[PARTITION_FANOUT];$/;" m class:doris::NewPartitionedAggregationNode +hash_uint src/runtime/decimal_value.h /^ uint32_t hash_uint(uint32_t value, uint32_t seed) const {$/;" f class:doris::DecimalValue +hash_value src/runtime/datetime_value.cpp /^std::size_t hash_value(DateTimeValue const& value) {$/;" f namespace:doris +hash_value src/runtime/decimal_value.cpp /^std::size_t hash_value(DecimalValue const& value) {$/;" f namespace:doris +hash_value src/runtime/decimalv2_value.cpp /^std::size_t hash_value(DecimalV2Value const& value) {$/;" f namespace:doris +hash_value src/runtime/large_int_value.cpp /^std::size_t hash_value(__int128 const& value) {$/;" f namespace:doris +hash_value src/runtime/string_value.h /^inline std::size_t hash_value(const StringValue& v) {$/;" f namespace:doris +hash_value src/util/container_util.hpp /^inline std::size_t hash_value(const TNetworkAddress& host_port) {$/;" f namespace:doris +hash_value src/util/uid_util.h /^inline std::size_t hash_value(const doris::TUniqueId& id) {$/;" f namespace:doris +hash_variable_len_row src/exec/hash_table.cpp /^uint32_t HashTable::hash_variable_len_row() {$/;" f class:doris::HashTable +hash_variable_len_row src/exec/partitioned_hash_table.cc /^uint32_t PartitionedHashTableCtx::hash_variable_len_row() {$/;" f class:doris::PartitionedHashTableCtx +haveUnprocessed src/util/tdigest.h /^ bool haveUnprocessed() const { return _unprocessed.size() > 0; }$/;" f class:doris::TDigest +have_futex src/gutil/spinlock_linux-inl.h /^static bool have_futex;$/;" v +hdfsFS src/common/hdfs.h /^typedef void* hdfsFS;$/;" t +hdfsFS src/runtime/disk_io_mgr.h /^typedef void* hdfsFS;$/;" t namespace:doris +hdfsFile src/common/hdfs.h /^typedef void* hdfsFile;$/;" t +hdfsFile src/runtime/disk_io_mgr.h /^typedef void* hdfsFile;$/;" t namespace:doris +head src/http/http_client.h /^ Status head() {$/;" f class:doris::HttpClient +head src/olap/rowset/segment_v2/column_writer.h /^ Page* head = nullptr;$/;" m struct:doris::segment_v2::ColumnWriter::PageHead +head src/util/internal_queue.h /^ T* head() const {$/;" f class:doris::InternalQueueBase +head_ src/olap/skiplist.h /^ Node* const head_;$/;" m class:doris::SkipList +head_ src/util/internal_queue.h /^ Node *head_, *tail_;$/;" m class:doris::InternalQueueBase +header src/http/http_request.cpp /^const std::string& HttpRequest::header(const std::string& key) const {$/;" f class:doris::HttpRequest +header_checksum_type src/exec/decompressor.h /^ LzoChecksum header_checksum_type;$/;" m struct:doris::LzopDecompressor::HeaderInfo +header_size src/exec/decompressor.h /^ uint32_t header_size;$/;" m struct:doris::LzopDecompressor::HeaderInfo +header_size src/olap/push_handler.h /^ size_t header_size() const {$/;" f class:doris::BinaryFile +header_type src/exec/decompressor.h /^ inline LzoChecksum header_type(int flags) {$/;" f class:doris::LzopDecompressor +headers src/http/http_request.h /^ const StringCaseUnorderedMap& headers() {$/;" f class:doris::HttpRequest +headers src/http/http_response.h /^ const std::map>& headers() const {$/;" f class:doris::HttpResponse +health_check src/olap/data_dir.cpp /^void DataDir::health_check() {$/;" f class:doris::DataDir +heapify src/runtime/sorted_run_merger.cc /^void SortedRunMerger::heapify(int parent_index) {$/;" f class:doris::SortedRunMerger +heartbeat src/agent/heartbeat_server.cpp /^void HeartbeatServer::heartbeat($/;" f class:doris::HeartbeatServer +heartbeat_flags src/runtime/exec_env.h /^ HeartbeatFlags* heartbeat_flags() { return _heartbeat_flags; }$/;" f class:doris::ExecEnv +help src/service/doris_main.cpp /^static void help(const char* progname) {$/;" f file: +helper test/olap/column_reader_test.cpp /^ FileHandler helper;$/;" m class:doris::TestColumn file: +helper test/olap/run_length_byte_test.cpp /^ FileHandler helper;$/;" m class:doris::TestRunLengthByte file: +helper test/olap/run_length_integer_test.cpp /^ FileHandler helper;$/;" m class:doris::TestRunLengthSignInteger file: +helper test/olap/run_length_integer_test.cpp /^ FileHandler helper;$/;" m class:doris::TestRunLengthUnsignInteger file: +helper test/olap/schema_change_test.cpp /^ FileHandler helper;$/;" m class:doris::TestColumn file: +hex src/util/md5.h /^ const std::string& hex() const {$/;" f class:doris::Md5Digest +hex_char src/gutil/strings/escaping.cc /^static char hex_char[] = "0123456789abcdef";$/;" m namespace:strings file: +hex_digit_to_int src/gutil/strings/escaping.h /^inline int hex_digit_to_int(char c) {$/;" f namespace:strings +hex_dump src/exec/read_write_util.cpp /^std::string ReadWriteUtil::hex_dump(const char* buf, int64_t length) {$/;" f class:doris::ReadWriteUtil +hex_dump src/exec/read_write_util.cpp /^std::string ReadWriteUtil::hex_dump(const uint8_t* buf, int64_t length) {$/;" f class:doris::ReadWriteUtil +hex_int src/exprs/math_functions.cpp /^StringVal MathFunctions::hex_int(FunctionContext* ctx, const BigIntVal& v) {$/;" f class:doris::MathFunctions +hex_string src/exprs/math_functions.cpp /^StringVal MathFunctions::hex_string(FunctionContext* ctx, const StringVal& s) {$/;" f class:doris::MathFunctions +hex_value src/gutil/strings/escaping.cc /^static char hex_value[256] = {$/;" m namespace:strings file: +hexdump src/util/debug_util.cpp /^std::string hexdump(const char* buf, int len) {$/;" f namespace:doris +hi output/udf/include/udf.h /^ int64_t hi;$/;" m struct:doris_udf::FunctionContext::UniqueId +hi src/gutil/int128.h /^ uint64 hi;$/;" m struct:uint128_pod +hi src/olap/olap_common.h /^ int64_t hi = 0;$/;" m struct:doris::RowsetId +hi src/udf/udf.h /^ int64_t hi;$/;" m struct:doris_udf::FunctionContext::UniqueId +hi src/util/uid_util.h /^ int64_t hi = 0;$/;" m struct:doris::UniqueId +hi_ src/gutil/int128.h /^ uint64 hi_;$/;" m class:uint128 +hide src/util/metrics.cpp /^void Metric::hide() {$/;" f class:doris::Metric +highBytes src/util/bitmap_value.h /^ static uint32_t highBytes(const uint64_t in) { return uint32_t(in >> 32); }$/;" f class:doris::detail::Roaring64Map +high_bits src/runtime/multi_precision.h /^inline uint64_t high_bits(__int128 x) {$/;" f namespace:doris +high_nibbles src/util/simdutf8check.h /^ __m128i high_nibbles;$/;" m struct:processed_utf_bytes +high_nibbles src/util/simdutf8check.h /^ __m256i high_nibbles;$/;" m struct:avx_processed_utf_bytes +hll_algorithm src/exprs/aggregate_functions.cpp /^int64_t AggregateFunctions::hll_algorithm(uint8_t *pdata, int data_len) {$/;" f class:doris::AggregateFunctions +hll_algorithm src/exprs/aggregate_functions.h /^ static int64_t hll_algorithm(const HllVal &dst) {$/;" f class:doris::AggregateFunctions +hll_algorithm src/exprs/aggregate_functions.h /^ static int64_t hll_algorithm(const StringVal &dst) {$/;" f class:doris::AggregateFunctions +hll_cardinality src/exprs/hll_function.cpp /^BigIntVal HllFunctions::hll_cardinality(FunctionContext* ctx, const StringVal& input) {$/;" f class:doris::HllFunctions +hll_cardinality src/exprs/hll_hash_function.cpp /^BigIntVal HllHashFunctions::hll_cardinality(FunctionContext* ctx, const HllVal& input) {$/;" f class:doris::HllHashFunctions +hll_empty src/exprs/hll_function.cpp /^StringVal HllFunctions::hll_empty(FunctionContext* ctx) {$/;" f class:doris::HllFunctions +hll_finalize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::hll_finalize(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::AggregateFunctions +hll_finalize src/exprs/hll_function.cpp /^BigIntVal HllFunctions::hll_finalize(FunctionContext*, const StringVal &src) {$/;" f class:doris::HllFunctions +hll_get_value src/exprs/hll_function.cpp /^BigIntVal HllFunctions::hll_get_value(FunctionContext*, const StringVal &src) {$/;" f class:doris::HllFunctions +hll_hash src/exec/csv_scan_node.cpp /^void CsvScanNode::hll_hash(const char* src, int len, std::string* result) {$/;" f class:doris::CsvScanNode +hll_hash src/exprs/hll_function.cpp /^StringVal HllFunctions::hll_hash(FunctionContext* ctx, const StringVal& input) {$/;" f class:doris::HllFunctions +hll_hash src/exprs/hll_hash_function.cpp /^StringVal HllHashFunctions::hll_hash(FunctionContext* ctx, const StringVal& input) {$/;" f class:doris::HllHashFunctions +hll_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::hll_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +hll_init src/exprs/hll_function.cpp /^void HllFunctions::hll_init(FunctionContext *, StringVal* dst) {$/;" f class:doris::HllFunctions +hll_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::hll_merge(FunctionContext* ctx, const StringVal& src,$/;" f class:doris::AggregateFunctions +hll_merge src/exprs/hll_function.cpp /^void HllFunctions::hll_merge(FunctionContext*, const StringVal& src, StringVal* dst) {$/;" f class:doris::HllFunctions +hll_raw_agg_finalize src/exprs/aggregate_functions.cpp /^doris_udf::HllVal AggregateFunctions::hll_raw_agg_finalize($/;" f class:doris::AggregateFunctions +hll_raw_agg_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::hll_raw_agg_init($/;" f class:doris::AggregateFunctions +hll_raw_agg_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::hll_raw_agg_merge($/;" f class:doris::AggregateFunctions +hll_raw_agg_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::hll_raw_agg_update($/;" f class:doris::AggregateFunctions +hll_serialize src/exprs/hll_function.cpp /^StringVal HllFunctions::hll_serialize(FunctionContext *ctx, const StringVal &src) {$/;" f class:doris::HllFunctions +hll_union_agg_finalize src/exprs/aggregate_functions.cpp /^doris_udf::BigIntVal AggregateFunctions::hll_union_agg_finalize(doris_udf::FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +hll_union_agg_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::hll_union_agg_init(FunctionContext* ctx, HllVal* dst) {$/;" f class:doris::AggregateFunctions +hll_union_agg_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::hll_union_agg_merge(FunctionContext* ctx, const HllVal& src, HllVal* dst) {$/;" f class:doris::AggregateFunctions +hll_union_agg_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::hll_union_agg_update(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +hll_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::hll_update(FunctionContext* ctx, const T& src, StringVal* dst) {$/;" f class:doris::AggregateFunctions +hll_update src/exprs/hll_function.cpp /^void HllFunctions::hll_update(FunctionContext *, const T &src, StringVal* dst) {$/;" f class:doris::HllFunctions +home_core_ src/runtime/bufferpool/buffer_pool.h /^ int home_core_;$/;" m class:doris::BufferPool::BufferHandle +host src/exec/mysql_scanner.h /^ std::string host;$/;" m struct:doris::MysqlScannerParam +host src/exec/tablet_info.h /^ std::string host;$/;" m struct:doris::NodeInfo +host src/runtime/descriptors.h /^ const std::string host() const {$/;" f class:doris::MySQLTableDescriptor +host src/runtime/mysql_table_writer.h /^ std::string host;$/;" m struct:doris::MysqlConnInfo +host src/util/mysql_load_error_hub.h /^ std::string host;$/;" m struct:doris::MysqlLoadErrorHub::MysqlInfo +hostname test/http/http_client_test.cpp /^static std::string hostname = "";$/;" m namespace:doris file: +hostname test/runtime/user_function_cache_test.cpp /^static std::string hostname = "";$/;" m namespace:doris file: +hostname_to_ip_addrs src/util/network_util.cpp /^Status hostname_to_ip_addrs(const std::string& name, std::vector* addresses) {$/;" f namespace:doris +hour src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::hour($/;" f class:doris::TimestampFunctions +hour src/runtime/datetime_value.h /^ int hour() const {$/;" f class:doris::DateTimeValue +hour src/runtime/datetime_value.h /^ int32_t hour;$/;" m struct:doris::TimeInterval +hours_add src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::hours_add($/;" f class:doris::TimestampFunctions +hours_diff src/exprs/timestamp_functions.cpp /^BigIntVal TimestampFunctions::hours_diff($/;" f class:doris::TimestampFunctions +hours_sub src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::hours_sub($/;" f class:doris::TimestampFunctions +ht_allocator_ src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr ht_allocator_;$/;" m class:doris::NewPartitionedAggregationNode +ht_ctx_ src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr ht_ctx_;$/;" m class:doris::NewPartitionedAggregationNode +ht_resize_timer_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* ht_resize_timer_;$/;" m class:doris::NewPartitionedAggregationNode +htdata src/exec/new_partitioned_hash_table.h /^ HtData htdata;$/;" m union:doris::NewPartitionedHashTable::Bucket::__anon41 +htdata src/exec/new_partitioned_hash_table.h /^ HtData htdata;$/;" m struct:doris::NewPartitionedHashTable::DuplicateNode +htdata src/exec/partitioned_hash_table.h /^ HtData htdata;$/;" m union:doris::PartitionedHashTable::Bucket::__anon42 +htdata src/exec/partitioned_hash_table.h /^ HtData htdata;$/;" m struct:doris::PartitionedHashTable::DuplicateNode +htonll src/gutil/endian.h 101;" d +http_parse_chunked src/http/http_parser.cpp /^HttpParser::ParseState HttpParser::http_parse_chunked($/;" f class:doris::HttpParser +http_request_duration_us src/util/doris_metrics.cpp /^IntCounter DorisMetrics::http_request_duration_us;$/;" m class:doris::DorisMetrics file: +http_request_duration_us src/util/doris_metrics.h /^ static IntCounter http_request_duration_us;$/;" m class:doris::DorisMetrics +http_request_send_bytes src/util/doris_metrics.cpp /^IntCounter DorisMetrics::http_request_send_bytes;$/;" m class:doris::DorisMetrics file: +http_request_send_bytes src/util/doris_metrics.h /^ static IntCounter http_request_send_bytes;$/;" m class:doris::DorisMetrics +http_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::http_requests_total;$/;" m class:doris::DorisMetrics file: +http_requests_total src/util/doris_metrics.h /^ static IntCounter http_requests_total;$/;" m class:doris::DorisMetrics +hybird_set src/exprs/in_predicate.h /^ HybirdSetBase* hybird_set() const {$/;" f class:doris::InPredicate +hypot src/gutil/port.h 839;" d +hypotf src/gutil/port.h 840;" d +i src/util/bitmap_value.h /^ roaring_uint32_iterator_t i;$/;" m class:doris::detail::final +id src/exec/exec_node.h /^ int id() const {$/;" f class:doris::ExecNode +id src/exec/olap_scan_node.h /^ int id;$/;" m struct:doris::OlapScanNode::__anon40 +id src/exec/olap_scanner.h /^ int id() const { return _id; }$/;" f class:doris::OlapScanner +id src/exec/schema_scanner/schema_collations_scanner.h /^ int64_t id;$/;" m struct:doris::SchemaCollationsScanner::CollationStruct +id src/exec/tablet_info.h /^ int64_t id = 0;$/;" m struct:doris::OlapTablePartition +id src/exec/tablet_info.h /^ int64_t id;$/;" m struct:doris::NodeInfo +id src/olap/fs/block_id.h /^ uint64_t id() const { return _id; }$/;" f class:doris::BlockId +id src/olap/fs/file_block_manager.cpp /^const BlockId& FileReadableBlock::id() const {$/;" f class:doris::fs::internal::FileReadableBlock +id src/olap/fs/file_block_manager.cpp /^const BlockId& FileWritableBlock::id() const {$/;" f class:doris::fs::internal::FileWritableBlock +id src/olap/rowset/segment_v2/segment.h /^ uint64_t id() const { return _segment_id; }$/;" f class:doris::segment_v2::Segment +id src/runtime/descriptors.h /^ SlotId id() const {$/;" f class:doris::SlotDescriptor +id src/runtime/descriptors.h /^ TupleId id() const {$/;" f class:doris::TupleDescriptor +id src/runtime/dpp_sink_internal.h /^ int64_t id() const {$/;" f class:doris::PartitionInfo +id src/runtime/routine_load/data_consumer.h /^ const UniqueId& id() { return _id; }$/;" f class:doris::DataConsumer +id src/runtime/stream_load/stream_load_context.h /^ UniqueId id;$/;" m class:doris::StreamLoadContext +id src/runtime/tablets_channel.h /^ UniqueId id;$/;" m struct:doris::TabletsChannelKey +id src/util/disk_info.h /^ int id;$/;" m struct:doris::DiskInfo::Disk +id_in_use src/olap/rowset/unique_rowset_id_generator.cpp /^bool UniqueRowsetIdGenerator::id_in_use(const RowsetId& rowset_id) {$/;" f class:doris::UniqueRowsetIdGenerator +identity_ src/gutil/template_util.h /^struct identity_ {$/;" s namespace:base +idx src/exec/new_partitioned_aggregation_node.h /^ const int idx;$/;" m struct:doris::NewPartitionedAggregationNode::Partition +idx src/exec/partitioned_hash_table.h /^ BufferedTupleStream2::RowIdx idx;$/;" m union:doris::PartitionedHashTable::HtData +idx src/exprs/json_functions.h /^ int idx; \/\/ array index of a json array, -1 means not set$/;" m struct:doris::JsonPath +idx src/runtime/buffered_tuple_stream.h /^ uint64_t idx() const {$/;" f struct:doris::BufferedTupleStream::RowIdx +idx src/runtime/buffered_tuple_stream2.h /^ uint64_t idx() const {$/;" f struct:doris::BufferedTupleStream2::RowIdx +iequals src/common/logconfig.cpp /^static bool iequals(const std::string& a, const std::string& b)$/;" f namespace:doris +if_ src/gutil/template_util.h /^struct if_ {$/;" s namespace:base +if_ src/gutil/template_util.h /^struct if_{$/;" s namespace:base +iface src/util/thrift_client.h /^ InterfaceType* iface() {$/;" f class:doris::ThriftClient +ignore_cast src/exec/es/es_predicate.cpp /^static bool ignore_cast(const SlotDescriptor* slot, const Expr* expr) {$/;" f namespace:doris +ignore_cast src/exec/es_scan_node.cpp /^bool EsScanNode::ignore_cast(SlotDescriptor* slot, Expr* expr) {$/;" f class:doris::EsScanNode +ignore_cast src/exec/olap_scan_node.cpp /^static bool ignore_cast(SlotDescriptor* slot, Expr* expr) {$/;" f namespace:doris +ignore_result src/gutil/basictypes.h /^inline void ignore_result(const T&) {$/;" f +ignored src/olap/stream_index_common.h /^ bool ignored() const {$/;" f class:doris::ColumnStatistics +impala test/util/perf_counters_test.cpp /^namespace impala {$/;" n file: +impala test/util/runtime_profile_test.cpp /^namespace impala {$/;" n file: +impl output/udf/include/udf.h /^ doris::FunctionContextImpl* impl() {$/;" f class:doris_udf::FunctionContext +impl src/udf/udf.h /^ doris::FunctionContextImpl* impl() {$/;" f class:doris_udf::FunctionContext +impl_ src/gutil/gscoped_ptr.h /^ doris::internal::gscoped_ptr_impl impl_;$/;" m class:gscoped_ptr +impl_ src/runtime/bufferpool/buffer_pool.h /^ Client* impl_;$/;" m class:doris::BufferPool::ClientHandle +implicit_cast src/gutil/casts.h /^inline To implicit_cast(typename base::identity_::type const &f) {$/;" f +implicit_cast src/gutil/casts.h /^inline To implicit_cast(typename base::identity_::type to) {$/;" f +import_label src/runtime/runtime_state.h /^ const std::string& import_label() {$/;" f class:doris::RuntimeState +in_cache src/olap/lru_cache.h /^ bool in_cache; \/\/ Whether entry is in the cache.$/;" m struct:doris::CachePriority::LRUHandle +in_dtor_ src/gutil/ref_counted.h /^ mutable bool in_dtor_;$/;" m class:doris::subtle::RefCountedBase +in_dtor_ src/gutil/ref_counted.h /^ mutable bool in_dtor_;$/;" m class:doris::subtle::RefCountedThreadSafeBase +in_eco_mode src/olap/tablet.h /^ inline bool in_eco_mode() { return false; }$/;" f class:doris::Tablet +in_flight src/runtime/row_batch.h /^ bool in_flight() const {$/;" f class:doris::RowBatch +in_flight_ranges src/runtime/disk_io_mgr_internal.h /^ InternalQueue* in_flight_ranges() { return &_in_flight_ranges; }$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +in_flight_ranges src/runtime/disk_io_mgr_internal.h /^ const InternalQueue* in_flight_ranges() const {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +in_flight_write_pages_ src/runtime/bufferpool/buffer_pool_internal.h /^ PageList in_flight_write_pages_;$/;" m class:doris::BufferPool::Client +in_queue src/util/internal_queue.h /^ bool in_queue() const { return parent_queue != NULL; }$/;" f struct:doris::InternalQueueBase::Node +in_restore_mode src/olap/tablet_meta.h /^inline bool TabletMeta::in_restore_mode() const {$/;" f class:doris::TabletMeta +in_use_ src/runtime/bufferpool/suballocator.h /^ bool in_use_;$/;" m class:doris::Suballocation +inc src/util/counter_cond_variable.hpp /^ void inc(int inc = 1) {$/;" f class:doris::CounterCondVariable +include_lower src/olap/iterators.h /^ bool include_lower;$/;" m struct:doris::StorageReadOptions::KeyRange +include_upper src/olap/iterators.h /^ bool include_upper;$/;" m struct:doris::StorageReadOptions::KeyRange +increase_buf src/runtime/dpp_writer.cpp /^void DppWriter::increase_buf(int len) {$/;" f class:doris::DppWriter +increase_deny_probability_ src/runtime/bufferpool/reservation_tracker.h /^ double increase_deny_probability_ = 0.0;$/;" m class:doris::ReservationTracker +increment src/util/metrics.h /^ void increment(const T& delta) {$/;" f class:doris::CoreLocalCounter +increment src/util/metrics.h /^ void increment(const T& delta) {$/;" f class:doris::LockSimpleMetric +increment_num_removes src/udf/udf_internal.h /^ void increment_num_removes() { $/;" f class:doris::FunctionContextImpl +increment_num_removes src/udf/udf_internal.h /^ void increment_num_removes(int64_t n) { $/;" f class:doris::FunctionContextImpl +increment_num_updates src/udf/udf_internal.h /^ void increment_num_updates() { $/;" f class:doris::FunctionContextImpl +increment_num_updates src/udf/udf_internal.h /^ void increment_num_updates(int64_t n) { $/;" f class:doris::FunctionContextImpl +increment_request_thread_and_dequeue src/runtime/disk_io_mgr_internal.h /^ void increment_request_thread_and_dequeue() {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +index src/olap/rowset/column_writer.h /^ StreamIndexWriter* index() {$/;" f class:doris::ColumnWriter +index_cmp src/olap/field.h /^int Field::index_cmp(const LhsCellType& lhs, const RhsCellType& rhs) const {$/;" f class:doris::Field +index_compare_row src/olap/row.h /^int index_compare_row(const LhsRowType& lhs, const RhsRowType& rhs) {$/;" f namespace:doris +index_disk_size src/olap/rowset/rowset.h /^ size_t index_disk_size() const { return rowset_meta()->index_disk_size(); }$/;" f class:doris::Rowset +index_disk_size src/olap/rowset/rowset_meta.h /^ size_t index_disk_size() const {$/;" f class:doris::RowsetMeta +index_entry src/olap/rowset/column_writer.h /^ PositionEntryWriter* index_entry() {$/;" f class:doris::ColumnWriter +index_id src/exec/tablet_info.h /^ int64_t index_id;$/;" m struct:doris::OlapTableIndexSchema +index_id src/runtime/tablets_channel.h /^ int64_t index_id;$/;" m struct:doris::TabletsChannelKey +index_length src/olap/tablet_schema.h /^ size_t index_length() const { return _index_length; }$/;" f class:doris::TabletColumn +index_load_ns src/olap/olap_common.h /^ int64_t index_load_ns = 0;$/;" m struct:doris::OlapReaderStatistics +index_loaded src/olap/rowset/segment_group.cpp /^bool SegmentGroup::index_loaded() {$/;" f class:doris::SegmentGroup +index_of src/exprs/string_functions.cpp /^static int index_of(const uint8_t* source, int source_offset, int source_count,$/;" f namespace:doris +index_offset src/olap/olap_index.h /^ uint32_t index_offset; \/\/ offset in index file$/;" m struct:doris::RowBlockPosition +index_page_size src/olap/rowset/segment_v2/indexed_column_writer.h /^ size_t index_page_size = 64 * 1024;$/;" m struct:doris::segment_v2::IndexedColumnWriterOptions +index_size src/olap/field.h /^ inline size_t index_size() const { return _index_size; }$/;" f class:doris::Field +index_size src/olap/olap_index.h /^ const size_t index_size() const {$/;" f class:doris::MemIndex +index_size src/olap/rowset/segment_group.h /^ size_t index_size() const {$/;" f class:doris::SegmentGroup +index_size src/olap/schema.h /^ size_t index_size(ColumnId cid) const {$/;" f class:doris::Schema +index_stream_lru_cache src/olap/storage_engine.h /^ Cache* index_stream_lru_cache() {$/;" f class:doris::StorageEngine +index_to_value src/runtime/dpp_sink.cpp /^ std::map index_to_value;$/;" m struct:doris::HllDppSinkMerge::HllMergeValue file: +indexes src/exec/tablet_info.h /^ const std::vector& indexes() const {$/;" f class:doris::OlapTableSchemaParam +indexes src/exec/tablet_info.h /^ std::vector indexes;$/;" m struct:doris::OlapTablePartition +info_arch build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^char const* info_arch = "INFO" ":" "arch[" ARCHITECTURE_ID "]";$/;" v +info_arch build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^char const* info_arch = "INFO" ":" "arch[" ARCHITECTURE_ID "]";$/;" v +info_compiler build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^char const* info_compiler = "INFO" ":" "compiler[" COMPILER_ID "]";$/;" v +info_compiler build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^char const* info_compiler = "INFO" ":" "compiler[" COMPILER_ID "]";$/;" v +info_cray build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^char const *info_cray = "INFO" ":" "compiler_wrapper[CrayPrgEnv]";$/;" v +info_cray build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^char const *info_cray = "INFO" ":" "compiler_wrapper[CrayPrgEnv]";$/;" v +info_language_dialect_default build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^const char* info_language_dialect_default =$/;" v +info_language_dialect_default build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^const char* info_language_dialect_default = "INFO" ":" "dialect_default["$/;" v +info_platform build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^char const* info_platform = "INFO" ":" "platform[" PLATFORM_ID "]";$/;" v +info_platform build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^char const* info_platform = "INFO" ":" "platform[" PLATFORM_ID "]";$/;" v +info_simulate build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^char const* info_simulate = "INFO" ":" "simulate[" SIMULATE_ID "]";$/;" v +info_simulate build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^char const* info_simulate = "INFO" ":" "simulate[" SIMULATE_ID "]";$/;" v +info_simulate_version build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^char const info_simulate_version[] = {$/;" v +info_simulate_version build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^char const info_simulate_version[] = {$/;" v +info_version build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^char const info_version[] = {$/;" v +info_version build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^char const info_version[] = {$/;" v +info_version_internal build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^char const info_version_internal[] = {$/;" v +info_version_internal build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^char const info_version_internal[] = {$/;" v +init src/agent/pusher.cpp /^AgentStatus Pusher::init() {$/;" f class:doris::Pusher +init src/common/configbase.cpp /^bool init(const char* filename, bool fillconfmap) {$/;" f namespace:doris::config +init src/common/resource_tls.cpp /^void ResourceTls::init() {$/;" f class:doris::ResourceTls +init src/exec/aggregation_node.cpp /^Status AggregationNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::AggregationNode +init src/exec/analytic_eval_node.cpp /^Status AnalyticEvalNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::AnalyticEvalNode +init src/exec/assert_num_rows_node.cpp /^Status AssertNumRowsNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::AssertNumRowsNode +init src/exec/blocking_join_node.cpp /^Status BlockingJoinNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::BlockingJoinNode +init src/exec/broker_scan_node.cpp /^Status BrokerScanNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::BrokerScanNode +init src/exec/csv_scan_node.cpp /^Status CsvScanNode::init(const TPlanNode& tnode) {$/;" f class:doris::CsvScanNode +init src/exec/data_sink.cpp /^Status DataSink::init(const TDataSink& thrift_sink) {$/;" f class:doris::DataSink +init src/exec/decompressor.cpp /^Status Bzip2Decompressor::init() {$/;" f class:doris::Bzip2Decompressor +init src/exec/decompressor.cpp /^Status GzipDecompressor::init() {$/;" f class:doris::GzipDecompressor +init src/exec/decompressor.cpp /^Status Lz4FrameDecompressor::init() {$/;" f class:doris::Lz4FrameDecompressor +init src/exec/es_http_scan_node.cpp /^Status EsHttpScanNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::EsHttpScanNode +init src/exec/except_node.cpp /^Status ExceptNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::ExceptNode +init src/exec/exchange_node.cpp /^Status ExchangeNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::ExchangeNode +init src/exec/exec_node.cpp /^Status ExecNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::ExecNode +init src/exec/hash_join_node.cpp /^Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::HashJoinNode +init src/exec/intersect_node.cpp /^Status IntersectNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::IntersectNode +init src/exec/lzo_decompressor.cpp /^Status LzopDecompressor::init() {$/;" f class:doris::LzopDecompressor +init src/exec/merge_join_node.cpp /^Status MergeJoinNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::MergeJoinNode +init src/exec/merge_node.cpp /^Status MergeNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::MergeNode +init src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::NewPartitionedAggregationNode +init src/exec/olap_rewrite_node.cpp /^Status OlapRewriteNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::OlapRewriteNode +init src/exec/olap_scan_node.cpp /^Status OlapScanNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::OlapScanNode +init src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::PartitionedAggregationNode +init src/exec/partitioned_hash_table.cc /^bool PartitionedHashTable::init() {$/;" f class:doris::PartitionedHashTable +init src/exec/pl_task_root.cpp /^Status ExchangeNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::ExchangeNode +init src/exec/schema_scan_node.cpp /^Status SchemaScanNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::SchemaScanNode +init src/exec/schema_scanner.cpp /^Status SchemaScanner::init(SchemaScannerParam* param, ObjectPool* pool) {$/;" f class:doris::SchemaScanner +init src/exec/sort_exec_exprs.cpp /^Status SortExecExprs::init($/;" f class:doris::SortExecExprs +init src/exec/sort_exec_exprs.cpp /^Status SortExecExprs::init(const TSortInfo& sort_info, ObjectPool* pool) {$/;" f class:doris::SortExecExprs +init src/exec/sort_exec_exprs.cpp /^Status SortExecExprs::init(const std::vector& lhs_ordering_expr_ctxs,$/;" f class:doris::SortExecExprs +init src/exec/sort_node.cpp /^Status SortNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::SortNode +init src/exec/spill_sort_node.cc /^Status SpillSortNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::SpillSortNode +init src/exec/tablet_info.cpp /^Status OlapTablePartitionParam::init() {$/;" f class:doris::OlapTablePartitionParam +init src/exec/tablet_info.cpp /^Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) {$/;" f class:doris::OlapTableSchemaParam +init src/exec/tablet_info.cpp /^Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) {$/;" f class:doris::OlapTableSchemaParam +init src/exec/tablet_sink.cpp /^Status IndexChannel::init(RuntimeState* state,$/;" f class:doris::stream_load::IndexChannel +init src/exec/tablet_sink.cpp /^Status NodeChannel::init(RuntimeState* state) {$/;" f class:doris::stream_load::NodeChannel +init src/exec/tablet_sink.cpp /^Status OlapTableSink::init(const TDataSink& t_sink) {$/;" f class:doris::stream_load::OlapTableSink +init src/exec/topn_node.cpp /^Status TopNNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::TopNNode +init src/exec/union_node.cpp /^Status UnionNode::init(const TPlanNode& tnode, RuntimeState* state) {$/;" f class:doris::UnionNode +init src/exprs/agg_fn_evaluator.cpp /^void AggFnEvaluator::init(FunctionContext* agg_fn_ctx, Tuple* dst) {$/;" f class:doris::AggFnEvaluator +init src/exprs/agg_fn_evaluator.h /^inline void AggFnEvaluator::init(const std::vector& evaluators,$/;" f class:doris::AggFnEvaluator +init src/exprs/bitmap_function.cpp /^void BitmapFunctions::init() {$/;" f class:doris::BitmapFunctions +init src/exprs/cast_functions.cpp /^void CastFunctions::init() {$/;" f class:doris::CastFunctions +init src/exprs/compound_predicate.cpp /^void CompoundPredicate::init() {$/;" f class:doris::CompoundPredicate +init src/exprs/decimal_operators.cpp /^void DecimalOperators::init() {$/;" f class:doris::DecimalOperators +init src/exprs/decimalv2_operators.cpp /^void DecimalV2Operators::init() {$/;" f class:doris::DecimalV2Operators +init src/exprs/encryption_functions.cpp /^void EncryptionFunctions::init() {$/;" f class:doris::EncryptionFunctions +init src/exprs/es_functions.cpp /^void ESFunctions::init() {$/;" f class:doris::ESFunctions +init src/exprs/grouping_sets_functions.cpp /^void GroupingSetsFunctions::init() {$/;" f class:doris::GroupingSetsFunctions +init src/exprs/hash_functions.cpp /^void HashFunctions::init() { }$/;" f class:doris::HashFunctions +init src/exprs/hll_function.cpp /^void HllFunctions::init() {$/;" f class:doris::HllFunctions +init src/exprs/hll_hash_function.cpp /^void HllHashFunctions::init() {$/;" f class:doris::HllHashFunctions +init src/exprs/is_null_predicate.cpp /^void IsNullPredicate::init() {$/;" f class:doris::IsNullPredicate +init src/exprs/json_functions.cpp /^void JsonFunctions::init() {$/;" f class:doris::JsonFunctions +init src/exprs/like_predicate.cpp /^void LikePredicate::init() {$/;" f class:doris::LikePredicate +init src/exprs/math_functions.cpp /^void MathFunctions::init() {$/;" f class:doris::MathFunctions +init src/exprs/new_in_predicate.cpp /^void InPredicate::init() {$/;" f class:doris::InPredicate +init src/exprs/operators.cpp /^void Operators::init() {$/;" f class:doris::Operators +init src/exprs/string_functions.cpp /^void StringFunctions::init() {$/;" f class:doris::StringFunctions +init src/exprs/time_operators.cpp /^void TimeOperators::init() {$/;" f class:doris::TimeOperators +init src/exprs/timestamp_functions.cpp /^void TimestampFunctions::init() {$/;" f class:doris::TimestampFunctions +init src/exprs/timezone_db.cpp /^void TimezoneDatabase::init() {$/;" f class:doris::TimezoneDatabase +init src/exprs/utility_functions.cpp /^void UtilityFunctions::init() {$/;" f class:doris::UtilityFunctions +init src/geo/geo_functions.cpp /^void GeoFunctions::init() {$/;" f class:doris::GeoFunctions +init src/geo/geo_types.cpp /^GeoParseStatus GeoCircle::init(double lng, double lat, double radius_meter) {$/;" f class:doris::GeoCircle +init src/http/http_client.cpp /^Status HttpClient::init(const std::string& url) {$/;" f class:doris::HttpClient +init src/olap/aggregate_func.h /^ inline void init(RowCursorCell* dst, const char* src, bool src_null, MemPool* mem_pool, ObjectPool* agg_pool) const {$/;" f class:doris::AggregateInfo +init src/olap/aggregate_func.h /^ static void init(RowCursorCell* dst, const char* src, bool src_null, MemPool* mem_pool, ObjectPool* agg_pool) {$/;" f struct:doris::AggregateFuncTraits +init src/olap/aggregate_func.h /^ static void init(RowCursorCell* dst, const char* src, bool src_null, MemPool* mem_pool, ObjectPool* agg_pool) {$/;" f struct:doris::BaseAggregateFuncs +init src/olap/bloom_filter.hpp /^ bool init(int64_t expected_entries) {$/;" f class:doris::BloomFilter +init src/olap/bloom_filter.hpp /^ bool init(int64_t expected_entries, double fpp) {$/;" f class:doris::BloomFilter +init src/olap/bloom_filter.hpp /^ bool init(uint32_t bit_num) {$/;" f class:doris::BitSet +init src/olap/bloom_filter.hpp /^ bool init(uint64_t* data, uint32_t data_len) {$/;" f class:doris::BitSet +init src/olap/bloom_filter.hpp /^ bool init(uint64_t* data, uint32_t len, uint32_t hash_function_num) {$/;" f class:doris::BloomFilter +init src/olap/bloom_filter_reader.cpp /^OLAPStatus BloomFilterIndexReader::init($/;" f class:doris::BloomFilterIndexReader +init src/olap/compaction.cpp /^OLAPStatus Compaction::init(int concurreny) {$/;" f class:doris::Compaction +init src/olap/data_dir.cpp /^Status DataDir::init() {$/;" f class:doris::DataDir +init src/olap/delete_handler.cpp /^OLAPStatus DeleteHandler::init(const TabletSchema& schema,$/;" f class:doris::DeleteHandler +init src/olap/delta_writer.cpp /^OLAPStatus DeltaWriter::init() {$/;" f class:doris::DeltaWriter +init src/olap/file_stream.h /^ inline OLAPStatus init() {$/;" f class:doris::ReadOnlyFileStream +init src/olap/generic_iterators.cpp /^Status AutoIncrementIterator::init(const StorageReadOptions& opts) {$/;" f class:doris::AutoIncrementIterator +init src/olap/generic_iterators.cpp /^Status MergeIterator::init(const StorageReadOptions& opts) {$/;" f class:doris::MergeIterator +init src/olap/generic_iterators.cpp /^Status MergeIteratorContext::init(const StorageReadOptions& opts) {$/;" f class:doris::MergeIteratorContext +init src/olap/generic_iterators.cpp /^Status UnionIterator::init(const StorageReadOptions& opts) {$/;" f class:doris::UnionIterator +init src/olap/hll.h /^ void init(char* buf, int len){$/;" f class:doris::HllSetResolver +init src/olap/memtable_flush_executor.cpp /^void MemTableFlushExecutor::init(const std::vector& data_dirs) {$/;" f class:doris::MemTableFlushExecutor +init src/olap/olap_common.h /^ void init(const std::string& rowset_id_str) {$/;" f struct:doris::RowsetId +init src/olap/olap_common.h /^ void init(int64_t id_version, int64_t high, int64_t middle, int64_t low) {$/;" f struct:doris::RowsetId +init src/olap/olap_common.h /^ void init(int64_t rowset_id) {$/;" f struct:doris::RowsetId +init src/olap/olap_cond.cpp /^OLAPStatus Cond::init(const TCondition& tcond, const TabletColumn& column) {$/;" f class:doris::Cond +init src/olap/olap_index.cpp /^OLAPStatus MemIndex::init(size_t short_key_len, size_t new_short_key_len,$/;" f class:doris::MemIndex +init src/olap/olap_meta.cpp /^OLAPStatus OlapMeta::init() {$/;" f class:doris::OlapMeta +init src/olap/push_handler.cpp /^OLAPStatus BinaryFile::init(const char* path) {$/;" f class:doris::BinaryFile +init src/olap/push_handler.cpp /^OLAPStatus BinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) {$/;" f class:doris::BinaryReader +init src/olap/push_handler.cpp /^OLAPStatus LzoBinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) {$/;" f class:doris::LzoBinaryReader +init src/olap/reader.cpp /^ OLAPStatus init() {$/;" f class:doris::CollectIterator::ChildCtx +init src/olap/reader.cpp /^OLAPStatus CollectIterator::init(Reader* reader) {$/;" f class:doris::CollectIterator +init src/olap/reader.cpp /^OLAPStatus Reader::init(const ReaderParams& read_params) {$/;" f class:doris::Reader +init src/olap/row_block.cpp /^OLAPStatus RowBlock::init(const RowBlockInfo& block_info) {$/;" f class:doris::RowBlock +init src/olap/row_cursor.cpp /^OLAPStatus RowCursor::init(const TabletSchema& schema) {$/;" f class:doris::RowCursor +init src/olap/row_cursor.cpp /^OLAPStatus RowCursor::init(const TabletSchema& schema, size_t column_count) {$/;" f class:doris::RowCursor +init src/olap/row_cursor.cpp /^OLAPStatus RowCursor::init(const TabletSchema& schema,$/;" f class:doris::RowCursor +init src/olap/row_cursor.cpp /^OLAPStatus RowCursor::init(const std::vector& schema) {$/;" f class:doris::RowCursor +init src/olap/row_cursor.cpp /^OLAPStatus RowCursor::init(const std::vector& schema, size_t column_count) {$/;" f class:doris::RowCursor +init src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::init() {$/;" f class:doris::AlphaRowset +init src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::init(RowsetReaderContext* read_context) {$/;" f class:doris::AlphaRowsetReader +init src/olap/rowset/alpha_rowset_writer.cpp /^OLAPStatus AlphaRowsetWriter::init(const RowsetWriterContext& rowset_writer_context) {$/;" f class:doris::AlphaRowsetWriter +init src/olap/rowset/beta_rowset.cpp /^OLAPStatus BetaRowset::init() {$/;" f class:doris::BetaRowset +init src/olap/rowset/beta_rowset_reader.cpp /^OLAPStatus BetaRowsetReader::init(RowsetReaderContext* read_context) {$/;" f class:doris::BetaRowsetReader +init src/olap/rowset/beta_rowset_writer.cpp /^OLAPStatus BetaRowsetWriter::init(const RowsetWriterContext& rowset_writer_context) {$/;" f class:doris::BetaRowsetWriter +init src/olap/rowset/bit_field_reader.cpp /^OLAPStatus BitFieldReader::init() {$/;" f class:doris::BitFieldReader +init src/olap/rowset/bit_field_writer.cpp /^OLAPStatus BitFieldWriter::init() {$/;" f class:doris::BitFieldWriter +init src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::init() {$/;" f class:doris::ColumnData +init src/olap/rowset/column_data_writer.cpp /^OLAPStatus ColumnDataWriter::init() {$/;" f class:doris::ColumnDataWriter +init src/olap/rowset/column_reader.cpp /^OLAPStatus ColumnReader::init($/;" f class:doris::ColumnReader +init src/olap/rowset/column_reader.cpp /^OLAPStatus DecimalColumnReader::init($/;" f class:doris::DecimalColumnReader +init src/olap/rowset/column_reader.cpp /^OLAPStatus IntegerColumnReader::init($/;" f class:doris::IntegerColumnReader +init src/olap/rowset/column_reader.cpp /^OLAPStatus LargeIntColumnReader::init($/;" f class:doris::LargeIntColumnReader +init src/olap/rowset/column_reader.cpp /^OLAPStatus StringColumnDictionaryReader::init($/;" f class:doris::StringColumnDictionaryReader +init src/olap/rowset/column_reader.cpp /^OLAPStatus StringColumnDirectReader::init($/;" f class:doris::StringColumnDirectReader +init src/olap/rowset/column_reader.cpp /^OLAPStatus TinyColumnReader::init($/;" f class:doris::TinyColumnReader +init src/olap/rowset/column_reader.h /^ virtual OLAPStatus init(std::map* streams,$/;" f class:doris::DefaultValueReader +init src/olap/rowset/column_reader.h /^ virtual OLAPStatus init(std::map* streams,$/;" f class:doris::FixLengthStringColumnReader +init src/olap/rowset/column_reader.h /^ virtual OLAPStatus init(std::map* streams,$/;" f class:doris::FloatintPointColumnReader +init src/olap/rowset/column_reader.h /^ virtual OLAPStatus init(std::map* streams,$/;" f class:doris::IntegerColumnReaderWrapper +init src/olap/rowset/column_reader.h /^ virtual OLAPStatus init(std::map* streams,$/;" f class:doris::VarStringColumnReader +init src/olap/rowset/column_writer.cpp /^OLAPStatus ByteColumnWriter::init() {$/;" f class:doris::ByteColumnWriter +init src/olap/rowset/column_writer.cpp /^OLAPStatus ColumnWriter::init() {$/;" f class:doris::ColumnWriter +init src/olap/rowset/column_writer.cpp /^OLAPStatus DecimalColumnWriter::init() {$/;" f class:doris::DecimalColumnWriter +init src/olap/rowset/column_writer.cpp /^OLAPStatus IntegerColumnWriter::init() {$/;" f class:doris::IntegerColumnWriter +init src/olap/rowset/column_writer.cpp /^OLAPStatus LargeIntColumnWriter::init() {$/;" f class:doris::LargeIntColumnWriter +init src/olap/rowset/column_writer.cpp /^OLAPStatus VarStringColumnWriter::init() {$/;" f class:doris::VarStringColumnWriter +init src/olap/rowset/column_writer.h /^ virtual OLAPStatus init() {$/;" f class:doris::DoubleColumnWriterBase +init src/olap/rowset/column_writer.h /^ virtual OLAPStatus init() {$/;" f class:doris::IntegerColumnWriterWrapper +init src/olap/rowset/rowset_meta.h /^ virtual bool init(const std::string& pb_rowset_meta) {$/;" f class:doris::RowsetMeta +init src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::init(bool is_using_cache) {$/;" f class:doris::SegmentReader +init src/olap/rowset/segment_v2/binary_dict_page.cpp /^Status BinaryDictPageDecoder::init() {$/;" f class:doris::segment_v2::BinaryDictPageDecoder +init src/olap/rowset/segment_v2/binary_prefix_page.cpp /^Status BinaryPrefixPageDecoder::init() {$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +init src/olap/rowset/segment_v2/bloom_filter.h /^ Status init(char* buf, uint32_t size, HashStrategyPB strategy) {$/;" f class:doris::segment_v2::BloomFilter +init src/olap/rowset/segment_v2/bloom_filter.h /^ Status init(uint64_t n, double fpp, HashStrategyPB strategy) {$/;" f class:doris::segment_v2::BloomFilter +init src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::init() {$/;" f class:doris::segment_v2::ColumnReader +init src/olap/rowset/segment_v2/column_reader.cpp /^Status DefaultValueColumnIterator::init(const ColumnIteratorOptions& opts) {$/;" f class:doris::segment_v2::DefaultValueColumnIterator +init src/olap/rowset/segment_v2/column_reader.h /^ virtual Status init(const ColumnIteratorOptions& opts) {$/;" f class:doris::segment_v2::ColumnIterator +init src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::init() {$/;" f class:doris::segment_v2::ColumnWriter +init src/olap/rowset/segment_v2/indexed_column_writer.cpp /^Status IndexedColumnWriter::init() {$/;" f class:doris::segment_v2::IndexedColumnWriter +init src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::init(const StorageReadOptions& opts) {$/;" f class:doris::segment_v2::SegmentIterator +init src/olap/rowset/segment_v2/segment_writer.cpp /^Status SegmentWriter::init(uint32_t write_mbytes_per_sec) {$/;" f class:doris::segment_v2::SegmentWriter +init src/olap/rowset/segment_writer.cpp /^OLAPStatus SegmentWriter::init(uint32_t write_mbytes_per_sec) {$/;" f class:doris::SegmentWriter +init src/olap/stream_index_common.cpp /^OLAPStatus ColumnStatistics::init(const FieldType& type, bool null_supported) {$/;" f class:doris::ColumnStatistics +init src/olap/stream_index_reader.cpp /^OLAPStatus PositionEntryReader::init($/;" f class:doris::PositionEntryReader +init src/olap/stream_index_reader.cpp /^OLAPStatus StreamIndexReader::init($/;" f class:doris::StreamIndexReader +init src/olap/tablet.cpp /^OLAPStatus Tablet::init() {$/;" f class:doris::Tablet +init src/plugin/plugin.h /^ int (*init)(void *);$/;" m struct:doris::Plugin +init src/runtime/broker_mgr.cpp /^void BrokerMgr::init() {$/;" f class:doris::BrokerMgr +init src/runtime/buffer_control_block.cpp /^Status BufferControlBlock::init() {$/;" f class:doris::BufferControlBlock +init src/runtime/buffered_block_mgr.cpp /^void BufferedBlockMgr::Block::init() {$/;" f class:doris::BufferedBlockMgr::Block +init src/runtime/buffered_block_mgr.cpp /^void BufferedBlockMgr::init(RuntimeState* state) {$/;" f class:doris::BufferedBlockMgr +init src/runtime/buffered_block_mgr2.cc /^void BufferedBlockMgr2::Block::init() {$/;" f class:doris::BufferedBlockMgr2::Block +init src/runtime/buffered_block_mgr2.cc /^void BufferedBlockMgr2::init($/;" f class:doris::BufferedBlockMgr2 +init src/runtime/buffered_tuple_stream.cpp /^Status BufferedTupleStream::init(RuntimeProfile* profile) {$/;" f class:doris::BufferedTupleStream +init src/runtime/buffered_tuple_stream.h /^ Status init() {$/;" f class:doris::BufferedTupleStream +init src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::init(int node_id, RuntimeProfile* profile, bool pinned) {$/;" f class:doris::BufferedTupleStream2 +init src/runtime/data_stream_sender.cpp /^Status DataStreamSender::Channel::init(RuntimeState* state) {$/;" f class:doris::DataStreamSender::Channel +init src/runtime/data_stream_sender.cpp /^Status DataStreamSender::init(const TDataSink& tsink) {$/;" f class:doris::DataStreamSender +init src/runtime/disk_io_mgr.cc /^Status DiskIoMgr::init(MemTracker* process_mem_tracker) {$/;" f class:doris::DiskIoMgr +init src/runtime/dpp_sink.cpp /^Status DppSink::init(RuntimeState* state) {$/;" f class:doris::DppSink +init src/runtime/etl_job_mgr.cpp /^Status EtlJobMgr::init() {$/;" f class:doris::EtlJobMgr +init src/runtime/exec_env_init.cpp /^Status ExecEnv::init(ExecEnv* env, const std::vector& store_paths) {$/;" f class:doris::ExecEnv +init src/runtime/export_sink.cpp /^Status ExportSink::init(const TDataSink& t_sink) {$/;" f class:doris::ExportSink +init src/runtime/export_task_mgr.cpp /^Status ExportTaskMgr::init() {$/;" f class:doris::ExportTaskMgr +init src/runtime/load_channel_mgr.cpp /^Status LoadChannelMgr::init(int64_t process_mem_limit) {$/;" f class:doris::LoadChannelMgr +init src/runtime/load_path_mgr.cpp /^Status LoadPathMgr::init() {$/;" f class:doris::LoadPathMgr +init src/runtime/merge_sorter.cpp /^Status MergeSorter::Run::init() {$/;" f class:doris::MergeSorter::Run +init src/runtime/mysql_table_sink.cpp /^Status MysqlTableSink::init(const TDataSink& t_sink) {$/;" f class:doris::MysqlTableSink +init src/runtime/result_buffer_mgr.cpp /^Status ResultBufferMgr::init() {$/;" f class:doris::ResultBufferMgr +init src/runtime/result_writer.cpp /^Status ResultWriter::init(RuntimeState* state) {$/;" f class:doris::ResultWriter +init src/runtime/routine_load/data_consumer.cpp /^Status KafkaDataConsumer::init(StreamLoadContext* ctx) {$/;" f class:doris::KafkaDataConsumer +init src/runtime/runtime_state.cpp /^Status RuntimeState::init($/;" f class:doris::RuntimeState +init src/runtime/small_file_mgr.cpp /^Status SmallFileMgr::init() {$/;" f class:doris::SmallFileMgr +init src/runtime/sorted_run_merger.cc /^ Status init(bool* done) {$/;" f class:doris::SortedRunMerger::BatchedRowSupplier +init src/runtime/spill_sorter.cc /^Status SpillSorter::Run::init() {$/;" f class:doris::SpillSorter::Run +init src/runtime/spill_sorter.cc /^Status SpillSorter::init() {$/;" f class:doris::SpillSorter +init src/runtime/tmp_file_mgr.cc /^Status TmpFileMgr::init(MetricRegistry* metrics) {$/;" f class:doris::TmpFileMgr +init src/runtime/tuple.h /^ void init(int size) {$/;" f class:doris::Tuple +init src/runtime/user_function_cache.cpp /^Status UserFunctionCache::init(const std::string& lib_dir) {$/;" f class:doris::UserFunctionCache +init src/service/backend_options.cpp /^bool BackendOptions::init() {$/;" f class:doris::BackendOptions +init src/udf/udf.cpp /^void HllVal::init(FunctionContext* ctx) {$/;" f class:doris_udf::HllVal +init src/util/cpu_info.cpp /^void CpuInfo::init() {$/;" f class:doris::CpuInfo +init src/util/disk_info.cpp /^void DiskInfo::init() {$/;" f class:doris::DiskInfo +init src/util/frame_of_reference_coding.cpp /^bool ForDecoder::init() {$/;" f class:doris::ForDecoder +init src/util/mem_info.cpp /^void MemInfo::init() {$/;" f class:doris::MemInfo +init src/util/threadpool.cpp /^Status ThreadPool::init() {$/;" f class:doris::ThreadPool +init test/exec/broker_reader_test.cpp /^void BrokerReaderTest::init() {$/;" f class:doris::BrokerReaderTest +init test/exec/broker_scan_node_test.cpp /^void BrokerScanNodeTest::init() {$/;" f class:doris::BrokerScanNodeTest +init test/exec/broker_scanner_test.cpp /^void BrokerScannerTest::init() {$/;" f class:doris::BrokerScannerTest +init test/exec/csv_scan_bench_test.cpp /^void CsvScanNodeBenchTest::init() {$/;" f class:doris::CsvScanNodeBenchTest +init test/exec/csv_scan_node_test.cpp /^void CsvScanNodeTest::init() {$/;" f class:doris::CsvScanNodeTest +init test/exec/csv_scanner_test.cpp /^void CsvScannerTest::init() {$/;" f class:doris::CsvScannerTest +init test/exec/parquet_scanner_test.cpp /^void ParquetSannerTest::init() {$/;" f class:doris::ParquetSannerTest +init test/runtime/load_channel_mgr_test.cpp /^OLAPStatus DeltaWriter::init() {$/;" f class:doris::DeltaWriter +init test/runtime/memory_scratch_sink_test.cpp /^void MemoryScratchSinkTest::init() {$/;" f class:doris::MemoryScratchSinkTest +init test/util/arrow/arrow_work_flow_test.cpp /^void ArrowWorkFlowTest::init() {$/;" f class:doris::ArrowWorkFlowTest +init_bfd src/util/bfd_parser.cpp /^void BfdParser::init_bfd() {$/;" f class:doris::BfdParser +init_buffer_poolstate src/runtime/runtime_state.cpp /^Status RuntimeState::init_buffer_poolstate() {$/;" f class:doris::RuntimeState +init_builtins_dummy src/exprs/expr.cpp /^void init_builtins_dummy() {$/;" f namespace:doris +init_cgroups src/agent/cgroups_mgr.cpp /^AgentStatus CgroupsMgr::init_cgroups() {$/;" f class:doris::CgroupsMgr +init_cluster_id src/agent/heartbeat_server.cpp /^void HeartbeatServer::init_cluster_id() {$/;" f class:doris::HeartbeatServer +init_custom src/runtime/tmp_file_mgr.cc /^Status TmpFileMgr::init_custom($/;" f class:doris::TmpFileMgr +init_daemon src/common/daemon.cpp /^void init_daemon(int argc, char** argv, const std::vector& paths) {$/;" f namespace:doris +init_desc_table test/exec/broker_scan_node_test.cpp /^void BrokerScanNodeTest::init_desc_table() {$/;" f class:doris::BrokerScanNodeTest +init_desc_table test/exec/broker_scanner_test.cpp /^void BrokerScannerTest::init_desc_table() {$/;" f class:doris::BrokerScannerTest +init_desc_table test/exec/parquet_scanner_test.cpp /^void ParquetSannerTest::init_desc_table() {$/;" f class:doris::ParquetSannerTest +init_desc_tbl test/exec/csv_scan_bench_test.cpp /^void CsvScanNodeBenchTest::init_desc_tbl() {$/;" f class:doris::CsvScanNodeBenchTest +init_desc_tbl test/exec/csv_scan_node_test.cpp /^void CsvScanNodeTest::init_desc_tbl() {$/;" f class:doris::CsvScanNodeTest +init_desc_tbl test/runtime/data_spliter_test.cpp /^void DataSplitTest::init_desc_tbl() {$/;" f class:doris::DataSplitTest +init_desc_tbl test/runtime/dpp_sink_test.cpp /^void DppSinkTest::init_desc_tbl() {$/;" f class:doris::DppSinkTest +init_desc_tbl test/runtime/memory_scratch_sink_test.cpp /^void MemoryScratchSinkTest::init_desc_tbl() {$/;" f class:doris::MemoryScratchSinkTest +init_desc_tbl test/runtime/qsorter_test.cpp /^void QSorterTest::init_desc_tbl() {$/;" f class:doris::QSorterTest +init_desc_tbl test/util/arrow/arrow_work_flow_test.cpp /^void ArrowWorkFlowTest::init_desc_tbl() {$/;" f class:doris::ArrowWorkFlowTest +init_doris_metrics src/common/daemon.cpp /^static void init_doris_metrics(const std::vector& store_paths) {$/;" f namespace:doris +init_event_attr src/util/perf_counters.cpp /^static bool init_event_attr(perf_event_attr* attr, PerfCounters::Counter counter) {$/;" f namespace:doris +init_expr_ctxes src/exec/base_scanner.cpp /^Status BaseScanner::init_expr_ctxes() {$/;" f class:doris::BaseScanner +init_fn_ src/exprs/agg_fn.h /^ void* init_fn_ = nullptr;$/;" m class:doris::AggFn +init_from_evhttp src/http/http_request.cpp /^int HttpRequest::init_from_evhttp() {$/;" f class:doris::HttpRequest +init_from_json src/olap/rowset/rowset_meta.h /^ virtual bool init_from_json(const std::string& json_rowset_meta) {$/;" f class:doris::RowsetMeta +init_from_pb src/olap/rowset/rowset_meta.h /^ virtual bool init_from_pb(const RowsetMetaPB& rowset_meta_pb) {$/;" f class:doris::RowsetMeta +init_from_pb src/olap/tablet_meta.cpp /^OLAPStatus AlterTabletTask::init_from_pb(const AlterTabletPB& alter_task) {$/;" f class:doris::AlterTabletTask +init_from_pb src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) {$/;" f class:doris::TabletMeta +init_from_pb src/olap/tablet_schema.cpp /^OLAPStatus TabletColumn::init_from_pb(const ColumnPB& column) {$/;" f class:doris::TabletColumn +init_from_pb src/olap/tablet_schema.cpp /^OLAPStatus TabletSchema::init_from_pb(const TabletSchemaPB& schema) {$/;" f class:doris::TabletSchema +init_get_next src/exec/cross_join_node.cpp /^void CrossJoinNode::init_get_next(TupleRow* first_left_row) {$/;" f class:doris::CrossJoinNode +init_glog src/common/logconfig.cpp /^bool init_glog(const char* basename, bool install_signal_handler) {$/;" f namespace:doris +init_has_varlen_slots src/runtime/descriptors.cpp /^void RowDescriptor::init_has_varlen_slots() {$/;" f class:doris::RowDescriptor +init_hash_table src/exec/partitioned_aggregation_node.cc /^bool PartitionedAggregationNode::Partition::init_hash_table() {$/;" f class:doris::PartitionedAggregationNode::Partition +init_in_pre test/exprs/in_predicate_test.cpp /^ void init_in_pre(InPredicate* in_pre) {$/;" f class:doris::InPredicateTest +init_instance src/runtime/memory/chunk_allocator.cpp /^void ChunkAllocator::init_instance(size_t reserve_limit) {$/;" f class:doris::ChunkAllocator +init_instance_mem_tracker src/runtime/runtime_state.cpp /^Status RuntimeState::init_instance_mem_tracker() {$/;" f class:doris::RuntimeState +init_internal src/runtime/disk_io_mgr_scan_range.cc /^void DiskIoMgr::ScanRange::init_internal(DiskIoMgr* io_mgr, RequestContext* reader) {$/;" f class:doris::DiskIoMgr::ScanRange +init_mem_trackers src/runtime/runtime_state.cpp /^Status RuntimeState::init_mem_trackers(const TUniqueId& query_id) {$/;" f class:doris::RuntimeState +init_metrics src/runtime/client_cache.cpp /^void ClientCacheHelper::init_metrics(MetricRegistry* metrics, const std::string& key_prefix) {$/;" f class:doris::ClientCacheHelper +init_metrics src/runtime/client_cache.h /^ void init_metrics(MetricRegistry* metrics, const std::string& key_prefix) {$/;" f class:doris::ClientCache +init_metrics src/runtime/test_env.cc /^void TestEnv::init_metrics() {$/;" f class:doris::TestEnv +init_mock test/exec/schema_scanner/schema_columns_scanner_test.cpp /^void init_mock() {$/;" f namespace:doris +init_mock test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^void init_mock() {$/;" f namespace:doris +init_mock test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^void init_mock() {$/;" f namespace:doris +init_mock test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^void init_mock() {$/;" f namespace:doris +init_mock test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^void init_mock() {$/;" f namespace:doris +init_mock test/exec/schema_scanner/schema_tables_scanner_test.cpp /^void init_mock() {$/;" f namespace:doris +init_module src/gutil/spinlock_linux-inl.h /^} init_module;$/;" m namespace:__anon20 typeref:struct:__anon20::InitModule +init_next_partition src/exec/analytic_eval_node.cpp /^inline void AnalyticEvalNode::init_next_partition(int64_t stream_idx) {$/;" f class:doris::AnalyticEvalNode +init_null src/exprs/aggregate_functions.cpp /^void AggregateFunctions::init_null(FunctionContext*, AnyVal* dst) {$/;" f class:doris::AggregateFunctions +init_null_string src/exprs/aggregate_functions.cpp /^void AggregateFunctions::init_null_string(FunctionContext* c, StringVal* dst) {$/;" f class:doris::AggregateFunctions +init_olap test/exec/new_olap_scan_node_test.cpp /^ void init_olap() {$/;" f class:doris::TestOlapScanNode +init_olap test/olap/olap_reader_test.cpp /^ void init_olap() {$/;" f class:doris::TestOLAPReaderColumn +init_olap test/olap/olap_reader_test.cpp /^ void init_olap() {$/;" f class:doris::TestOLAPReaderColumnDeleteCondition +init_olap test/olap/olap_reader_test.cpp /^ void init_olap() {$/;" f class:doris::TestOLAPReaderRow +init_olap_column test/olap/vectorized_olap_reader_test.cpp /^ void init_olap_column() {$/;" f class:doris::TestVectorizedOLAPReader +init_olap_row test/olap/vectorized_olap_reader_test.cpp /^ void init_olap_row() {$/;" f class:doris::TestVectorizedOLAPReader +init_order_expr test/runtime/qsorter_test.cpp /^void QSorterTest::init_order_expr() {$/;" f class:doris::QSorterTest +init_params test/exec/broker_scanner_test.cpp /^void BrokerScannerTest::init_params() {$/;" f class:doris::BrokerScannerTest +init_parquet_reader src/exec/parquet_reader.cpp /^Status ParquetReaderWrap::init_parquet_reader(const std::vector& tuple_slot_descs) {$/;" f class:doris::ParquetReaderWrap +init_proc_self_io_counter src/util/perf_counters.cpp /^bool PerfCounters::init_proc_self_io_counter(Counter counter) {$/;" f class:doris::PerfCounters +init_proc_self_status_counter src/util/perf_counters.cpp /^bool PerfCounters::init_proc_self_status_counter(Counter counter) {$/;" f class:doris::PerfCounters +init_range test/runtime/disk_io_mgr_test.cpp /^ DiskIoMgr::ScanRange* init_range(int num_buffers, const char* file_path, int offset,$/;" f class:doris::DiskIoMgrTest +init_rollups test/runtime/dpp_sink_test.cpp /^void DppSinkTest::init_rollups() {$/;" f class:doris::DppSinkTest +init_row_desc test/runtime/data_spliter_test.cpp /^void DataSplitTest::init_row_desc() {$/;" f class:doris::DataSplitTest +init_row_desc test/runtime/dpp_sink_test.cpp /^void DppSinkTest::init_row_desc() {$/;" f class:doris::DppSinkTest +init_row_desc test/runtime/qsorter_test.cpp /^void QSorterTest::init_row_desc() {$/;" f class:doris::QSorterTest +init_row_with_others src/olap/row.h /^void init_row_with_others(DstRowType* dst, const SrcRowType& src, MemPool* mem_pool, ObjectPool* agg_pool) {$/;" f namespace:doris +init_runtime_profile src/exec/exec_node.cpp /^void ExecNode::init_runtime_profile(const std::string& name) {$/;" f class:doris::ExecNode +init_runtime_state test/exec/csv_scan_bench_test.cpp /^void CsvScanNodeBenchTest::init_runtime_state() {$/;" f class:doris::CsvScanNodeBenchTest +init_runtime_state test/exec/csv_scan_node_test.cpp /^void CsvScanNodeTest::init_runtime_state() {$/;" f class:doris::CsvScanNodeTest +init_runtime_state test/runtime/data_spliter_test.cpp /^void DataSplitTest::init_runtime_state() {$/;" f class:doris::DataSplitTest +init_runtime_state test/runtime/dpp_sink_test.cpp /^void DppSinkTest::init_runtime_state() {$/;" f class:doris::DppSinkTest +init_runtime_state test/runtime/memory_scratch_sink_test.cpp /^void MemoryScratchSinkTest::init_runtime_state() {$/;" f class:doris::MemoryScratchSinkTest +init_runtime_state test/runtime/qsorter_test.cpp /^void QSorterTest::init_runtime_state() {$/;" f class:doris::QSorterTest +init_runtime_state test/util/arrow/arrow_work_flow_test.cpp /^void ArrowWorkFlowTest::init_runtime_state() {$/;" f class:doris::ArrowWorkFlowTest +init_scan_key src/olap/row_cursor.cpp /^OLAPStatus RowCursor::init_scan_key(const TabletSchema& schema,$/;" f class:doris::RowCursor +init_scan_node test/exec/new_olap_scan_node_test.cpp /^ void init_scan_node() {$/;" f class:doris::TestOlapScanNode +init_scan_node test/olap/olap_reader_test.cpp /^ void init_scan_node() {$/;" f class:doris::TestOLAPReaderColumn +init_scan_node test/olap/olap_reader_test.cpp /^ void init_scan_node() {$/;" f class:doris::TestOLAPReaderColumnDeleteCondition +init_scan_node test/olap/olap_reader_test.cpp /^ void init_scan_node() {$/;" f class:doris::TestOLAPReaderRow +init_scan_node test/olap/vectorized_olap_reader_test.cpp /^ void init_scan_node() {$/;" f class:doris::TestVectorizedOLAPReader +init_scan_node_k1_v test/olap/olap_reader_test.cpp /^ void init_scan_node_k1_v() {$/;" f class:doris::TestOLAPReaderColumn +init_scan_node_k1_v test/olap/olap_reader_test.cpp /^ void init_scan_node_k1_v() {$/;" f class:doris::TestOLAPReaderColumnDeleteCondition +init_scan_node_k1_v test/olap/olap_reader_test.cpp /^ void init_scan_node_k1_v() {$/;" f class:doris::TestOLAPReaderRow +init_signals src/common/daemon.cpp /^void init_signals() {$/;" f namespace:doris +init_streams src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::Partition::init_streams() {$/;" f class:doris::PartitionedAggregationNode::Partition +init_succeeded src/olap/tablet.h /^inline bool Tablet::init_succeeded() {$/;" f class:doris::Tablet +init_sys_counter src/util/perf_counters.cpp /^bool PerfCounters::init_sys_counter(Counter counter) {$/;" f class:doris::PerfCounters +init_tablet_schema test/olap/row_block_test.cpp /^void init_tablet_schema(TabletSchema* tablet_schema) {$/;" f namespace:doris +init_tablet_schema test/olap/row_block_v2_test.cpp /^void init_tablet_schema(TabletSchema* tablet_schema, bool is_nullable) {$/;" f namespace:doris +init_tablet_schema test/olap/txn_manager_test.cpp /^ void init_tablet_schema() {$/;" f class:doris::TxnManagerTest +init_threadmgr src/util/thread.cpp /^void Thread::init_threadmgr() {$/;" f class:doris::Thread +init_thrift_logging src/util/thrift_util.cpp /^void init_thrift_logging() {$/;" f namespace:doris +init_tmp_file_mgr src/runtime/test_env.cc /^void TestEnv::init_tmp_file_mgr(const std::vector& tmp_dirs,$/;" f class:doris::TestEnv +init_tmp_files src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::init_tmp_files() {$/;" f class:doris::BufferedBlockMgr2 +init_tuple_idx_map src/runtime/descriptors.cpp /^void RowDescriptor::init_tuple_idx_map() {$/;" f class:doris::RowDescriptor +init_zero src/exprs/aggregate_functions.cpp /^void AggregateFunctions::init_zero(FunctionContext*, DecimalV2Val* dst) {$/;" f class:doris::AggregateFunctions +init_zero src/exprs/aggregate_functions.cpp /^void AggregateFunctions::init_zero(FunctionContext*, DecimalVal* dst) {$/;" f class:doris::AggregateFunctions +init_zero src/exprs/aggregate_functions.cpp /^void AggregateFunctions::init_zero(FunctionContext*, T* dst) {$/;" f class:doris::AggregateFunctions +initial_data_ src/util/faststring.h /^ uint8_t initial_data_[kInitialCapacity];$/;" m class:doris::faststring +initial_reservation_mem_tracker_ src/runtime/initial_reservations.h /^ MemTracker* const initial_reservation_mem_tracker_;$/;" m class:doris::InitialReservations +initial_reservations src/runtime/runtime_state.h /^ InitialReservations* initial_reservations() const { $/;" f class:doris::RuntimeState +initial_reservations_ src/runtime/initial_reservations.h /^ ReservationTracker initial_reservations_;$/;" m class:doris::InitialReservations +initial_scan_range_queue_capacity src/runtime/disk_io_mgr_internal.h /^ int initial_scan_range_queue_capacity() const { return _initial_queue_capacity; }$/;" f class:doris::DiskIoMgr::RequestContext +initialize src/util/doris_metrics.cpp /^void DorisMetrics::initialize($/;" f class:doris::DorisMetrics +initialized_ src/runtime/bufferpool/reservation_tracker.h /^ bool initialized_ = false;$/;" m class:doris::ReservationTracker +initialized_ src/util/cpu_info.cpp /^bool CpuInfo::initialized_ = false;$/;" m class:doris::CpuInfo file: +initialized_ src/util/cpu_info.h /^ static bool initialized_;$/;" m class:doris::CpuInfo +initialized_ src/util/os_info.cpp /^bool OsInfo::initialized_ = false;$/;" m class:doris::OsInfo file: +initialized_ src/util/os_info.h /^ static bool initialized_;$/;" m class:doris::OsInfo +inlined_string_slots_ src/runtime/buffered_tuple_stream3.h /^ std::vector>> inlined_string_slots_;$/;" m class:doris::BufferedTupleStream3 +input_buf_read_remaining src/exec/plain_text_line_reader.h /^ inline size_t input_buf_read_remaining() {$/;" f class:doris::PlainTextLineReader +input_checksum_type src/exec/decompressor.h /^ LzoChecksum input_checksum_type;$/;" m struct:doris::LzopDecompressor::HeaderInfo +input_done src/runtime/merge_sorter.cpp /^Status MergeSorter::input_done() {$/;" f class:doris::MergeSorter +input_done src/runtime/qsorter.cpp /^Status QSorter::input_done() {$/;" f class:doris::QSorter +input_done src/runtime/spill_sorter.cc /^Status SpillSorter::input_done() {$/;" f class:doris::SpillSorter +input_evals src/exprs/new_agg_fn_evaluator_ir.cc /^ExprContext* const* NewAggFnEvaluator::input_evals() const {$/;" f class:NewAggFnEvaluator +input_evals_ src/exprs/new_agg_fn_evaluator.h /^ std::vector input_evals_;$/;" m class:doris::NewAggFnEvaluator +input_expr_ctxs src/exprs/agg_fn_evaluator.h /^ const std::vector& input_expr_ctxs() const {$/;" f class:doris::AggFnEvaluator +input_expr_ctxs src/exprs/new_agg_fn_evaluator.h /^ const std::vector& input_expr_ctxs() const {$/;" f class:doris::NewAggFnEvaluator +input_type src/exec/decompressor.h /^ inline LzoChecksum input_type(int flags) {$/;" f class:doris::LzopDecompressor +insert src/exec/hash_table.h /^ void IR_ALWAYS_INLINE insert(TupleRow* row) {$/;" f class:doris::HashTable +insert src/exec/partitioned_hash_table.inline.h /^inline bool PartitionedHashTable::insert($/;" f class:doris::PartitionedHashTable +insert src/exec/partitioned_hash_table.inline.h /^inline bool PartitionedHashTable::insert(PartitionedHashTableCtx* ht_ctx,$/;" f class:doris::PartitionedHashTable +insert src/exprs/hybird_set.h /^ virtual void insert(HybirdSetBase* set) {$/;" f class:doris::HybirdSet +insert src/exprs/hybird_set.h /^ virtual void insert(void* data) {$/;" f class:doris::HybirdSet +insert src/exprs/hybird_set.h /^ virtual void insert(void* data) {$/;" f class:doris::StringValueSet +insert src/exprs/hybird_set.h /^ void insert(HybirdSetBase* set) {$/;" f class:doris::StringValueSet +insert src/exprs/in_predicate.cpp /^void InPredicate::insert(void* value) {$/;" f class:doris::InPredicate +insert src/olap/lru_cache.cpp /^Cache::Handle* LRUCache::insert($/;" f class:doris::LRUCache +insert src/olap/lru_cache.cpp /^Cache::Handle* ShardedLRUCache::insert($/;" f class:doris::ShardedLRUCache +insert src/olap/lru_cache.cpp /^LRUHandle* HandleTable::insert(LRUHandle* h) {$/;" f class:doris::HandleTable +insert src/olap/memtable.cpp /^void MemTable::insert(const Tuple* tuple) {$/;" f class:doris::MemTable +insert src/olap/page_cache.cpp /^void StoragePageCache::insert(const CacheKey& key, const Slice& data, PageCacheHandle* handle,$/;" f class:doris::StoragePageCache +insert src/runtime/query_statistics.cpp /^void QueryStatisticsRecvr::insert(const PQueryStatistics& statistics, int sender_id) {$/;" f class:doris::QueryStatisticsRecvr +insert src/util/file_cache.cpp /^void FileCache::insert(const std::string& file_name, FileType* file, OpenedFileHandle* file_handle) {$/;" f class:doris::FileCache +insert src/util/path_trie.hpp /^ bool insert(const std::vector path, int index, const T& value) {$/;" f class:doris::PathTrie::TrieNode +insert src/util/path_trie.hpp /^ bool insert(const std::string& path, const T& value) {$/;" f class:doris::PathTrie +insert_LRUCache test/olap/lru_cache_test.cpp /^static void insert_LRUCache(LRUCache& cache, const CacheKey& key, int value, CachePriority priority) {$/;" f namespace:doris +insert_duplicate_node src/exec/partitioned_hash_table.inline.h /^inline PartitionedHashTable::DuplicateNode* PartitionedHashTable::insert_duplicate_node($/;" f class:doris::PartitionedHashTable +insert_impl src/exec/hash_table.hpp /^inline void HashTable::insert_impl(TupleRow* row) {$/;" f class:doris::HashTable +insert_internal src/exec/partitioned_hash_table.inline.h /^inline PartitionedHashTable::HtData* PartitionedHashTable::insert_internal($/;" f class:doris::PartitionedHashTable +insert_row src/runtime/mysql_table_writer.cpp /^Status MysqlTableWriter::insert_row(TupleRow* row) {$/;" f class:doris::MysqlTableWriter +insert_tuple_row src/exec/topn_node.cpp /^void TopNNode::insert_tuple_row(TupleRow* input_row) {$/;" f class:doris::TopNNode +insert_tuple_row src/runtime/qsorter.cpp /^Status QSorter::insert_tuple_row(TupleRow* input_row) {$/;" f class:doris::QSorter +inserter_thread test/util/blocking_queue_test.cpp /^ void inserter_thread(int arg) {$/;" f class:doris::MultiThreadTest +insertion_sort src/runtime/merge_sorter.cpp /^void MergeSorter::TupleSorter::insertion_sort(const TupleIterator& first,$/;" f class:doris::MergeSorter::TupleSorter +insertion_sort src/runtime/spill_sorter.cc /^void SpillSorter::TupleSorter::insertion_sort(const TupleIterator& first,$/;" f class:doris::SpillSorter::TupleSorter +install src/util/system_metrics.cpp /^void SystemMetrics::install(MetricRegistry* registry,$/;" f class:doris::SystemMetrics +install_signal src/common/daemon.cpp /^int install_signal(int signo, void(*handler)(int)) {$/;" f namespace:doris +instance src/olap/key_coder.cpp /^ static KeyCoderResolver* instance() {$/;" f class:doris::KeyCoderResolver +instance src/olap/page_cache.h /^ static StoragePageCache* instance() { return _s_instance; }$/;" f class:doris::StoragePageCache +instance src/olap/snapshot_manager.cpp /^SnapshotManager* SnapshotManager::instance() {$/;" f class:doris::SnapshotManager +instance src/olap/storage_engine.h /^ static StorageEngine* instance() {$/;" f class:doris::StorageEngine +instance src/runtime/memory/chunk_allocator.cpp /^ChunkAllocator* ChunkAllocator::instance() {$/;" f class:doris::ChunkAllocator +instance src/runtime/memory/chunk_allocator.h /^ static ChunkAllocator* instance() {$/;" f class:doris::ChunkAllocator +instance src/runtime/user_function_cache.cpp /^UserFunctionCache* UserFunctionCache::instance() {$/;" f class:doris::UserFunctionCache +instance src/util/block_compression.cpp /^ static const Lz4BlockCompression* instance() {$/;" f class:doris::Lz4BlockCompression +instance src/util/block_compression.cpp /^ static const Lz4fBlockCompression* instance() {$/;" f class:doris::Lz4fBlockCompression +instance src/util/block_compression.cpp /^ static const SnappyBlockCompression* instance() {$/;" f class:doris::SnappyBlockCompression +instance src/util/block_compression.cpp /^ static const ZlibBlockCompression* instance() {$/;" f class:doris::ZlibBlockCompression +instance src/util/core_local.cpp /^CoreDataAllocatorFactory* CoreDataAllocatorFactory::instance() {$/;" f class:doris::CoreDataAllocatorFactory +instance src/util/core_local.h /^ static CoreLocalValueController* instance() {$/;" f class:doris::CoreLocalValueController +instance src/util/doris_metrics.h /^ static DorisMetrics* instance() { return &_s_doris_metrics; }$/;" f class:doris::DorisMetrics +instance src/util/file_manager.cpp /^FileManager* FileManager::instance() {$/;" f class:doris::FileManager +instance src/util/uuid_generator.h /^ static UUIDGenerator* instance() {$/;" f class:doris::UUIDGenerator +instance_buffer_reservation src/runtime/runtime_state.h /^ ReservationTracker* instance_buffer_reservation() {$/;" f class:doris::RuntimeState +instance_mem_tracker src/runtime/runtime_state.h /^ MemTracker* instance_mem_tracker() { {$/;" f class:doris::RuntimeState +instr src/exprs/string_functions.cpp /^IntVal StringFunctions::instr($/;" f class:doris::StringFunctions +int128_t src/olap/olap_common.h /^typedef __int128 int128_t;$/;" t namespace:doris +int128_t src/runtime/decimalv2_value.h /^typedef __int128_t int128_t;$/;" t namespace:doris +int16 src/gutil/integral_types.h /^typedef int16_t int16;$/;" t +int16_t src/gutil/port.h /^typedef short int16_t;$/;" t +int1store src/util/mysql_global.h 27;" d +int2store src/util/mysql_global.h 28;" d +int32 src/gutil/integral_types.h /^typedef int32_t int32;$/;" t +int3store src/util/mysql_global.h 29;" d +int64 src/gutil/integral_types.h /^typedef __int64 int64;$/;" t +int64 src/gutil/integral_types.h /^typedef int64_t int64;$/;" t +int8 src/gutil/integral_types.h /^typedef int8_t int8;$/;" t +int8store src/util/mysql_global.h 32;" d +int_len output/udf/include/udf.h /^ int8_t int_len;$/;" m struct:doris_udf::DecimalVal +int_len src/udf/udf.h /^ int8_t int_len;$/;" m struct:doris_udf::DecimalVal +int_memmatch src/gutil/strings/memutil.cc /^const char *int_memmatch(const char *phaystack, size_t haylen,$/;" f +int_to_hex_digit src/gutil/strings/escaping.h /^inline int int_to_hex_digit(int i) {$/;" f namespace:strings +int_to_lower_hex_digit src/gutil/strings/escaping.h /^inline int int_to_lower_hex_digit(int i) {$/;" f namespace:strings +int_to_str src/runtime/datetime_value.cpp /^static char* int_to_str(uint64_t val, char* to) {$/;" f namespace:doris +int_val src/exprs/expr_value.h /^ int32_t int_val;$/;" m struct:doris::ExprValue +int_val test/runtime/dpp_writer_test.cpp /^ int32_t int_val;$/;" m struct:doris::TestDataTuple file: +int_val test/runtime/mysql_table_writer_test.cpp /^ int32_t int_val;$/;" m struct:doris::TestDataTuple file: +int_value src/runtime/decimal_value.h /^ int64_t int_value() const {$/;" f class:doris::DecimalValue +int_value src/runtime/decimalv2_value.h /^ int64_t int_value() const {$/;" f class:doris::DecimalV2Value +integer src/olap/decimal12.h /^ int64_t integer;$/;" m struct:doris::decimal12_t +integral_constant src/gutil/template_util.h /^struct integral_constant {$/;" s namespace:base +integratedLocation src/util/tdigest.h /^ inline Value integratedLocation(Value q) const {$/;" f class:doris::TDigest +integratedQ src/util/tdigest.h /^ inline Value integratedQ(Value k) const {$/;" f class:doris::TDigest +intermediate_row_desc_ src/exec/new_partitioned_aggregation_node.h /^ RowDescriptor intermediate_row_desc_;$/;" m class:doris::NewPartitionedAggregationNode +intermediate_slot_desc src/exprs/agg_fn.h /^ const SlotDescriptor& intermediate_slot_desc() const { return intermediate_slot_desc_; }$/;" f class:doris::AggFn +intermediate_slot_desc src/exprs/new_agg_fn_evaluator.cc /^const SlotDescriptor& NewAggFnEvaluator::intermediate_slot_desc() const {$/;" f class:NewAggFnEvaluator +intermediate_slot_desc_ src/exprs/agg_fn.h /^ const SlotDescriptor& intermediate_slot_desc_;$/;" m class:doris::AggFn +intermediate_tuple_desc_ src/exec/new_partitioned_aggregation_node.h /^ TupleDescriptor* intermediate_tuple_desc_;$/;" m class:doris::NewPartitionedAggregationNode +intermediate_tuple_id_ src/exec/new_partitioned_aggregation_node.h /^ TupleId intermediate_tuple_id_;$/;" m class:doris::NewPartitionedAggregationNode +intermediate_type src/exprs/agg_fn.h /^ const TypeDescriptor& intermediate_type() const { return intermediate_slot_desc_.type(); }$/;" f class:doris::AggFn +intermediate_type src/exprs/agg_fn_evaluator.h /^ const TypeDescriptor& intermediate_type() const {$/;" f class:doris::AggFnEvaluator +intermediate_type src/exprs/new_agg_fn_evaluator.cc /^const TypeDescriptor& NewAggFnEvaluator::intermediate_type() const {$/;" f class:NewAggFnEvaluator +internal src/gutil/casts.h /^namespace internal {$/;" n namespace:base +internal src/gutil/cpu.cc /^namespace internal {$/;" n namespace:base file: +internal src/gutil/cpu.h /^namespace internal {$/;" n namespace:base +internal src/gutil/gscoped_ptr.h /^namespace internal {$/;" n namespace:doris +internal src/gutil/spinlock_internal.cc /^namespace base { namespace internal { static int SuggestedDelayNS(int loop); }}$/;" n namespace:base file: +internal src/gutil/spinlock_internal.cc /^namespace internal {$/;" n namespace:base file: +internal src/gutil/spinlock_internal.h /^namespace internal {$/;" n namespace:base +internal src/gutil/spinlock_linux-inl.h /^namespace internal {$/;" n namespace:base +internal src/gutil/spinlock_posix-inl.h /^namespace internal {$/;" n namespace:base +internal src/gutil/spinlock_win32-inl.h /^namespace internal {$/;" n namespace:base +internal src/gutil/strings/split.h /^namespace internal {$/;" n namespace:strings +internal src/gutil/strings/split_internal.h /^namespace internal {$/;" n namespace:strings +internal src/gutil/strings/strcat.cc /^namespace internal {$/;" n namespace:strings file: +internal src/gutil/strings/strcat.h /^namespace internal {$/;" n namespace:strings +internal src/gutil/strings/substitute.cc /^namespace internal {$/;" n namespace:strings file: +internal src/gutil/strings/substitute.h /^namespace internal { \/\/ Implementation details.$/;" n namespace:strings +internal src/gutil/type_traits.h /^namespace internal {$/;" n namespace:base +internal src/olap/fs/block_manager_metrics.cpp /^namespace internal {$/;" n namespace:doris::fs file: +internal src/olap/fs/block_manager_metrics.h /^namespace internal {$/;" n namespace:doris::fs +internal src/olap/fs/file_block_manager.cpp /^namespace internal {$/;" n namespace:doris::fs file: +internal src/olap/fs/file_block_manager.h /^namespace internal {$/;" n namespace:doris::fs +internal_fa src/util/minizip/unzip.h /^ uLong internal_fa; \/* internal file attributes 2 bytes *\/$/;" m struct:unz_file_info64_s +internal_fa src/util/minizip/unzip.h /^ uLong internal_fa; \/* internal file attributes 2 bytes *\/$/;" m struct:unz_file_info_s +interpolate src/util/tdigest.h /^ static Value interpolate(Value x, Value x0, Value x1) { return (x - x0) \/ (x1 - x0); }$/;" f class:doris::TDigest +interpret_eval src/exprs/scalar_fn_call.cpp /^RETURN_TYPE ScalarFnCall::interpret_eval(ExprContext* context, TupleRow* row) {$/;" f class:doris::ScalarFnCall +intersect_count src/exprs/bitmap_function.cpp /^ int64_t intersect_count() const {$/;" f struct:doris::BitmapIntersect +io_error src/env/env_posix.cpp /^static Status io_error(const std::string& context, int err_number) {$/;" f namespace:doris +io_mgr_tracker src/runtime/test_env.h /^ MemTracker* io_mgr_tracker() {$/;" f class:doris::TestEnv +io_ns src/olap/olap_common.h /^ int64_t io_ns = 0;$/;" m struct:doris::OlapReaderStatistics +io_time_ms src/util/system_metrics.cpp /^ IntLockCounter io_time_ms;$/;" m struct:doris::DiskMetrics file: +io_time_weigthed src/util/system_metrics.cpp /^ IntLockCounter io_time_weigthed;$/;" m struct:doris::DiskMetrics file: +ip src/exec/schema_scanner.h /^ const std::string* ip; \/\/ frontend ip$/;" m struct:doris::SchemaScannerParam +ip_to_int src/util/cidr.cpp /^bool CIDR::ip_to_int(const std::string& ip_str, uint32_t* value) {$/;" f class:doris::CIDR +ipaddress src/util/thrift_client.h /^ const std::string& ipaddress() {$/;" f class:doris::ThriftClientImpl +ir_crc_hash src/util/hash_util_ir.cpp /^uint32_t ir_crc_hash(const void* data, int32_t bytes, uint32_t hash) {$/;" f namespace:doris +ir_fnv_hash src/util/hash_util_ir.cpp /^uint32_t ir_fnv_hash(const void* data, int32_t bytes, uint32_t hash) {$/;" f namespace:doris +ir_generic_is_null_string src/exec/scanner_ir.cpp /^bool ir_generic_is_null_string(const char* s, int slen, const char* n, int nlen) {$/;" f +ir_is_null_string src/exec/scanner_ir.cpp /^bool ir_is_null_string(const char* data, int len) {$/;" f +ir_string_compare src/runtime/string_value_ir.cpp /^int ir_string_compare(const char* s1, int n1, const char* s2, int n2) {$/;" f namespace:doris +ir_string_to_bool src/exec/scanner_ir.cpp /^bool ir_string_to_bool(const char* s, int len, StringParser::ParseResult* result) {$/;" f +ir_string_to_double src/exec/scanner_ir.cpp /^double ir_string_to_double(const char* s, int len, StringParser::ParseResult* result) {$/;" f +ir_string_to_float src/exec/scanner_ir.cpp /^float ir_string_to_float(const char* s, int len, StringParser::ParseResult* result) {$/;" f +ir_string_to_int16 src/exec/scanner_ir.cpp /^int16_t ir_string_to_int16(const char* s, int len, StringParser::ParseResult* result) {$/;" f +ir_string_to_int32 src/exec/scanner_ir.cpp /^int32_t ir_string_to_int32(const char* s, int len, StringParser::ParseResult* result) {$/;" f +ir_string_to_int64 src/exec/scanner_ir.cpp /^int64_t ir_string_to_int64(const char* s, int len, StringParser::ParseResult* result) {$/;" f +ir_string_to_int8 src/exec/scanner_ir.cpp /^int8_t ir_string_to_int8(const char* s, int len, StringParser::ParseResult* result) {$/;" f +is32BitsEnough src/util/bitmap_value.h /^ bool is32BitsEnough() const { return maximum() <= std::numeric_limits::max(); }$/;" f class:doris::detail::Roaring64Map +is64bitOpenFunction src/util/minizip/unzip.c /^ int is64bitOpenFunction;$/;" m struct:__anon34 file: +isDirty src/util/tdigest.h /^ bool isDirty() { return _processed.size() > _max_processed || _unprocessed.size() > _max_unprocessed; }$/;" f class:doris::TDigest +isEmpty src/util/bitmap_value.h /^ bool isEmpty() const {$/;" f class:doris::detail::Roaring64Map +isFull src/util/bitmap_value.h /^ bool isFull() const {$/;" f class:doris::detail::Roaring64Map +isStrictSubset src/util/bitmap_value.h /^ bool isStrictSubset(const Roaring64Map& r) const {$/;" f class:doris::detail::Roaring64Map +isSubset src/util/bitmap_value.h /^ bool isSubset(const Roaring64Map& r) const {$/;" f class:doris::detail::Roaring64Map +isZip64 src/util/minizip/unzip.c /^ int isZip64;$/;" m struct:__anon34 file: +is_aborted src/common/status.h /^ bool is_aborted() const { return code() == TStatusCode::ABORTED; }$/;" f class:doris::Status +is_active src/util/threadpool.h /^ bool is_active() const {$/;" f class:doris::ThreadPoolToken +is_address_v4 src/util/network_util.cpp /^bool InetAddress::is_address_v4 () const {$/;" f class:doris::InetAddress +is_after src/olap/rowset/segment_v2/row_ranges.h /^ bool is_after(const RowRange& other) const {$/;" f class:doris::segment_v2::RowRange +is_all_whitespace src/util/string_parser.hpp /^ static inline bool is_all_whitespace(const char* s, int len) {$/;" f class:doris::StringParser +is_already_exist src/common/status.h /^ bool is_already_exist() const { return code() == TStatusCode::ALREADY_EXIST; }$/;" f class:doris::Status +is_any_tuple_nullable src/runtime/descriptors.cpp /^bool RowDescriptor::is_any_tuple_nullable() const {$/;" f class:doris::RowDescriptor +is_arg_constant src/udf/udf_ir.cpp /^bool FunctionContext::is_arg_constant(int i) const {$/;" f class:doris_udf::FunctionContext +is_array src/gutil/type_traits.h /^template struct is_array : public true_type {};$/;" s namespace:base +is_array src/gutil/type_traits.h /^template struct is_array : public true_type {};$/;" s namespace:base +is_array src/gutil/type_traits.h /^template struct is_array : public false_type {};$/;" s namespace:base +is_before src/olap/rowset/segment_v2/row_ranges.h /^ bool is_before(const RowRange& other) const {$/;" f class:doris::segment_v2::RowRange +is_begin_include src/exec/olap_common.h /^ bool is_begin_include() const {$/;" f class:doris::ColumnValueRange +is_bf_column src/olap/rowset/column_writer.h /^ bool is_bf_column() {$/;" f class:doris::ColumnWriter +is_bf_column src/olap/tablet_schema.h /^ inline bool is_bf_column() const { return _is_bf_column; }$/;" f class:doris::TabletColumn +is_blacklisted src/runtime/tmp_file_mgr.cc /^bool TmpFileMgr::is_blacklisted(DeviceId device_id) {$/;" f class:doris::TmpFileMgr +is_blacklisted src/runtime/tmp_file_mgr.h /^ bool is_blacklisted() const {$/;" f class:doris::TmpFileMgr::Dir +is_blacklisted src/runtime/tmp_file_mgr.h /^ bool is_blacklisted() const {$/;" f class:doris::TmpFileMgr::File +is_bound src/exprs/expr.cpp /^bool Expr::is_bound(std::vector* tuple_ids) const {$/;" f class:doris::Expr +is_bound src/exprs/slot_ref.cpp /^bool SlotRef::is_bound(std::vector* tuple_ids) const {$/;" f class:doris::SlotRef +is_builtin src/exprs/agg_fn.h /^ bool is_builtin() const { return _fn.binary_type == TFunctionBinaryType::BUILTIN; }$/;" f class:doris::AggFn +is_builtin src/exprs/agg_fn_evaluator.h /^ bool is_builtin() const {$/;" f class:doris::AggFnEvaluator +is_cancelled src/common/status.h /^ bool is_cancelled() const { return code() == TStatusCode::CANCELLED; }$/;" f class:doris::Status +is_cancelled src/runtime/buffered_block_mgr2.cc /^bool BufferedBlockMgr2::is_cancelled() {$/;" f class:doris::BufferedBlockMgr2 +is_cancelled src/runtime/runtime_state.h /^ bool is_cancelled() const {$/;" f class:doris::RuntimeState +is_child_passthrough src/exec/union_node.h /^ bool is_child_passthrough(int child_idx) const {$/;" f class:doris::UnionNode +is_class src/gutil/template_util.h /^struct is_class$/;" s namespace:base +is_class_or_union src/gutil/type_traits.h /^template struct is_class_or_union {$/;" s namespace:base::internal +is_clone_ src/exprs/new_agg_fn_evaluator.h /^ const bool is_clone_;$/;" m class:doris::NewAggFnEvaluator +is_closed src/exec/exec_node.h /^ bool is_closed() const {$/;" f class:doris::ExecNode +is_closed src/exec/new_partitioned_aggregation_node.h /^ bool is_closed;$/;" m struct:doris::NewPartitionedAggregationNode::Partition +is_closed src/exec/partitioned_aggregation_node.h /^ bool is_closed;$/;" m struct:doris::PartitionedAggregationNode::Partition +is_closed src/runtime/buffered_tuple_stream3.h /^ bool is_closed() const { return closed_; }$/;" f class:doris::BufferedTupleStream3 +is_closed src/runtime/bufferpool/buffer_pool.h /^ bool is_closed() const { return tracker_ == nullptr; }$/;" f class:doris::BufferPool::SubReservation +is_clustering_col src/runtime/descriptors.h /^ bool is_clustering_col(const SlotDescriptor* slot_desc) const {$/;" f class:doris::TableDescriptor +is_collection_type src/runtime/types.h /^ inline bool is_collection_type() const {$/;" f struct:doris::TypeDescriptor +is_compile src/exec/schema_scanner/schema_collations_scanner.h /^ const char *is_compile;$/;" m struct:doris::SchemaCollationsScanner::CollationStruct +is_complex_type src/runtime/types.h /^ inline bool is_complex_type() const {$/;" f struct:doris::TypeDescriptor +is_const src/gutil/template_util.h /^template struct is_const : false_type {};$/;" s namespace:base +is_const src/gutil/template_util.h /^template struct is_const : true_type {};$/;" s namespace:base +is_constant src/exprs/expr.cpp /^bool Expr::is_constant() const {$/;" f class:doris::Expr +is_constant src/exprs/scalar_fn_call.cpp /^bool ScalarFnCall::is_constant() const {$/;" f class:doris::ScalarFnCall +is_constant src/exprs/slot_ref.h /^ virtual bool is_constant() const {$/;" f class:doris::SlotRef +is_consumption_metric_null src/runtime/mem_tracker.h /^ bool is_consumption_metric_null() {$/;" f class:doris::MemTracker +is_convertible src/gutil/type_traits.h /^struct is_convertible$/;" s namespace:base +is_count_distinct src/exprs/agg_fn.h /^ bool is_count_distinct() const { return agg_op_ == COUNT_DISTINCT; }$/;" f class:doris::AggFn +is_count_star src/exprs/agg_fn.h /^ bool is_count_star() const { return agg_op_ == COUNT && _children.empty(); }$/;" f class:doris::AggFn +is_count_star src/exprs/agg_fn_evaluator.h /^ bool is_count_star() const {$/;" f class:doris::AggFnEvaluator +is_date_type src/runtime/types.h /^ inline bool is_date_type() const {$/;" f struct:doris::TypeDescriptor +is_decimal_type src/runtime/types.h /^ inline bool is_decimal_type() const {$/;" f struct:doris::TypeDescriptor +is_default src/exec/schema_scanner/schema_collations_scanner.h /^ const char *is_default;$/;" m struct:doris::SchemaCollationsScanner::CollationStruct +is_dict_encoding src/olap/rowset/segment_v2/binary_dict_page.cpp /^bool BinaryDictPageDecoder::is_dict_encoding() const {$/;" f class:doris::segment_v2::BinaryDictPageDecoder +is_dir src/util/file_utils.cpp /^bool FileUtils::is_dir(const std::string& file_path, Env* env) {$/;" f class:doris::FileUtils +is_dir src/util/file_utils.cpp /^bool FileUtils::is_dir(const std::string& path) {$/;" f class:doris::FileUtils +is_directory src/agent/cgroups_mgr.cpp /^bool CgroupsMgr::is_directory(const char* file_path) {$/;" f class:doris::CgroupsMgr +is_dot_or_dotdot src/util/file_utils.h /^inline bool is_dot_or_dotdot(const char* name) {$/;" f namespace:doris +is_downloaded src/runtime/user_function_cache.cpp /^ bool is_downloaded = false;$/;" m struct:doris::UserFunctionCacheEntry file: +is_empty src/olap/rowset/segment_v2/row_ranges.h /^ bool is_empty() {$/;" f class:doris::segment_v2::RowRanges +is_empty_value_range src/exec/olap_common.h /^bool ColumnValueRange::is_empty_value_range() const {$/;" f class:doris::ColumnValueRange +is_end_ src/gutil/strings/split_internal.h /^ bool is_end_;$/;" m class:strings::internal::SplitIterator +is_end_include src/exec/olap_common.h /^ bool is_end_include() const {$/;" f class:doris::ColumnValueRange +is_end_of_file src/common/status.h /^ bool is_end_of_file() const { return code() == TStatusCode::END_OF_FILE; }$/;" f class:doris::Status +is_enum src/gutil/type_traits.h /^template struct is_enum$/;" s namespace:base +is_enum src/gutil/type_traits.h /^template struct is_enum : is_enum { };$/;" s namespace:base +is_enum src/gutil/type_traits.h /^template struct is_enum : is_enum { };$/;" s namespace:base +is_enum src/gutil/type_traits.h /^template struct is_enum : is_enum { };$/;" s namespace:base +is_enum_impl src/gutil/type_traits.h /^template struct is_enum_impl$/;" s namespace:base::internal +is_enum_impl src/gutil/type_traits.h /^template struct is_enum_impl : false_type { };$/;" s namespace:base::internal +is_enumeration_type src/runtime/primitive_type.h /^inline bool is_enumeration_type(PrimitiveType type) {$/;" f namespace:doris +is_eof src/olap/rowset/alpha_rowset_reader.h /^ bool is_eof = false;$/;" m struct:doris::AlphaMergeContext +is_eos src/exec/merge_join_node.h /^ bool is_eos;$/;" m struct:doris::MergeJoinNode::ChildReaderContext +is_file_exist src/agent/cgroups_mgr.cpp /^bool CgroupsMgr::is_file_exist(const char* file_path) {$/;" f class:doris::CgroupsMgr +is_file_exist src/agent/cgroups_mgr.cpp /^bool CgroupsMgr::is_file_exist(const std::string& file_path) {$/;" f class:doris::CgroupsMgr +is_filter_data src/olap/delete_handler.cpp /^bool DeleteHandler::is_filter_data(const int32_t data_version,$/;" f class:doris::DeleteHandler +is_finished src/runtime/load_channel.cpp /^bool LoadChannel::is_finished() {$/;" f class:doris::LoadChannel +is_fixed_value_convertible src/exec/olap_common.h /^bool ColumnValueRange::is_fixed_value_convertible() const {$/;" f class:doris::ColumnValueRange +is_fixed_value_range src/exec/olap_common.h /^bool ColumnValueRange::is_fixed_value_range() const {$/;" f class:doris::ColumnValueRange +is_floating_point src/gutil/type_traits.h /^template struct is_floating_point : false_type { };$/;" s namespace:base +is_floating_point src/gutil/type_traits.h /^template struct is_floating_point$/;" s namespace:base +is_floating_point src/gutil/type_traits.h /^template struct is_floating_point$/;" s namespace:base +is_floating_point src/gutil/type_traits.h /^template struct is_floating_point$/;" s namespace:base +is_floating_point src/gutil/type_traits.h /^template<> struct is_floating_point : true_type { };$/;" s namespace:base +is_floating_point src/gutil/type_traits.h /^template<> struct is_floating_point : true_type { };$/;" s namespace:base +is_floating_point src/gutil/type_traits.h /^template<> struct is_floating_point : true_type { };$/;" s namespace:base +is_format_support_streaming src/http/action/stream_load.cpp /^static bool is_format_support_streaming(TFileFormatType::type format) {$/;" f namespace:doris +is_full src/olap/rowset/segment_v2/index_page.cpp /^bool IndexPageBuilder::is_full() const {$/;" f class:doris::segment_v2::IndexPageBuilder +is_full src/runtime/row_batch.h /^ bool is_full() {$/;" f class:doris::RowBatch +is_high_value_maximum src/exec/olap_common.h /^ bool is_high_value_maximum() const {$/;" f class:doris::ColumnValueRange +is_in_hybirdmap src/exprs/agg_fn_evaluator.cpp /^bool AggFnEvaluator::is_in_hybirdmap(void* input_val, Tuple* dst, bool* is_add_buckets) {$/;" f class:doris::AggFnEvaluator +is_in_memory src/olap/tablet_schema.h /^ inline bool is_in_memory() const {return _is_in_memory; }$/;" f class:doris::TabletSchema +is_in_prior_network src/service/backend_options.cpp /^bool BackendOptions::is_in_prior_network(const std::string& ip) {$/;" f class:doris::BackendOptions +is_in_range src/exec/olap_common.h /^bool ColumnValueRange::is_in_range(const T& value) {$/;" f class:doris::ColumnValueRange +is_in_subplan src/exec/exec_node.h /^ bool is_in_subplan() const { return false; }$/;" f class:doris::ExecNode +is_in_use src/olap/rowset/segment_group.cpp /^bool SegmentGroup::is_in_use() {$/;" f class:doris::SegmentGroup +is_initialized src/util/bit_stream_utils.h /^ bool is_initialized() const { return buffer_ != NULL; }$/;" f class:doris::BitReader +is_integral src/gutil/type_traits.h /^template struct is_integral : false_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template struct is_integral : is_integral { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template struct is_integral : is_integral { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template struct is_integral : is_integral { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral<__wchar_t> : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_integral src/gutil/type_traits.h /^template<> struct is_integral : true_type { };$/;" s namespace:base +is_invalid_argument src/common/status.h /^ bool is_invalid_argument() const { return code() == TStatusCode::INVALID_ARGUMENT; }$/;" f class:doris::Status +is_io_error src/common/status.h /^ bool is_io_error() const {return code() == TStatusCode::IO_ERROR; }$/;" f class:doris::Status +is_io_error src/olap/utils.h /^inline bool is_io_error(OLAPStatus status) {$/;" f namespace:doris +is_key src/olap/olap_cond.h /^ inline bool is_key() const {$/;" f class:doris::CondColumn +is_key src/olap/tablet_schema.h /^ inline bool is_key() const { return _is_key; }$/;" f class:doris::TabletColumn +is_lazy_materialization_read src/olap/iterators.h /^ virtual bool is_lazy_materialization_read() const { return false; }$/;" f class:doris::RowwiseIterator +is_leaf src/olap/rowset/segment_v2/index_page.h /^ inline bool is_leaf() const {$/;" f class:doris::segment_v2::IndexPageReader +is_leap src/runtime/datetime_value.cpp /^static bool is_leap(uint32_t year) {$/;" f namespace:doris +is_literal_node src/exec/es/es_predicate.cpp /^static bool is_literal_node(const Expr* expr) {$/;" f namespace:doris +is_loop_closed src/geo/geo_types.cpp /^static bool is_loop_closed(const std::vector& points) {$/;" f namespace:doris +is_loopback_v4 src/util/network_util.cpp /^bool InetAddress::is_loopback_v4() {$/;" f class:doris::InetAddress +is_low_value_mininum src/exec/olap_common.h /^ bool is_low_value_mininum() const {$/;" f class:doris::ColumnValueRange +is_lower_keys_included src/olap/rowset/rowset_reader_context.h /^ const std::vector* is_lower_keys_included = nullptr;$/;" m struct:doris::RowsetReaderContext +is_mangled src/util/symbols_util.cpp /^bool SymbolsUtil::is_mangled(const std::string& symbol) {$/;" f class:doris::SymbolsUtil +is_map src/gutil/strings/split_internal.h /^ struct is_map {$/;" s class:strings::internal::Splitter +is_match_func src/exec/es/es_predicate.cpp /^bool EsPredicate::is_match_func(const Expr* conjunct) {$/;" f class:doris::EsPredicate +is_match_func src/exec/es_scan_node.cpp /^bool EsScanNode::is_match_func(Expr* conjunct) {$/;" f class:doris::EsScanNode +is_matched src/exec/partitioned_hash_table.inline.h /^inline bool PartitionedHashTable::Iterator::is_matched() const {$/;" f class:doris::PartitionedHashTable::Iterator +is_materialized src/runtime/descriptor_helper.h /^ TSlotDescriptorBuilder& is_materialized(bool is_materialized) {$/;" f class:doris::TSlotDescriptorBuilder +is_materialized src/runtime/descriptors.h /^ bool is_materialized() const {$/;" f class:doris::SlotDescriptor +is_max_size src/runtime/buffered_block_mgr.h /^ bool is_max_size() const {$/;" f class:doris::BufferedBlockMgr::Block +is_max_size src/runtime/buffered_block_mgr2.h /^ bool is_max_size() const {$/;" f class:doris::BufferedBlockMgr2::Block +is_mem_limit_exceeded src/common/status.h /^ bool is_mem_limit_exceeded() const { return code() == TStatusCode::MEM_LIMIT_EXCEEDED; }$/;" f class:doris::Status +is_merge src/exprs/agg_fn.h /^ bool is_merge() const { return is_merge_; }$/;" f class:doris::AggFn +is_merge src/exprs/agg_fn_evaluator.h /^ bool is_merge() const {$/;" f class:doris::AggFnEvaluator +is_merge_ src/exprs/agg_fn.h /^ const bool is_merge_;$/;" m class:doris::AggFn +is_multi_distinct src/exprs/agg_fn_evaluator.h /^ bool is_multi_distinct() {$/;" f class:doris::AggFnEvaluator +is_multi_distinct src/exprs/new_agg_fn_evaluator.h /^ bool is_multi_distinct() {$/;" f class:doris::NewAggFnEvaluator +is_name_valid src/http/action/mini_load.cpp /^static bool is_name_valid(const std::string& name) {$/;" f namespace:doris +is_named_wildcard src/util/path_trie.hpp /^ bool is_named_wildcard(const std::string& key) {$/;" f class:doris::PathTrie::TrieNode +is_neg src/runtime/datetime_value.h /^ bool is_neg;$/;" m struct:doris::TimeInterval +is_negative_vint src/exec/read_write_util.h /^inline bool ReadWriteUtil::is_negative_vint(int8_t byte) {$/;" f class:doris::ReadWriteUtil +is_non_const_reference src/gutil/template_util.h /^template struct is_non_const_reference : false_type {};$/;" s namespace:base +is_non_const_reference src/gutil/template_util.h /^template struct is_non_const_reference : true_type {};$/;" s namespace:base +is_non_const_reference src/gutil/template_util.h /^template struct is_non_const_reference : false_type {};$/;" s namespace:base +is_not_found src/common/status.h /^ bool is_not_found() const { return code() == TStatusCode::NOT_FOUND; }$/;" f class:doris::Status +is_not_in src/exec/es/es_predicate.h /^ bool is_not_in;$/;" m struct:doris::ExtInPredicate +is_not_in src/exprs/in_predicate.h /^ bool is_not_in() const {$/;" f class:doris::InPredicate +is_not_null src/exec/es/es_predicate.h /^ bool is_not_null;$/;" m struct:doris::ExtIsNullPredicate +is_not_null src/exprs/is_null_predicate.cpp /^BooleanVal IsNullPredicate::is_not_null(FunctionContext* ctx, const T& val) {$/;" f class:doris::IsNullPredicate +is_null output/udf/include/udf.h /^ bool is_null;$/;" m struct:doris_udf::AnyVal +is_null src/exec/broker_scanner.cpp /^bool is_null(const Slice& slice) {$/;" f namespace:doris +is_null src/exec/csv_scan_node.cpp /^static bool is_null(const char* value, int value_length) {$/;" f namespace:doris +is_null src/exec/schema_scanner.h /^ bool is_null;$/;" m struct:doris::SchemaScanner::ColumnDesc +is_null src/exprs/is_null_predicate.cpp /^BooleanVal IsNullPredicate::is_null(FunctionContext* ctx, const T& val) {$/;" f class:doris::IsNullPredicate +is_null src/geo/geo_functions.cpp /^ bool is_null;$/;" m struct:doris::StConstructState file: +is_null src/geo/geo_functions.cpp /^ bool is_null;$/;" m struct:doris::StContainsState file: +is_null src/olap/column_block.h /^ bool is_null() const { return _block.is_null(_idx); }$/;" f struct:doris::ColumnBlockCell +is_null src/olap/column_block.h /^ bool is_null(size_t idx) const {$/;" f class:doris::ColumnBlock +is_null src/olap/fs/block_id.h /^ bool is_null() const { return _id == kInvalidId; }$/;" f class:doris::BlockId +is_null src/olap/row_block2.h /^ bool is_null(size_t col_idx) const {$/;" f class:doris::RowBlockRow +is_null src/olap/row_cursor.h /^ bool is_null(size_t index) const {$/;" f class:doris::RowCursor +is_null src/olap/row_cursor_cell.h /^ bool is_null() const { return *reinterpret_cast(_ptr); }$/;" f struct:doris::RowCursorCell +is_null src/olap/schema.h /^ bool is_null(const char* row, int index) const {$/;" f class:doris::Schema +is_null src/olap/tuple.h /^ bool is_null(size_t i) const { return _nulls[i]; }$/;" f class:doris::OlapTuple +is_null src/olap/wrapper_field.h /^ bool is_null() const { return *reinterpret_cast(_field_buf); }$/;" f class:doris::WrapperField +is_null src/runtime/tuple.h /^ bool is_null(const NullIndicatorOffset& offset) const {$/;" f class:doris::Tuple +is_null src/runtime/vectorized_row_batch.h /^ bool* is_null() const {$/;" f class:doris::ColumnVector +is_null src/udf/udf.h /^ bool is_null;$/;" m struct:doris_udf::AnyVal +is_null_bit_set src/exprs/slot_ref.h /^inline bool SlotRef::is_null_bit_set(TupleRow* row) {$/;" f class:doris::SlotRef +is_null_scalar_function src/exprs/expr.h /^ bool is_null_scalar_function(std::string &str) {$/;" f class:doris::Expr +is_nullable src/exprs/expr_context.cpp /^bool ExprContext::is_nullable() {$/;" f class:doris::ExprContext +is_nullable src/exprs/slot_ref.h /^inline bool SlotRef::is_nullable(Expr* expr) {$/;" f class:doris::SlotRef +is_nullable src/olap/column_block.h /^ bool is_nullable() const { return _block->is_nullable(); }$/;" f class:doris::ColumnBlockView +is_nullable src/olap/column_block.h /^ bool is_nullable() const { return _null_bitmap != nullptr; }$/;" f class:doris::ColumnBlock +is_nullable src/olap/field.h /^ bool is_nullable() const { return _is_nullable; }$/;" f class:doris::Field +is_nullable src/olap/rowset/segment_v2/column_reader.h /^ bool is_nullable() const { return _meta.is_nullable(); }$/;" f class:doris::segment_v2::ColumnReader +is_nullable src/olap/tablet_schema.h /^ inline bool is_nullable() const { return _is_nullable; }$/;" f class:doris::TabletColumn +is_nullable src/runtime/descriptors.h /^ bool is_nullable() const {$/;" f class:doris::SlotDescriptor +is_on_queue src/runtime/disk_io_mgr_internal.h /^ bool is_on_queue() const {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +is_open src/exec/olap_scanner.h /^ bool is_open() const { return _is_open; }$/;" f class:doris::OlapScanner +is_open src/runtime/bufferpool/buffer_pool.h /^ bool is_open() const { return data_ != NULL; }$/;" f class:doris::BufferPool::BufferHandle +is_open src/runtime/bufferpool/buffer_pool.h /^ bool is_open() const { return page_ != NULL; }$/;" f class:doris::BufferPool::PageHandle +is_page_full src/olap/rowset/segment_v2/binary_dict_page.cpp /^bool BinaryDictPageBuilder::is_page_full() {$/;" f class:doris::segment_v2::BinaryDictPageBuilder +is_partial_delete src/olap/generic_iterators.cpp /^ int is_partial_delete() const { return _block.delete_state() == DEL_PARTIAL_SATISFIED; }$/;" f class:doris::MergeIteratorContext +is_pending src/olap/rowset/rowset.h /^ bool is_pending() const { return _is_pending; }$/;" f class:doris::Rowset +is_pending src/olap/rowset/segment_group.h /^ inline bool is_pending() const { return _is_pending; }$/;" f class:doris::SegmentGroup +is_pinned src/runtime/buffered_block_mgr2.h /^ bool is_pinned() const {$/;" f class:doris::BufferedBlockMgr2::Block +is_pinned src/runtime/buffered_tuple_stream2.h /^ bool is_pinned() const { return _pinned; }$/;" f class:doris::BufferedTupleStream2 +is_pinned src/runtime/buffered_tuple_stream3.h /^ inline bool is_pinned() const { return handle.is_pinned(); }$/;" f struct:doris::BufferedTupleStream3::Page +is_pinned src/runtime/buffered_tuple_stream3.h /^ bool is_pinned() const { return pinned_; }$/;" f class:doris::BufferedTupleStream3 +is_pinned src/runtime/bufferpool/buffer_pool.h /^ bool is_pinned() const { return pin_count() > 0; }$/;" f class:doris::BufferPool::PageHandle +is_pod src/gutil/type_traits.h /^template struct is_pod$/;" s namespace:base +is_pod src/gutil/type_traits.h /^template struct is_pod : is_pod { };$/;" s namespace:base +is_pod src/gutil/type_traits.h /^template struct is_pod : is_pod { };$/;" s namespace:base +is_pod src/gutil/type_traits.h /^template struct is_pod : is_pod { };$/;" s namespace:base +is_pointer src/gutil/type_traits.h /^template struct is_pointer : false_type { };$/;" s namespace:base +is_pointer src/gutil/type_traits.h /^template struct is_pointer : true_type { };$/;" s namespace:base +is_pointer src/gutil/type_traits.h /^template struct is_pointer : is_pointer { };$/;" s namespace:base +is_pointer src/gutil/type_traits.h /^template struct is_pointer : is_pointer { };$/;" s namespace:base +is_pointer src/gutil/type_traits.h /^template struct is_pointer : is_pointer { };$/;" s namespace:base +is_prefix_of src/runtime/descriptors.cpp /^bool RowDescriptor::is_prefix_of(const RowDescriptor& other_desc) const {$/;" f class:doris::RowDescriptor +is_primary_replica src/olap/tablet.h /^ inline bool is_primary_replica() { return false; }$/;" f class:doris::Tablet +is_range_value_convertible src/exec/olap_common.h /^bool ColumnValueRange::is_range_value_convertible() const {$/;" f class:doris::ColumnValueRange +is_read_page src/runtime/buffered_tuple_stream3.h /^ bool is_read_page(const Page* page) const {$/;" f class:doris::BufferedTupleStream3 +is_reference src/gutil/type_traits.h /^template struct is_reference : false_type {};$/;" s namespace:base +is_reference src/gutil/type_traits.h /^template struct is_reference : true_type {};$/;" s namespace:base +is_registered src/runtime/bufferpool/buffer_pool.h /^ bool is_registered() const { return impl_ != NULL; }$/;" f class:doris::BufferPool::ClientHandle +is_remove_from_rowset_meta src/olap/rowset/rowset_meta.h /^ bool is_remove_from_rowset_meta() const {$/;" f class:doris::RowsetMeta +is_rotational src/util/disk_info.h /^ bool is_rotational;$/;" m struct:doris::DiskInfo::Disk +is_rotational src/util/disk_info.h /^ static bool is_rotational(int disk_id) {$/;" f class:doris::DiskInfo +is_row_selected src/olap/selection_vector.h /^ bool is_row_selected(size_t row) const {$/;" f class:doris::SelectionVector +is_running_in_vm src/gutil/cpu.h /^ bool is_running_in_vm() const { return is_running_in_vm_; }$/;" f class:base::final +is_running_in_vm_ src/gutil/cpu.h /^ bool is_running_in_vm_;$/;" m class:base::final +is_safe_subtract src/olap/serialize.h /^inline bool is_safe_subtract(int64_t left, int64_t right) {$/;" f namespace:doris::ser +is_same src/gutil/type_traits.h /^template struct is_same : public false_type { };$/;" s namespace:base +is_same src/gutil/type_traits.h /^template struct is_same : public true_type { };$/;" s namespace:base +is_scan_node src/exec/exec_node.h /^ virtual bool is_scan_node() const {$/;" f class:doris::ExecNode +is_scan_node src/exec/scan_node.h /^ virtual bool is_scan_node() const {$/;" f class:doris::ScanNode +is_segments_overlapping src/olap/rowset/rowset_meta.h /^ bool is_segments_overlapping() const {$/;" f class:doris::RowsetMeta +is_service_unavailable src/common/status.h /^ bool is_service_unavailable() const { return code() == TStatusCode::SERVICE_UNAVAILABLE; }$/;" f class:doris::Status +is_set_default_rowset_type_to_beta src/runtime/heartbeat_flags.h /^ bool is_set_default_rowset_type_to_beta() {$/;" f class:doris::HeartbeatFlags +is_shutdown src/util/batch_process_thread_pool.hpp /^ bool is_shutdown() {$/;" f class:doris::BatchProcessThreadPool +is_shutdown src/util/priority_thread_pool.hpp /^ bool is_shutdown() {$/;" f class:doris::PriorityThreadPool +is_singleton_delta src/olap/rowset/rowset_meta.h /^ bool is_singleton_delta() const {$/;" f class:doris::RowsetMeta +is_slotref src/exprs/expr.h /^ bool is_slotref() const {$/;" f class:doris::Expr +is_specialized src/gutil/casts.h /^ static const bool is_specialized = false;$/;" m class:enum_limits +is_spilled src/exec/new_partitioned_aggregation_node.h /^ bool is_spilled() const { return hash_tbl.get() == NULL; }$/;" f struct:doris::NewPartitionedAggregationNode::Partition +is_spilled src/exec/partitioned_aggregation_node.h /^ bool is_spilled() const {$/;" f struct:doris::PartitionedAggregationNode::Partition +is_ssd_disk src/olap/data_dir.h /^ bool is_ssd_disk() const {$/;" f class:doris::DataDir +is_streaming src/http/action/mini_load.cpp /^ bool is_streaming = false;$/;" m struct:doris::MiniLoadCtx file: +is_streaming_preagg_ src/exec/new_partitioned_aggregation_node.h /^ bool is_streaming_preagg_;$/;" m class:doris::NewPartitionedAggregationNode +is_string_type src/olap/wrapper_field.h /^ bool is_string_type() const { return _is_string_type; }$/;" f class:doris::WrapperField +is_string_type src/runtime/types.h /^ inline bool is_string_type() const {$/;" f struct:doris::TypeDescriptor +is_sum_distinct src/exprs/agg_fn.h /^ bool is_sum_distinct() const { return agg_op_ == SUM_DISTINCT; }$/;" f class:doris::AggFn +is_supported src/util/cpu_info.h /^ inline static bool is_supported(long flag) {$/;" f class:doris::CpuInfo +is_suppressed src/olap/out_stream.h /^ bool is_suppressed() const {$/;" f class:doris::OutStream +is_thrift_rpc_error src/common/status.h /^ bool is_thrift_rpc_error() const { return code() == TStatusCode::THRIFT_RPC_ERROR; }$/;" f class:doris::Status +is_timeout src/runtime/fragment_mgr.cpp /^ bool is_timeout(const DateTimeValue& now) const {$/;" f class:doris::FragmentExecState +is_too_old src/runtime/load_path_mgr.cpp /^bool LoadPathMgr::is_too_old(time_t cur_time, const std::string& label_dir, int64_t reserve_hours) {$/;" f class:doris::LoadPathMgr +is_type_compatible src/runtime/primitive_type.h /^inline bool is_type_compatible(PrimitiveType lhs, PrimitiveType rhs) {$/;" f namespace:doris +is_uninitialized src/common/status.h /^ bool is_uninitialized() const { return code() == TStatusCode::UNINITIALIZED; }$/;" f class:doris::Status +is_upper_keys_included src/olap/rowset/rowset_reader_context.h /^ const std::vector* is_upper_keys_included = nullptr;$/;" m struct:doris::RowsetReaderContext +is_used src/olap/data_dir.h /^ bool is_used() const { return _is_used; }$/;" f class:doris::DataDir +is_used src/olap/olap_common.h /^ bool is_used; \/\/ 是否可用标识$/;" m struct:doris::DataDirInfo +is_used src/olap/storage_engine.h /^ bool is_used;$/;" m struct:doris::StorageEngine::CompactionDiskStat +is_used src/olap/tablet.h /^inline bool Tablet::is_used() {$/;" f class:doris::Tablet +is_using_mmap src/olap/rowset/segment_reader.h /^ bool is_using_mmap() {$/;" f class:doris::SegmentReader +is_valid src/exprs/json_functions.h /^ bool is_valid; \/\/ true if the path is successfully parsed$/;" m struct:doris::JsonPath +is_valid src/olap/hll.cpp /^bool HyperLogLog::is_valid(const Slice& slice) {$/;" f class:doris::HyperLogLog +is_valid src/olap/rowset/segment_v2/row_ranges.h /^ bool is_valid() const {$/;" f class:doris::segment_v2::RowRange +is_valid_key test/olap/skiplist_test.cpp /^ static bool is_valid_key(Key k) {$/;" f class:doris::ConcurrentTest file: +is_valid_lng_lat src/geo/geo_types.cpp /^static inline bool is_valid_lng_lat(double lng, double lat) {$/;" f namespace:doris +is_var_len_string_type src/runtime/types.h /^ inline bool is_var_len_string_type() const {$/;" f struct:doris::TypeDescriptor +is_var_length src/runtime/descriptors.h /^ static bool is_var_length(const std::vector& descs) {$/;" f class:doris::TupleDescriptor +is_vectorized src/exprs/compound_predicate.h /^ virtual bool is_vectorized() const {$/;" f class:doris::CompoundPredicate +is_vectorized src/exprs/expr.cpp /^bool Expr::is_vectorized() const {$/;" f class:doris::Expr +is_vectorized src/exprs/slot_ref.h /^ virtual bool is_vectorized() const {$/;" f class:doris::SlotRef +is_void src/gutil/template_util.h /^template <> struct is_void : true_type {};$/;" s namespace:base +is_void src/gutil/template_util.h /^template struct is_void : false_type {};$/;" s namespace:base +is_whitespace src/util/string_parser.hpp /^ static inline bool is_whitespace(const char& c) {$/;" f class:doris::StringParser +is_write_page src/runtime/buffered_tuple_stream3.h /^ bool is_write_page(const Page* page) const { return write_page_ == page; }$/;" f class:doris::BufferedTupleStream3 +is_zero src/runtime/decimal_value.h /^ bool is_zero() const {$/;" f class:doris::DecimalValue +is_zero src/runtime/decimalv2_value.h /^ bool is_zero() const {$/;" f class:doris::DecimalV2Value +isinf src/gutil/port.h /^inline int isinf(double x) {$/;" f +isnan src/gutil/port.h 922;" d +isvalidcharntorune src/gutil/utf/rune.c /^isvalidcharntorune(const char* str, int length, Rune* rune, int* consumed) {$/;" f +iter src/util/tdigest.h /^ std::vector::const_iterator iter;$/;" m struct:doris::CentroidList +iterate src/exprs/new_in_predicate.cpp /^BooleanVal InPredicate::iterate($/;" f class:doris::InPredicate +iterate src/olap/olap_meta.cpp /^OLAPStatus OlapMeta::iterate(int column_family_index, const std::string& prefix,$/;" f class:doris::OlapMeta +iterate src/runtime/bufferpool/buffer_pool_internal.h /^ void iterate(boost::function fn) { list_.iterate(fn); }$/;" f class:doris::BufferPool::PageList +iterate src/util/bitmap_value.h /^ void iterate(roaring_iterator64 iterator, void* ptr) const {$/;" f class:doris::detail::Roaring64Map +iterate src/util/internal_queue.h /^ void iterate(boost::function fn) {$/;" f class:doris::InternalQueueBase +iterator src/exec/row_batch_list.h /^ TupleRowIterator iterator() {$/;" f class:doris::RowBatchList +iterator src/gutil/strings/stringpiece.h /^ typedef const char* iterator;$/;" t class:StringPiece +iterator_category src/util/bitmap_value.h /^ typedef std::forward_iterator_tag iterator_category;$/;" t class:doris::detail::final +iterator_offset_t src/olap/utils.h /^typedef uint32_t iterator_offset_t;$/;" t namespace:doris +iterator_type src/exec/olap_common.h /^ typedef typename std::set::iterator iterator_type;$/;" t class:doris::ColumnValueRange +job_id src/runtime/stream_load/stream_load_context.h /^ int64_t job_id = -1;$/;" m class:doris::StreamLoadContext +job_id src/util/load_error_hub.h /^ int64_t job_id;$/;" m struct:doris::LoadErrorHub::ErrorMsg +join src/http/ev_http_server.cpp /^void EvHttpServer::join() {$/;" f class:doris::EvHttpServer +join src/service/brpc_service.cpp /^void BRpcService::join() {$/;" f class:doris::BRpcService +join src/util/batch_process_thread_pool.hpp /^ void join() {$/;" f class:doris::BatchProcessThreadPool +join src/util/priority_thread_pool.hpp /^ void join() {$/;" f class:doris::PriorityThreadPool +join src/util/ref_count_closure.h /^ void join() {$/;" f class:doris::RefCountClosure +join src/util/thread.cpp /^Status ThreadJoiner::join() {$/;" f class:doris::ThreadJoiner +join src/util/thread.cpp /^void Thread::join() {$/;" f class:doris::Thread +join src/util/thrift_server.cpp /^void ThriftServer::join() {$/;" f class:doris::ThriftServer +join_path_segments src/util/path_util.cpp /^string join_path_segments(const string& a, const string& b) {$/;" f namespace:doris::path_util +join_path_segments_v src/util/path_util.cpp /^vector join_path_segments_v(const vector& v, const string& s) {$/;" f namespace:doris::path_util +join_receivers test/runtime/data_stream_test.cpp /^ void join_receivers() {$/;" f class:doris::DataStreamTest +join_senders test/runtime/data_stream_test.cpp /^ void join_senders() {$/;" f class:doris::DataStreamTest +join_strings src/olap/fs/block_id.cpp /^string BlockId::join_strings(const vector& blocks) {$/;" f class:doris::BlockId +json_path_close src/exprs/json_functions.cpp /^void JsonFunctions::json_path_close($/;" f class:doris::JsonFunctions +json_path_prepare src/exprs/json_functions.cpp /^void JsonFunctions::json_path_prepare($/;" f class:doris::JsonFunctions +json_rowset_meta src/olap/rowset/rowset_meta.h /^ virtual bool json_rowset_meta(std::string* json_rowset_meta) {$/;" f class:doris::RowsetMeta +json_type_to_string src/exec/es/es_scroll_parser.cpp /^std::string json_type_to_string(rapidjson::Type type) {$/;" f namespace:doris +json_value_to_string src/exec/es/es_scroll_parser.cpp /^std::string json_value_to_string(const rapidjson::Value& value) {$/;" f namespace:doris +k0 src/gutil/hash/city.cc /^static const uint64 k0 = 0xa5b85c5e198ed849ULL;$/;" m namespace:util_hash file: +k1 src/gutil/hash/city.cc /^static const uint64 k1 = 0x8d58ac26afe12e47ULL;$/;" m namespace:util_hash file: +k2 src/gutil/hash/city.cc /^static const uint64 k2 = 0xc47b6e9e3a970ed3ULL;$/;" m namespace:util_hash file: +k3 src/gutil/hash/city.cc /^static const uint64 k3 = 0xc70f6907e782aa0bULL;$/;" m namespace:util_hash file: +kApb src/gutil/strings/ascii_ctype.h 32;" d +kApb src/gutil/strings/ascii_ctype.h 68;" d +kAsciiPropertyBits src/gutil/strings/ascii_ctype.cc /^const unsigned char kAsciiPropertyBits[256] = {$/;" v +kAsciiToInt src/gutil/strings/numbers.cc /^static const int8 kAsciiToInt[256] = {$/;" m namespace:__anon28 file: +kAsciiToLower src/gutil/strings/ascii_ctype.cc /^const unsigned char kAsciiToLower[256] = {$/;" v +kAsciiToUpper src/gutil/strings/ascii_ctype.cc /^const unsigned char kAsciiToUpper[256] = {$/;" v +kBase32NumUnescapedBytes src/gutil/strings/escaping.cc /^static const int kBase32NumUnescapedBytes[] = {$/;" m namespace:strings file: +kBase64Chars src/gutil/strings/escaping.cc /^static const char kBase64Chars[] =$/;" m namespace:strings file: +kBatchSize src/olap/rowset/segment_v2/segment_iterator.cpp /^ static const uint32_t kBatchSize = 256;$/;" m class:doris::segment_v2::SegmentIterator::BitmapRangeIterator file: +kBlockManagerDir test/olap/fs/file_block_manager_test.cpp /^ const string kBlockManagerDir = ".\/ut_dir\/file_block_manager";$/;" m class:doris::FileBlockManagerTest file: +kCacheSize test/olap/lru_cache_test.cpp /^ static const int kCacheSize = 1000;$/;" m class:doris::CacheTest file: +kCutStr src/gutil/strings/split.cc /^static const char kCutStr[] = "...";$/;" v file: +kCutStrSize src/gutil/strings/split.cc /^static const int kCutStrSize = sizeof(kCutStr) - 1;$/;" v file: +kDefaultGiveUpAfterMs src/util/thread.h /^ kDefaultGiveUpAfterMs = -1 \/\/ forever$/;" e enum:doris::ThreadJoiner::__anon39 +kDefaultPoolName test/util/threadpool_test.cpp /^static const char* kDefaultPoolName = "test";$/;" m namespace:doris file: +kDefaultWarnAfterMs src/util/thread.h /^ kDefaultWarnAfterMs = 1000,$/;" e enum:doris::ThreadJoiner::__anon39 +kDefaultWarnEveryMs src/util/thread.h /^ kDefaultWarnEveryMs = 1000,$/;" e enum:doris::ThreadJoiner::__anon39 +kDontNeedShellEscapeChars src/gutil/strings/escaping.cc /^const char* kDontNeedShellEscapeChars =$/;" m namespace:strings file: +kDoubleToBufferSize src/gutil/strings/numbers.h /^static const int kDoubleToBufferSize = 32;$/;" v +kFastToBufferSize src/gutil/strings/numbers.h /^static const int kFastToBufferSize = 32;$/;" v +kFingerprintSeed0 src/gutil/hash/hash.cc /^static const uint32 kFingerprintSeed0 = 0;$/;" v file: +kFingerprintSeed0 src/gutil/hash/hash.cc /^static const uint32 kFingerprintSeed0 = 0xabc;$/;" v file: +kFingerprintSeed1 src/gutil/hash/hash.cc /^static const uint32 kFingerprintSeed1 = 0xdef;$/;" v file: +kFingerprintSeed1 src/gutil/hash/hash.cc /^static const uint32 kFingerprintSeed1 = 102072;$/;" v file: +kFinished src/runtime/tablets_channel.h /^ kFinished \/\/ closed or cancelled$/;" e enum:doris::TabletsChannel::State +kFloatToBufferSize src/gutil/strings/numbers.h /^static const int kFloatToBufferSize = 24;$/;" v +kHexTable src/gutil/strings/escaping.cc /^static const char kHexTable[513]=$/;" m namespace:strings file: +kHighWater src/util/tdigest.h /^const size_t kHighWater = 40000;$/;" m namespace:doris +kIllegalFprint src/gutil/integral_types.h /^static const Fprint kIllegalFprint = 0;$/;" v +kIllegalHash16 src/gutil/hash/legacy_hash.h /^const Hash16 kIllegalHash16 = static_cast(0xffff);$/;" v +kIllegalHash32 src/gutil/hash/legacy_hash.h /^const Hash32 kIllegalHash32 = static_cast(0xffffffffUL);$/;" v +kInitialCapacity src/util/faststring.h /^ kInitialCapacity = 32$/;" e enum:doris::faststring::__anon37 +kInitialized src/runtime/tablets_channel.h /^ kInitialized,$/;" e enum:doris::TabletsChannel::State +kInvalidId src/olap/fs/block_id.cpp /^const uint64_t BlockId::kInvalidId = 0;$/;" m class:doris::BlockId file: +kInvalidId src/olap/fs/block_id.h /^ static const uint64_t kInvalidId;$/;" m class:doris::BlockId +kLeaveNullsEscaped src/gutil/strings/escaping.cc /^static bool kLeaveNullsEscaped = true;$/;" m namespace:strings file: +kLibShardNum src/runtime/user_function_cache.cpp /^static const int kLibShardNum = 128;$/;" m namespace:doris file: +kMaxFprint src/gutil/integral_types.h /^static const Fprint kMaxFprint = GG_ULONGLONG(0xFFFFFFFFFFFFFFFF);$/;" v +kMaxHeight src/olap/skiplist.h /^ enum { kMaxHeight = 12 };$/;" e enum:doris::SkipList::__anon55 +kMaxOverCut src/gutil/strings/split.cc /^static const int kMaxOverCut = 12;$/;" v file: +kMaxWidth test/util/bit_stream_utils_test.cpp /^const int kMaxWidth = 64;$/;" m namespace:doris file: +kMaxWidth test/util/rle_encoding_test.cpp /^const int kMaxWidth = 64;$/;" m namespace:doris file: +kMicrosecondsPerSecond src/util/monotime.h /^ static const int64_t kMicrosecondsPerSecond = 1000000L;$/;" m class:doris::MonoTime +kMix32 src/gutil/hash/string_hash.h /^static const uint32 kMix32 = 0x12b9b0a1UL;$/;" m namespace:hash_internal +kMix64 src/gutil/hash/string_hash.h /^static const uint64 kMix64 = GG_ULONGLONG(0x2b992ddfa23249d6);$/;" m namespace:hash_internal +kMtabPath src/olap/data_dir.cpp /^static const char* const kMtabPath = "\/etc\/mtab";$/;" m namespace:doris file: +kNanosecondsPerMicrosecond src/util/monotime.h /^ static const int64_t kNanosecondsPerMicrosecond = 1000L;$/;" m class:doris::MonoTime +kNanosecondsPerMillisecond src/util/monotime.h /^ static const int64_t kNanosecondsPerMillisecond = 1000000L;$/;" m class:doris::MonoTime +kNanosecondsPerSecond src/util/monotime.h /^ static const int64_t kNanosecondsPerSecond = 1000000000L;$/;" m class:doris::MonoTime +kNumShardBits src/olap/lru_cache.h /^ static const int kNumShardBits = 4;$/;" m class:doris::CachePriority +kNumShards src/olap/lru_cache.h /^ static const int kNumShards = 1 << kNumShardBits;$/;" m class:doris::CachePriority +kOpened src/runtime/tablets_channel.h /^ kOpened,$/;" e enum:doris::TabletsChannel::State +kPlatformUsesOPDSections src/gutil/port.h /^enum { kPlatformUsesOPDSections = 0 };$/;" e enum:__anon6 +kPlatformUsesOPDSections src/gutil/port.h /^enum { kPlatformUsesOPDSections = 1 };$/;" e enum:__anon5 +kPprofActionMutex src/http/action/pprof_actions.cpp /^static std::mutex kPprofActionMutex;$/;" m namespace:doris file: +kPprofDefaultSampleSecs src/http/action/pprof_actions.cpp /^static const int kPprofDefaultSampleSecs = 30;$/;" m namespace:doris file: +kRowsetDir test/olap/rowset/beta_rowset_test.cpp /^ const string kRowsetDir = ".\/ut_dir\/beta_rowset_test";$/;" m class:doris::BetaRowsetTest file: +kSegmentDir test/olap/rowset/segment_v2/segment_test.cpp /^ const string kSegmentDir = ".\/ut_dir\/segment_test";$/;" m class:doris::segment_v2::SegmentReaderWriterTest file: +kStringPrintfVectorMaxArgs src/gutil/stringprintf.cc /^const int kStringPrintfVectorMaxArgs = 32;$/;" v +kTestDir test/olap/rowset/segment_v2/bitmap_index_test.cpp /^ const std::string kTestDir = ".\/ut_dir\/bitmap_index_test";$/;" m class:doris::segment_v2::BitmapIndexTest file: +kTestDir test/olap/rowset/segment_v2/ordinal_page_index_test.cpp /^ const std::string kTestDir = ".\/ut_dir\/ordinal_page_index_test";$/;" m class:doris::segment_v2::OrdinalPageIndexTest file: +kTestDir test/olap/rowset/segment_v2/zone_map_index_test.cpp /^ const std::string kTestDir = ".\/ut_dir\/zone_map_index_test";$/;" m class:doris::segment_v2::ColumnZoneMapTest file: +kTestFilePath src/olap/data_dir.cpp /^static const char* const kTestFilePath = "\/.testfile";$/;" m namespace:doris file: +kTmpInfix src/util/path_util.cpp /^const string kTmpInfix = ".doristmp";$/;" m namespace:doris::path_util file: +kUnBase64 src/gutil/strings/escaping.cc /^static const signed char kUnBase64[] = {$/;" m namespace:strings file: +kUnWebSafeBase64 src/gutil/strings/escaping.cc /^static const signed char kUnWebSafeBase64[] = {$/;" m namespace:strings file: +kUnescapeNulls src/gutil/strings/escaping.cc /^static bool kUnescapeNulls = false;$/;" m namespace:strings file: +kUninitialized src/util/monotime.cpp /^const int64_t MonoDelta::kUninitialized = kint64min;$/;" m class:doris::MonoDelta file: +kUninitialized src/util/monotime.h /^ static const int64_t kUninitialized;$/;" m class:doris::MonoDelta +kWebSafeBase64Chars src/gutil/strings/escaping.cc /^static const char kWebSafeBase64Chars[] =$/;" m namespace:strings file: +k_100_continue src/http/action/mini_load.cpp /^const char* k_100_continue = "100-continue";$/;" m namespace:doris file: +k_add_batch_status test/exec/tablet_sink_test.cpp /^Status k_add_batch_status;$/;" m namespace:doris::stream_load file: +k_doris_exit src/common/daemon.cpp /^bool k_doris_exit = false;$/;" m namespace:doris file: +k_engine test/olap/delete_handler_test.cpp /^static StorageEngine* k_engine = nullptr;$/;" m namespace:doris file: +k_engine test/olap/delta_writer_test.cpp /^StorageEngine* k_engine = nullptr;$/;" m namespace:doris file: +k_engine test/olap/memtable_flush_executor_test.cpp /^StorageEngine* k_engine = nullptr;$/;" m namespace:doris file: +k_engine test/olap/olap_snapshot_converter_test.cpp /^static StorageEngine* k_engine = nullptr;$/;" m namespace:doris file: +k_engine test/olap/rowset/rowset_meta_manager_test.cpp /^static StorageEngine* k_engine = nullptr;$/;" m namespace:doris file: +k_engine test/olap/tablet_mgr_test.cpp /^static StorageEngine* k_engine = nullptr;$/;" m namespace:doris file: +k_engine test/olap/txn_manager_test.cpp /^static StorageEngine* k_engine = nullptr;$/;" m namespace:doris file: +k_flush_executor test/olap/memtable_flush_executor_test.cpp /^MemTableFlushExecutor* k_flush_executor = nullptr;$/;" m namespace:doris file: +k_is_downloaded test/runtime/user_function_cache_test.cpp /^bool k_is_downloaded = false;$/;" m namespace:doris file: +k_mem_tracker test/olap/delta_writer_test.cpp /^MemTracker* k_mem_tracker = nullptr;$/;" m namespace:doris file: +k_names src/util/system_metrics.cpp /^ static const char* k_names[k_num_metrics];$/;" m struct:doris::CpuMetrics file: +k_names src/util/system_metrics.cpp /^const char* CpuMetrics::k_names[] = {$/;" m class:doris::CpuMetrics file: +k_num_metrics src/util/system_metrics.cpp /^ static constexpr int k_num_metrics = 10;$/;" m struct:doris::CpuMetrics file: +k_response_str test/http/stream_load_test.cpp /^std::string k_response_str;$/;" m namespace:doris file: +k_segment_magic src/olap/rowset/segment_v2/segment_writer.cpp /^const char* k_segment_magic = "D0R1";$/;" m namespace:doris::segment_v2 file: +k_segment_magic_length src/olap/rowset/segment_v2/segment_writer.cpp /^const uint32_t k_segment_magic_length = 4;$/;" m namespace:doris::segment_v2 file: +k_stream_load_begin_result src/runtime/stream_load/stream_load_executor.cpp /^TLoadTxnBeginResult k_stream_load_begin_result;$/;" m namespace:doris file: +k_stream_load_commit_result src/runtime/stream_load/stream_load_executor.cpp /^TLoadTxnCommitResult k_stream_load_commit_result;$/;" m namespace:doris file: +k_stream_load_plan_status src/runtime/stream_load/stream_load_executor.cpp /^Status k_stream_load_plan_status;$/;" m namespace:doris file: +k_stream_load_put_result src/http/action/stream_load.cpp /^TStreamLoadPutResult k_stream_load_put_result;$/;" m namespace:doris file: +k_stream_load_rollback_result src/runtime/stream_load/stream_load_executor.cpp /^TLoadTxnRollbackResult k_stream_load_rollback_result;$/;" m namespace:doris file: +k_streaming_load_bytes src/http/action/stream_load.cpp /^IntCounter k_streaming_load_bytes;$/;" m namespace:doris file: +k_streaming_load_current_processing src/http/action/stream_load.cpp /^static IntGauge k_streaming_load_current_processing;$/;" m namespace:doris file: +k_streaming_load_duration_ms src/http/action/stream_load.cpp /^IntCounter k_streaming_load_duration_ms;$/;" m namespace:doris file: +k_streaming_load_requests_total src/http/action/stream_load.cpp /^IntCounter k_streaming_load_requests_total;$/;" m namespace:doris file: +k_ut_diskstats_path src/util/system_metrics.cpp /^const char* k_ut_diskstats_path;$/;" m namespace:doris file: +k_ut_fd_path src/util/system_metrics.cpp /^const char* k_ut_fd_path;$/;" m namespace:doris file: +k_ut_net_dev_path src/util/system_metrics.cpp /^const char* k_ut_net_dev_path;$/;" m namespace:doris file: +k_ut_stat_path src/util/system_metrics.cpp /^const char* k_ut_stat_path;$/;" m namespace:doris file: +k_zero_length_region_ src/runtime/mem_pool.h /^ static uint32_t k_zero_length_region_;$/;" m class:doris::MemPool +kafka_info src/runtime/stream_load/stream_load_context.h /^ std::unique_ptr kafka_info;$/;" m class:doris::StreamLoadContext +keep_alive_min src/runtime/external_scan_context_mgr.h /^ short keep_alive_min;$/;" m struct:doris::ScanContext +keep_kudu_scanner_alive src/exec/kudu_scanner.cpp /^void KuduScanner::keep_kudu_scanner_alive() {$/;" f class:doris::KuduScanner +kept_in_memory src/olap/rowset/segment_v2/column_reader.h /^ bool kept_in_memory = false;$/;" m struct:doris::segment_v2::ColumnReaderOptions +kept_in_memory src/olap/rowset/segment_v2/page_io.h /^ bool kept_in_memory = false;$/;" m struct:doris::segment_v2::PageReadOptions +kernel_dirent src/gutil/linux_syscall_support.h /^struct kernel_dirent {$/;" s +kernel_dirent64 src/gutil/linux_syscall_support.h /^struct kernel_dirent64 {$/;" s +kernel_old_sigaction src/gutil/linux_syscall_support.h /^struct kernel_old_sigaction {$/;" s +kernel_old_sigaction src/gutil/linux_syscall_support.h 266;" d +kernel_rusage src/gutil/linux_syscall_support.h /^struct kernel_rusage {$/;" s +kernel_sigaction src/gutil/linux_syscall_support.h /^struct kernel_sigaction {$/;" s +kernel_sigset_t src/gutil/linux_syscall_support.h /^struct kernel_sigset_t {$/;" s +kernel_stat src/gutil/linux_syscall_support.h /^struct kernel_stat {$/;" s +kernel_stat64 src/gutil/linux_syscall_support.h /^struct kernel_stat64 {$/;" s +kernel_timespec src/gutil/linux_syscall_support.h /^struct kernel_timespec {$/;" s +kernel_timeval src/gutil/linux_syscall_support.h /^struct kernel_timeval {$/;" s +key src/exprs/json_functions.h /^ std::string key; \/\/ key of a json object$/;" m struct:doris::JsonPath +key src/olap/lru_cache.h /^ CacheKey key() const {$/;" f struct:doris::CachePriority::LRUHandle +key src/olap/short_key_index.h /^ Slice key(ssize_t ordinal) const {$/;" f class:doris::ShortKeyIndexDecoder +key src/olap/skiplist.h /^ Key const key;$/;" m struct:doris::SkipList::Node +key src/olap/skiplist.h /^inline const Key& SkipList::Iterator::key() const {$/;" f class:doris::SkipList::Iterator +key test/olap/skiplist_test.cpp /^ static uint64_t key(Key key) { return (key >> 40); }$/;" f class:doris::ConcurrentTest file: +keyToBits src/util/radix_sort.h /^ static KeyBits keyToBits(Key x) { return bit_cast(x); }$/;" f struct:doris::RadixSort +key_columns src/runtime/descriptors.h /^ const std::vector& key_columns() const { return key_columns_; }$/;" f class:doris::KuduTableDescriptor +key_columns_ src/runtime/descriptors.h /^ std::vector key_columns_;$/;" m class:doris::KuduTableDescriptor +key_data src/olap/lru_cache.h /^ char key_data[1]; \/\/ Beginning of key$/;" m struct:doris::CachePriority::LRUHandle +key_length src/olap/lru_cache.h /^ size_t key_length;$/;" m struct:doris::CachePriority::LRUHandle +key_range_index src/olap/rowset/alpha_rowset_reader.h /^ int key_range_index = -1;$/;" m struct:doris::AlphaMergeContext +key_ranges src/olap/iterators.h /^ std::vector key_ranges;$/;" m class:doris::StorageReadOptions +key_t src/olap/aggregate_func.cpp /^ typedef std::pair key_t;$/;" t class:doris::AggregateFuncResolver file: +key_type src/olap/skiplist.h /^ typedef Key key_type;$/;" t class:doris::SkipList +keys src/runtime/dpp_sink_internal.h /^ const std::vector& keys() const {$/;" f class:doris::RollupSchema +keys src/util/minizip/unzip.c /^ unsigned long keys[3]; \/* keys defining the pseudo-random sequence *\/$/;" m struct:__anon34 file: +keys_type src/olap/tablet.h /^inline KeysType Tablet::keys_type() const {$/;" f class:doris::Tablet +keys_type src/olap/tablet_schema.h /^ inline KeysType keys_type() const { return _keys_type; }$/;" f class:doris::TabletSchema +keys_type src/runtime/dpp_sink_internal.h /^ const std::string& keys_type() const {$/;" f class:doris::RollupSchema +kind src/olap/stream_name.h /^ StreamInfoMessage::Kind kind() const {$/;" f class:doris::StreamName +kint16max src/gutil/integral_types.h /^static const int16 kint16max = (( int16) 0x7FFF);$/;" v +kint16min src/gutil/integral_types.h /^static const int16 kint16min = (( int16) ~0x7FFF);$/;" v +kint32max src/gutil/integral_types.h /^static const int32 kint32max = (( int32) 0x7FFFFFFF);$/;" v +kint32min src/gutil/integral_types.h /^static const int32 kint32min = (( int32) ~0x7FFFFFFF);$/;" v +kint64max src/gutil/integral_types.h /^static const int64 kint64max = (( int64) GG_LONGLONG(0x7FFFFFFFFFFFFFFF));$/;" v +kint64min src/gutil/integral_types.h /^static const int64 kint64min = (( int64) GG_LONGLONG(~0x7FFFFFFFFFFFFFFF));$/;" v +kint8max src/gutil/integral_types.h /^static const int8 kint8max = (( int8) 0x7F);$/;" v +kint8min src/gutil/integral_types.h /^static const int8 kint8min = (( int8) ~0x7F);$/;" v +knuth_stddev_finalize src/exprs/aggregate_functions.cpp /^DoubleVal AggregateFunctions::knuth_stddev_finalize(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +knuth_stddev_pop_finalize src/exprs/aggregate_functions.cpp /^DoubleVal AggregateFunctions::knuth_stddev_pop_finalize(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +knuth_var_finalize src/exprs/aggregate_functions.cpp /^DoubleVal AggregateFunctions::knuth_var_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +knuth_var_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::knuth_var_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +knuth_var_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::knuth_var_merge(FunctionContext* ctx, const StringVal& src,$/;" f class:doris::AggregateFunctions +knuth_var_pop_finalize src/exprs/aggregate_functions.cpp /^DoubleVal AggregateFunctions::knuth_var_pop_finalize(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +knuth_var_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::knuth_var_update(FunctionContext* ctx, const T& src,$/;" f class:doris::AggregateFunctions +kudu_client src/exec/kudu_scan_node.h /^ kudu::client::KuduClient* kudu_client() { return _client.get(); }$/;" f class:doris::KuduScanNode +kudu_master_addresses src/runtime/descriptors.h /^ const std::vector& kudu_master_addresses() const {$/;" f class:doris::KuduTableDescriptor +kudu_round_trips src/exec/kudu_scan_node.h /^ RuntimeProfile::Counter* kudu_round_trips() const { return _kudu_round_trips; }$/;" f class:doris::KuduScanNode +kuint128max src/gutil/int128.cc /^const uint128_pod kuint128max = {$/;" v +kuint16max src/gutil/integral_types.h /^static const uint16 kuint16max = ((uint16) 0xFFFF);$/;" v +kuint32max src/gutil/integral_types.h /^static const uint32 kuint32max = ((uint32) 0xFFFFFFFF);$/;" v +kuint64max src/gutil/integral_types.h /^static const uint64 kuint64max = ((uint64) GG_LONGLONG(0xFFFFFFFFFFFFFFFF));$/;" v +kuint8max src/gutil/integral_types.h /^static const uint8 kuint8max = (( uint8) 0xFF);$/;" v +l src/gutil/bits.h /^ static const T l = (sizeof(T) == 1) ? 1 :$/;" m struct:BitPattern +label src/http/action/mini_load.h /^ std::string label;$/;" m struct:doris::LoadHandle +label src/runtime/mem_tracker.h /^ const std::string& label() const {$/;" f class:doris::MemTracker +label src/runtime/stream_load/stream_load_context.h /^ std::string label;$/;" m class:doris::StreamLoadContext +labels src/util/metrics.h /^ std::set labels;$/;" m struct:doris::MetricLabels +large_int_val src/exprs/expr_value.h /^ __int128 large_int_val;$/;" m struct:doris::ExprValue +largest_partition_percent_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::HighWaterMarkCounter* largest_partition_percent_;$/;" m class:doris::NewPartitionedAggregationNode +largest_spilled_partition src/exec/partitioned_aggregation_node.cc /^int64_t PartitionedAggregationNode::largest_spilled_partition() const {$/;" f class:doris::PartitionedAggregationNode +last src/olap/olap_index.h /^ uint32_t last;$/;" m struct:doris::IDRange +last_access_time src/runtime/external_scan_context_mgr.h /^ time_t last_access_time;$/;" m struct:doris::ScanContext +last_base_compaction_failure_time src/olap/tablet.h /^ int64_t last_base_compaction_failure_time() { return _last_base_compaction_failure_millis; }$/;" f class:doris::Tablet +last_base_compaction_success_time src/olap/tablet.h /^ int64_t last_base_compaction_success_time() { return _last_base_compaction_success_millis; }$/;" f class:doris::Tablet +last_block src/olap/rowset/segment_v2/segment.h /^ uint32_t last_block() const {$/;" f class:doris::segment_v2::Segment +last_cumu_compaction_failure_time src/olap/tablet.h /^ int64_t last_cumu_compaction_failure_time() { return _last_cumu_compaction_failure_millis; }$/;" f class:doris::Tablet +last_cumu_compaction_success_time src/olap/tablet.h /^ int64_t last_cumu_compaction_success_time() { return _last_cumu_compaction_success_millis; }$/;" f class:doris::Tablet +last_expr_value src/exec/hash_table.h /^ void* last_expr_value(int expr_idx) const {$/;" f class:doris::HashTable +last_expr_value src/exec/partitioned_hash_table.h /^ void* last_expr_value(int expr_idx) const {$/;" f class:doris::PartitionedHashTableCtx +last_expr_value_null src/exec/hash_table.h /^ bool last_expr_value_null(int expr_idx) const {$/;" f class:doris::HashTable +last_expr_value_null src/exec/partitioned_hash_table.h /^ bool last_expr_value_null(int expr_idx) const {$/;" f class:doris::PartitionedHashTableCtx +last_ordinal src/olap/rowset/segment_v2/ordinal_page_index.h /^ ordinal_t last_ordinal() const { return _index->get_last_ordinal(_cur_idx); }$/;" f class:doris::segment_v2::OrdinalPageIndexIterator +last_updated_time src/runtime/load_channel.h /^ time_t last_updated_time() const { return _last_updated_time.load(); }$/;" f class:doris::LoadChannel +last_val_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::last_val_remove(FunctionContext* ctx, const StringVal& src,$/;" f class:doris::AggregateFunctions +last_val_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::last_val_remove(FunctionContext* ctx, const T& src, T* dst) {$/;" f class:doris::AggregateFunctions +last_val_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::last_val_update(FunctionContext* ctx, const StringVal& src,$/;" f class:doris::AggregateFunctions +last_val_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::last_val_update(FunctionContext* ctx, const T& src, T* dst) {$/;" f class:doris::AggregateFunctions +last_visit_time src/runtime/routine_load/data_consumer.h /^ time_t last_visit_time() { return _last_visit_time; }$/;" f class:doris::DataConsumer +latch_ src/util/countdown_latch.h /^ CountDownLatch *latch_;$/;" m class:doris::CountDownOnScopeExit +layout_equals src/runtime/descriptors.cpp /^bool RowDescriptor::layout_equals(const RowDescriptor& other_desc) const {$/;" f class:doris::RowDescriptor +layout_equals src/runtime/descriptors.cpp /^bool SlotDescriptor::layout_equals(const SlotDescriptor& other_desc) const {$/;" f class:doris::SlotDescriptor +layout_equals src/runtime/descriptors.cpp /^bool TupleDescriptor::layout_equals(const TupleDescriptor& other_desc) const {$/;" f class:doris::TupleDescriptor +layout_is_prefix_of src/runtime/descriptors.cpp /^bool RowDescriptor::layout_is_prefix_of(const RowDescriptor& other_desc) const {$/;" f class:doris::RowDescriptor +lcpyT src/gutil/strings/util.cc /^size_t lcpyT(CHAR* dst, const CHAR* src, size_t dst_size) {$/;" f namespace:__anon27 +le src/runtime/string_value.h /^ bool le(const StringValue& other) const {$/;" f struct:doris::StringValue +left src/exprs/string_functions.cpp /^StringVal StringFunctions::left($/;" f class:doris::StringFunctions +left_join_get_next src/exec/hash_join_node.cpp /^Status HashJoinNode::left_join_get_next(RuntimeState* state,$/;" f class:doris::HashJoinNode +len output/udf/include/udf.h /^ int len;$/;" m struct:doris_udf::FunctionContext::TypeDesc +len output/udf/include/udf.h /^ int len;$/;" m struct:doris_udf::StringVal +len src/runtime/buffered_block_mgr.h /^ int64_t len;$/;" m struct:doris::BufferedBlockMgr::BufferDescriptor +len src/runtime/buffered_block_mgr2.h /^ int64_t len;$/;" m struct:doris::BufferedBlockMgr2::BufferDescriptor +len src/runtime/buffered_tuple_stream3.h /^ inline int len() const { return handle.len(); }$/;" f struct:doris::BufferedTupleStream3::Page +len src/runtime/bufferpool/buffer_pool.cc /^int64_t BufferPool::PageHandle::len() const {$/;" f class:doris::BufferPool::PageHandle +len src/runtime/bufferpool/buffer_pool.h /^ int64_t len() const {$/;" f class:doris::BufferPool::BufferHandle +len src/runtime/bufferpool/buffer_pool_internal.h /^ const int64_t len;$/;" m class:doris::BufferPool::Page +len src/runtime/bufferpool/suballocator.h /^ int64_t len() const { return len_; }$/;" f class:doris::Suballocation +len src/runtime/disk_io_mgr.h /^ int64_t len() const { return _len; }$/;" f class:doris::DiskIoMgr::RequestRange +len src/runtime/disk_io_mgr.h /^ int64_t len() { return _len; }$/;" f class:doris::DiskIoMgr::BufferDescriptor +len src/runtime/string_value.h /^ size_t len;$/;" m struct:doris::StringValue +len src/runtime/types.h /^ int len;$/;" m struct:doris::TypeDescriptor +len src/udf/udf.h /^ int len;$/;" m struct:doris_udf::FunctionContext::TypeDesc +len src/udf/udf.h /^ int len;$/;" m struct:doris_udf::StringVal +len src/util/frame_of_reference_coding.h /^ uint32_t len() {$/;" f class:doris::ForEncoder +len src/util/mem_range.h /^ int64_t len() const { return len_; }$/;" f class:doris::MemRange +len src/util/rle_encoding.h /^ int32_t len() const { return bit_writer_.bytes_written(); }$/;" f class:doris::RleEncoder +len test/udf/uda_test.cpp /^ int len;$/;" m struct:doris_udf::MinState file: +len test/util/utf8_check_test.cpp /^ int len;$/;" m struct:doris::test file: +len_ src/runtime/bufferpool/buffer_pool.h /^ int64_t len_;$/;" m class:doris::BufferPool::BufferHandle +len_ src/runtime/bufferpool/suballocator.h /^ int64_t len_;$/;" m class:doris::Suballocation +len_ src/util/faststring.h /^ size_t len_;$/;" m class:doris::faststring +len_ src/util/mem_range.h /^ int64_t len_;$/;" m class:doris::MemRange +length src/exec/csv_scan_node.cpp /^ int length() const {$/;" f class:doris::StringRef +length src/exprs/string_functions.cpp /^IntVal StringFunctions::length(FunctionContext* context, const StringVal& str) {$/;" f class:doris::StringFunctions +length src/gutil/strings/stringpiece.h /^ int length() const { return length_; }$/;" f class:StringPiece +length src/http/http_parser.h /^ size_t length; \/\/ minimal length need to read$/;" m struct:doris::HttpChunkParseCtx +length src/olap/field.h /^ inline int32_t length() const { return _length; }$/;" f class:doris::Field +length src/olap/file_helper.cpp /^off_t FileHandler::length() const {$/;" f class:doris::FileHandler +length src/olap/file_helper.cpp /^off_t FileHandlerWithBuf::length() const {$/;" f class:doris::FileHandlerWithBuf +length src/olap/file_stream.h /^ size_t length() {$/;" f class:doris::ReadOnlyFileStream::FileCursor +length src/olap/olap_index.h /^ size_t length;$/;" m struct:doris::EntrySlice +length src/olap/out_stream.h /^ uint32_t length : 24; \/\/ 24位长度$/;" m struct:doris::StreamHead +length src/olap/rowset/run_length_integer_writer.h /^ inline uint16_t length() const {$/;" f struct:doris::RunLengthIntegerWriter::DeltaHead +length src/olap/rowset/run_length_integer_writer.h /^ inline uint16_t length() const {$/;" f struct:doris::RunLengthIntegerWriter::DirectHead +length src/olap/rowset/run_length_integer_writer.h /^ inline uint16_t length() const {$/;" f struct:doris::RunLengthIntegerWriter::PatchedBaseHead +length src/olap/tablet_schema.h /^ size_t length() const { return _length; }$/;" f class:doris::TabletColumn +length src/util/faststring.h /^ size_t length() const {$/;" f class:doris::faststring +length src/util/mysql_row_buffer.h /^ int length() const {$/;" f class:doris::MysqlRowBuffer +length_ src/gutil/strings/stringpiece.h /^ int length_;$/;" m class:StringPiece +length_low src/olap/rowset/run_length_integer_writer.h /^ uint8_t length_low: 8;$/;" m struct:doris::RunLengthIntegerWriter::DeltaHead +length_low src/olap/rowset/run_length_integer_writer.h /^ uint8_t length_low: 8;$/;" m struct:doris::RunLengthIntegerWriter::DirectHead +length_low src/olap/rowset/run_length_integer_writer.h /^ uint8_t length_low: 8;$/;" m struct:doris::RunLengthIntegerWriter::PatchedBaseHead +length_msb src/olap/rowset/run_length_integer_writer.h /^ uint8_t length_msb: 1,$/;" m struct:doris::RunLengthIntegerWriter::DeltaHead +length_msb src/olap/rowset/run_length_integer_writer.h /^ uint8_t length_msb: 1,$/;" m struct:doris::RunLengthIntegerWriter::DirectHead +length_msb src/olap/rowset/run_length_integer_writer.h /^ uint8_t length_msb: 1,$/;" m struct:doris::RunLengthIntegerWriter::PatchedBaseHead +less src/util/radix_sort.h /^ static bool less(Key x, Key y) {$/;" f struct:doris::RadixSortFloatTraits +less src/util/radix_sort.h /^ static bool less(Key x, Key y) {$/;" f struct:doris::RadixSortIntTraits +less src/util/radix_sort.h /^ static bool less(Key x, Key y) {$/;" f struct:doris::RadixSortUIntTraits +level src/exec/new_partitioned_aggregation_node.h /^ const int level;$/;" m struct:doris::NewPartitionedAggregationNode::Partition +level src/exec/new_partitioned_hash_table.h /^ int ALWAYS_INLINE level() const { return level_; }$/;" f class:doris::NewPartitionedHashTableCtx +level src/exec/partitioned_aggregation_node.h /^ const int level;$/;" m struct:doris::PartitionedAggregationNode::Partition +level src/exec/partitioned_hash_table.h /^ int level() const { return _level; }$/;" f class:doris::PartitionedHashTableCtx +level_ src/exec/new_partitioned_hash_table.h /^ int level_;$/;" m class:doris::NewPartitionedHashTableCtx +lhs_ordering_expr_ctxs src/exec/sort_exec_exprs.h /^ const std::vector& lhs_ordering_expr_ctxs() const {$/;" f class:doris::SortExecExprs +lib_file src/runtime/user_function_cache.cpp /^ std::string lib_file;$/;" m struct:doris::UserFunctionCacheEntry file: +lib_handle src/runtime/user_function_cache.cpp /^ void* lib_handle = nullptr;$/;" m struct:doris::UserFunctionCacheEntry file: +lib_version src/exec/decompressor.h /^ uint16_t lib_version;$/;" m struct:doris::LzopDecompressor::HeaderInfo +like src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::like($/;" f class:doris::LikePredicate +like_close src/exprs/like_predicate.cpp /^void LikePredicate::like_close($/;" f class:doris::LikePredicate +like_fn src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::like_fn($/;" f class:doris::LikePredicate +like_prepare src/exprs/like_predicate.cpp /^void LikePredicate::like_prepare($/;" f class:doris::LikePredicate +limit src/exec/exec_node.h /^ int64_t limit() const {$/;" f class:doris::ExecNode +limit src/olap/byte_buffer.h /^ inline uint64_t limit() const {$/;" f class:doris::StorageByteBuffer +limit src/olap/row_block.h /^ size_t limit() const { return _limit; }$/;" f class:doris::RowBlock +limit src/runtime/mem_tracker.h /^ int64_t limit() const {$/;" f class:doris::MemTracker +limit src/runtime/vectorized_row_batch.h /^ uint16_t limit() const { return _limit; }$/;" f class:doris::VectorizedRowBatch +limit src/util/byte_buffer.h /^ size_t limit;$/;" m struct:doris::ByteBuffer +limit_ src/gutil/strings/split.h /^ const int limit_;$/;" m class:strings::delimiter::LimitImpl +limit_exceeded src/runtime/mem_tracker.h /^ bool limit_exceeded() const{$/;" f class:doris::MemTracker +limit_exceeded src/runtime/mem_tracker.h /^ static bool limit_exceeded(const std::vector& limits) {$/;" f class:doris::MemTracker +line_to_src_tuple src/exec/broker_scanner.cpp /^bool BrokerScanner::line_to_src_tuple(const Slice& line) {$/;" f class:doris::BrokerScanner +lineno src/util/bfd_parser.cpp /^ unsigned int lineno;$/;" m struct:doris::BfdFindCtx file: +linestring src/geo/wkt_yacc.y /^linestring:$/;" l +link_files_to src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::link_files_to(const std::string& dir, RowsetId new_rowset_id) {$/;" f class:doris::AlphaRowset +link_files_to src/olap/rowset/beta_rowset.cpp /^OLAPStatus BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset_id) {$/;" f class:doris::BetaRowset +link_segments_to_path src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::link_segments_to_path(const std::string& dest_path, const RowsetId& rowset_id) {$/;" f class:doris::SegmentGroup +list src/geo/wkt_parse_type.h /^ std::vector list;$/;" m struct:doris::GeoCoordinateList +list src/geo/wkt_parse_type.h /^ std::vector list;$/;" m struct:doris::GeoCoordinateListList +list src/runtime/free_pool.hpp /^ FreeListNode* list; \/\/ Used when it is being used by the caller.$/;" m union:doris::FreePool::FreeListNode::__anon1 +list_ src/olap/skiplist.h /^ const SkipList* list_;$/;" m class:doris::SkipList::Iterator +list_ src/runtime/bufferpool/buffer_pool_internal.h /^ InternalList list_;$/;" m class:doris::BufferPool::PageList +list_dirs_files src/util/file_utils.cpp /^Status FileUtils::list_dirs_files(const std::string& path, std::set* dirs,$/;" f class:doris::FileUtils +list_files src/util/file_utils.cpp /^Status FileUtils::list_files(Env* env, const std::string& dir,$/;" f class:doris::FileUtils +list_matching_formats src/util/bfd_parser.cpp /^static void list_matching_formats(char **p, std::string* message) {$/;" f namespace:doris +list_sections src/util/bfd_parser.cpp /^void BfdParser::list_sections(std::string* ss) {$/;" f class:doris::BfdParser +list_table_status src/exec/schema_scanner/schema_helper.cpp /^Status SchemaHelper::list_table_status($/;" f class:doris::SchemaHelper +list_targets src/util/bfd_parser.cpp /^void BfdParser::list_targets(std::vector* out) {$/;" f class:doris::BfdParser +list_versions src/olap/tablet.cpp /^void Tablet::list_versions(vector* versions) const {$/;" f class:doris::Tablet +literal_count_ src/util/rle_encoding.h /^ int literal_count_;$/;" m class:doris::RleEncoder +literal_count_ src/util/rle_encoding.h /^ uint32_t literal_count_;$/;" m class:doris::RleDecoder +literal_indicator_byte_idx_ src/util/rle_encoding.h /^ int literal_indicator_byte_idx_;$/;" m class:doris::RleEncoder +lo output/udf/include/udf.h /^ int64_t lo;$/;" m struct:doris_udf::FunctionContext::UniqueId +lo src/gutil/int128.h /^ uint64 lo;$/;" m struct:uint128_pod +lo src/olap/olap_common.h /^ int64_t lo = 0;$/;" m struct:doris::RowsetId +lo src/udf/udf.h /^ int64_t lo;$/;" m struct:doris_udf::FunctionContext::UniqueId +lo src/util/uid_util.h /^ int64_t lo = 0;$/;" m struct:doris::UniqueId +lo_ src/gutil/int128.h /^ uint64 lo_;$/;" m class:uint128 +load src/common/atomic.h /^ ALWAYS_INLINE T load() const {$/;" f class:doris::AtomicInt +load src/common/atomic.h /^ inline T* load() const { return reinterpret_cast(_ptr.load()); }$/;" f class:doris::AtomicPtr +load src/common/configbase.cpp /^bool Properties::load(const char* filename) {$/;" f class:doris::config::Properties +load src/olap/data_dir.cpp /^OLAPStatus DataDir::load() {$/;" f class:doris::DataDir +load src/olap/rowset/rowset.cpp /^OLAPStatus Rowset::load(bool use_cache) {$/;" f class:doris::Rowset +load src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::load(bool use_cache) {$/;" f class:doris::SegmentGroup +load src/olap/rowset/segment_v2/bitmap_index_reader.cpp /^Status BitmapIndexReader::load(bool use_page_cache, bool kept_in_memory) {$/;" f class:doris::segment_v2::BitmapIndexReader +load src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp /^Status BloomFilterIndexReader::load(bool use_page_cache, bool kept_in_memory) {$/;" f class:doris::segment_v2::BloomFilterIndexReader +load src/olap/rowset/segment_v2/indexed_column_reader.cpp /^Status IndexedColumnReader::load(bool use_page_cache, bool kept_in_memory) {$/;" f class:doris::segment_v2::IndexedColumnReader +load src/olap/rowset/segment_v2/ordinal_page_index.cpp /^Status OrdinalIndexReader::load(bool use_page_cache, bool kept_in_memory) {$/;" f class:doris::segment_v2::OrdinalIndexReader +load src/olap/rowset/segment_v2/zone_map_index.cpp /^Status ZoneMapIndexReader::load(bool use_page_cache, bool kept_in_memory) {$/;" f class:doris::segment_v2::ZoneMapIndexReader +load_bf_columns src/olap/rowset/rowset_reader_context.h /^ const std::set* load_bf_columns = nullptr;$/;" m struct:doris::RowsetReaderContext +load_channel_mgr src/runtime/exec_env.h /^ LoadChannelMgr* load_channel_mgr() { return _load_channel_mgr; }$/;" f class:doris::ExecEnv +load_check_req src/http/action/mini_load.cpp /^ TLoadCheckRequest load_check_req;$/;" m struct:doris::MiniLoadAsyncCtx file: +load_cost_nanos src/runtime/stream_load/stream_load_context.h /^ int64_t load_cost_nanos = 0;$/;" m class:doris::StreamLoadContext +load_data src/olap/tablet_sync_service.h /^ bool load_data;$/;" m struct:doris::FetchRowsetMetaTask +load_data src/olap/tablet_sync_service.h /^ bool load_data;$/;" m struct:doris::FetchTabletMetaTask +load_data_dirs src/olap/storage_engine.cpp /^void StorageEngine::load_data_dirs(const std::vector& data_dirs) {$/;" f class:doris::StorageEngine +load_dir src/runtime/runtime_state.h /^ const std::string& load_dir() const {$/;" f class:doris::RuntimeState +load_doris_home src/util/path_builder.cpp /^void PathBuilder::load_doris_home() {$/;" f class:doris::PathBuilder +load_error_hub_info src/runtime/runtime_state.h /^ const TLoadErrorHubInfo* load_error_hub_info() {$/;" f class:doris::RuntimeState +load_factor src/exec/hash_table.h /^ float load_factor() {$/;" f class:doris::HashTable +load_factor src/exec/new_partitioned_hash_table.h /^ double load_factor() const {$/;" f class:doris::NewPartitionedHashTable +load_factor src/exec/partitioned_hash_table.h /^ double load_factor() const {$/;" f class:doris::PartitionedHashTable +load_handle src/http/action/mini_load.cpp /^ LoadHandle load_handle;$/;" m struct:doris::MiniLoadAsyncCtx file: +load_header src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::load_header($/;" f class:doris::StorageEngine +load_id src/olap/delta_writer.h /^ PUniqueId load_id;$/;" m struct:doris::WriteRequest +load_id src/olap/rowset/rowset.h /^ PUniqueId load_id() const { return rowset_meta()->load_id(); }$/;" f class:doris::Rowset +load_id src/olap/rowset/rowset_meta.h /^ PUniqueId load_id() const {$/;" f class:doris::RowsetMeta +load_id src/olap/rowset/rowset_writer_context.h /^ PUniqueId load_id;$/;" m struct:doris::RowsetWriterContext +load_id src/olap/rowset/segment_group.h /^ inline PUniqueId load_id() const { return _load_id; }$/;" f class:doris::SegmentGroup +load_id src/olap/txn_manager.h /^ PUniqueId load_id;$/;" m struct:doris::TabletTxnInfo +load_id src/runtime/load_channel.h /^ const UniqueId& load_id() const { return _load_id; }$/;" f class:doris::LoadChannel +load_id test/olap/txn_manager_test.cpp /^ PUniqueId load_id;$/;" m class:doris::TxnManagerTest file: +load_index_page src/olap/rowset/segment_v2/indexed_column_reader.cpp /^Status IndexedColumnReader::load_index_page(RandomAccessFile* file,$/;" f class:doris::segment_v2::IndexedColumnReader +load_job_id src/runtime/runtime_state.h /^ const int64_t load_job_id() {$/;" f class:doris::RuntimeState +load_json_meta src/olap/tablet_meta_manager.cpp /^OLAPStatus TabletMetaManager::load_json_meta(DataDir* store, const std::string& meta_path) {$/;" f class:doris::TabletMetaManager +load_json_rowset_meta src/olap/rowset/rowset_meta_manager.cpp /^OLAPStatus RowsetMetaManager::load_json_rowset_meta(OlapMeta* meta, const std::string& rowset_meta_path) {$/;" f class:doris::RowsetMetaManager +load_lock src/runtime/user_function_cache.cpp /^ std::mutex load_lock;$/;" m struct:doris::UserFunctionCacheEntry file: +load_meta src/tools/meta_tool.cpp /^void load_meta(DataDir* data_dir) {$/;" f +load_path_mgr src/runtime/exec_env.h /^ LoadPathMgr* load_path_mgr() { return _load_path_mgr; }$/;" f class:doris::ExecEnv +load_pb src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::load_pb(const char* file, uint32_t seg_id) {$/;" f class:doris::SegmentGroup +load_segment src/olap/olap_index.cpp /^OLAPStatus MemIndex::load_segment(const char* file, size_t *current_num_rows_per_row_block, bool use_cache) {$/;" f class:doris::MemIndex +load_src_type src/runtime/stream_load/stream_load_context.h /^ TLoadSourceType::type load_src_type;$/;" m class:doris::StreamLoadContext +load_stream_mgr src/runtime/exec_env.h /^ LoadStreamMgr* load_stream_mgr() { return _load_stream_mgr; }$/;" f class:doris::ExecEnv +load_symbols src/util/bfd_parser.cpp /^int BfdParser::load_symbols() {$/;" f class:doris::BfdParser +load_tablet_from_dir src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::load_tablet_from_dir(DataDir* store, TTabletId tablet_id,$/;" f class:doris::TabletManager +load_tablet_from_meta src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tablet_id,$/;" f class:doris::TabletManager +load_type src/runtime/stream_load/stream_load_context.h /^ TLoadType::type load_type;$/;" m class:doris::StreamLoadContext +loaded_bytes src/runtime/stream_load/stream_load_context.h /^ int64_t loaded_bytes = 0;$/;" m class:doris::StreamLoadContext +local src/util/minizip/unzip.c 91;" d file: +local_core_alloc_count src/runtime/memory/chunk_allocator.cpp /^static IntCounter local_core_alloc_count;$/;" m namespace:doris file: +local_time src/runtime/datetime_value.cpp /^DateTimeValue DateTimeValue::local_time() {$/;" f class:doris::DateTimeValue +locate src/exprs/string_functions.cpp /^IntVal StringFunctions::locate($/;" f class:doris::StringFunctions +locate_pos src/exprs/string_functions.cpp /^IntVal StringFunctions::locate_pos($/;" f class:doris::StringFunctions +lock src/runtime/disk_io_mgr_internal.h /^ boost::mutex lock;$/;" m struct:doris::DiskIoMgr::DiskQueue +lock src/util/fake_lock.h /^ void lock() {}$/;" f class:doris::FakeLock +lock src/util/mutex.cpp /^OLAPStatus Mutex::lock() {$/;" f class:doris::Mutex +lock src/util/mutex.h /^ void lock() {$/;" f class:doris::MutexLock +lock src/util/runtime_profile.h /^ boost::mutex lock;$/;" m struct:doris::RuntimeProfile::PeriodicCounterUpdateState +lock src/util/spinlock.h /^ void lock() {$/;" f class:doris::SpinLock +lock_ src/exec/exec_node.h /^ std::mutex lock_;$/;" m class:doris::ExecNode::RowBatchQueue +lock_ src/runtime/bufferpool/buffer_allocator.cc /^ SpinLock lock_;$/;" m class:doris::BufferPool::FreeBufferArena file: +lock_ src/runtime/bufferpool/buffer_pool_internal.h /^ boost::mutex lock_;$/;" m class:doris::BufferPool::Client +lock_ src/runtime/bufferpool/reservation_tracker.h /^ SpinLock lock_;$/;" m class:doris::ReservationTracker +lock_ src/runtime/initial_reservations.h /^ SpinLock lock_;$/;" m class:doris::InitialReservations +lock_ src/util/countdown_latch.h /^ mutable Mutex lock_;$/;" m class:doris::CountDownLatch +lock_ src/util/internal_queue.h /^ mutable LockType lock_;$/;" m class:doris::InternalQueueBase +lock_ src/util/streaming_sampler.h /^ mutable SpinLock lock_;$/;" m class:doris::StreamingSampler +lock_sibling_subtree_first src/runtime/bufferpool/reservation_tracker.h /^ static bool lock_sibling_subtree_first($/;" f class:doris::ReservationTracker +log src/exprs/math_functions.cpp /^DoubleVal MathFunctions::log($/;" f class:doris::MathFunctions +log2 src/exprs/math_functions.cpp /^DoubleVal MathFunctions::log2($/;" f class:doris::MathFunctions +log2 src/util/bit_util.h /^ static inline int log2(uint64_t x) {$/;" f class:doris::BitUtil +log_10 src/exprs/math_functions.cpp /^const double log_10[] = {$/;" m namespace:doris file: +log_10_int src/runtime/datetime_value.cpp /^const uint64_t log_10_int[] = {$/;" m namespace:doris file: +log_error src/runtime/runtime_state.cpp /^bool RuntimeState::log_error(const std::string& error) {$/;" f class:doris::RuntimeState +log_error src/runtime/runtime_state.cpp /^void RuntimeState::log_error(const Status& status) {$/;" f class:doris::RuntimeState +log_error src/udf/udf.cpp /^ bool log_error(const std::string& error) {$/;" f class:doris::RuntimeState +log_has_space src/runtime/runtime_state.h /^ bool log_has_space() {$/;" f class:doris::RuntimeState +log_max_buffer_len_ src/runtime/bufferpool/buffer_allocator.h /^ const int log_max_buffer_len_;$/;" m struct:doris::BufferPool::BufferAllocator +log_min_buffer_len_ src/runtime/bufferpool/buffer_allocator.h /^ const int log_min_buffer_len_;$/;" m struct:doris::BufferPool::BufferAllocator +log_udf test/udf/udf_test.cpp /^StringVal log_udf(FunctionContext* context, const StringVal& arg1) {$/;" f namespace:doris_udf +logging_initialized src/common/logconfig.cpp /^static bool logging_initialized = false;$/;" m namespace:doris file: +logging_mutex src/common/logconfig.cpp /^static std::mutex logging_mutex;$/;" m namespace:doris file: +logs_handler src/http/default_path_handlers.cpp /^void logs_handler(const WebPageHandler::ArgumentMap& args, std::stringstream* output) {$/;" f namespace:doris +lookup src/olap/lru_cache.cpp /^Cache::Handle* LRUCache::lookup(const CacheKey& key, uint32_t hash) {$/;" f class:doris::LRUCache +lookup src/olap/lru_cache.cpp /^Cache::Handle* ShardedLRUCache::lookup(const CacheKey& key) {$/;" f class:doris::ShardedLRUCache +lookup src/olap/lru_cache.cpp /^LRUHandle* HandleTable::lookup(const CacheKey& key, uint32_t hash) {$/;" f class:doris::HandleTable +lookup src/olap/page_cache.cpp /^bool StoragePageCache::lookup(const CacheKey& key, PageCacheHandle* handle) {$/;" f class:doris::StoragePageCache +lookup src/util/file_cache.cpp /^bool FileCache::lookup(const std::string& file_name, OpenedFileHandle* file_handle) {$/;" f class:doris::FileCache +loose_enum_cast src/gutil/casts.h /^inline Enum loose_enum_cast(int e_val) {$/;" f +loose_enum_test src/gutil/casts.h /^inline bool loose_enum_test(int e_val) {$/;" f +loose_enum_test_cast src/gutil/casts.h /^inline bool loose_enum_test_cast(int e_val, Enum* e_var) {$/;" f +lowBytes src/util/bitmap_value.h /^ static uint32_t lowBytes(const uint64_t in) { return uint32_t(in); }$/;" f class:doris::detail::Roaring64Map +low_bits src/runtime/multi_precision.h /^inline uint64_t low_bits(__int128 x) {$/;" f namespace:doris +low_water_mark src/runtime/bufferpool/buffer_allocator.cc /^ int low_water_mark;$/;" m struct:doris::BufferPool::FreeBufferArena::PerSizeLists file: +lower src/exprs/string_functions.cpp /^StringVal StringFunctions::lower(FunctionContext* context, const StringVal& str) {$/;" f class:doris::StringFunctions +lower src/exprs/udf_builtins.cpp /^StringVal UdfBuiltins::lower(FunctionContext* context, const StringVal& v) {$/;" f class:doris::UdfBuiltins +lower_bound src/olap/rowset/segment_v2/segment.h /^ ShortKeyIndexIterator lower_bound(const Slice& key) const {$/;" f class:doris::segment_v2::Segment +lower_bound src/olap/short_key_index.h /^ ShortKeyIndexIterator lower_bound(const Slice& key) const {$/;" f class:doris::ShortKeyIndexDecoder +lower_bound_keys src/olap/rowset/rowset_reader_context.h /^ const std::vector* lower_bound_keys = nullptr;$/;" m struct:doris::RowsetReaderContext +lower_key src/olap/iterators.h /^ const RowCursor* lower_key;$/;" m struct:doris::StorageReadOptions::KeyRange +lowest_limit src/runtime/mem_tracker.h /^ int64_t lowest_limit() const {$/;" f class:doris::MemTracker +lpad src/exprs/string_functions.cpp /^StringVal StringFunctions::lpad($/;" f class:doris::StringFunctions +lseek src/gutil/linux_syscall_support.h /^ LSS_INLINE off_t LSS_NAME(lseek)(int f, off_t o, int w) {$/;" f +lt src/runtime/raw_value.h /^inline bool RawValue::lt(const void* v1, const void* v2, const TypeDescriptor& type) {$/;" f class:doris::RawValue +lt src/runtime/string_value.h /^ bool lt(const StringValue& other) const {$/;" f struct:doris::StringValue +ltrim src/exprs/string_functions.cpp /^StringVal StringFunctions::ltrim(FunctionContext* context, const StringVal& str) {$/;" f class:doris::StringFunctions +lz4_compress src/olap/compress.cpp /^OLAPStatus lz4_compress(StorageByteBuffer* in, StorageByteBuffer* out, bool* smaller) {$/;" f namespace:doris +lz4_decompress src/olap/compress.cpp /^OLAPStatus lz4_decompress(StorageByteBuffer* in, StorageByteBuffer* out) {$/;" f namespace:doris +lzo_compress src/olap/compress.cpp /^OLAPStatus lzo_compress(StorageByteBuffer* in, StorageByteBuffer* out, bool* smaller) {$/;" f namespace:doris +lzo_decompress src/olap/compress.cpp /^OLAPStatus lzo_decompress(StorageByteBuffer* in, StorageByteBuffer* out) {$/;" f namespace:doris +m2 src/exprs/aggregate_functions.cpp /^ double m2;$/;" m struct:doris::KnuthVarianceState file: +m_ src/gutil/charmap.h /^ uint32 m_[8];$/;" m class:Charmap +magic_number src/olap/file_helper.h /^ uint64_t magic_number;$/;" m struct:doris::_FixedFileHeaderV2 +main build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^int main(argc, argv) int argc; char *argv[];$/;" f +main build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^void main() {}$/;" f +main build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^int main(int argc, char* argv[])$/;" f +main build/CMakeFiles/feature_tests.c /^int main(int argc, char** argv) { (void)argv; return features[argc]; }$/;" f +main build/CMakeFiles/feature_tests.cxx /^int main(int argc, char** argv) { (void)argv; return features[argc]; }$/;" f +main src/service/doris_main.cpp /^int main(int argc, char** argv) {$/;" f +main src/tools/meta_tool.cpp /^int main(int argc, char** argv) {$/;" f +main test/agent/agent_server_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/agent/cgroups_mgr_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/agent/heartbeat_server_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/agent/utils_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/common/resource_tls_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/common/status_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/env/env_posix_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/exec/broker_reader_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/broker_scan_node_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/broker_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/csv_scan_bench_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/csv_scan_node_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/csv_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/es_http_scan_node_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/es_predicate_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/es_query_builder_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/exec/es_scan_node_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/es_scan_reader_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/exec/hash_table_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/mysql_scan_node_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/mysql_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/new_olap_scan_node_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/olap_common_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/olap_scan_node_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/olap_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/orc_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/parquet_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/partitioned_hash_table_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/plain_text_line_reader_bzip_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/plain_text_line_reader_gzip_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/plain_text_line_reader_lz4frame_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/plain_text_line_reader_lzop_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/plain_text_line_reader_uncompressed_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scan_node_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_authors_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_charsets_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_collations_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_columns_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_engines_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_tables_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner/schema_variables_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/schema_scanner_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/set_executor_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exec/tablet_info_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/exec/tablet_sink_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/exprs/bitmap_function_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exprs/hll_function_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exprs/hybird_set_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exprs/in_predicate_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exprs/json_function_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exprs/percentile_approx_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exprs/string_functions_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/exprs/timestamp_functions_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/geo/geo_functions_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/geo/geo_types_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/geo/wkt_parse_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/http/http_client_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/http/http_utils_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/http/message_body_sink_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/http/metrics_action_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/http/stream_load_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/olap/aggregate_func_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/bit_field_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/bloom_filter_index_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/bloom_filter_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/byte_buffer_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/column_reader_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/comparison_predicate_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/decimal12_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/delete_handler_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/delta_writer_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/file_helper_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/file_utils_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/fs/file_block_manager_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/generic_iterators_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/hll_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/in_list_predicate_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/key_coder_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/lru_cache_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/memtable_flush_executor_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/null_predicate_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/olap_meta_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/olap_reader_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/olap_snapshot_converter_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/options_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/page_cache_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/row_block_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/row_block_v2_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/row_cursor_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/alpha_rowset_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/rowset/beta_rowset_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/rowset/rowset_converter_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/rowset/rowset_meta_manager_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/rowset/rowset_meta_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/rowset/segment_v2/binary_dict_page_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/rowset/segment_v2/binary_plain_page_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/binary_prefix_page_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/rowset/segment_v2/bitmap_index_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/block_bloom_filter_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/bloom_filter_page_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/encoding_info_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/ordinal_page_index_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/plain_page_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/rle_page_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/row_ranges_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/segment_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/segment_v2/zone_map_index_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/rowset/unique_rowset_id_generator_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/run_length_byte_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/run_length_integer_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/schema_change_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/selection_vector_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/serialize_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/short_key_index_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/skiplist_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/storage_types_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/stream_index_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/olap/tablet_meta_manager_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/tablet_mgr_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/txn_manager_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/olap/vectorized_olap_reader_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/buffer_control_block_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/buffered_block_mgr2_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/buffered_tuple_stream2_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/buffered_tuple_stream_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/runtime/data_spliter_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/data_stream_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/datetime_value_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/decimal_value_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/decimalv2_value_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/disk_io_mgr_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/dpp_sink_internal_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/dpp_sink_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/dpp_writer_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/etl_job_mgr_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/export_task_mgr_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/external_scan_context_mgr_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/fragment_mgr_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/free_list_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/heartbeat_flags_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/runtime/kafka_consumer_pipe_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/runtime/large_int_value_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/load_channel_mgr_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/runtime/mem_limit_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/mem_pool_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/memory/chunk_allocator_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/memory/system_allocator_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/memory_scratch_sink_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/mysql_table_writer_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/qsorter_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/raw_value_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/result_buffer_mgr_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/result_queue_mgr_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/result_sink_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/result_writer_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/routine_load_task_executor_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/runtime/small_file_mgr_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/runtime/snapshot_loader_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/sorter_test.cpp /^int main(int argc, char **argv) {$/;" f +main test/runtime/stream_load_pipe_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/runtime/string_buffer_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/string_value_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/thread_resource_mgr_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/tmp_file_mgr_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/runtime/user_function_cache_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/udf/uda_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/udf/udf_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/aes_util_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/arrow/arrow_row_batch_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/arrow/arrow_row_block_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/arrow/arrow_work_flow_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/bit_stream_utils_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/bit_util_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/bitmap_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/bitmap_value_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/block_compression_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/brpc_stub_cache_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/byte_buffer_test2.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/cgroup_util_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/cidr_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/coding_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/core_local_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/countdown_latch_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/counter_cond_variable_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/crc32c_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/decompress_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/doris_metrics_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/faststring_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/file_cache_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/file_manager_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/filesystem_util_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/frame_of_reference_coding_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/internal_queue_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/json_util_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/lru_cache_util_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/md5_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/monotime_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/new_metrics_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/parse_util_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/path_trie_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/path_util_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/perf_counters_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/radix_sort_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/rle_encoding_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/runtime_profile_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/scoped_cleanup_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/string_parser_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/string_util_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/system_metrics_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/tdigest_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/thread_pool_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/thread_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/threadpool_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/types_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/uid_util_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/url_coding_test.cpp /^int main(int argc, char** argv) {$/;" f +main test/util/utf8_check_test.cpp /^int main(int argc, char* argv[]) {$/;" f +main test/util/zip_util_test.cpp /^int main(int argc, char** argv) {$/;" f +make_client src/runtime/client_cache.h /^ ThriftClientImpl* make_client(const TNetworkAddress& hostport, void** client_key) {$/;" f class:doris::ClientCache +make_gscoped_ptr src/gutil/gscoped_ptr.h /^gscoped_ptr make_gscoped_ptr(T* ptr) {$/;" f +make_key test/olap/skiplist_test.cpp /^ static Key make_key(uint64_t k, uint64_t g) {$/;" f class:doris::ConcurrentTest file: +make_network_address src/util/network_util.cpp /^TNetworkAddress make_network_address(const std::string& hostname, int port) {$/;" f namespace:doris +make_packed_time src/runtime/datetime_value.h /^ int64_t make_packed_time(int64_t time, int64_t second_part) const {$/;" f class:doris::DateTimeValue +make_scoped_refptr src/gutil/ref_counted.h /^scoped_refptr make_scoped_refptr(T* t) {$/;" f +make_slot_descriptor src/testutil/desc_tbl_builder.cc /^static TSlotDescriptor make_slot_descriptor($/;" f namespace:doris +make_snapshot src/agent/agent_server.cpp /^void AgentServer::make_snapshot(TAgentResult& t_agent_result,$/;" f class:doris::AgentServer +make_snapshot src/http/action/snapshot_action.cpp /^int64_t SnapshotAction::make_snapshot(int64_t tablet_id, int32_t schema_hash,$/;" f class:doris::SnapshotAction +make_snapshot src/olap/snapshot_manager.cpp /^OLAPStatus SnapshotManager::make_snapshot($/;" f class:doris::SnapshotManager +make_snapshot src/service/backend_service.h /^ virtual void make_snapshot($/;" f class:doris::BackendService +make_snapshot test/runtime/data_stream_test.cpp /^ virtual void make_snapshot($/;" f class:doris::DorisTestBackend +make_tuple_descriptor src/testutil/desc_tbl_builder.cc /^static TTupleDescriptor make_tuple_descriptor(int id, int byte_size, int num_null_bytes) {$/;" f namespace:doris +make_visible src/olap/rowset/rowset.cpp /^void Rowset::make_visible(Version version, VersionHash version_hash) {$/;" f class:doris::Rowset +make_visible_extra src/olap/rowset/alpha_rowset.cpp /^void AlphaRowset::make_visible_extra(Version version, VersionHash version_hash) {$/;" f class:doris::AlphaRowset +make_visible_extra src/olap/rowset/rowset.h /^ virtual void make_visible_extra(Version version, VersionHash version_hash) {}$/;" f class:doris::Rowset +mangle_prepare_or_close_function src/util/symbols_util.cpp /^std::string SymbolsUtil::mangle_prepare_or_close_function(const std::string& fn_name) {$/;" f class:doris::SymbolsUtil +mangle_user_function src/util/symbols_util.cpp /^std::string SymbolsUtil::mangle_user_function(const std::string& fn_name,$/;" f class:doris::SymbolsUtil +map_ src/util/bitmap.h /^ const uint8_t *map_;$/;" m class:doris::BitmapIterator +map_end src/util/bitmap_value.h /^ std::map::const_iterator map_end;$/;" m class:doris::detail::final +map_iter src/util/bitmap_value.h /^ std::map::const_iterator map_iter;$/;" m class:doris::detail::final +map_lock src/runtime/user_function_cache.cpp /^ SpinLock map_lock;$/;" m struct:doris::UserFunctionCacheEntry file: +mark_event src/util/runtime_profile.h /^ void mark_event(const std::string& label) {$/;" f class:doris::RuntimeProfile::EventSequence +mark_flush_resources src/runtime/row_batch.h /^ void mark_flush_resources() {$/;" f class:doris::RowBatch +mark_need_to_return src/runtime/row_batch.h /^ void mark_need_to_return() {$/;" f class:doris::RowBatch +mark_needs_deep_copy src/runtime/row_batch.h /^ void mark_needs_deep_copy() {$/;" f class:doris::RowBatch +master_addresses_ src/runtime/descriptors.h /^ std::vector master_addresses_;$/;" m class:doris::KuduTableDescriptor +master_info src/runtime/exec_env.h /^ TMasterInfo* master_info() { return _master_info; }$/;" f class:doris::ExecEnv +match src/exprs/es_functions.cpp /^BooleanVal ESFunctions::match(FunctionContext* ctx, const StringVal& col, $/;" f class:doris::ESFunctions +match src/runtime/routine_load/data_consumer.cpp /^bool KafkaDataConsumer::match(StreamLoadContext* ctx) {$/;" f class:doris::KafkaDataConsumer +match_condition src/olap/rowset/segment_v2/column_reader.cpp /^bool ColumnReader::match_condition(CondColumn* cond) const {$/;" f class:doris::segment_v2::ColumnReader +matched src/exec/hash_table.h /^ bool matched() {$/;" f class:doris::HashTable::Iterator +matched src/exec/hash_table.h /^ bool matched;$/;" m struct:doris::HashTable::Node +matched src/exec/new_partitioned_hash_table.h /^ bool matched;$/;" m struct:doris::NewPartitionedHashTable::Bucket +matched src/exec/new_partitioned_hash_table.h /^ bool matched;$/;" m struct:doris::NewPartitionedHashTable::DuplicateNode +matched src/exec/partitioned_hash_table.h /^ bool matched;$/;" m struct:doris::PartitionedHashTable::Bucket +matched src/exec/partitioned_hash_table.h /^ bool matched;$/;" m struct:doris::PartitionedHashTable::DuplicateNode +materialize_batch src/exec/union_node_ir.cpp /^void UnionNode::materialize_batch(RowBatch* dst_batch, uint8_t** tuple_buf) {$/;" f class:doris::UnionNode +materialize_exprs src/exec/union_node_ir.cpp /^void IR_ALWAYS_INLINE UnionNode::materialize_exprs(const std::vector& exprs,$/;" f class:doris::UnionNode +materialize_exprs src/runtime/tuple.cpp /^void Tuple::materialize_exprs($/;" f class:doris::Tuple +materialize_row src/exec/es_scan_node.cpp /^Status EsScanNode::materialize_row(MemPool* tuple_pool, Tuple* tuple,$/;" f class:doris::EsScanNode +materialize_tuple_timer src/exec/scan_node.h /^ RuntimeProfile::Counter* materialize_tuple_timer() const {$/;" f class:doris::ScanNode +max src/exprs/aggregate_functions.cpp /^void AggregateFunctions::max(FunctionContext* ctx, const StringVal& src, StringVal* dst) {$/;" f class:doris::AggregateFunctions +max src/exprs/aggregate_functions.cpp /^void AggregateFunctions::max(FunctionContext*, const DecimalV2Val& src, DecimalV2Val* dst) {$/;" f class:doris::AggregateFunctions +max src/exprs/aggregate_functions.cpp /^void AggregateFunctions::max(FunctionContext*, const DecimalVal& src, DecimalVal* dst) {$/;" f class:doris::AggregateFunctions +max src/exprs/aggregate_functions.cpp /^void AggregateFunctions::max(FunctionContext*, const LargeIntVal& src, LargeIntVal* dst) {$/;" f class:doris::AggregateFunctions +max src/exprs/aggregate_functions.cpp /^void AggregateFunctions::max(FunctionContext*, const T& src, T* dst) {$/;" f class:doris::AggregateFunctions +max src/exprs/aggregate_functions.cpp /^void AggregateFunctions::max(FunctionContext*,$/;" f class:doris::AggregateFunctions +maxProcessed src/util/tdigest.h /^ Index maxProcessed() const { return _max_processed; }$/;" f class:doris::TDigest +maxUnprocessed src/util/tdigest.h /^ Index maxUnprocessed() const { return _max_unprocessed; }$/;" f class:doris::TDigest +max_align_t src/runtime/mem_pool.cpp /^uint32_t MemPool::k_zero_length_region_ alignas(std::max_align_t) = MEM_POOL_POISON;$/;" m namespace:doris file: +max_ascii_len src/util/string_parser.hpp /^inline int StringParser::StringParseTraits<__int128>::max_ascii_len() {$/;" f class:doris::StringParser::StringParseTraits +max_ascii_len src/util/string_parser.hpp /^inline int StringParser::StringParseTraits::max_ascii_len() {$/;" f class:doris::StringParser::StringParseTraits +max_ascii_len src/util/string_parser.hpp /^inline int StringParser::StringParseTraits::max_ascii_len() {$/;" f class:doris::StringParser::StringParseTraits +max_ascii_len src/util/string_parser.hpp /^inline int StringParser::StringParseTraits::max_ascii_len() {$/;" f class:doris::StringParser::StringParseTraits +max_ascii_len src/util/string_parser.hpp /^inline int StringParser::StringParseTraits::max_ascii_len() {$/;" f class:doris::StringParser::StringParseTraits +max_batch_rows src/runtime/stream_load/stream_load_context.h /^ int64_t max_batch_rows = 100000;$/;" m class:doris::StreamLoadContext +max_batch_rows src/runtime/stream_load/stream_load_context.h /^ int64_t max_batch_rows = 1024;$/;" m class:doris::KafkaLoadInfo +max_batch_size src/runtime/stream_load/stream_load_context.h /^ int64_t max_batch_size = 100 * 1024 * 1024; \/\/ 100MB$/;" m class:doris::KafkaLoadInfo +max_batch_size src/runtime/stream_load/stream_load_context.h /^ int64_t max_batch_size = 100 * 1024 * 1024; \/\/ 100MB$/;" m class:doris::StreamLoadContext +max_block_size src/runtime/buffered_block_mgr.h /^ int64_t max_block_size() const {$/;" f class:doris::BufferedBlockMgr +max_block_size src/runtime/buffered_block_mgr2.h /^ int64_t max_block_size() const { {$/;" f class:doris::BufferedBlockMgr2 +max_buffer_len_ src/runtime/bufferpool/buffer_allocator.h /^ const int64_t max_buffer_len_;$/;" m struct:doris::BufferPool::BufferAllocator +max_bytes_ src/util/bit_stream_utils.h /^ int max_bytes_;$/;" m class:doris::BitReader +max_continuous_version_from_begining src/olap/tablet.cpp /^OLAPStatus Tablet::max_continuous_version_from_begining(Version* version,$/;" f class:doris::Tablet +max_disk_io_util_percent src/util/doris_metrics.cpp /^IntGauge DorisMetrics::max_disk_io_util_percent;$/;" m class:doris::DorisMetrics file: +max_disk_io_util_percent src/util/doris_metrics.h /^ static IntGauge max_disk_io_util_percent;$/;" m class:doris::DorisMetrics +max_enumerator src/gutil/casts.h /^ static const Enum max_enumerator = 0;$/;" m class:enum_limits +max_errors src/runtime/runtime_state.h /^ int max_errors() const {$/;" f class:doris::RuntimeState +max_filter_ratio src/runtime/stream_load/stream_load_context.h /^ double max_filter_ratio = 0.0;$/;" m class:doris::StreamLoadContext +max_height_ src/olap/skiplist.h /^ std::atomic max_height_; \/\/ Height of the entire list$/;" m class:doris::SkipList +max_interval_s src/runtime/stream_load/stream_load_context.h /^ int64_t max_interval_s = 5;$/;" m class:doris::KafkaLoadInfo +max_interval_s src/runtime/stream_load/stream_load_context.h /^ int64_t max_interval_s = 5;$/;" m class:doris::StreamLoadContext +max_io_buffers src/runtime/runtime_state.h /^ int max_io_buffers() const {$/;" f class:doris::RuntimeState +max_network_receive_bytes_rate src/util/doris_metrics.cpp /^IntGauge DorisMetrics::max_network_receive_bytes_rate;$/;" m class:doris::DorisMetrics file: +max_network_receive_bytes_rate src/util/doris_metrics.h /^ static IntGauge max_network_receive_bytes_rate;$/;" m class:doris::DorisMetrics +max_network_send_bytes_rate src/util/doris_metrics.cpp /^IntGauge DorisMetrics::max_network_send_bytes_rate;$/;" m class:doris::DorisMetrics file: +max_network_send_bytes_rate src/util/doris_metrics.h /^ static IntGauge max_network_send_bytes_rate;$/;" m class:doris::DorisMetrics +max_num_buckets_ src/exec/new_partitioned_hash_table.h /^ const int64_t max_num_buckets_;$/;" m class:doris::NewPartitionedHashTable +max_num_cores_ src/util/cpu_info.cpp /^int CpuInfo::max_num_cores_ = 1;$/;" m class:doris::CpuInfo file: +max_num_cores_ src/util/cpu_info.h /^ static int max_num_cores_;$/;" m class:doris::CpuInfo +max_num_file_handles src/util/filesystem_util.cc /^uint64_t FileSystemUtil::max_num_file_handles() {$/;" f class:doris::FileSystemUtil +max_num_numa_nodes_ src/util/cpu_info.cpp /^int CpuInfo::max_num_numa_nodes_;$/;" m class:doris::CpuInfo file: +max_num_numa_nodes_ src/util/cpu_info.h /^ static int max_num_numa_nodes_;$/;" m class:doris::CpuInfo +max_page_len_ src/runtime/buffered_tuple_stream3.h /^ const int64_t max_page_len_;$/;" m class:doris::BufferedTupleStream3 +max_partition_level_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::HighWaterMarkCounter* max_partition_level_;$/;" m class:doris::NewPartitionedAggregationNode +max_read_buffer_size src/runtime/disk_io_mgr.h /^ int max_read_buffer_size() const { return _max_buffer_size; }$/;" f class:doris::DiskIoMgr +max_reservation src/runtime/runtime_state.h /^ int64_t max_reservation() {$/;" f class:doris::RuntimeState +max_rows_per_segment src/olap/rowset/rowset_writer_context.h /^ uint32_t max_rows_per_segment = INT32_MAX;$/;" m struct:doris::RowsetWriterContext +max_scavenge_attempts_ src/runtime/bufferpool/buffer_allocator.h /^ int max_scavenge_attempts_;$/;" m struct:doris::BufferPool::BufferAllocator +max_serialized_size src/olap/hll.cpp /^size_t HyperLogLog::max_serialized_size() const {$/;" f class:doris::HyperLogLog +max_size src/gutil/strings/stringpiece.h /^ int max_size() const { return length_; }$/;" f class:StringPiece +max_tuple_buffer_size src/runtime/row_batch.cpp /^int RowBatch::max_tuple_buffer_size() {$/;" f class:doris::RowBatch +max_value src/olap/rowset/segment_v2/zone_map_index.h /^ char* max_value = nullptr;$/;" m struct:doris::segment_v2::ZoneMap +max_version src/olap/tablet.h /^inline Version Tablet::max_version() const {$/;" f class:doris::Tablet +max_version src/olap/tablet_meta.cpp /^Version TabletMeta::max_version() const {$/;" f class:doris::TabletMeta +maximum src/olap/stream_index_common.h /^ const WrapperField* maximum() const {$/;" f class:doris::ColumnStatistics +maximum src/util/bitmap_value.h /^ uint64_t maximum() const {$/;" f class:doris::detail::Roaring64Map +maxlen src/exec/schema_scanner/schema_charsets_scanner.h /^ int64_t maxlen;$/;" m struct:doris::SchemaCharsetsScanner::CharsetStruct +may_submit_new_tasks src/util/threadpool.h /^ bool may_submit_new_tasks() const {$/;" f class:doris::ThreadPoolToken +md5 src/exprs/encryption_functions.cpp /^StringVal EncryptionFunctions::md5(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::EncryptionFunctions +md5 src/runtime/small_file_mgr.h /^ std::string md5;$/;" m struct:doris::CacheEntry +md5 src/runtime/snapshot_loader.h /^ std::string md5;$/;" m struct:doris::FileStat +md5sum src/exprs/encryption_functions.cpp /^StringVal EncryptionFunctions::md5sum($/;" f class:doris::EncryptionFunctions +md5sum src/util/file_utils.cpp /^Status FileUtils::md5sum(const std::string& file, std::string* md5sum) {$/;" f class:doris::FileUtils +mean src/exprs/aggregate_functions.cpp /^ double mean;$/;" m struct:doris::KnuthVarianceState file: +mem_compare src/util/slice.h /^ static int mem_compare(const void* a, const void* b, size_t n) {$/;" f struct:doris::Slice +mem_consumption src/olap/delta_writer.cpp /^int64_t DeltaWriter::mem_consumption() const {$/;" f class:doris::DeltaWriter +mem_consumption src/runtime/load_channel.h /^ int64_t mem_consumption() const { return _mem_tracker->consumption(); }$/;" f class:doris::LoadChannel +mem_consumption src/runtime/tablets_channel.h /^ int64_t mem_consumption() const { return _mem_tracker->consumption(); }$/;" f class:doris::TabletsChannel +mem_consumption test/runtime/load_channel_mgr_test.cpp /^int64_t DeltaWriter::mem_consumption() const { return 1024L; }$/;" f class:doris::DeltaWriter +mem_equal src/util/slice.h /^ static bool mem_equal(const void* a, const void* b, size_t n) {$/;" f struct:doris::Slice +mem_limit_exceeded src/exec/hash_table.cpp /^void HashTable::mem_limit_exceeded(int64_t allocation_size) {$/;" f class:doris::HashTable +mem_limit_too_low_error src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::mem_limit_too_low_error(Client* client, int node_id) {$/;" f class:doris::BufferedBlockMgr2 +mem_pool src/olap/row_block.h /^ MemPool* mem_pool() const {$/;" f class:doris::RowBlock +mem_pool src/olap/rowset/column_data_writer.cpp /^MemPool* ColumnDataWriter::mem_pool() {$/;" f class:doris::ColumnDataWriter +mem_pool src/runtime/vectorized_row_batch.h /^ MemPool* mem_pool() {$/;" f class:doris::VectorizedRowBatch +mem_pool_ src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr mem_pool_;$/;" m class:doris::NewPartitionedAggregationNode +mem_pool_ src/exec/new_partitioned_hash_table.h /^ MemPool* mem_pool_;$/;" m class:doris::NewPartitionedHashTableCtx +mem_pool_ src/exprs/new_agg_fn_evaluator.h /^ MemPool* mem_pool_ = nullptr;$/;" m class:doris::NewAggFnEvaluator +mem_range src/runtime/bufferpool/buffer_pool.h /^ MemRange mem_range() const { return MemRange(data(), len()); }$/;" f class:doris::BufferPool::BufferHandle +mem_tracker src/exec/exec_node.h /^ MemTracker* mem_tracker() const {$/;" f class:doris::ExecNode +mem_tracker src/runtime/data_stream_recvr.h /^ MemTracker* mem_tracker() const { return _mem_tracker.get(); }$/;" f class:doris::DataStreamRecvr +mem_tracker src/runtime/mem_pool.h /^ MemTracker* mem_tracker() { return mem_tracker_; }$/;" f class:doris::MemPool +mem_tracker_ src/runtime/bufferpool/reservation_tracker.h /^ MemTracker* mem_tracker_ = nullptr;$/;" m class:doris::ReservationTracker +mem_tracker_ src/runtime/mem_pool.h /^ MemTracker* mem_tracker_;$/;" m class:doris::MemPool +mem_trackers src/runtime/runtime_state.h /^ std::vector* mem_trackers() {$/;" f class:doris::RuntimeState +mem_usage_handler src/http/default_path_handlers.cpp /^void mem_usage_handler(MemTracker* mem_tracker, const WebPageHandler::ArgumentMap& args,$/;" f namespace:doris +memcasecmp src/gutil/strings/memutil.cc /^int memcasecmp(const char *s1, const char *s2, size_t len) {$/;" f +memcaseis src/gutil/strings/memutil.h 139;" d +memcasemem src/gutil/strings/memutil.h /^inline const char * memcasemem(const char *phaystack, size_t haylen,$/;" f +memcaseprefix src/gutil/strings/memutil.h 114;" d +memcasestr src/gutil/strings/memutil.h /^inline const char * memcasestr(const char *phaystack, size_t haylen,$/;" f +memcasesuffix src/gutil/strings/memutil.h 128;" d +memcat src/gutil/strings/memutil.h /^inline char *memcat(char *dest, size_t destlen,$/;" f +memchr_sse src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline void* memchr_sse(const void* s, int c, size_t n) {$/;" f namespace:doris +memcmp_sse src/olap/bhp_lib.h /^inline int memcmp_sse(const void* buf1, const void* buf2, unsigned int count) {$/;" f namespace:doris +memcmp_sse32 src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline int memcmp_sse32(const void* buf1, const void* buf2,$/;" f namespace:doris +memcmp_sse64 src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline int memcmp_sse64(const void* buf1, const void* buf2,$/;" f namespace:doris +memcount src/gutil/strings/memutil.h /^inline int memcount(const char* buf, size_t len, char c) {$/;" f +memcpy_inlined src/gutil/strings/fastmem.h /^inline void memcpy_inlined(void *dst, const void *src, size_t size) {$/;" f namespace:strings +memcspn src/gutil/strings/memutil.cc /^size_t memcspn(const char *s, size_t slen, const char *reject) {$/;" f +memdup src/gutil/strings/memutil.cc /^char *memdup(const char *s, size_t slen) {$/;" f +memeq src/gutil/strings/fastmem.h /^inline bool memeq(const void* a_v, const void* b_v, size_t n) {$/;" f namespace:strings +memis src/gutil/strings/memutil.h 135;" d +memmatch src/gutil/strings/memutil.cc /^const char *memmatch(const char *phaystack, size_t haylen,$/;" f +memmem src/gutil/strings/memutil.h /^inline const char * memmem(const char *phaystack, size_t haylen,$/;" f +memory_barrier src/common/atomic.h /^ static ALWAYS_INLINE void memory_barrier() {$/;" f class:doris::AtomicUtil +memory_copy src/util/mem_util.hpp /^inline void memory_copy(void* dst, const void* src, size_t size) {$/;" f namespace:doris +memory_maintenance_thread src/common/daemon.cpp /^void* memory_maintenance_thread(void* dummy) {$/;" f namespace:doris +memory_pool_bytes_total src/util/doris_metrics.cpp /^IntGauge DorisMetrics::memory_pool_bytes_total;$/;" m class:doris::DorisMetrics file: +memory_pool_bytes_total src/util/doris_metrics.h /^ static IntGauge memory_pool_bytes_total;$/;" m class:doris::DorisMetrics +memory_usage src/olap/memtable.h /^ size_t memory_usage() const { return _mem_tracker->consumption(); }$/;" f class:doris::MemTable +memory_used_counter src/exec/exec_node.h /^ RuntimeProfile::Counter* memory_used_counter() const {$/;" f class:doris::ExecNode +mempbrk src/gutil/strings/memutil.cc /^char *mempbrk(const char *s, size_t slen, const char *accept) {$/;" f +memprefix src/gutil/strings/memutil.h 108;" d +memrchr src/gutil/strings/memutil.cc /^char *memrchr(const char *s, int c, size_t slen) {$/;" f +memrm src/gutil/strings/strip.cc /^int memrm(char* str, int strlen, char c) {$/;" f +memspn src/gutil/strings/memutil.cc /^size_t memspn(const char *s, size_t slen, const char *accept) {$/;" f +memstr src/gutil/strings/memutil.h /^inline const char * memstr(const char *phaystack, size_t haylen,$/;" f +memsuffix src/gutil/strings/memutil.h 121;" d +memtable_flush_duration_us src/util/doris_metrics.cpp /^IntCounter DorisMetrics::memtable_flush_duration_us;$/;" m class:doris::DorisMetrics file: +memtable_flush_duration_us src/util/doris_metrics.h /^ static IntCounter memtable_flush_duration_us;$/;" m class:doris::DorisMetrics +memtable_flush_executor src/olap/storage_engine.h /^ MemTableFlushExecutor* memtable_flush_executor() { return _memtable_flush_executor.get(); }$/;" f class:doris::StorageEngine +memtable_flush_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::memtable_flush_total;$/;" m class:doris::DorisMetrics file: +memtable_flush_total src/util/doris_metrics.h /^ static IntCounter memtable_flush_total;$/;" m class:doris::DorisMetrics +merge src/exprs/agg_fn_evaluator.cpp /^void AggFnEvaluator::merge($/;" f class:doris::AggFnEvaluator +merge src/exprs/agg_fn_evaluator.cpp /^void AggFnEvaluator::merge(FunctionContext* agg_fn_ctx, Tuple* src, Tuple* dst) {$/;" f class:doris::AggFnEvaluator +merge src/exprs/aggregate_functions.cpp /^ void merge(MultiDistinctCountDateState& state) {$/;" f class:doris::MultiDistinctCountDateState +merge src/exprs/aggregate_functions.cpp /^ void merge(MultiDistinctDecimalState& state) {$/;" f class:doris::MultiDistinctDecimalState +merge src/exprs/aggregate_functions.cpp /^ void merge(MultiDistinctDecimalV2State& state) {$/;" f class:doris::MultiDistinctDecimalV2State +merge src/exprs/aggregate_functions.cpp /^ void merge(MultiDistinctNumericState& state) {$/;" f class:doris::MultiDistinctNumericState +merge src/exprs/aggregate_functions.cpp /^ void merge(MultiDistinctStringCountState& state) {$/;" f class:doris::MultiDistinctStringCountState +merge src/exprs/bitmap_function.cpp /^ void merge(const BitmapIntersect& other) {$/;" f struct:doris::BitmapIntersect +merge src/olap/bloom_filter.hpp /^ bool merge(const BitSet& set) {$/;" f class:doris::BitSet +merge src/olap/bloom_filter.hpp /^ bool merge(const BloomFilter& that) {$/;" f class:doris::BloomFilter +merge src/olap/hll.cpp /^void HyperLogLog::merge(const HyperLogLog& other) {$/;" f class:doris::HyperLogLog +merge src/olap/schema_change.cpp /^bool RowBlockMerger::merge($/;" f class:doris::RowBlockMerger +merge src/olap/stream_index_common.cpp /^void ColumnStatistics::merge(ColumnStatistics* other) {$/;" f class:doris::ColumnStatistics +merge src/runtime/query_statistics.cpp /^void QueryStatistics::merge(QueryStatisticsRecvr* recvr) {$/;" f class:doris::QueryStatistics +merge src/runtime/query_statistics.h /^ void merge(QueryStatistics* statistics) {$/;" f class:doris::QueryStatisticsRecvr +merge src/runtime/query_statistics.h /^ void merge(const QueryStatistics& other) {$/;" f class:doris::QueryStatistics +merge src/util/runtime_profile.cpp /^void RuntimeProfile::merge(RuntimeProfile* other) {$/;" f class:doris::RuntimeProfile +merge src/util/tdigest.h /^ inline void merge(const TDigest* other) {$/;" f class:doris::TDigest +mergeProcessed src/util/tdigest.h /^ void mergeProcessed(const std::vector& tdigests) {$/;" f class:doris::TDigest +mergeUnprocessed src/util/tdigest.h /^ void mergeUnprocessed(const std::vector& tdigests) {$/;" f class:doris::TDigest +merge_fn_ src/exprs/agg_fn.h /^ void* merge_fn_ = nullptr;$/;" m class:doris::AggFn +merge_intermediate_runs src/runtime/spill_sorter.cc /^Status SpillSorter::merge_intermediate_runs() {$/;" f class:doris::SpillSorter +merge_or_update_fn src/exprs/agg_fn.h /^ void* merge_or_update_fn() const { return is_merge_ ? merge_fn_ : update_fn_; }$/;" f class:doris::AggFn +merge_pb src/runtime/query_statistics.h /^ void merge_pb(const PQueryStatistics& statistics) {$/;" f class:doris::QueryStatistics +merge_rowsets src/olap/merger.cpp /^OLAPStatus Merger::merge_rowsets(TabletSharedPtr tablet,$/;" f class:doris::Merger +merged_rows src/olap/merger.h /^ int64_t merged_rows = 0;$/;" m struct:doris::Merger::Statistics +merged_rows src/olap/reader.h /^ uint64_t merged_rows() const {$/;" f class:doris::Reader +merged_rows src/olap/schema_change.h /^ uint64_t merged_rows() const {$/;" f class:doris::SchemaChange +message src/common/status.cpp /^Slice Status::message() const {$/;" f class:doris::Status +message src/olap/file_helper.h /^ const MessageType& message() const {$/;" f class:doris::FileHeader +meta src/olap/rowset/segment_v2/column_writer.h /^ ColumnMetaPB* meta;$/;" m struct:doris::segment_v2::ColumnWriterOptions +meta_data src/runtime/disk_io_mgr.h /^ void* meta_data() const { return _meta_data; }$/;" f class:doris::DiskIoMgr::ScanRange +meta_path test/olap/tablet_meta_manager_test.cpp /^const std::string meta_path = ".\/be\/test\/olap\/test_data\/header.txt";$/;" m namespace:doris file: +meta_read_request_duration_us src/util/doris_metrics.cpp /^IntCounter DorisMetrics::meta_read_request_duration_us;$/;" m class:doris::DorisMetrics file: +meta_read_request_duration_us src/util/doris_metrics.h /^ static IntCounter meta_read_request_duration_us;$/;" m class:doris::DorisMetrics +meta_read_request_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::meta_read_request_total;$/;" m class:doris::DorisMetrics file: +meta_read_request_total src/util/doris_metrics.h /^ static IntCounter meta_read_request_total;$/;" m class:doris::DorisMetrics +meta_write_request_duration_us src/util/doris_metrics.cpp /^IntCounter DorisMetrics::meta_write_request_duration_us;$/;" m class:doris::DorisMetrics file: +meta_write_request_duration_us src/util/doris_metrics.h /^ static IntCounter meta_write_request_duration_us;$/;" m class:doris::DorisMetrics +meta_write_request_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::meta_write_request_total;$/;" m class:doris::DorisMetrics file: +meta_write_request_total src/util/doris_metrics.h /^ static IntCounter meta_write_request_total;$/;" m class:doris::DorisMetrics +metadata src/util/runtime_profile.h /^ int64_t metadata() const {$/;" f class:doris::RuntimeProfile +method src/exec/decompressor.h /^ uint8_t method;$/;" m struct:doris::LzopDecompressor::HeaderInfo +method src/http/http_request.h /^ HttpMethod method() const {$/;" f class:doris::HttpRequest +metrics src/runtime/exec_env.h /^ MetricRegistry* metrics() const { return _metrics; }$/;" f class:doris::ExecEnv +metrics src/runtime/test_env.h /^ MetricRegistry* metrics() {$/;" f class:doris::TestEnv +metrics src/util/doris_metrics.h /^ static MetricRegistry* metrics() { return _s_doris_metrics._metrics; }$/;" f class:doris::DorisMetrics +metrics src/util/doris_metrics.h /^ std::unordered_map metrics;$/;" m class:doris::IntGaugeMetricsMap +metrics src/util/metrics.h /^ const std::map& metrics() const {$/;" f class:doris::MetricCollector +metrics src/util/system_metrics.cpp /^ IntLockCounter metrics[k_num_metrics];$/;" m struct:doris::CpuMetrics file: +mi src/olap/olap_common.h /^ int64_t mi = 0;$/;" m struct:doris::RowsetId +micros_add src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::micros_add($/;" f class:doris::TimestampFunctions +micros_sub src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::micros_sub($/;" f class:doris::TimestampFunctions +microsecond src/runtime/datetime_value.h /^ int32_t microsecond;$/;" m struct:doris::TimeInterval +min src/exprs/aggregate_functions.cpp /^void AggregateFunctions::min(FunctionContext* ctx, const StringVal& src, StringVal* dst) {$/;" f class:doris::AggregateFunctions +min src/exprs/aggregate_functions.cpp /^void AggregateFunctions::min(FunctionContext*, const DecimalV2Val& src, DecimalV2Val* dst) {$/;" f class:doris::AggregateFunctions +min src/exprs/aggregate_functions.cpp /^void AggregateFunctions::min(FunctionContext*, const DecimalVal& src, DecimalVal* dst) {$/;" f class:doris::AggregateFunctions +min src/exprs/aggregate_functions.cpp /^void AggregateFunctions::min(FunctionContext*, const LargeIntVal& src, LargeIntVal* dst) {$/;" f class:doris::AggregateFunctions +min src/exprs/aggregate_functions.cpp /^void AggregateFunctions::min(FunctionContext*, const T& src, T* dst) {$/;" f class:doris::AggregateFunctions +min src/exprs/aggregate_functions.cpp /^void AggregateFunctions::min(FunctionContext*,$/;" f class:doris::AggregateFunctions +min src/gutil/strings/util.cc 36;" d file: +min src/runtime/datetime_value.cpp /^static int min(int a, int b) {$/;" f namespace:doris +min3 test/udf/udf_test.cpp /^FloatVal min3(FunctionContext* context, const FloatVal& f1,$/;" f namespace:doris_udf +min_buckets src/gutil/hash/hash.h /^ static const size_t min_buckets = 8; \/\/ 4 and 8 are defaults.$/;" m struct:GoodFastHash +min_buckets src/gutil/hash/hash.h /^ static const size_t min_buckets = 8; \/\/ 4 and 8 are defaults.$/;" m struct:__gnu_cxx::hash +min_buckets src/gutil/strings/stringpiece.h /^ static const size_t min_buckets = 8; \/\/ 4 and 8 are defaults.$/;" m struct:GoodFastHash +min_buffer_len src/runtime/bufferpool/buffer_pool.h /^ int64_t min_buffer_len() const { return min_buffer_len_; }$/;" f class:doris::BufferPool +min_buffer_len_ src/runtime/bufferpool/buffer_allocator.h /^ const int64_t min_buffer_len_;$/;" m struct:doris::BufferPool::BufferAllocator +min_buffer_len_ src/runtime/bufferpool/buffer_pool.h /^ const int64_t min_buffer_len_;$/;" m class:doris::BufferPool +min_buffer_len_ src/runtime/bufferpool/suballocator.h /^ const int64_t min_buffer_len_;$/;" m class:doris::Suballocator +min_buffer_len_ src/runtime/bufferpool/system_allocator.h /^ const int64_t min_buffer_len_;$/;" m class:doris::SystemAllocator +min_enumerator src/gutil/casts.h /^ static const Enum min_enumerator = 0;$/;" m class:enum_limits +min_ht_mem src/exec/new_partitioned_aggregation_node.cc /^ int min_ht_mem;$/;" m struct:doris::StreamingHtMinReductionEntry file: +min_required_buffers src/exec/partitioned_aggregation_node.h /^ int min_required_buffers() const {$/;" f class:doris::PartitionedAggregationNode +min_reservation src/runtime/runtime_state.h /^ int64_t min_reservation() {$/;" f class:doris::RuntimeState +min_size src/runtime/free_list.hpp /^ static int min_size() {$/;" f class:doris::FreeList +min_value src/olap/rowset/segment_v2/zone_map_index.h /^ char* min_value = nullptr;$/;" m struct:doris::segment_v2::ZoneMap +mini_load_async_ctx src/http/action/mini_load.cpp /^ MiniLoadAsyncCtx* mini_load_async_ctx = nullptr;$/;" m struct:doris::MiniLoadCtx file: +minimum src/olap/stream_index_common.h /^ const WrapperField* minimum() const {$/;" f class:doris::ColumnStatistics +minimum src/util/bitmap_value.h /^ uint64_t minimum() const {$/;" f class:doris::detail::Roaring64Map +minute src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::minute($/;" f class:doris::TimestampFunctions +minute src/runtime/datetime_value.h /^ int minute() const {$/;" f class:doris::DateTimeValue +minute src/runtime/datetime_value.h /^ int32_t minute;$/;" m struct:doris::TimeInterval +minutes_add src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::minutes_add($/;" f class:doris::TimestampFunctions +minutes_diff src/exprs/timestamp_functions.cpp /^BigIntVal TimestampFunctions::minutes_diff($/;" f class:doris::TimestampFunctions +minutes_sub src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::minutes_sub($/;" f class:doris::TimestampFunctions +mix src/gutil/hash/jenkins_lookup2.h /^static inline void mix(uint32& a, uint32& b, uint32& c) { \/\/ 32bit version$/;" f +mix src/gutil/hash/jenkins_lookup2.h /^static inline void mix(uint64& a, uint64& b, uint64& c) { \/\/ 64bit version$/;" f +mmap src/gutil/linux_syscall_support.h /^ LSS_INLINE void* LSS_NAME(mmap)(void *s, size_t l, int p, int f, int d,$/;" f +mmap src/olap/byte_buffer.cpp /^StorageByteBuffer* StorageByteBuffer::mmap(FileHandler* handler, uint64_t offset, int prot, int flags) {$/;" f class:doris::StorageByteBuffer +mmap src/olap/byte_buffer.cpp /^StorageByteBuffer* StorageByteBuffer::mmap(void* start, uint64_t length, int prot, int flags,$/;" f class:doris::StorageByteBuffer +mock_es_server test/exec/es_scan_reader_test.cpp /^static EvHttpServer* mock_es_server = nullptr;$/;" m namespace:doris file: +mod src/util/pretty_printer.h /^ static ENABLE_IF_FLOAT(T, double) mod(const T& value, int modulus) {$/;" f class:doris::PrettyPrinter +mod src/util/pretty_printer.h /^ static ENABLE_IF_INTEGRAL(T, int64_t) mod(const T& value, const int modulus) {$/;" f class:doris::PrettyPrinter +mod_table src/exec/olap_utils.h /^static int mod_table[] = {0, 2, 1};$/;" m namespace:doris +mod_table src/util/url_coding.cpp /^static int mod_table[] = {0, 2, 1};$/;" m namespace:doris file: +mode src/env/env.h /^ Env::OpenMode mode = Env::CREATE_OR_OPEN_WITH_TRUNCATE;$/;" m struct:doris::RandomRWFileOptions +mode src/env/env.h /^ Env::OpenMode mode = Env::CREATE_OR_OPEN_WITH_TRUNCATE;$/;" m struct:doris::WritableFileOptions +mode src/util/threadpool.h /^ ThreadPool::ExecutionMode mode() const { return _mode; }$/;" f class:doris::ThreadPoolToken +mode_t src/gutil/port.h /^typedef unsigned int mode_t;$/;" t +model src/gutil/cpu.h /^ int model() const { return model_; }$/;" f class:base::final +model_ src/gutil/cpu.h /^ int model_; \/\/ model of processor$/;" m class:base::final +model_name src/util/cpu_info.h /^ static std::string model_name() {$/;" f class:doris::CpuInfo +model_name_ src/util/cpu_info.cpp /^string CpuInfo::model_name_ = "unknown";$/;" m class:doris::CpuInfo file: +model_name_ src/util/cpu_info.h /^ static std::string model_name_;$/;" m class:doris::CpuInfo +modify_rowsets src/olap/compaction.cpp /^OLAPStatus Compaction::modify_rowsets() {$/;" f class:doris::Compaction +modify_rowsets src/olap/tablet.cpp /^OLAPStatus Tablet::modify_rowsets(const vector& to_add,$/;" f class:doris::Tablet +modify_rs_metas src/olap/tablet_meta.cpp /^void TabletMeta::modify_rs_metas(const vector& to_add,$/;" f class:doris::TabletMeta +modify_user_cgroups src/agent/cgroups_mgr.cpp /^AgentStatus CgroupsMgr::modify_user_cgroups(const string& user_name,$/;" f class:doris::CgroupsMgr +money_format src/exprs/string_functions.cpp /^StringVal StringFunctions::money_format(FunctionContext *context, const BigIntVal &v) {$/;" f class:doris::StringFunctions +money_format src/exprs/string_functions.cpp /^StringVal StringFunctions::money_format(FunctionContext *context, const DecimalV2Val &v) {$/;" f class:doris::StringFunctions +money_format src/exprs/string_functions.cpp /^StringVal StringFunctions::money_format(FunctionContext *context, const DecimalVal &v) {$/;" f class:doris::StringFunctions +money_format src/exprs/string_functions.cpp /^StringVal StringFunctions::money_format(FunctionContext *context, const LargeIntVal &v) {$/;" f class:doris::StringFunctions +money_format src/exprs/string_functions.cpp /^StringVal StringFunctions::money_format(FunctionContext* context, const DoubleVal& v) {$/;" f class:doris::StringFunctions +month src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::month($/;" f class:doris::TimestampFunctions +month src/runtime/datetime_value.h /^ int month() const {$/;" f class:doris::DateTimeValue +month src/runtime/datetime_value.h /^ int32_t month;$/;" m struct:doris::TimeInterval +month_name src/exprs/timestamp_functions.cpp /^StringVal TimestampFunctions::month_name($/;" f class:doris::TimestampFunctions +month_name src/runtime/datetime_value.cpp /^const char* DateTimeValue::month_name() const {$/;" f class:doris::DateTimeValue +months_add src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::months_add($/;" f class:doris::TimestampFunctions +months_diff src/exprs/timestamp_functions.cpp /^BigIntVal TimestampFunctions::months_diff($/;" f class:doris::TimestampFunctions +months_sub src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::months_sub($/;" f class:doris::TimestampFunctions +move src/olap/rowset/segment_v2/page_handle.h /^ _cache_data(std::move(other._cache_data)) {$/;" f class:doris::segment_v2::PageHandle +move src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::move($/;" f class:doris::SnapshotLoader +move_hash_partitions src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::move_hash_partitions(int64_t num_input_rows) {$/;" f class:doris::PartitionedAggregationNode +move_next src/olap/rowset/segment_v2/index_page.h /^ bool move_next() {$/;" f class:doris::segment_v2::IndexPageIterator +move_node src/exec/hash_table.hpp /^inline void HashTable::move_node(Bucket* from_bucket, Bucket* to_bucket,$/;" f class:doris::HashTable +move_to_trash src/olap/utils.cpp /^OLAPStatus move_to_trash(const boost::filesystem::path& schema_hash_root,$/;" f namespace:doris +mremap src/gutil/linux_syscall_support.h /^ LSS_INLINE void *LSS_NAME(mremap)(void *old_address, size_t old_size,$/;" f +msg src/util/load_error_hub.h /^ std::string msg;$/;" m struct:doris::LoadErrorHub::ErrorMsg +mt_updater test/util/new_metrics_test.cpp /^void mt_updater(IntCounter* counter, std::atomic* used_time) {$/;" f namespace:doris +mtime src/runtime/disk_io_mgr.h /^ int64_t mtime() const { return _mtime; }$/;" f class:doris::DiskIoMgr::HdfsCachedFileHandle +mtime src/runtime/disk_io_mgr.h /^ int64_t mtime() const { return _mtime; }$/;" f class:doris::DiskIoMgr::ScanRange +murmur_hash2_64 src/util/hash_util.hpp /^ static uint64_t murmur_hash2_64(const void* input, int len, uint64_t seed) {$/;" f class:doris::HashUtil +murmur_hash3_32 src/exprs/hash_functions.cpp /^IntVal HashFunctions::murmur_hash3_32($/;" f class:doris::HashFunctions +murmur_hash3_32 src/util/hash_util.hpp /^ static uint32_t murmur_hash3_32(const void* key, int32_t len, uint32_t seed) {$/;" f class:doris::HashUtil +murmur_hash3_x64_128 src/util/murmur_hash3.cpp /^void murmur_hash3_x64_128(const void* key, const int len, const uint32_t seed, void* out) {$/;" f +murmur_hash3_x64_64 src/util/murmur_hash3.cpp /^void murmur_hash3_x64_64(const void* key, const int len, const uint64_t seed, void* out) {$/;" f +murmur_hash3_x86_128 src/util/murmur_hash3.cpp /^void murmur_hash3_x86_128(const void* key, const int len, uint32_t seed, void* out) {$/;" f +murmur_hash3_x86_32 src/util/murmur_hash3.cpp /^void murmur_hash3_x86_32(const void* key, int len, uint32_t seed, void* out) {$/;" f +murmur_hash64A src/util/hash_util.hpp /^ static uint64_t murmur_hash64A (const void* key, int32_t len, unsigned int seed) {$/;" f class:doris::HashUtil +must src/exec/es/es_query_builder.cpp /^void BooleanQueryBuilder::must(QueryBuilder* filter) {$/;" f class:doris::BooleanQueryBuilder +must_not src/exec/es/es_query_builder.cpp /^void BooleanQueryBuilder::must_not(QueryBuilder* filter) {$/;" f class:doris::BooleanQueryBuilder +mutable_bitmap src/olap/selection_vector.h /^ uint8_t* mutable_bitmap() { return _bitmap.get(); }$/;" f class:doris::SelectionVector +mutable_cell_ptr src/olap/column_block.h /^ uint8_t* mutable_cell_ptr() const { return _block.mutable_cell_ptr(_idx); }$/;" f struct:doris::ColumnBlockCell +mutable_cell_ptr src/olap/column_block.h /^ uint8_t* mutable_cell_ptr(size_t idx) const { return _data + idx * _type_info->size(); }$/;" f class:doris::ColumnBlock +mutable_cell_ptr src/olap/row_block2.h /^ uint8_t* mutable_cell_ptr(size_t col_idx) const {$/;" f class:doris::RowBlockRow +mutable_cell_ptr src/olap/row_cursor_cell.h /^ void* mutable_cell_ptr() const { return (char*)_ptr + 1; }$/;" f struct:doris::RowCursorCell +mutable_cell_ptr src/olap/wrapper_field.h /^ void* mutable_cell_ptr() const { return _field_buf + 1; }$/;" f class:doris::WrapperField +mutable_data src/util/slice.h /^ char* mutable_data() { return const_cast(data); }$/;" f struct:doris::Slice +mutable_delete_predicate src/olap/rowset/rowset_meta.h /^ DeletePredicatePB* mutable_delete_predicate() {$/;" f class:doris::RowsetMeta +mutable_entry src/olap/stream_index_writer.cpp /^PositionEntryWriter* StreamIndexWriter::mutable_entry(uint32_t index) {$/;" f class:doris::StreamIndexWriter +mutable_extra src/olap/file_helper.h /^ ExtraType* mutable_extra() {$/;" f class:doris::FileHeader +mutable_message src/olap/file_helper.h /^ MessageType* mutable_message() {$/;" f class:doris::FileHeader +mutable_serialize_batch_ns src/exec/tablet_sink.h /^ int64_t* mutable_serialize_batch_ns() { return &_serialize_batch_ns; }$/;" f class:doris::stream_load::OlapTableSink +mutable_stats src/olap/reader.h /^ OlapReaderStatistics* mutable_stats() { return &_stats; }$/;" f class:doris::Reader +mutable_tablet_schema src/olap/tablet_meta.h /^inline TabletSchema* TabletMeta::mutable_tablet_schema() {$/;" f class:doris::TabletMeta +mutable_wait_in_flight_packet_ns src/exec/tablet_sink.h /^ int64_t* mutable_wait_in_flight_packet_ns() { return &_wait_in_flight_packet_ns; }$/;" f class:doris::stream_load::OlapTableSink +my_add test/runtime/test_data/user_function_cache/lib/my_add.cc /^void my_add() {$/;" f +my_add_md5sum test/runtime/user_function_cache_test.cpp /^std::string my_add_md5sum;$/;" m namespace:doris file: +my_del test/runtime/test_data/user_function_cache/lib/my_add.cc /^void my_del() {$/;" f +my_double_round src/exprs/math_functions.cpp /^double MathFunctions::my_double_round(double value, int64_t dec, bool dec_unsigned, bool truncate) {$/;" f class:doris::MathFunctions +mysql_db src/runtime/descriptors.h /^ const std::string mysql_db() const {$/;" f class:doris::MySQLTableDescriptor +mysql_table src/runtime/descriptors.h /^ const std::string mysql_table() const {$/;" f class:doris::MySQLTableDescriptor +mysql_week_mode src/runtime/datetime_value.cpp /^uint8_t mysql_week_mode(uint32_t mode) {$/;" f namespace:doris +name src/common/configbase.h /^ const char* name;$/;" m struct:doris::config::Register::Field +name src/exec/es/es_predicate.h /^ std::string name;$/;" m struct:doris::ExtColumnDesc +name src/exec/schema_scanner.h /^ const char* name;$/;" m struct:doris::SchemaScanner::ColumnDesc +name src/exec/schema_scanner/schema_collations_scanner.h /^ const char *name;$/;" m struct:doris::SchemaCollationsScanner::CollationStruct +name src/exec/schema_scanner/schema_variables_scanner.h /^ const char *name;$/;" m struct:doris::SchemaVariablesScanner::VariableStruct +name src/olap/tablet_schema.h /^ inline std::string name() const { return _col_name; }$/;" f class:doris::TabletColumn +name src/runtime/descriptors.h /^ const std::string& name() const {$/;" f class:doris::TableDescriptor +name src/runtime/dpp_sink.cpp /^ const std::string& name() const {$/;" f class:doris::Translator +name src/runtime/snapshot_loader.h /^ std::string name;$/;" m struct:doris::FileStat +name src/util/cpu_info.cpp /^ string name;$/;" m struct:doris::__anon36 file: +name src/util/disk_info.h /^ std::string name;$/;" m struct:doris::DiskInfo::Disk +name src/util/metrics.h /^ std::string name;$/;" m struct:doris::MetricLabel +name src/util/runtime_profile.h /^ const std::string& name() const {$/;" f class:doris::RuntimeProfile +name src/util/thread.cpp /^ const std::string& name() const { return _name; }$/;" f class:doris::ThreadMgr::ThreadDescriptor +name src/util/thread.cpp /^const std::string& Thread::name() const {$/;" f class:doris::Thread +name_ src/runtime/bufferpool/buffer_pool_internal.h /^ const std::string name_;$/;" m class:doris::BufferPool::Client +nano_delta_ src/util/monotime.h /^ int64_t nano_delta_;$/;" m class:doris::MonoDelta +nanos_ src/util/monotime.h /^ int64_t nanos_;$/;" m class:doris::MonoTime +ne src/runtime/string_value.h /^ bool ne(const StringValue& other) const {$/;" f struct:doris::StringValue +need_agg_finalize src/olap/reader.h /^ bool need_agg_finalize = true;$/;" m struct:doris::ReaderParams +need_bitmap_index src/olap/rowset/segment_v2/column_writer.h /^ bool need_bitmap_index = false;$/;" m struct:doris::segment_v2::ColumnWriterOptions +need_bloom_filter src/olap/rowset/segment_v2/column_writer.h /^ bool need_bloom_filter = false;$/;" m struct:doris::segment_v2::ColumnWriterOptions +need_delete_file src/olap/rowset/rowset.h /^ bool need_delete_file() const {$/;" f class:doris::Rowset +need_gen_rollup src/olap/delta_writer.h /^ bool need_gen_rollup;$/;" m struct:doris::WriteRequest +need_ordered_result src/olap/rowset/rowset_reader_context.h /^ bool need_ordered_result = true;$/;" m struct:doris::RowsetReaderContext +need_remove_handle src/http/action/mini_load.cpp /^ bool need_remove_handle = false;$/;" m struct:doris::MiniLoadAsyncCtx file: +need_rollback src/runtime/stream_load/stream_load_context.h /^ bool need_rollback = false;$/;" m class:doris::StreamLoadContext +need_to_return src/runtime/row_batch.h /^ bool need_to_return() {$/;" f class:doris::RowBatch +need_zone_map src/olap/rowset/segment_v2/column_writer.h /^ bool need_zone_map = false;$/;" m struct:doris::segment_v2::ColumnWriterOptions +needs_deep_copy src/runtime/row_batch.h /^ bool needs_deep_copy() { return _needs_deep_copy; }$/;" f class:doris::RowBatch +needs_finalize_ src/exec/new_partitioned_aggregation_node.h /^ const bool needs_finalize_;$/;" m class:doris::NewPartitionedAggregationNode +needs_serialize_ src/exec/new_partitioned_aggregation_node.h /^ bool needs_serialize_;$/;" m class:doris::NewPartitionedAggregationNode +neg test/util/utf8_check_test.cpp /^ std::vector neg = {$/;" m class:doris::Utf8CheckTest file: +neg_infinite src/runtime/dpp_sink_internal.h /^ static const PartRangeKey& neg_infinite() {$/;" f class:doris::PartRangeKey +negative_bigint src/exprs/math_functions.cpp /^BigIntVal MathFunctions::negative_bigint($/;" f class:doris::MathFunctions +negative_decimal src/exprs/math_functions.cpp /^DecimalV2Val MathFunctions::negative_decimal($/;" f class:doris::MathFunctions +negative_decimal src/exprs/math_functions.cpp /^DecimalVal MathFunctions::negative_decimal($/;" f class:doris::MathFunctions +negative_double src/exprs/math_functions.cpp /^DoubleVal MathFunctions::negative_double($/;" f class:doris::MathFunctions +nelem src/gutil/utf/utfdef.h 13;" d +new_auto_increment_iterator src/olap/generic_iterators.cpp /^RowwiseIterator* new_auto_increment_iterator(const Schema& schema, size_t num_rows) {$/;" f namespace:doris +new_bitmap_index_iterator src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::new_bitmap_index_iterator(BitmapIndexIterator** iterator) {$/;" f class:doris::segment_v2::ColumnReader +new_bitmap_index_iterator src/olap/rowset/segment_v2/segment.cpp /^Status Segment::new_bitmap_index_iterator(uint32_t cid, BitmapIndexIterator** iter) {$/;" f class:doris::segment_v2::Segment +new_block_for_write src/runtime/buffered_tuple_stream.cpp /^Status BufferedTupleStream::new_block_for_write(int min_size, bool* got_block) {$/;" f class:doris::BufferedTupleStream +new_block_for_write src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::new_block_for_write(int64_t min_size, bool* got_block) {$/;" f class:doris::BufferedTupleStream2 +new_builder_options test/olap/rowset/segment_v2/plain_page_test.cpp /^ PageBuilderOptions* new_builder_options() {$/;" f class:doris::segment_v2::PlainPageTest +new_column_iterator src/olap/rowset/segment_v2/segment.cpp /^Status Segment::new_column_iterator(uint32_t cid, ColumnIterator** iter) {$/;" f class:doris::segment_v2::Segment +new_entry_length src/olap/olap_index.h /^ const size_t new_entry_length() const {$/;" f class:doris::MemIndex +new_id src/olap/lru_cache.cpp /^uint64_t ShardedLRUCache::new_id() {$/;" f class:doris::ShardedLRUCache +new_iterator src/olap/rowset/segment_v2/bitmap_index_reader.cpp /^Status BitmapIndexReader::new_iterator(BitmapIndexIterator** iterator) {$/;" f class:doris::segment_v2::BitmapIndexReader +new_iterator src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp /^Status BloomFilterIndexReader::new_iterator(std::unique_ptr* iterator) {$/;" f class:doris::segment_v2::BloomFilterIndexReader +new_iterator src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::new_iterator(ColumnIterator** iterator) {$/;" f class:doris::segment_v2::ColumnReader +new_iterator src/olap/rowset/segment_v2/segment.cpp /^Status Segment::new_iterator(const Schema& schema,$/;" f class:doris::segment_v2::Segment +new_lru_cache src/olap/lru_cache.cpp /^Cache* new_lru_cache(size_t capacity) {$/;" f namespace:doris +new_merge_iterator src/olap/generic_iterators.cpp /^RowwiseIterator* new_merge_iterator(std::vector inputs) {$/;" f namespace:doris +new_short_key_length src/olap/olap_index.h /^ const size_t new_short_key_length() const {$/;" f class:doris::MemIndex +new_short_key_length src/olap/rowset/segment_group.h /^ const size_t new_short_key_length() const {$/;" f class:doris::SegmentGroup +new_slow_task test/util/threadpool_test.cpp /^ static shared_ptr new_slow_task(CountDownLatch* latch) {$/;" f class:doris::SlowTask +new_tablet src/olap/schema_change.h /^ TabletSharedPtr new_tablet;$/;" m struct:doris::SchemaChangeHandler::SchemaChangeParams +new_token src/util/threadpool.cpp /^std::unique_ptr ThreadPool::new_token(ExecutionMode mode) {$/;" f class:doris::ThreadPool +new_union_iterator src/olap/generic_iterators.cpp /^RowwiseIterator* new_union_iterator(std::vector inputs) {$/;" f namespace:doris +next src/exec/hash_table.hpp /^inline void HashTable::Iterator::next() {$/;" f class:doris::HashTable::Iterator +next src/exec/new_partitioned_hash_table.h /^ DuplicateNode* next;$/;" m struct:doris::NewPartitionedHashTable::DuplicateNode +next src/exec/partitioned_hash_table.h /^ DuplicateNode* next;$/;" m struct:doris::PartitionedHashTable::DuplicateNode +next src/exec/partitioned_hash_table.inline.h /^inline void PartitionedHashTable::Iterator::next() {$/;" f class:doris::PartitionedHashTable::Iterator +next src/exec/row_batch_list.h /^ void next() {$/;" f class:doris::RowBatchList::TupleRowIterator +next src/exprs/hybird_set.h /^ virtual void next() {$/;" f class:doris::HybirdSet::Iterator +next src/exprs/hybird_set.h /^ virtual void next() {$/;" f class:doris::StringValueSet::Iterator +next src/olap/lru_cache.h /^ LRUHandle* next;$/;" m struct:doris::CachePriority::LRUHandle +next src/olap/olap_index.cpp /^const OLAPIndexOffset MemIndex::next(const OLAPIndexOffset& pos) const {$/;" f class:doris::MemIndex +next src/olap/push_handler.cpp /^OLAPStatus BinaryReader::next(RowCursor* row) {$/;" f class:doris::BinaryReader +next src/olap/push_handler.cpp /^OLAPStatus LzoBinaryReader::next(RowCursor* row) {$/;" f class:doris::LzoBinaryReader +next src/olap/reader.cpp /^ OLAPStatus next(const RowCursor** row, bool* delete_flag) {$/;" f class:doris::CollectIterator::ChildCtx +next src/olap/reader.cpp /^ OLAPStatus next(const RowCursor** row, bool* delete_flag) {$/;" f class:doris::CollectIterator +next src/olap/rowset/bit_field_reader.cpp /^OLAPStatus BitFieldReader::next(char* value) {$/;" f class:doris::BitFieldReader +next src/olap/rowset/column_data_writer.cpp /^void ColumnDataWriter::next(const RowType& row) {$/;" f class:doris::ColumnDataWriter +next src/olap/rowset/column_reader.cpp /^OLAPStatus IntegerColumnReader::next(int64_t* value) {$/;" f class:doris::IntegerColumnReader +next src/olap/rowset/column_reader.cpp /^OLAPStatus StringColumnDictionaryReader::next(char* buffer, uint32_t* length) {$/;" f class:doris::StringColumnDictionaryReader +next src/olap/rowset/column_reader.cpp /^OLAPStatus StringColumnDirectReader::next(char* buffer, uint32_t* length) {$/;" f class:doris::StringColumnDirectReader +next src/olap/rowset/run_length_byte_reader.cpp /^OLAPStatus RunLengthByteReader::next(char* value) {$/;" f class:doris::RunLengthByteReader +next src/olap/rowset/run_length_integer_reader.h /^ inline OLAPStatus next(int64_t* value) {$/;" f class:doris::RunLengthIntegerReader +next src/olap/rowset/segment_v2/column_writer.h /^ Page* next = nullptr;$/;" m struct:doris::segment_v2::ColumnWriter::Page +next src/olap/rowset/segment_v2/ordinal_page_index.h /^ void next() {$/;" f class:doris::segment_v2::OrdinalPageIndexIterator +next src/runtime/free_list.hpp /^ FreeListNode* next;$/;" m struct:doris::FreeList::FreeListNode +next src/runtime/free_pool.hpp /^ FreeListNode* next; \/\/ Used when it is in the free list$/;" m union:doris::FreePool::FreeListNode::__anon1 +next src/runtime/merge_sorter.cpp /^ void next() {$/;" f class:doris::MergeSorter::TupleSorter::TupleIterator +next src/runtime/row_batch.h /^ TupleRow* IR_ALWAYS_INLINE next() {$/;" f class:doris::RowBatch::Iterator +next src/runtime/sorted_run_merger.cc /^ Status next(RowBatch* transfer_batch, bool* done) {$/;" f class:doris::SortedRunMerger::BatchedRowSupplier +next src/runtime/spill_sorter.cc /^ void next() {$/;" f class:doris::SpillSorter::TupleSorter::TupleIterator +next src/util/internal_queue.h /^ T* next() const {$/;" f struct:doris::InternalQueueBase::Node +next_ src/olap/skiplist.h /^ std::atomic next_[1];$/;" m struct:doris::SkipList::Node +next_batch src/olap/generic_iterators.cpp /^Status AutoIncrementIterator::next_batch(RowBlockV2* block) {$/;" f class:doris::AutoIncrementIterator +next_batch src/olap/generic_iterators.cpp /^Status MergeIterator::next_batch(RowBlockV2* block) {$/;" f class:doris::MergeIterator +next_batch src/olap/generic_iterators.cpp /^Status UnionIterator::next_batch(RowBlockV2* block) {$/;" f class:doris::UnionIterator +next_batch src/olap/rowset/segment_v2/binary_dict_page.cpp /^Status BinaryDictPageDecoder::next_batch(size_t* n, ColumnBlockView* dst) {$/;" f class:doris::segment_v2::BinaryDictPageDecoder +next_batch src/olap/rowset/segment_v2/binary_prefix_page.cpp /^Status BinaryPrefixPageDecoder::next_batch(size_t* n, ColumnBlockView* dst) {$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +next_batch src/olap/rowset/segment_v2/column_reader.cpp /^Status DefaultValueColumnIterator::next_batch(size_t* n, ColumnBlockView* dst) {$/;" f class:doris::segment_v2::DefaultValueColumnIterator +next_batch src/olap/rowset/segment_v2/column_reader.cpp /^Status FileColumnIterator::next_batch(size_t* n, ColumnBlockView* dst) {$/;" f class:doris::segment_v2::FileColumnIterator +next_batch src/olap/rowset/segment_v2/empty_segment_iterator.cpp /^Status EmptySegmentIterator::next_batch(RowBlockV2* block) {$/;" f class:doris::segment_v2::EmptySegmentIterator +next_batch src/olap/rowset/segment_v2/indexed_column_reader.cpp /^Status IndexedColumnIterator::next_batch(size_t* n, ColumnBlockView* column_view) {$/;" f class:doris::segment_v2::IndexedColumnIterator +next_batch src/olap/rowset/segment_v2/segment_iterator.cpp /^Status SegmentIterator::next_batch(RowBlockV2* block) {$/;" f class:doris::segment_v2::SegmentIterator +next_block src/olap/rowset/alpha_rowset_reader.cpp /^OLAPStatus AlphaRowsetReader::next_block(RowBlock** block) {$/;" f class:doris::AlphaRowsetReader +next_block src/olap/rowset/beta_rowset_reader.cpp /^OLAPStatus BetaRowsetReader::next_block(RowBlock** block) {$/;" f class:doris::BetaRowsetReader +next_block_for_read src/runtime/buffered_tuple_stream.cpp /^Status BufferedTupleStream::next_block_for_read() {$/;" f class:doris::BufferedTupleStream +next_block_for_read src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::next_block_for_read() {$/;" f class:doris::BufferedTupleStream2 +next_bucket src/exec/hash_table.hpp /^inline HashTable::Bucket* HashTable::next_bucket(int64_t* bucket_idx) {$/;" f class:doris::HashTable +next_chunk_size_ src/runtime/mem_pool.h /^ int next_chunk_size_;$/;" m class:doris::MemPool +next_column_unique_id src/olap/tablet_schema.h /^ inline size_t next_column_unique_id() const { return _next_column_unique_id; }$/;" f class:doris::TabletSchema +next_duplicate src/exec/partitioned_hash_table.inline.h /^inline void PartitionedHashTable::Iterator::next_duplicate() {$/;" f class:doris::PartitionedHashTable::Iterator +next_filled_bucket src/exec/partitioned_hash_table.inline.h /^inline void PartitionedHashTable::next_filled_bucket(int64_t* bucket_idx, DuplicateNode** node) {$/;" f class:doris::PartitionedHashTable +next_free_ src/runtime/bufferpool/suballocator.h /^ std::unique_ptr next_free_;$/;" m class:doris::Suballocation +next_hash src/olap/lru_cache.h /^ LRUHandle* next_hash;$/;" m struct:doris::CachePriority::LRUHandle +next_id src/olap/rowset/unique_rowset_id_generator.cpp /^RowsetId UniqueRowsetIdGenerator::next_id() {$/;" f class:doris::UniqueRowsetIdGenerator +next_node src/util/internal_queue.h /^ Node* next_node;$/;" m struct:doris::InternalQueueBase::Node +next_node_ src/exec/new_partitioned_hash_table.h /^ DuplicateNode* next_node_;$/;" m class:doris::NewPartitionedHashTable +next_partition src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::next_partition() {$/;" f class:doris::PartitionedAggregationNode +next_power_of_two src/util/bit_util.h /^ static inline int64_t next_power_of_two(int64_t v) {$/;" f class:doris::BitUtil +next_range src/olap/rowset/segment_v2/segment_iterator.cpp /^ bool next_range(uint32_t max_range_size, uint32_t* from, uint32_t* to) {$/;" f class:doris::segment_v2::SegmentIterator::BitmapRangeIterator +next_row_with_aggregation src/olap/reader.h /^ OLAPStatus next_row_with_aggregation(RowCursor *row_cursor, MemPool* mem_pool, ObjectPool* agg_pool, bool *eof) {$/;" f class:doris::Reader +next_rowset_id src/olap/storage_engine.h /^ RowsetId next_rowset_id() { return _rowset_id_generator->next_id(); };$/;" f class:doris::StorageEngine +next_scan_range_to_start src/runtime/disk_io_mgr_internal.h /^ ScanRange* next_scan_range_to_start() { return _next_scan_range_to_start; }$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +next_slot_id src/runtime/descriptor_helper.h /^ TSlotId next_slot_id() { return _next_slot_id++; }$/;" f class:doris::TDescriptorTableBuilder +next_tuple src/exec/kudu_scanner.h /^ inline Tuple* next_tuple(Tuple* t) const {$/;" f class:doris::KuduScanner +next_tuple_id src/runtime/descriptor_helper.h /^ TTupleId next_tuple_id() { return _next_tuple_id++; }$/;" f class:doris::TDescriptorTableBuilder +next_unique_id src/olap/tablet.h /^inline size_t Tablet::next_unique_id() const {$/;" f class:doris::Tablet +next_unmatched src/exec/partitioned_hash_table.inline.h /^inline void PartitionedHashTable::Iterator::next_unmatched() {$/;" f class:doris::PartitionedHashTable::Iterator +next_uuid src/util/uuid_generator.h /^ boost::uuids::uuid next_uuid() {$/;" f class:doris::UUIDGenerator +next_vector src/olap/rowset/column_reader.cpp /^OLAPStatus ColumnReader::next_vector($/;" f class:doris::ColumnReader +next_vector src/olap/rowset/column_reader.cpp /^OLAPStatus DecimalColumnReader::next_vector($/;" f class:doris::DecimalColumnReader +next_vector src/olap/rowset/column_reader.cpp /^OLAPStatus LargeIntColumnReader::next_vector($/;" f class:doris::LargeIntColumnReader +next_vector src/olap/rowset/column_reader.cpp /^OLAPStatus StringColumnDictionaryReader::next_vector($/;" f class:doris::StringColumnDictionaryReader +next_vector src/olap/rowset/column_reader.cpp /^OLAPStatus StringColumnDirectReader::next_vector($/;" f class:doris::StringColumnDirectReader +next_vector src/olap/rowset/column_reader.cpp /^OLAPStatus TinyColumnReader::next_vector($/;" f class:doris::TinyColumnReader +next_vector src/olap/rowset/column_reader.h /^ virtual OLAPStatus next_vector($/;" f class:doris::DefaultValueReader +next_vector src/olap/rowset/column_reader.h /^ virtual OLAPStatus next_vector($/;" f class:doris::FixLengthStringColumnReader +next_vector src/olap/rowset/column_reader.h /^ virtual OLAPStatus next_vector($/;" f class:doris::FloatintPointColumnReader +next_vector src/olap/rowset/column_reader.h /^ virtual OLAPStatus next_vector($/;" f class:doris::IntegerColumnReaderWrapper +next_vector src/olap/rowset/column_reader.h /^ virtual OLAPStatus next_vector($/;" f class:doris::NullValueReader +next_vector src/olap/rowset/column_reader.h /^ virtual OLAPStatus next_vector($/;" f class:doris::VarStringColumnReader +nextafter src/gutil/port.h 837;" d +nice src/olap/storage_engine.h /^ uint32_t nice; \/\/ 优先度$/;" m struct:doris::StorageEngine::CompactionCandidate +nil src/gutil/utf/utfdef.h 14;" d +no src/olap/utils.cpp /^int Errno::no() {$/;" f class:doris::Errno +no_nulls src/runtime/vectorized_row_batch.h /^ bool no_nulls() const {$/;" f class:doris::ColumnVector +no_string_slots src/runtime/descriptors.h /^ const std::vector& no_string_slots() const {$/;" f class:doris::TupleDescriptor +node_ src/exec/new_partitioned_hash_table.h /^ DuplicateNode* node_;$/;" m class:doris::NewPartitionedHashTable::Iterator +node_ src/olap/skiplist.h /^ Node* node_;$/;" m class:doris::SkipList::Iterator +node_id src/exec/tablet_sink.h /^ int64_t node_id() const { return _node_id; }$/;" f class:doris::stream_load::NodeChannel +node_id_ src/runtime/buffered_tuple_stream3.h /^ int node_id_;$/;" m class:doris::BufferedTupleStream3 +node_info src/exec/tablet_sink.h /^ const NodeInfo* node_info() const { return _node_info; }$/;" f class:doris::stream_load::NodeChannel +node_remaining_current_page_ src/exec/new_partitioned_hash_table.h /^ int node_remaining_current_page_;$/;" m class:doris::NewPartitionedHashTable +node_type src/exec/es/es_predicate.h /^ TExprNodeType::type node_type;$/;" m struct:doris::ExtPredicate +node_type src/exprs/expr.h /^ TExprNodeType::type node_type() const {$/;" f class:doris::Expr +noexcept src/exec/new_partitioned_hash_table.h /^ const uint8_t* expr_values, const uint8_t* expr_values_null) const noexcept;$/;" m class:doris::NewPartitionedHashTableCtx +noexcept src/exec/new_partitioned_hash_table.h /^ const uint8_t* expr_values_null) const noexcept;$/;" m class:doris::NewPartitionedHashTableCtx +noexcept src/exec/new_partitioned_hash_table.h /^ uint8_t* expr_values, uint8_t* expr_values_null) noexcept;$/;" m class:doris::NewPartitionedHashTableCtx +noexcept src/exec/new_partitioned_hash_table.h /^ void Reset() noexcept;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +noexcept src/exec/tablet_sink.h /^ ~NodeChannel() noexcept;$/;" m class:doris::stream_load::NodeChannel +noexcept src/runtime/buffered_tuple_stream3.h /^ bool AddRowSlow(TupleRow* row, Status* status) noexcept;$/;" m class:doris::BufferedTupleStream3 +noexcept src/runtime/buffered_tuple_stream3.h /^ bool DeepCopy(TupleRow* row, uint8_t** data, const uint8_t* data_end) noexcept;$/;" m class:doris::BufferedTupleStream3 +noexcept src/runtime/buffered_tuple_stream3.h /^ bool DeepCopyInternal(TupleRow* row, uint8_t** data, const uint8_t* data_end) noexcept;$/;" m class:doris::BufferedTupleStream3 +noexcept src/runtime/buffered_tuple_stream3.h /^ int64_t ComputeRowSize(TupleRow* row) const noexcept;$/;" m class:doris::BufferedTupleStream3 +noexcept src/runtime/buffered_tuple_stream3.h /^ uint8_t* AddRowCustomBeginSlow(int64_t size, Status* status) noexcept;$/;" m class:doris::BufferedTupleStream3 +noexcept src/runtime/buffered_tuple_stream3.h /^ void AddLargeRowCustomEnd(int64_t size) noexcept;$/;" m class:doris::BufferedTupleStream3 +non_blocking_get src/util/blocking_priority_queue.hpp /^ bool non_blocking_get(T* out) {$/;" f class:doris::BlockingPriorityQueue +normalize_binary_predicate src/exec/olap_scan_node.cpp /^Status OlapScanNode::normalize_binary_predicate(SlotDescriptor* slot, ColumnValueRange* range) {$/;" f class:doris::OlapScanNode +normalize_conjuncts src/exec/olap_scan_node.cpp /^Status OlapScanNode::normalize_conjuncts() {$/;" f class:doris::OlapScanNode +normalize_in_predicate src/exec/olap_scan_node.cpp /^Status OlapScanNode::normalize_in_predicate(SlotDescriptor* slot, ColumnValueRange* range) {$/;" f class:doris::OlapScanNode +normalize_predicate src/exec/olap_scan_node.cpp /^Status OlapScanNode::normalize_predicate(ColumnValueRange& range, SlotDescriptor* slot) {$/;" f class:doris::OlapScanNode +not_empty src/util/threadpool.h /^ ConditionVariable not_empty;$/;" m struct:doris::ThreadPool::IdleThread +notify_all src/util/condition_variable.cpp /^void ConditionVariable::notify_all() {$/;" f class:doris::ConditionVariable +notify_one src/util/condition_variable.cpp /^void ConditionVariable::notify_one() {$/;" f class:doris::ConditionVariable +now src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::now(FunctionContext* context) {$/;" f class:doris::TimestampFunctions +npos src/gutil/strings/stringpiece.cc /^const StringPiece::size_type StringPiece::npos = size_type(-1);$/;" m class:StringPiece file: +npos src/gutil/strings/stringpiece.h /^ static const size_type npos;$/;" m class:StringPiece +nraddr src/gutil/valgrind.h /^ unsigned int nraddr; \/* where's the code? *\/$/;" m struct:__anon11 +nraddr src/gutil/valgrind.h /^ unsigned int nraddr; \/* where's the code? *\/$/;" m struct:__anon13 +nraddr src/gutil/valgrind.h /^ unsigned int nraddr; \/* where's the code? *\/$/;" m struct:__anon9 +nraddr src/gutil/valgrind.h /^ unsigned long long int nraddr; \/* where's the code? *\/$/;" m struct:__anon10 +nraddr src/gutil/valgrind.h /^ unsigned long long int nraddr; \/* where's the code? *\/$/;" m struct:__anon12 +nraddr src/gutil/valgrind.h /^ unsigned long long int nraddr; \/* where's the code? *\/$/;" m struct:__anon14 +nrows src/olap/column_block.h /^ size_t nrows() const { return _nrows; }$/;" f class:doris::ColumnBlock +nrows src/olap/selection_vector.h /^ size_t nrows() const { return _n_rows; }$/;" f class:doris::SelectionVector +ntohll src/gutil/endian.h 102;" d +null output/udf/include/udf.h /^ static BigIntVal null() {$/;" f struct:doris_udf::BigIntVal +null output/udf/include/udf.h /^ static BooleanVal null() {$/;" f struct:doris_udf::BooleanVal +null output/udf/include/udf.h /^ static DateTimeVal null() {$/;" f struct:doris_udf::DateTimeVal +null output/udf/include/udf.h /^ static DecimalV2Val null() {$/;" f struct:doris_udf::DecimalV2Val +null output/udf/include/udf.h /^ static DecimalVal null() {$/;" f struct:doris_udf::DecimalVal +null output/udf/include/udf.h /^ static DoubleVal null() {$/;" f struct:doris_udf::DoubleVal +null output/udf/include/udf.h /^ static FloatVal null() {$/;" f struct:doris_udf::FloatVal +null output/udf/include/udf.h /^ static IntVal null() {$/;" f struct:doris_udf::IntVal +null output/udf/include/udf.h /^ static LargeIntVal null() {$/;" f struct:doris_udf::LargeIntVal +null output/udf/include/udf.h /^ static SmallIntVal null() {$/;" f struct:doris_udf::SmallIntVal +null output/udf/include/udf.h /^ static StringVal null() {$/;" f struct:doris_udf::StringVal +null output/udf/include/udf.h /^ static TinyIntVal null() {$/;" f struct:doris_udf::TinyIntVal +null src/udf/udf.h /^ static BigIntVal null() {$/;" f struct:doris_udf::BigIntVal +null src/udf/udf.h /^ static BooleanVal null() {$/;" f struct:doris_udf::BooleanVal +null src/udf/udf.h /^ static DateTimeVal null() {$/;" f struct:doris_udf::DateTimeVal +null src/udf/udf.h /^ static DecimalV2Val null() {$/;" f struct:doris_udf::DecimalV2Val +null src/udf/udf.h /^ static DecimalVal null() {$/;" f struct:doris_udf::DecimalVal +null src/udf/udf.h /^ static DoubleVal null() {$/;" f struct:doris_udf::DoubleVal +null src/udf/udf.h /^ static FloatVal null() {$/;" f struct:doris_udf::FloatVal +null src/udf/udf.h /^ static IntVal null() {$/;" f struct:doris_udf::IntVal +null src/udf/udf.h /^ static LargeIntVal null() {$/;" f struct:doris_udf::LargeIntVal +null src/udf/udf.h /^ static SmallIntVal null() {$/;" f struct:doris_udf::SmallIntVal +null src/udf/udf.h /^ static StringVal null() {$/;" f struct:doris_udf::StringVal +null src/udf/udf.h /^ static TinyIntVal null() {$/;" f struct:doris_udf::TinyIntVal +null src/util/mem_range.h /^ static MemRange null() { return MemRange(nullptr, 0); }$/;" f class:doris::MemRange +null_bitmap src/olap/column_block.h /^ uint8_t* null_bitmap() const { return _null_bitmap; }$/;" f class:doris::ColumnBlock +null_bitmap src/olap/rowset/segment_v2/parsed_page.h /^ Slice null_bitmap;$/;" m struct:doris::segment_v2::ParsedPage +null_bitmap_ src/exec/new_partitioned_hash_table.h /^ Bitmap null_bitmap_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +null_decoder src/olap/rowset/segment_v2/parsed_page.h /^ RleDecoder null_decoder;$/;" m struct:doris::segment_v2::ParsedPage +null_indicator_offset src/exprs/slot_ref.h /^ inline NullIndicatorOffset null_indicator_offset() const {$/;" f class:doris::SlotRef +null_indicator_offset src/runtime/descriptors.h /^ const NullIndicatorOffset& null_indicator_offset() const {$/;" f class:doris::SlotDescriptor +null_or_empty src/exprs/string_functions.cpp /^BooleanVal StringFunctions::null_or_empty($/;" f class:doris::StringFunctions +null_supported src/olap/row_block.h /^ bool null_supported;$/;" m struct:doris::RowBlockInfo +null_terminator_ok src/gutil/strings/numbers.h /^ bool null_terminator_ok;$/;" m struct:DoubleRangeOptions +nullable src/runtime/descriptor_helper.h /^ TSlotDescriptorBuilder& nullable(bool nullable) {$/;" f class:doris::TSlotDescriptorBuilder +nullable_cell_ptr src/olap/row_cursor.h /^ char* nullable_cell_ptr(uint32_t cid) const {$/;" f class:doris::RowCursor +nullable_cell_ptr src/olap/wrapper_field.h /^ char* nullable_cell_ptr() const { return _field_buf; }$/;" f class:doris::WrapperField +num test/util/lru_cache_util_test.cpp /^ int num;$/;" m struct:doris::Foo file: +num_active_tmp_devices src/runtime/tmp_file_mgr.cc /^int TmpFileMgr::num_active_tmp_devices() {$/;" f class:doris::TmpFileMgr +num_allocated_buffers src/runtime/disk_io_mgr.h /^ int num_allocated_buffers() const { return _num_allocated_buffers; }$/;" f class:doris::DiskIoMgr +num_available_threads src/runtime/thread_resource_mgr.h /^ int num_available_threads() const {$/;" f class:doris::ThreadResourceMgr::ResourcePool +num_bits src/gutil/bits.cc /^const char Bits::num_bits[] = {$/;" m class:Bits file: +num_bits src/gutil/bits.h /^ static const char num_bits[];$/;" m class:Bits +num_bits src/util/bitmap.h /^ int64_t num_bits() const { return num_bits_; }$/;" f class:doris::Bitmap +num_bits_ src/util/bitmap.h /^ size_t num_bits_;$/;" m class:doris::BitmapIterator +num_bits_ src/util/bitmap.h /^ int64_t num_bits_;$/;" m class:doris::Bitmap +num_buckets src/exec/hash_table.h /^ int64_t num_buckets() {$/;" f class:doris::HashTable +num_buckets src/exec/new_partitioned_hash_table.h /^ int64_t num_buckets() const { return num_buckets_; }$/;" f class:doris::NewPartitionedHashTable +num_buckets src/exec/partitioned_hash_table.h /^ int64_t num_buckets() const { return _num_buckets; }$/;" f class:doris::PartitionedHashTable +num_buckets src/exec/tablet_info.h /^ int64_t num_buckets = 0;$/;" m struct:doris::OlapTablePartition +num_buckets_ src/exec/new_partitioned_hash_table.h /^ int64_t num_buckets_;$/;" m class:doris::NewPartitionedHashTable +num_buckets_with_duplicates_ src/exec/new_partitioned_hash_table.h /^ int64_t num_buckets_with_duplicates_;$/;" m class:doris::NewPartitionedHashTable +num_buffered_values_ src/util/rle_encoding.h /^ int num_buffered_values_;$/;" m class:doris::RleEncoder +num_buffers src/runtime/row_batch.h /^ int num_buffers() const { $/;" f class:doris::RowBatch +num_buffers_in_readers src/runtime/disk_io_mgr.h /^ int num_buffers_in_readers() const { return _num_buffers_in_readers; }$/;" f class:doris::DiskIoMgr +num_build_tuples_ src/exec/new_partitioned_hash_table.h /^ const int num_build_tuples_;$/;" m class:doris::NewPartitionedHashTable +num_bytes src/olap/rowset/segment_v2/bloom_filter.h /^ uint32_t num_bytes() const { return _num_bytes; }$/;" f class:doris::segment_v2::BloomFilter +num_bytes_sent test/runtime/data_stream_test.cpp /^ int num_bytes_sent;$/;" m struct:doris::DataStreamTest::SenderInfo file: +num_clean_pages src/runtime/bufferpool/buffer_allocator.cc /^ AtomicInt64 num_clean_pages;$/;" m struct:doris::BufferPool::FreeBufferArena::PerSizeLists file: +num_clustering_cols src/runtime/descriptors.h /^ int num_clustering_cols() const {$/;" f class:doris::TableDescriptor +num_cols src/runtime/descriptors.h /^ int num_cols() const {$/;" f class:doris::TableDescriptor +num_column_ids src/olap/schema.h /^ size_t num_column_ids() const { return _col_ids.size(); }$/;" f class:doris::Schema +num_columns src/olap/schema.h /^ size_t num_columns() const { return _cols.size(); }$/;" f class:doris::Schema +num_columns src/olap/tablet.h /^inline size_t Tablet::num_columns() const {$/;" f class:doris::Tablet +num_columns src/olap/tablet_schema.h /^ inline size_t num_columns() const { return _num_columns; }$/;" f class:doris::TabletSchema +num_complete src/util/progress_updater.h /^ int64_t num_complete() const {$/;" f class:doris::ProgressUpdater +num_cores src/util/cpu_info.h /^ static int num_cores() {$/;" f class:doris::CpuInfo +num_cores_ src/util/cpu_info.cpp /^int CpuInfo::num_cores_ = 1;$/;" m class:doris::CpuInfo file: +num_cores_ src/util/cpu_info.h /^ static int num_cores_;$/;" m class:doris::CpuInfo +num_counters src/util/runtime_profile.h /^ int num_counters() const {$/;" f class:doris::RuntimeProfile +num_data_bytes_sent src/runtime/data_stream_sender.cpp /^ int64_t num_data_bytes_sent() const {$/;" f class:doris::DataStreamSender::Channel +num_data_pages src/olap/rowset/segment_v2/ordinal_page_index.h /^ int32_t num_data_pages() const { return _num_pages; }$/;" f class:doris::segment_v2::OrdinalIndexReader +num_disks src/util/disk_info.h /^ static int num_disks() {$/;" f class:doris::DiskInfo +num_duplicate_nodes_ src/exec/new_partitioned_hash_table.h /^ int64_t num_duplicate_nodes_;$/;" m class:doris::NewPartitionedHashTable +num_exprs_ src/exec/new_partitioned_hash_table.h /^ int num_exprs_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +num_failed_probes_ src/exec/new_partitioned_hash_table.h /^ int64_t num_failed_probes_;$/;" m class:doris::NewPartitionedHashTable +num_file src/util/minizip/unzip.c /^ ZPOS64_T num_file; \/* number of the current file in the zipfile*\/$/;" m struct:__anon34 file: +num_filled_buckets_ src/exec/new_partitioned_hash_table.h /^ int64_t num_filled_buckets_;$/;" m class:doris::NewPartitionedHashTable +num_fixed_args src/exprs/scalar_fn_call.h /^ int num_fixed_args() const {$/;" f class:doris::ScalarFnCall +num_flags src/util/cpu_info.cpp /^static const long num_flags = sizeof(flag_mappings) \/ sizeof(flag_mappings[0]);$/;" m namespace:doris file: +num_free_buffers src/runtime/buffered_block_mgr2.h /^ int num_free_buffers() const {$/;" f class:doris::BufferedBlockMgr2 +num_free_buffers src/runtime/bufferpool/buffer_allocator.cc /^ AtomicInt64 num_free_buffers;$/;" m struct:doris::BufferPool::FreeBufferArena::PerSizeLists file: +num_hash_buckets_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* num_hash_buckets_;$/;" m class:doris::NewPartitionedAggregationNode +num_hash_collisions_ src/exec/new_partitioned_hash_table.h /^ int64_t num_hash_collisions_;$/;" m class:doris::NewPartitionedHashTable +num_index_entries src/olap/rowset/segment_group.cpp /^uint64_t SegmentGroup::num_index_entries() const {$/;" f class:doris::SegmentGroup +num_io_buffers src/runtime/row_batch.h /^ int num_io_buffers() const {$/;" f class:doris::RowBatch +num_items src/olap/short_key_index.h /^ uint32_t num_items() const {$/;" f class:doris::ShortKeyIndexDecoder +num_key_columns src/olap/schema.h /^ size_t num_key_columns() const {$/;" f class:doris::Schema +num_key_columns src/olap/tablet.h /^inline size_t Tablet::num_key_columns() const {$/;" f class:doris::Tablet +num_key_columns src/olap/tablet_schema.h /^ inline size_t num_key_columns() const { return _num_key_columns; }$/;" f class:doris::TabletSchema +num_local_disks src/runtime/disk_io_mgr.h /^ int num_local_disks() const { return num_total_disks() - num_remote_disks(); }$/;" f class:doris::DiskIoMgr +num_null_bytes src/runtime/descriptors.h /^ int num_null_bytes() const {$/;" f class:doris::RowDescriptor +num_null_bytes src/runtime/descriptors.h /^ int num_null_bytes() const {$/;" f class:doris::TupleDescriptor +num_null_columns src/olap/tablet.h /^inline size_t Tablet::num_null_columns() const {$/;" f class:doris::Tablet +num_null_columns src/olap/tablet_schema.h /^ inline size_t num_null_columns() const { return _num_null_columns; }$/;" f class:doris::TabletSchema +num_null_slots src/runtime/descriptors.h /^ int num_null_slots() const {$/;" f class:doris::RowDescriptor +num_null_slots src/runtime/descriptors.h /^ int num_null_slots() const {$/;" f class:doris::TupleDescriptor +num_of_file src/util/minizip/unzip.h /^ ZPOS64_T num_of_file; \/* # of file *\/$/;" m struct:unz64_file_pos_s +num_of_file src/util/minizip/unzip.h /^ uLong num_of_file; \/* # of file *\/$/;" m struct:unz_file_pos_s +num_optional_threads src/runtime/thread_resource_mgr.h /^ int num_optional_threads() const {$/;" f class:doris::ThreadResourceMgr::ResourcePool +num_output_rows_ready src/exec/analytic_eval_node.cpp /^inline int64_t AnalyticEvalNode::num_output_rows_ready() const {$/;" f class:doris::AnalyticEvalNode +num_pages src/olap/rowset/segment_v2/zone_map_index.h /^ int32_t num_pages() const { return _page_zone_maps.size(); }$/;" f class:doris::segment_v2::ZoneMapIndexReader +num_pages_ src/runtime/buffered_tuple_stream3.h /^ int64_t num_pages_;$/;" m class:doris::BufferedTupleStream3 +num_pages_ src/runtime/bufferpool/buffer_pool_internal.h /^ int64_t num_pages_;$/;" m class:doris::BufferPool::Client +num_passthrough_rows_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* num_passthrough_rows_;$/;" m class:doris::NewPartitionedAggregationNode +num_per_fragment_instances src/runtime/runtime_state.h /^ int num_per_fragment_instances() const {$/;" f class:doris::RuntimeState +num_pinned src/runtime/buffered_tuple_stream2.cc /^int num_pinned(const list& blocks) {$/;" f namespace:doris +num_pinned_buffers src/runtime/buffered_block_mgr2.cc /^int BufferedBlockMgr2::num_pinned_buffers(Client* client) const {$/;" f class:doris::BufferedBlockMgr2 +num_probes_ src/exec/new_partitioned_hash_table.h /^ int64_t num_probes_;$/;" m class:doris::NewPartitionedHashTable +num_remaining_ranges src/runtime/disk_io_mgr_internal.h /^ int num_remaining_ranges() const { return _num_remaining_ranges; }$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +num_remaining_ranges src/runtime/disk_io_mgr_internal.h /^ int& num_remaining_ranges() { return _num_remaining_ranges; }$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +num_remote_disks src/runtime/disk_io_mgr.h /^ int num_remote_disks() const { return REMOTE_NUM_DISKS; }$/;" f class:doris::DiskIoMgr +num_remote_ranges src/runtime/disk_io_mgr.cc /^int DiskIoMgr::num_remote_ranges(RequestContext* reader) const {$/;" f class:doris::DiskIoMgr +num_removes src/udf/udf_internal.h /^ int64_t num_removes() const {$/;" f class:doris::FunctionContextImpl +num_repartitions_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* num_repartitions_;$/;" m class:doris::NewPartitionedAggregationNode +num_required_bounds src/gutil/strings/numbers.h /^ uint32 num_required_bounds;$/;" m struct:DoubleRangeOptions +num_required_threads src/runtime/thread_resource_mgr.h /^ int num_required_threads() const {$/;" f class:doris::ThreadResourceMgr::ResourcePool +num_reserved_buffers_remaining src/runtime/buffered_block_mgr2.cc /^int BufferedBlockMgr2::num_reserved_buffers_remaining(Client* client) const {$/;" f class:doris::BufferedBlockMgr2 +num_resizes_ src/exec/new_partitioned_hash_table.h /^ int64_t num_resizes_;$/;" m class:doris::NewPartitionedHashTable +num_row_repartitioned_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* num_row_repartitioned_;$/;" m class:doris::NewPartitionedAggregationNode +num_rows src/olap/olap_index.h /^ const size_t num_rows() const {$/;" f class:doris::MemIndex +num_rows src/olap/olap_index.h /^ uint64_t num_rows;$/;" m struct:doris::OLAPIndexFixedHeader +num_rows src/olap/row_block2.h /^ size_t num_rows() const { return _num_rows; }$/;" f class:doris::RowBlockV2 +num_rows src/olap/rowset/column_data.h /^ int64_t num_rows() const { return _segment_group->num_rows(); }$/;" f class:doris::ColumnData +num_rows src/olap/rowset/rowset.h /^ size_t num_rows() const { return rowset_meta()->num_rows(); }$/;" f class:doris::Rowset +num_rows src/olap/rowset/rowset_meta.h /^ int64_t num_rows() const {$/;" f class:doris::RowsetMeta +num_rows src/olap/rowset/segment_group.h /^ int64_t num_rows() const {$/;" f class:doris::SegmentGroup +num_rows src/olap/rowset/segment_v2/parsed_page.h /^ ordinal_t num_rows = 0;$/;" m struct:doris::segment_v2::ParsedPage +num_rows src/olap/rowset/segment_v2/segment.h /^ uint32_t num_rows() const { return _footer.num_rows(); }$/;" f class:doris::segment_v2::Segment +num_rows src/olap/rowset/segment_v2/segment_iterator.h /^ uint32_t num_rows() const { return _segment->num_rows(); }$/;" f class:doris::segment_v2::SegmentIterator +num_rows src/olap/tablet.h /^inline size_t Tablet::num_rows() {$/;" f class:doris::Tablet +num_rows src/olap/tablet_meta.h /^inline size_t TabletMeta::num_rows() const {$/;" f class:doris::TabletMeta +num_rows src/runtime/buffered_block_mgr.h /^ int num_rows() const {$/;" f class:doris::BufferedBlockMgr::Block +num_rows src/runtime/buffered_block_mgr2.h /^ int num_rows() const {$/;" f class:doris::BufferedBlockMgr2::Block +num_rows src/runtime/buffered_tuple_stream.h /^ int64_t num_rows() const {$/;" f class:doris::BufferedTupleStream +num_rows src/runtime/buffered_tuple_stream2.h /^ int64_t num_rows() const { return _num_rows; }$/;" f class:doris::BufferedTupleStream2 +num_rows src/runtime/buffered_tuple_stream3.h /^ int num_rows;$/;" m struct:doris::BufferedTupleStream3::Page +num_rows src/runtime/buffered_tuple_stream3.h /^ int64_t num_rows() const { return num_rows_; }$/;" f class:doris::BufferedTupleStream3 +num_rows src/runtime/row_batch.h /^ int num_rows() const {$/;" f class:doris::RowBatch +num_rows src/runtime/vectorized_row_batch.h /^ inline int num_rows() {$/;" f class:doris::VectorizedRowBatch +num_rows_ src/runtime/buffered_tuple_stream3.h /^ int64_t num_rows_;$/;" m class:doris::BufferedTupleStream3 +num_rows_filtered src/exec/base_scanner.h /^ int64_t num_rows_filtered; \/\/ unqualified rows (unmatch the dest schema, or no partition)$/;" m struct:doris::ScannerCounter +num_rows_filtered src/exec/es_http_scanner.h /^ int64_t num_rows_filtered;$/;" m struct:doris::EsScanCounter +num_rows_in_block src/olap/rowset/segment_reader.h /^ uint32_t num_rows_in_block() {$/;" f class:doris::SegmentReader +num_rows_load_filtered src/runtime/runtime_state.h /^ int64_t num_rows_load_filtered() {$/;" f class:doris::RuntimeState +num_rows_load_success src/runtime/runtime_state.h /^ int64_t num_rows_load_success() {$/;" f class:doris::RuntimeState +num_rows_load_total src/runtime/runtime_state.h /^ int64_t num_rows_load_total() {$/;" f class:doris::RuntimeState +num_rows_load_unselected src/runtime/runtime_state.h /^ int64_t num_rows_load_unselected() {$/;" f class:doris::RuntimeState +num_rows_per_block src/olap/rowset/segment_v2/segment.h /^ uint32_t num_rows_per_block() const {$/;" f class:doris::segment_v2::Segment +num_rows_per_block src/olap/rowset/segment_v2/segment_writer.h /^ uint32_t num_rows_per_block = 1024;$/;" m struct:doris::segment_v2::SegmentWriterOptions +num_rows_per_block src/olap/short_key_index.h /^ uint32_t num_rows_per_block() const {$/;" f class:doris::ShortKeyIndexDecoder +num_rows_per_row_block src/olap/tablet.h /^inline size_t Tablet::num_rows_per_row_block() const {$/;" f class:doris::Tablet +num_rows_per_row_block src/olap/tablet_schema.h /^ inline size_t num_rows_per_row_block() const { return _num_rows_per_row_block; }$/;" f class:doris::TabletSchema +num_rows_received test/runtime/data_stream_test.cpp /^ int num_rows_received;$/;" m struct:doris::DataStreamTest::ReceiverInfo file: +num_rows_returned src/exec/es_http_scanner.h /^ int64_t num_rows_returned;$/;" m struct:doris::EsScanCounter +num_rows_unselected src/exec/base_scanner.h /^ int64_t num_rows_unselected; \/\/ rows filterd by predicates$/;" m struct:doris::ScannerCounter +num_rows_written src/olap/rowset/segment_v2/segment_writer.h /^ uint32_t num_rows_written() { return _row_count; }$/;" f class:doris::segment_v2::SegmentWriter +num_sampled src/util/runtime_profile.h /^ int64_t num_sampled; \/\/ number of samples taken$/;" m struct:doris::RuntimeProfile::BucketCountersInfo +num_sampled src/util/runtime_profile.h /^ int64_t num_sampled; \/\/ number of samples taken$/;" m struct:doris::RuntimeProfile::SamplingCounterInfo +num_scanner_threads src/runtime/runtime_state.h /^ int num_scanner_threads() const {$/;" f class:doris::RuntimeState +num_segments src/olap/rowset/column_data.h /^ uint32_t num_segments() const {$/;" f class:doris::ColumnData +num_segments src/olap/rowset/rowset.h /^ int64_t num_segments() const { return rowset_meta()->num_segments(); }$/;" f class:doris::Rowset +num_segments src/olap/rowset/rowset_meta.h /^ int64_t num_segments() const {$/;" f class:doris::RowsetMeta +num_segments src/olap/rowset/segment_group.h /^ inline int32_t num_segments() const { return _num_segments; }$/;" f class:doris::SegmentGroup +num_senders test/runtime/data_stream_test.cpp /^ int num_senders;$/;" m struct:doris::DataStreamTest::ReceiverInfo file: +num_short_key_columns src/olap/tablet.h /^inline size_t Tablet::num_short_key_columns() const {$/;" f class:doris::Tablet +num_short_key_columns src/olap/tablet_schema.h /^ inline size_t num_short_key_columns() const { return _num_short_key_columns; }$/;" f class:doris::TabletSchema +num_short_keys src/olap/rowset/segment_v2/segment.h /^ size_t num_short_keys() const { return _tablet_schema->num_short_key_columns(); }$/;" f class:doris::segment_v2::Segment +num_spilled_partitions_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* num_spilled_partitions_;$/;" m class:doris::NewPartitionedAggregationNode +num_symbols src/util/bfd_parser.h /^ long num_symbols() const {$/;" f class:doris::BfdParser +num_threads src/runtime/thread_resource_mgr.h /^ int64_t num_threads() const {$/;" f class:doris::ThreadResourceMgr::ResourcePool +num_threads src/util/threadpool.h /^ int num_threads() const {$/;" f class:doris::ThreadPool +num_threads_in_op src/runtime/disk_io_mgr_internal.h /^ int num_threads_in_op() const {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +num_total_disks src/runtime/disk_io_mgr.h /^ int num_total_disks() const { return _disk_queues.size(); }$/;" f class:doris::DiskIoMgr +num_tuple_streams src/runtime/row_batch.h /^ int num_tuple_streams() const {$/;" f class:doris::RowBatch +num_unstarted_ranges src/runtime/disk_io_mgr.cc /^int DiskIoMgr::num_unstarted_ranges(RequestContext* reader) const {$/;" f class:doris::DiskIoMgr +num_updates src/udf/udf_internal.h /^ int64_t num_updates() const { $/;" f class:doris::FunctionContextImpl +num_values src/olap/rowset/segment_v2/indexed_column_reader.h /^ int64_t num_values() const { return _num_values; }$/;" f class:doris::segment_v2::IndexedColumnReader +num_var_args test/udf/udf_test.cpp /^IntVal num_var_args(FunctionContext*, const BigIntVal& dummy, int n, const IntVal* args) {$/;" f namespace:doris_udf +numa_node_core_idx_ src/util/cpu_info.cpp /^vector CpuInfo::numa_node_core_idx_;$/;" m class:doris::CpuInfo file: +numa_node_core_idx_ src/util/cpu_info.h /^ static std::vector numa_node_core_idx_;$/;" m class:doris::CpuInfo +numa_node_to_cores_ src/util/cpu_info.cpp /^vector> CpuInfo::numa_node_to_cores_;$/;" m class:doris::CpuInfo file: +numa_node_to_cores_ src/util/cpu_info.h /^ static std::vector> numa_node_to_cores_;$/;" m class:doris::CpuInfo +number_entry src/util/minizip/unzip.h /^ ZPOS64_T number_entry; \/* total number of entries in$/;" m struct:unz_global_info64_s +number_entry src/util/minizip/unzip.h /^ uLong number_entry; \/* total number of entries in$/;" m struct:unz_global_info_s +number_filtered_rows src/runtime/stream_load/stream_load_context.h /^ int64_t number_filtered_rows = 0;$/;" m class:doris::StreamLoadContext +number_loaded_rows src/runtime/stream_load/stream_load_context.h /^ int64_t number_loaded_rows = 0;$/;" m class:doris::StreamLoadContext +number_total_rows src/runtime/stream_load/stream_load_context.h /^ int64_t number_total_rows = 0;$/;" m class:doris::StreamLoadContext +number_unselected_rows src/runtime/stream_load/stream_load_context.h /^ int64_t number_unselected_rows = 0;$/;" m class:doris::StreamLoadContext +numeric_limits src/util/string_parser.cpp /^__int128 StringParser::numeric_limits<__int128>(bool negative) {$/;" f class:doris::StringParser +numeric_limits src/util/string_parser.hpp /^T StringParser::numeric_limits(bool negative) {$/;" f class:doris::StringParser +numeric_limits_max src/util/frame_of_reference_coding.cpp /^const T ForEncoder::numeric_limits_max() {$/;" f class:doris::ForEncoder +numeric_limits_max src/util/frame_of_reference_coding.cpp /^const uint24_t ForEncoder::numeric_limits_max() {$/;" f class:doris::ForEncoder +obj_pool src/runtime/plan_fragment_executor.h /^ ObjectPool* obj_pool() {$/;" f class:doris::PlanFragmentExecutor +obj_pool src/runtime/runtime_state.h /^ ObjectPool* obj_pool() const {$/;" f class:doris::RuntimeState +obj_pool_ptr src/runtime/runtime_state.h /^ std::shared_ptr obj_pool_ptr() const {$/;" f class:doris::RuntimeState +object_pool test/exprs/binary_predicate_test.cpp /^ ObjectPool* object_pool() {$/;" f class:doris::BinaryOpTest +obtain_base_compaction_lock src/olap/tablet.h /^ inline void obtain_base_compaction_lock() { _base_lock.lock(); }$/;" f class:doris::Tablet +obtain_cumulative_lock src/olap/tablet.h /^ inline void obtain_cumulative_lock() { _cumulative_lock.lock(); }$/;" f class:doris::Tablet +obtain_header_rdlock src/olap/tablet.h /^ inline void obtain_header_rdlock() { _meta_lock.rdlock(); }$/;" f class:doris::Tablet +obtain_header_wrlock src/olap/tablet.h /^ inline void obtain_header_wrlock() { _meta_lock.wrlock(); }$/;" f class:doris::Tablet +obtain_push_lock src/olap/tablet.h /^ inline void obtain_push_lock() { _ingest_lock.lock(); }$/;" f class:doris::Tablet +obtain_shard_path src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::obtain_shard_path($/;" f class:doris::StorageEngine +offer src/util/batch_process_thread_pool.hpp /^ bool offer(T task) {$/;" f class:doris::BatchProcessThreadPool +offer src/util/priority_thread_pool.hpp /^ bool offer(Task task) {$/;" f class:doris::PriorityThreadPool +offer src/util/priority_thread_pool.hpp /^ bool offer(WorkFunction func) {$/;" f class:doris::PriorityThreadPool +offset src/olap/file_stream.h /^ size_t offset() const { return _offset; }$/;" f class:doris::ReadOnlyFileStream::FileCursor +offset src/olap/olap_index.h /^ iterator_offset_t offset;$/;" m struct:doris::OLAPIndexOffset +offset src/olap/page_cache.h /^ int64_t offset;$/;" m struct:doris::StoragePageCache::CacheKey +offset src/olap/rowset/segment_reader.h /^ int64_t offset;$/;" m struct:doris::SegmentReader::DiskRange +offset src/olap/rowset/segment_v2/binary_plain_page.h /^ uint32_t offset(int idx) const {$/;" f class:doris::segment_v2::BinaryPlainPageDecoder +offset src/olap/rowset/segment_v2/page_pointer.h /^ uint64_t offset;$/;" m struct:doris::segment_v2::PagePointer +offset src/runtime/buffered_tuple_stream.h /^ uint64_t offset() const {$/;" f struct:doris::BufferedTupleStream::RowIdx +offset src/runtime/buffered_tuple_stream2.h /^ uint64_t offset() const {$/;" f struct:doris::BufferedTupleStream2::RowIdx +offset src/runtime/disk_io_mgr.h /^ int64_t offset() const { return _offset; }$/;" f class:doris::DiskIoMgr::RequestRange +offset src/runtime/external_scan_context_mgr.h /^ int64_t offset;$/;" m struct:doris::ScanContext +offset_ src/util/bitmap.h /^ size_t offset_;$/;" m class:doris::BitmapIterator +offset_central_dir src/util/minizip/unzip.c /^ ZPOS64_T offset_central_dir; \/* offset of start of central directory with$/;" m struct:__anon34 file: +offset_curfile src/util/minizip/unzip.c /^ ZPOS64_T offset_curfile;\/* relative offset of local header 8 bytes *\/$/;" m struct:unz_file_info64_internal_s file: +offset_fn_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::offset_fn_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +offset_fn_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::offset_fn_init(FunctionContext* ctx, T* dst) {$/;" f class:doris::AggregateFunctions +offset_fn_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::offset_fn_update(FunctionContext* ctx, const IntVal& src,$/;" f class:doris::AggregateFunctions +offset_fn_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::offset_fn_update(FunctionContext* ctx, const T& src,$/;" f class:doris::AggregateFunctions +offset_in_page src/olap/rowset/segment_v2/parsed_page.h /^ ordinal_t offset_in_page = 0;$/;" m struct:doris::segment_v2::ParsedPage +offset_local_extrafield src/util/minizip/unzip.c /^ ZPOS64_T offset_local_extrafield;\/* offset of the local extra field *\/$/;" m struct:__anon33 file: +offset_position src/olap/rowset/segment_reader.h /^ uint32_t offset_position;$/;" m struct:doris::SegmentReader::VectorizedPositionInfo +ok src/common/status.h /^ bool ok() const { return _state == nullptr; }$/;" f class:doris::Status +ok src/runtime/disk_io_mgr.h /^ bool ok() const { return _hdfs_file != NULL; }$/;" f class:doris::DiskIoMgr::HdfsCachedFileHandle +olap_adler32 src/olap/utils.cpp /^uint32_t olap_adler32(uint32_t adler, const char* buf, size_t len) {$/;" f namespace:doris +olap_compress src/olap/utils.cpp /^OLAPStatus olap_compress(const char* src_buf,$/;" f namespace:doris +olap_crc32 src/olap/utils.cpp /^uint32_t olap_crc32(uint32_t crc32, const char* buf, size_t len) {$/;" f namespace:doris +olap_decompress src/olap/utils.cpp /^OLAPStatus olap_decompress(const char* src_buf,$/;" f namespace:doris +on_chunk_data src/http/action/mini_load.cpp /^void MiniLoadAction::on_chunk_data(HttpRequest* http_req) {$/;" f class:doris::MiniLoadAction +on_chunk_data src/http/action/stream_load.cpp /^void StreamLoadAction::on_chunk_data(HttpRequest* req) {$/;" f class:doris::StreamLoadAction +on_chunk_data src/http/http_handler.h /^ virtual void on_chunk_data(HttpRequest* req) { }$/;" f class:doris::HttpHandler +on_chunked src/http/ev_http_server.cpp /^static void on_chunked(struct evhttp_request* ev_req, void* param) {$/;" f namespace:doris +on_close src/olap/rowset/rowset.h /^ OLAPStatus on_close(uint64_t refs_by_reader) {$/;" f class:doris::RowsetStateMachine +on_close src/runtime/buffer_control_block.cpp /^void GetResultBatchCtx::on_close(int64_t packet_seq,$/;" f class:doris::GetResultBatchCtx +on_connection src/http/ev_http_server.cpp /^static int on_connection(struct evhttp_request* req, void* param) {$/;" f namespace:doris +on_data src/runtime/buffer_control_block.cpp /^void GetResultBatchCtx::on_data(TFetchDataResult* t_result, int64_t packet_seq, bool eos) {$/;" f class:doris::GetResultBatchCtx +on_failure src/runtime/buffer_control_block.cpp /^void GetResultBatchCtx::on_failure(const Status& status) {$/;" f class:doris::GetResultBatchCtx +on_free src/http/ev_http_server.cpp /^static void on_free(struct evhttp_request* ev_req, void* arg) {$/;" f namespace:doris +on_header src/http/action/mini_load.cpp /^int MiniLoadAction::on_header(HttpRequest* req) {$/;" f class:doris::MiniLoadAction +on_header src/http/action/stream_load.cpp /^int StreamLoadAction::on_header(HttpRequest* req) {$/;" f class:doris::StreamLoadAction +on_header src/http/ev_http_server.cpp /^int EvHttpServer::on_header(struct evhttp_request* ev_req) {$/;" f class:doris::EvHttpServer +on_header src/http/ev_http_server.cpp /^static int on_header(struct evhttp_request* ev_req, void* param) {$/;" f namespace:doris +on_header src/http/http_handler.h /^ virtual int on_header(HttpRequest* req) { return 0; }$/;" f class:doris::HttpHandler +on_load src/olap/rowset/rowset.h /^ OLAPStatus on_load() {$/;" f class:doris::RowsetStateMachine +on_release src/olap/rowset/rowset.h /^ OLAPStatus on_release() {$/;" f class:doris::RowsetStateMachine +on_request src/http/ev_http_server.cpp /^static void on_request(struct evhttp_request *ev_req, void *arg) {$/;" f namespace:doris +on_response_data src/http/http_client.cpp /^size_t HttpClient::on_response_data(const void* data, size_t length) {$/;" f class:doris::HttpClient +once src/util/thread.cpp /^static GoogleOnceType once = GOOGLE_ONCE_INIT;$/;" m namespace:doris file: +op src/exec/es/es_predicate.h /^ TExprOpcode::type op;$/;" m struct:doris::ExtBinaryPredicate +op src/exprs/expr.h /^ TExprOpcode::type op() const {$/;" f class:doris::Expr +op src/olap/olap_cond.h /^ CondOp op;$/;" m struct:doris::Cond +op_type src/olap/tablet_sync_service.h /^ MetaOpType op_type;$/;" m struct:doris::PushRowsetMetaTask +opaque src/util/minizip/ioapi.h /^ voidpf opaque;$/;" m struct:zlib_filefunc64_def_s +opaque src/util/minizip/ioapi.h /^ voidpf opaque;$/;" m struct:zlib_filefunc_def_s +open src/exec/aggregation_node.cpp /^Status AggregationNode::open(RuntimeState* state) {$/;" f class:doris::AggregationNode +open src/exec/analytic_eval_node.cpp /^Status AnalyticEvalNode::open(RuntimeState* state) {$/;" f class:doris::AnalyticEvalNode +open src/exec/assert_num_rows_node.cpp /^Status AssertNumRowsNode::open(RuntimeState* state) {$/;" f class:doris::AssertNumRowsNode +open src/exec/base_scanner.cpp /^Status BaseScanner::open() {$/;" f class:doris::BaseScanner +open src/exec/blocking_join_node.cpp /^Status BlockingJoinNode::open(RuntimeState* state) {$/;" f class:doris::BlockingJoinNode +open src/exec/broker_reader.cpp /^Status BrokerReader::open() {$/;" f class:doris::BrokerReader +open src/exec/broker_scan_node.cpp /^Status BrokerScanNode::open(RuntimeState* state) {$/;" f class:doris::BrokerScanNode +open src/exec/broker_scanner.cpp /^Status BrokerScanner::open() {$/;" f class:doris::BrokerScanner +open src/exec/broker_writer.cpp /^Status BrokerWriter::open() {$/;" f class:doris::BrokerWriter +open src/exec/csv_scan_node.cpp /^Status CsvScanNode::open(RuntimeState* state) {$/;" f class:doris::CsvScanNode +open src/exec/csv_scanner.cpp /^ Status CsvScanner::open() {$/;" f class:doris::CsvScanner +open src/exec/es/es_scan_reader.cpp /^Status ESScanReader::open() {$/;" f class:doris::ESScanReader +open src/exec/es_http_scan_node.cpp /^Status EsHttpScanNode::open(RuntimeState* state) {$/;" f class:doris::EsHttpScanNode +open src/exec/es_http_scanner.cpp /^Status EsHttpScanner::open() {$/;" f class:doris::EsHttpScanner +open src/exec/es_scan_node.cpp /^Status EsScanNode::open(RuntimeState* state) {$/;" f class:doris::EsScanNode +open src/exec/exchange_node.cpp /^Status ExchangeNode::open(RuntimeState* state) {$/;" f class:doris::ExchangeNode +open src/exec/exec_node.cpp /^Status ExecNode::open(RuntimeState* state) {$/;" f class:doris::ExecNode +open src/exec/hash_join_node.cpp /^Status HashJoinNode::open(RuntimeState* state) {$/;" f class:doris::HashJoinNode +open src/exec/intersect_node.cpp /^Status IntersectNode::open(RuntimeState* state) {$/;" f class:doris::IntersectNode +open src/exec/kudu_scan_node.cpp /^Status KuduScanNode::open(RuntimeState* state) {$/;" f class:doris::KuduScanNode +open src/exec/kudu_scanner.cpp /^Status KuduScanner::open() {$/;" f class:doris::KuduScanner +open src/exec/local_file_reader.cpp /^Status LocalFileReader::open() {$/;" f class:doris::LocalFileReader +open src/exec/local_file_writer.cpp /^Status LocalFileWriter::open() {$/;" f class:doris::LocalFileWriter +open src/exec/merge_join_node.cpp /^Status MergeJoinNode::open(RuntimeState* state) {$/;" f class:doris::MergeJoinNode +open src/exec/merge_node.cpp /^Status MergeNode::open(RuntimeState* state) {$/;" f class:doris::MergeNode +open src/exec/mysql_scan_node.cpp /^Status MysqlScanNode::open(RuntimeState* state) {$/;" f class:doris::MysqlScanNode +open src/exec/mysql_scanner.cpp /^Status MysqlScanner::open() {$/;" f class:doris::MysqlScanner +open src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::open(RuntimeState* state) {$/;" f class:doris::NewPartitionedAggregationNode +open src/exec/olap_rewrite_node.cpp /^Status OlapRewriteNode::open(RuntimeState* state) {$/;" f class:doris::OlapRewriteNode +open src/exec/olap_scan_node.cpp /^Status OlapScanNode::open(RuntimeState* state) {$/;" f class:doris::OlapScanNode +open src/exec/olap_scanner.cpp /^Status OlapScanner::open() {$/;" f class:doris::OlapScanner +open src/exec/orc_scanner.cpp /^Status ORCScanner::open() {$/;" f class:doris::ORCScanner +open src/exec/parquet_scanner.cpp /^Status ParquetScanner::open() {$/;" f class:doris::ParquetScanner +open src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::open(RuntimeState* state) {$/;" f class:doris::PartitionedAggregationNode +open src/exec/pl_task_root.cpp /^Status ExchangeNode::open(RuntimeState* state) {$/;" f class:doris::ExchangeNode +open src/exec/pre_aggregation_node.cpp /^Status PreAggregationNode::open(RuntimeState* state) {$/;" f class:doris::PreAggregationNode +open src/exec/repeat_node.cpp /^Status RepeatNode::open(RuntimeState* state) {$/;" f class:doris::RepeatNode +open src/exec/schema_scan_node.cpp /^Status SchemaScanNode::open(RuntimeState* state) {$/;" f class:doris::SchemaScanNode +open src/exec/select_node.cpp /^Status SelectNode::open(RuntimeState* state) {$/;" f class:doris::SelectNode +open src/exec/sort_exec_exprs.cpp /^Status SortExecExprs::open(RuntimeState* state) {$/;" f class:doris::SortExecExprs +open src/exec/sort_node.cpp /^Status SortNode::open(RuntimeState* state) {$/;" f class:doris::SortNode +open src/exec/spill_sort_node.cc /^Status SpillSortNode::open(RuntimeState* state) {$/;" f class:doris::SpillSortNode +open src/exec/tablet_sink.cpp /^Status IndexChannel::open() {$/;" f class:doris::stream_load::IndexChannel +open src/exec/tablet_sink.cpp /^Status OlapTableSink::open(RuntimeState* state) {$/;" f class:doris::stream_load::OlapTableSink +open src/exec/tablet_sink.cpp /^void NodeChannel::open() {$/;" f class:doris::stream_load::NodeChannel +open src/exec/topn_node.cpp /^Status TopNNode::open(RuntimeState* state) {$/;" f class:doris::TopNNode +open src/exec/union_node.cpp /^Status UnionNode::open(RuntimeState* state) {$/;" f class:doris::UnionNode +open src/exprs/agg_fn_evaluator.cpp /^Status AggFnEvaluator::open(RuntimeState* state, FunctionContext* agg_fn_ctx) {$/;" f class:doris::AggFnEvaluator +open src/exprs/case_expr.cpp /^Status CaseExpr::open($/;" f class:doris::CaseExpr +open src/exprs/expr.cpp /^Status Expr::open($/;" f class:doris::Expr +open src/exprs/expr.cpp /^Status Expr::open(const std::vector& ctxs, RuntimeState* state) {$/;" f class:doris::Expr +open src/exprs/expr.h /^ Status open(RuntimeState* state,$/;" f class:doris::Expr +open src/exprs/expr_context.cpp /^Status ExprContext::open(RuntimeState* state) {$/;" f class:doris::ExprContext +open src/exprs/expr_context.cpp /^Status ExprContext::open(std::vector evals, RuntimeState* state) {$/;" f class:doris::ExprContext +open src/exprs/in_predicate.cpp /^Status InPredicate::open($/;" f class:doris::InPredicate +open src/exprs/scalar_fn_call.cpp /^Status ScalarFnCall::open($/;" f class:doris::ScalarFnCall +open src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(open)(const char* p, int f, int m) {$/;" f +open src/olap/delta_writer.cpp /^OLAPStatus DeltaWriter::open(WriteRequest* req, MemTracker* mem_tracker, DeltaWriter** writer) {$/;" f class:doris::DeltaWriter +open src/olap/file_helper.cpp /^OLAPStatus FileHandler::open(const string& file_name, int flag) {$/;" f class:doris::FileHandler +open src/olap/file_helper.cpp /^OLAPStatus FileHandlerWithBuf::open(const string& file_name, const char* mode) {$/;" f class:doris::FileHandlerWithBuf +open src/olap/fs/file_block_manager.cpp /^Status FileBlockManager::open() {$/;" f class:doris::fs::FileBlockManager +open src/olap/rowset/segment_v2/segment.cpp /^Status Segment::open(std::string filename,$/;" f class:doris::segment_v2::Segment +open src/olap/storage_engine.cpp /^Status StorageEngine::open(const EngineOptions& options, StorageEngine** engine_ptr) {$/;" f class:doris::StorageEngine +open src/runtime/data_spliter.cpp /^Status DataSpliter::open(RuntimeState* state) {$/;" f class:doris::DataSpliter +open src/runtime/data_stream_sender.cpp /^Status DataStreamSender::open(RuntimeState* state) {$/;" f class:doris::DataStreamSender +open src/runtime/disk_io_mgr_scan_range.cc /^Status DiskIoMgr::ScanRange::open() {$/;" f class:doris::DiskIoMgr::ScanRange +open src/runtime/dpp_sink_internal.cpp /^Status PartitionInfo::open(RuntimeState* state) {$/;" f class:doris::PartitionInfo +open src/runtime/dpp_sink_internal.cpp /^Status RollupSchema::open(RuntimeState* state) {$/;" f class:doris::RollupSchema +open src/runtime/dpp_writer.cpp /^Status DppWriter::open() {$/;" f class:doris::DppWriter +open src/runtime/export_sink.cpp /^Status ExportSink::open(RuntimeState* state) {$/;" f class:doris::ExportSink +open src/runtime/load_channel.cpp /^Status LoadChannel::open(const PTabletWriterOpenRequest& params) {$/;" f class:doris::LoadChannel +open src/runtime/load_channel_mgr.cpp /^Status LoadChannelMgr::open(const PTabletWriterOpenRequest& params) {$/;" f class:doris::LoadChannelMgr +open src/runtime/memory_scratch_sink.cpp /^Status MemoryScratchSink::open(RuntimeState* state) {$/;" f class:doris::MemoryScratchSink +open src/runtime/message_body_sink.cpp /^Status MessageBodyFileSink::open() {$/;" f class:doris::MessageBodyFileSink +open src/runtime/mysql_table_sink.cpp /^Status MysqlTableSink::open(RuntimeState* state) {$/;" f class:doris::MysqlTableSink +open src/runtime/mysql_table_writer.cpp /^Status MysqlTableWriter::open(const MysqlConnInfo& conn_info, const std::string& tbl) {$/;" f class:doris::MysqlTableWriter +open src/runtime/plan_fragment_executor.cpp /^Status PlanFragmentExecutor::open() {$/;" f class:doris::PlanFragmentExecutor +open src/runtime/result_sink.cpp /^Status ResultSink::open(RuntimeState* state) {$/;" f class:doris::ResultSink +open src/runtime/tablets_channel.cpp /^Status TabletsChannel::open(const PTabletWriterOpenRequest& params) {$/;" f class:doris::TabletsChannel +open src/util/thrift_client.cpp /^Status ThriftClientImpl::open() {$/;" f class:doris::ThriftClientImpl +open test/runtime/fragment_mgr_test.cpp /^Status PlanFragmentExecutor::open() {$/;" f class:doris::PlanFragmentExecutor +open test/runtime/load_channel_mgr_test.cpp /^OLAPStatus DeltaWriter::open(WriteRequest* req, MemTracker* mem_tracker, DeltaWriter** writer) {$/;" f class:doris::DeltaWriter +open_bfd src/util/bfd_parser.cpp /^int BfdParser::open_bfd() {$/;" f class:doris::BfdParser +open_block src/olap/fs/file_block_manager.cpp /^Status FileBlockManager::open_block(const std::string& path, unique_ptr* block) {$/;" f class:doris::fs::FileBlockManager +open_es src/exec/es_scan_node.cpp /^Status EsScanNode::open_es(TNetworkAddress& address, TExtOpenResult& result, TExtOpenParams& params) {$/;" f class:doris::EsScanNode +open_file src/util/file_manager.cpp /^Status FileManager::open_file(const std::string& file_name, OpenedFileHandle* file_handle) {$/;" f class:doris::FileManager +open_file_for_random src/env/env_util.cpp /^Status open_file_for_random(Env *env, const string &path, shared_ptr *file) {$/;" f namespace:doris::env_util +open_file_for_write src/env/env_util.cpp /^Status open_file_for_write(Env* env, const string& path, shared_ptr* file) {$/;" f namespace:doris::env_util +open_file_for_write src/env/env_util.cpp /^Status open_file_for_write(const WritableFileOptions& opts,$/;" f namespace:doris::env_util +open_file_reader src/exec/broker_scanner.cpp /^Status BrokerScanner::open_file_reader() {$/;" f class:doris::BrokerScanner +open_file_writer src/runtime/export_sink.cpp /^Status ExportSink::open_file_writer() {$/;" f class:doris::ExportSink +open_internal src/runtime/plan_fragment_executor.cpp /^Status PlanFragmentExecutor::open_internal() {$/;" f class:doris::PlanFragmentExecutor +open_line_reader src/exec/broker_scanner.cpp /^Status BrokerScanner::open_line_reader() {$/;" f class:doris::BrokerScanner +open_mysql_conn src/util/mysql_load_error_hub.cpp /^Status MysqlLoadErrorHub::open_mysql_conn(MYSQL** my_conn) {$/;" f class:doris::MysqlLoadErrorHub +open_next_reader src/exec/broker_scanner.cpp /^Status BrokerScanner::open_next_reader() {$/;" f class:doris::BrokerScanner +open_next_reader src/exec/orc_scanner.cpp /^Status ORCScanner::open_next_reader() {$/;" f class:doris::ORCScanner +open_next_reader src/exec/parquet_scanner.cpp /^Status ParquetScanner::open_next_reader() {$/;" f class:doris::ParquetScanner +open_next_scan_token src/exec/kudu_scanner.cpp /^Status KuduScanner::open_next_scan_token(const string& scan_token) {$/;" f class:doris::KuduScanner +open_scanner src/service/backend_service.cpp /^void BackendService::open_scanner(TScanOpenResult& result_, const TScanOpenParams& params) {$/;" f class:doris::BackendService +open_status test/runtime/load_channel_mgr_test.cpp /^OLAPStatus open_status;$/;" m namespace:doris file: +open_wait src/exec/tablet_sink.cpp /^Status NodeChannel::open_wait() {$/;" f class:doris::stream_load::NodeChannel +open_with_cache src/olap/file_helper.cpp /^OLAPStatus FileHandler::open_with_cache(const string& file_name, int flag) {$/;" f class:doris::FileHandler +open_with_mode src/olap/file_helper.cpp /^OLAPStatus FileHandler::open_with_mode(const string& file_name, int flag, int mode) {$/;" f class:doris::FileHandler +open_with_mode src/olap/file_helper.cpp /^OLAPStatus FileHandlerWithBuf::open_with_mode(const string& file_name, const char* mode) {$/;" f class:doris::FileHandlerWithBuf +open_with_retry src/util/thrift_client.cpp /^Status ThriftClientImpl::open_with_retry(int num_tries, int wait_ms) {$/;" f class:doris::ThriftClientImpl +opened src/exprs/expr_context.h /^ bool opened() {$/;" f class:doris::ExprContext +opened_ src/exprs/new_agg_fn_evaluator.h /^ bool opened_ = false;$/;" m class:doris::NewAggFnEvaluator +operand_field src/olap/olap_cond.h /^ WrapperField* operand_field;$/;" m struct:doris::Cond +operand_set src/olap/olap_cond.h /^ FieldSet operand_set;$/;" m struct:doris::Cond +operator ! src/gutil/int128.h /^inline bool operator!(const uint128& val) {$/;" f +operator != output/udf/include/udf.h /^ bool operator!=(const BigIntVal& other) const {$/;" f struct:doris_udf::BigIntVal +operator != output/udf/include/udf.h /^ bool operator!=(const BooleanVal& other) const {$/;" f struct:doris_udf::BooleanVal +operator != output/udf/include/udf.h /^ bool operator!=(const DateTimeVal& other) const {$/;" f struct:doris_udf::DateTimeVal +operator != output/udf/include/udf.h /^ bool operator!=(const DecimalV2Val& other) const {$/;" f struct:doris_udf::DecimalV2Val +operator != output/udf/include/udf.h /^ bool operator!=(const DecimalVal& other) const {$/;" f struct:doris_udf::DecimalVal +operator != output/udf/include/udf.h /^ bool operator!=(const DoubleVal& other) const {$/;" f struct:doris_udf::DoubleVal +operator != output/udf/include/udf.h /^ bool operator!=(const FloatVal& other) const {$/;" f struct:doris_udf::FloatVal +operator != output/udf/include/udf.h /^ bool operator!=(const IntVal& other) const {$/;" f struct:doris_udf::IntVal +operator != output/udf/include/udf.h /^ bool operator!=(const LargeIntVal& other) const {$/;" f struct:doris_udf::LargeIntVal +operator != output/udf/include/udf.h /^ bool operator!=(const SmallIntVal& other) const {$/;" f struct:doris_udf::SmallIntVal +operator != output/udf/include/udf.h /^ bool operator!=(const StringVal& other) const {$/;" f struct:doris_udf::StringVal +operator != output/udf/include/udf.h /^ bool operator!=(const TinyIntVal& other) const {$/;" f struct:doris_udf::TinyIntVal +operator != src/exec/hash_table.h /^ bool operator!=(const Iterator& rhs) {$/;" f class:doris::HashTable::Iterator +operator != src/gutil/gscoped_ptr.h /^ bool operator!=(C* p) const { return array_ != p; }$/;" f class:gscoped_array +operator != src/gutil/gscoped_ptr.h /^ bool operator!=(C* p) const {$/;" f class:gscoped_ptr_malloc +operator != src/gutil/gscoped_ptr.h /^ bool operator!=(const element_type* p) const { return impl_.get() != p; }$/;" f class:gscoped_ptr +operator != src/gutil/gscoped_ptr.h /^ bool operator!=(element_type* array) const { return impl_.get() != array; }$/;" f class:gscoped_ptr +operator != src/gutil/gscoped_ptr.h /^bool operator!=(C* p, const gscoped_ptr_malloc& b) {$/;" f +operator != src/gutil/gscoped_ptr.h /^bool operator!=(C* p1, const gscoped_array& p2) {$/;" f +operator != src/gutil/gscoped_ptr.h /^bool operator!=(T* p1, const gscoped_ptr& p2) {$/;" f +operator != src/gutil/int128.h /^inline bool operator!=(const uint128& lhs, const uint128& rhs) {$/;" f +operator != src/gutil/strings/split_internal.h /^ bool operator!=(const SplitIterator& other) const {$/;" f class:strings::internal::SplitIterator +operator != src/gutil/strings/stringpiece.h /^inline bool operator!=(StringPiece x, StringPiece y) {$/;" f +operator != src/olap/decimal12.h /^ bool operator!=(const decimal12_t& value) const {$/;" f struct:doris::decimal12_t +operator != src/olap/fs/block_id.h /^ bool operator!=(const BlockId& other) const {$/;" f class:doris::BlockId +operator != src/olap/lru_cache.h /^ inline bool operator!=(const CacheKey& other) const {$/;" f class:doris::CacheKey +operator != src/olap/olap_common.h /^ bool operator!=(const RowsetId& rhs) const {$/;" f struct:doris::RowsetId +operator != src/olap/olap_common.h /^ bool operator!=(const Version& rhs) const {$/;" f struct:doris::Version +operator != src/olap/olap_index.h /^ bool operator!=(const RowBlockPosition& other) const {$/;" f struct:doris::RowBlockPosition +operator != src/olap/rowset/segment_v2/page_pointer.h /^ bool operator!=(const PagePointer& other) const {$/;" f struct:doris::segment_v2::PagePointer +operator != src/olap/short_key_index.h /^ bool operator!=(const ShortKeyIndexIterator& other) {$/;" f class:doris::ShortKeyIndexIterator +operator != src/olap/uint24.h /^ bool operator!=(const uint24_t& value) const {$/;" f struct:doris::uint24_t +operator != src/olap/utils.h /^ bool operator!=(const BinarySearchIterator& iterator) {$/;" f class:doris::BinarySearchIterator +operator != src/runtime/datetime_value.h /^ bool operator!=(const DateTimeValue& other) const {$/;" f class:doris::DateTimeValue +operator != src/runtime/decimal_value.h /^ bool operator!=(const DecimalValue& other) const {$/;" f class:doris::DecimalValue +operator != src/runtime/decimalv2_value.h /^ bool operator!=(const DecimalV2Value& other) const {$/;" f class:doris::DecimalV2Value +operator != src/runtime/dpp_sink_internal.h /^ bool operator!=(const PartRangeKey& other) const {$/;" f class:doris::PartRangeKey +operator != src/runtime/string_value.h /^ bool operator!=(const StringValue& other) const {$/;" f struct:doris::StringValue +operator != src/runtime/types.h /^ bool operator!=(const TypeDescriptor& other) const {$/;" f struct:doris::TypeDescriptor +operator != src/udf/udf.h /^ bool operator!=(const BigIntVal& other) const {$/;" f struct:doris_udf::BigIntVal +operator != src/udf/udf.h /^ bool operator!=(const BooleanVal& other) const {$/;" f struct:doris_udf::BooleanVal +operator != src/udf/udf.h /^ bool operator!=(const DateTimeVal& other) const {$/;" f struct:doris_udf::DateTimeVal +operator != src/udf/udf.h /^ bool operator!=(const DecimalV2Val& other) const {$/;" f struct:doris_udf::DecimalV2Val +operator != src/udf/udf.h /^ bool operator!=(const DecimalVal& other) const {$/;" f struct:doris_udf::DecimalVal +operator != src/udf/udf.h /^ bool operator!=(const DoubleVal& other) const {$/;" f struct:doris_udf::DoubleVal +operator != src/udf/udf.h /^ bool operator!=(const FloatVal& other) const {$/;" f struct:doris_udf::FloatVal +operator != src/udf/udf.h /^ bool operator!=(const IntVal& other) const {$/;" f struct:doris_udf::IntVal +operator != src/udf/udf.h /^ bool operator!=(const LargeIntVal& other) const {$/;" f struct:doris_udf::LargeIntVal +operator != src/udf/udf.h /^ bool operator!=(const SmallIntVal& other) const {$/;" f struct:doris_udf::SmallIntVal +operator != src/udf/udf.h /^ bool operator!=(const StringVal& other) const {$/;" f struct:doris_udf::StringVal +operator != src/udf/udf.h /^ bool operator!=(const TinyIntVal& other) const {$/;" f struct:doris_udf::TinyIntVal +operator != src/util/bitmap_value.h /^ bool operator!=(const Roaring64MapSetBitForwardIterator& o) {$/;" f class:doris::detail::final +operator != src/util/lru_cache.hpp /^ bool operator!=(const Iterator& rhs) const {$/;" f class:doris::LruCache::Iterator +operator != src/util/metrics.h /^ bool operator!=(const MetricLabel& other) const {$/;" f struct:doris::MetricLabel +operator != src/util/monotime.cpp /^bool operator!=(const MonoDelta &lhs, const MonoDelta &rhs) {$/;" f namespace:doris +operator != src/util/monotime.cpp /^bool operator!=(const MonoTime& lhs, const MonoTime& rhs) {$/;" f namespace:doris +operator != src/util/slice.h /^inline bool operator!=(const Slice& x, const Slice& y) {$/;" f namespace:doris +operator != src/util/uid_util.h /^ bool operator!=(const UniqueId& rhs) const {$/;" f struct:doris::UniqueId +operator % src/runtime/decimal_value.cpp /^DecimalValue operator%(const DecimalValue& v1, const DecimalValue& v2){$/;" f namespace:doris +operator % src/runtime/decimalv2_value.cpp /^DecimalV2Value operator%(const DecimalV2Value& v1, const DecimalV2Value& v2){$/;" f namespace:doris +operator & src/util/bitmap_value.h /^ Roaring64Map operator&(const Roaring64Map& o) const { return Roaring64Map(*this) &= o; }$/;" f class:doris::detail::Roaring64Map +operator &= src/common/atomic.h /^ AtomicInt& operator&=(T v) {$/;" f class:doris::AtomicInt +operator &= src/util/bitmap_value.h /^ BitmapValue& operator&=(const BitmapValue& rhs) {$/;" f class:doris::BitmapValue +operator &= src/util/bitmap_value.h /^ Roaring64Map& operator&=(const Roaring64Map& r) {$/;" f class:doris::detail::Roaring64Map +operator () src/exec/olap_scan_node.h /^ Status operator()(T& v) {$/;" f class:doris::OlapScanNode::ExtendScanKeyVisitor +operator () src/exec/olap_scan_node.h /^ bool operator()(T& v) const {$/;" f class:doris::OlapScanNode::IsFixedValueRangeVisitor +operator () src/exec/olap_scan_node.h /^ bool operator()(const HeapType& lhs, const HeapType& rhs) const {$/;" f class:doris::OlapScanNode::MergeComparison +operator () src/exec/olap_scan_node.h /^ size_t operator()(T& v) const {$/;" f class:doris::OlapScanNode::GetFixedValueSizeVisitor +operator () src/exec/olap_scan_node.h /^ std::string operator()(T& v, P& v2) const {$/;" f class:doris::OlapScanNode::ToOlapFilterVisitor +operator () src/exec/tablet_info.h /^ bool operator()(const Tuple* lhs, const Tuple* rhs) const {$/;" f class:doris::OlapTablePartKeyComparator +operator () src/exprs/aggregate_functions.cpp /^ size_t operator()(const DateTimeVal& obj) const {$/;" f class:doris::MultiDistinctCountDateState::DateTimeHashHelper +operator () src/exprs/aggregate_functions.cpp /^ size_t operator()(const T& obj) const {$/;" f class:doris::MultiDistinctNumericState::NumericHashHelper +operator () src/gutil/gscoped_ptr.h /^ inline void operator()(T* ptr) const {$/;" f struct:doris::DefaultDeleter +operator () src/gutil/gscoped_ptr.h /^ inline void operator()(void* ptr) const {$/;" f struct:doris::FreeDeleter +operator () src/gutil/hash/hash.h /^ bool operator()(char const* const a, char const* const b) const {$/;" f struct:__gnu_cxx::hash +operator () src/gutil/hash/hash.h /^ bool operator()(const char* a, const char* b) const {$/;" f struct:GoodFastHash +operator () src/gutil/hash/hash.h /^ bool operator()(const pair& a,$/;" f struct:__gnu_cxx::hash +operator () src/gutil/hash/hash.h /^ bool operator()(const std::basic_string<_CharT, _Traits, _Alloc>& a,$/;" f struct:GoodFastHash +operator () src/gutil/hash/hash.h /^ bool operator()(const std::string& a, const std::string& b) const {$/;" f struct:__gnu_cxx::hash +operator () src/gutil/hash/hash.h /^ bool operator()(const uint128& a, const uint128& b) const {$/;" f struct:__gnu_cxx::hash +operator () src/gutil/hash/hash.h /^ size_t operator()(T *x) const {$/;" f struct:__gnu_cxx::hash +operator () src/gutil/hash/hash.h /^ size_t operator()(bool x) const { return static_cast(x); }$/;" f struct:__gnu_cxx::hash +operator () src/gutil/hash/hash.h /^ size_t operator()(char const* const k) const {$/;" f struct:__gnu_cxx::hash +operator () src/gutil/hash/hash.h /^ size_t operator()(const char* s) const {$/;" f struct:GoodFastHash +operator () src/gutil/hash/hash.h /^ size_t operator()(const pair& p) const {$/;" f struct:__gnu_cxx::hash +operator () src/gutil/hash/hash.h /^ size_t operator()(const std::basic_string<_CharT, _Traits, _Alloc>& k) const {$/;" f struct:GoodFastHash +operator () src/gutil/hash/hash.h /^ size_t operator()(const std::basic_string<_CharT, _Traits, _Alloc>& k) const {$/;" f struct:__gnu_cxx::hash +operator () src/gutil/hash/hash.h /^ size_t operator()(const std::string& k) const {$/;" f struct:__gnu_cxx::hash +operator () src/gutil/hash/hash.h /^ size_t operator()(const uint128& x) const {$/;" f struct:__gnu_cxx::hash +operator () src/gutil/ref_counted.h /^ bool operator()(const scoped_refptr& x, const scoped_refptr& y) const {$/;" f struct:ScopedRefPtrEqualToFunctor +operator () src/gutil/ref_counted.h /^ size_t operator()(const scoped_refptr& p) const {$/;" f struct:ScopedRefPtrHashFunctor +operator () src/gutil/stl_util.h /^ typename BinaryOp::result_type operator()(const Pair& p1,$/;" f class:BinaryOperateOnFirst +operator () src/gutil/stl_util.h /^ typename BinaryOp::result_type operator()(const Pair& p1,$/;" f class:BinaryOperateOnSecond +operator () src/gutil/stl_util.h /^ typename F::result_type operator()(typename G1::argument_type x,$/;" f class:BinaryComposeBinary +operator () src/gutil/stl_util.h /^ typename UnaryOp::result_type operator()(const Pair& p) const {$/;" f class:UnaryOperateOnFirst +operator () src/gutil/stl_util.h /^ typename UnaryOp::result_type operator()(const Pair& p) const {$/;" f class:UnaryOperateOnSecond +operator () src/gutil/strings/numbers.h /^ bool operator()(const string& a, const string& b) const {$/;" f struct:autodigit_greater +operator () src/gutil/strings/numbers.h /^ bool operator()(const string& a, const string& b) const {$/;" f struct:autodigit_less +operator () src/gutil/strings/numbers.h /^ bool operator()(const string& a, const string& b) const {$/;" f struct:strict_autodigit_greater +operator () src/gutil/strings/numbers.h /^ bool operator()(const string& a, const string& b) const {$/;" f struct:strict_autodigit_less +operator () src/gutil/strings/split.h /^ bool operator()(StringPiece sp) const {$/;" f struct:strings::AllowEmpty +operator () src/gutil/strings/split.h /^ bool operator()(StringPiece sp) const {$/;" f struct:strings::SkipEmpty +operator () src/gutil/strings/split.h /^ bool operator()(StringPiece sp) const {$/;" f struct:strings::SkipWhitespace +operator () src/gutil/strings/split.h /^ void operator()(C* c, const V& v) const { c->insert(v); }$/;" f struct:strings::internal::BasicInsertPolicy +operator () src/gutil/strings/split.h /^ void operator()(C* c, const V& v) const { c->push_back(v); }$/;" f struct:strings::internal::BackInsertPolicy +operator () src/gutil/strings/split_internal.h /^ Container operator()(Splitter* splitter) const {$/;" f struct:strings::internal::Splitter::SelectContainer +operator () src/gutil/strings/split_internal.h /^ To operator()(StringPiece from) const {$/;" f struct:strings::internal::StringPieceTo +operator () src/gutil/strings/split_internal.h /^ bool operator()(StringPiece \/* ignored *\/) {$/;" f struct:strings::internal::NoFilter +operator () src/gutil/strings/split_internal.h /^ string operator()(StringPiece from) const {$/;" f struct:strings::internal::StringPieceTo +operator () src/gutil/strings/stringpiece.cc /^ size_t hash::operator()(StringPiece s) const {$/;" f class:std::hash +operator () src/gutil/strings/stringpiece.h /^ bool operator()(const StringPiece& s1, const StringPiece& s2) const {$/;" f struct:GoodFastHash +operator () src/gutil/strings/stringpiece.h /^ size_t operator()(StringPiece s) const {$/;" f struct:GoodFastHash +operator () src/gutil/strings/util.cc /^ Rune operator()(const char** p, const char* end) {$/;" f struct:NextCharUTF8 +operator () src/gutil/strings/util.h /^ bool operator()(const char* s1, const char* s2) const {$/;" f struct:streq +operator () src/gutil/strings/util.h /^ bool operator()(const char* s1, const char* s2) const {$/;" f struct:strlt +operator () src/http/action/mini_load.cpp /^bool LoadHandleCmp::operator() (const LoadHandle& lhs, const LoadHandle& rhs) const {$/;" f class:doris::LoadHandleCmp +operator () src/olap/aggregate_func.cpp /^ size_t operator()(const std::pair& pair) const {$/;" f struct:doris::AggregateFuncMapHash +operator () src/olap/byte_buffer.cpp /^void StorageByteBuffer::BufDeleter::operator()(char* p) {$/;" f class:doris::StorageByteBuffer::BufDeleter +operator () src/olap/fs/block_id.h /^ bool operator()(const BlockId& first, const BlockId& second) const {$/;" f struct:doris::BlockIdCompare +operator () src/olap/fs/block_id.h /^ bool operator()(const BlockId& first, const BlockId& second) const {$/;" f struct:doris::BlockIdEqual +operator () src/olap/fs/block_id.h /^ size_t operator()(const BlockId& block_id) const {$/;" f struct:doris::BlockIdHash +operator () src/olap/generic_iterators.cpp /^ bool operator()(const MergeIteratorContext* lhs, const MergeIteratorContext* rhs) const {$/;" f struct:doris::MergeIterator::MergeContextComparator +operator () src/olap/key_coder.cpp /^ std::size_t operator()(T t) const {$/;" f struct:doris::EnumClassHash +operator () src/olap/memtable.cpp /^int MemTable::RowCursorComparator::operator()(const char* left, const char* right) const {$/;" f class:doris::MemTable::RowCursorComparator +operator () src/olap/olap_common.h /^ size_t operator()(const Version& version) const {$/;" f struct:doris::HashOfVersion +operator () src/olap/olap_cond.h /^ bool operator()(const WrapperField* left, const WrapperField* right) const {$/;" f struct:doris::FieldEqual +operator () src/olap/olap_cond.h /^ size_t operator()(const WrapperField* field) const {$/;" f struct:doris::FieldHash +operator () src/olap/olap_index.h /^ bool operator()(const RowCursor& key, const iterator_offset_t& index) {$/;" f class:doris::IndexComparator +operator () src/olap/olap_index.h /^ bool operator()(const RowCursor& key, iterator_offset_t index) {$/;" f class:doris::SegmentComparator +operator () src/olap/olap_index.h /^ bool operator()(const iterator_offset_t index, const RowCursor& key) {$/;" f class:doris::SegmentComparator +operator () src/olap/olap_index.h /^ bool operator()(const iterator_offset_t& index, const RowCursor& key) {$/;" f class:doris::IndexComparator +operator () src/olap/reader.cpp /^bool CollectIterator::ChildCtxComparator::operator()(const ChildCtx* a, const ChildCtx* b) {$/;" f class:doris::CollectIterator::ChildCtxComparator +operator () src/olap/rowset/alpha_rowset_reader.cpp /^bool AlphaMergeContextComparator::operator() (const AlphaMergeContext* x, const AlphaMergeContext* y) const {$/;" f class:doris::AlphaMergeContextComparator +operator () src/olap/rowset/column_data.h /^ bool operator()(const RowCursor& key, const iterator_offset_t& index) const {$/;" f class:doris::ColumnDataComparator +operator () src/olap/rowset/column_data.h /^ bool operator()(const iterator_offset_t& index, const RowCursor& key) const {$/;" f class:doris::ColumnDataComparator +operator () src/olap/rowset/segment_v2/binary_dict_page.h /^ size_t operator()(const Slice& slice) const {$/;" f struct:doris::segment_v2::BinaryDictPageBuilder::HashOfSlice +operator () src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^ bool operator()(const PackedInt128& a, const PackedInt128& b) const {$/;" f struct:doris::segment_v2::__anon52::Int128Comparator +operator () src/olap/rowset/segment_v2/encoding_info.cpp /^ size_t operator()(const std::pair& pair) const {$/;" f struct:doris::segment_v2::EncodingMapHash +operator () src/olap/storage_engine.h /^ bool operator()(const CompactionCandidate& a, const CompactionCandidate& b) {$/;" f struct:doris::StorageEngine::CompactionCandidateComparator +operator () src/runtime/data_stream_mgr.h /^ bool operator()(const std::pair& a,$/;" f struct:doris::DataStreamMgr::ComparisonOp +operator () src/runtime/datetime_value.h /^ size_t operator()(const doris::DateTimeValue& v) const {$/;" f struct:std::hash +operator () src/runtime/decimal_value.h /^ size_t operator()(const doris::DecimalValue& v) const {$/;" f struct:std::hash +operator () src/runtime/decimalv2_value.h /^ size_t operator()(const doris::DecimalV2Value& v) const {$/;" f struct:std::hash +operator () src/runtime/dpp_sink_internal.h /^ std::size_t operator()(const doris::TabletDesc& desc) const {$/;" f struct:std::hash +operator () src/runtime/qsorter.cpp /^bool TupleRowLessThan::operator()(TupleRow* const& lhs, TupleRow* const& rhs) const {$/;" f class:doris::TupleRowLessThan +operator () src/util/container_util.hpp /^ bool operator()(const TNetworkAddress* const& p1,$/;" f struct:doris::TNetworkAddressPtrEquals +operator () src/util/container_util.hpp /^ size_t operator()(const TNetworkAddress* const& p) const {$/;" f struct:doris::HashTNetworkAddressPtr +operator () src/util/hash_util.hpp /^ size_t operator()(const doris::TNetworkAddress& address) const {$/;" f struct:std::hash +operator () src/util/hash_util.hpp /^ size_t operator()(const std::pair& pair) const {$/;" f struct:std::hash +operator () src/util/hash_util.hpp /^ std::size_t operator()(const __int128& val) const {$/;" f struct:std::hash +operator () src/util/hash_util.hpp /^ std::size_t operator()(const doris::TUniqueId& id) const {$/;" f struct:std::hash +operator () src/util/slice.h /^ bool operator()(const Slice& a, const Slice& b) const {$/;" f struct:doris::Slice::Comparator +operator () src/util/string_util.h /^ bool operator()(const std::string& lhs, const std::string& rhs) const {$/;" f struct:doris::StringCaseEqual +operator () src/util/string_util.h /^ bool operator()(const std::string& lhs, const std::string& rhs) const {$/;" f struct:doris::StringCaseLess +operator () src/util/string_util.h /^ std::size_t operator()(const std::string& value) const {$/;" f struct:doris::StringCaseHasher +operator () src/util/tdigest.h /^ bool operator()(const TDigest *left, const TDigest *right) const {$/;" f class:doris::TDigest::TDigestComparator +operator () src/util/tdigest.h /^ bool operator()(const Centroid &a, const Centroid &b) const { return a.mean() < b.mean(); }$/;" f struct:doris::CentroidComparator +operator () src/util/tdigest.h /^ bool operator()(const CentroidList &left, const CentroidList &right) const {$/;" f class:doris::CentroidListComparator +operator () src/util/tuple_row_compare.h /^ bool operator() (Tuple* lhs, Tuple* rhs) const {$/;" f class:doris::TupleRowComparator +operator () src/util/tuple_row_compare.h /^ bool operator() (TupleRow* lhs, TupleRow* rhs) const {$/;" f class:doris::TupleRowComparator +operator () src/util/uid_util.h /^ size_t operator()(const doris::UniqueId& uid) const {$/;" f struct:std::hash +operator () test/olap/skiplist_test.cpp /^ int operator()(const Key& a, const Key& b) const {$/;" f struct:doris::TestComparator +operator * src/gutil/gscoped_ptr.h /^ C& operator*() const {$/;" f class:gscoped_ptr_malloc +operator * src/gutil/gscoped_ptr.h /^ element_type& operator*() const {$/;" f class:gscoped_ptr +operator * src/gutil/int128.h /^inline uint128 operator*(const uint128& lhs, const uint128& rhs) {$/;" f +operator * src/gutil/strings/split_internal.h /^ StringPiece operator*() { return curr_piece_; }$/;" f class:strings::internal::SplitIterator +operator * src/olap/short_key_index.h /^inline Slice ShortKeyIndexIterator::operator*() const {$/;" f class:doris::ShortKeyIndexIterator +operator * src/olap/utils.h /^ iterator_offset_t operator*() const {$/;" f class:doris::BinarySearchIterator +operator * src/runtime/decimal_value.cpp /^DecimalValue operator*(const DecimalValue& v1, const DecimalValue& v2){$/;" f namespace:doris +operator * src/runtime/decimalv2_value.cpp /^DecimalV2Value operator*(const DecimalV2Value& v1, const DecimalV2Value& v2){$/;" f namespace:doris +operator * src/util/bitmap_value.h /^ value_type operator*() const {$/;" f class:doris::detail::final +operator * src/util/lru_cache.hpp /^ KeyValuePair& operator*() {$/;" f class:doris::LruCache::Iterator +operator *= src/gutil/int128.h /^inline uint128& uint128::operator*=(const uint128& b) {$/;" f class:uint128 +operator + src/gutil/int128.h /^inline uint128 operator+(const uint128& lhs, const uint128& rhs) {$/;" f +operator + src/runtime/decimal_value.cpp /^DecimalValue operator+(const DecimalValue& v1, const DecimalValue& v2) {$/;" f namespace:doris +operator + src/runtime/decimalv2_value.cpp /^DecimalV2Value operator+(const DecimalV2Value& v1, const DecimalV2Value& v2) {$/;" f namespace:doris +operator + src/util/monotime.cpp /^MonoTime operator+(const MonoTime& t, const MonoDelta& delta) {$/;" f namespace:doris +operator ++ src/common/atomic.h /^ AtomicInt& operator++() {$/;" f class:doris::AtomicInt +operator ++ src/common/atomic.h /^ AtomicInt operator++(int) {$/;" f class:doris::AtomicInt +operator ++ src/gutil/int128.h /^inline uint128 uint128::operator++(int) {$/;" f class:uint128 +operator ++ src/gutil/int128.h /^inline uint128& uint128::operator++() {$/;" f class:uint128 +operator ++ src/gutil/strings/split_internal.h /^ SplitIterator operator++(int \/* postincrement *\/) {$/;" f class:strings::internal::SplitIterator +operator ++ src/gutil/strings/split_internal.h /^ SplitIterator& operator++() {$/;" f class:strings::internal::SplitIterator +operator ++ src/olap/short_key_index.h /^ ShortKeyIndexIterator& operator++() {$/;" f class:doris::ShortKeyIndexIterator +operator ++ src/olap/tablet_sync_service.h /^ FetchRowsetMetaTask& operator++() {$/;" f struct:doris::FetchRowsetMetaTask +operator ++ src/olap/tablet_sync_service.h /^ FetchTabletMetaTask& operator++() {$/;" f struct:doris::FetchTabletMetaTask +operator ++ src/olap/tablet_sync_service.h /^ PushRowsetMetaTask& operator++() {$/;" f struct:doris::PushRowsetMetaTask +operator ++ src/olap/tablet_sync_service.h /^ PushTabletMetaTask& operator++() {$/;" f struct:doris::PushTabletMetaTask +operator ++ src/olap/utils.h /^ BinarySearchIterator& operator++() {$/;" f class:doris::BinarySearchIterator +operator ++ src/runtime/datetime_value.h /^ DateTimeValue& operator++() {$/;" f class:doris::DateTimeValue +operator ++ src/util/bitmap_value.h /^ type_of_iterator operator++(int) { \/\/ i++, must return orig. value$/;" f class:doris::detail::final +operator ++ src/util/bitmap_value.h /^ type_of_iterator& operator++() { \/\/ ++i, must returned inc. value$/;" f class:doris::detail::final +operator ++ src/util/lru_cache.hpp /^ Iterator& operator++() {$/;" f class:doris::LruCache::Iterator +operator ++ src/util/priority_thread_pool.hpp /^ Task& operator++() {$/;" f struct:doris::PriorityThreadPool::Task +operator += src/common/atomic.h /^ AtomicInt& operator+=(T delta) {$/;" f class:doris::AtomicInt +operator += src/gutil/int128.h /^inline uint128& uint128::operator+=(const uint128& b) {$/;" f class:uint128 +operator += src/olap/decimal12.h /^ decimal12_t& operator+=(const decimal12_t& value) {$/;" f struct:doris::decimal12_t +operator += src/olap/short_key_index.h /^ ShortKeyIndexIterator& operator+=(ssize_t step) {$/;" f class:doris::ShortKeyIndexIterator +operator += src/olap/uint24.h /^ uint24_t& operator+=(const uint24_t& value) {$/;" f struct:doris::uint24_t +operator += src/olap/utils.h /^ BinarySearchIterator& operator+=(size_t step) {$/;" f class:doris::BinarySearchIterator +operator += src/runtime/decimal_value.cpp /^DecimalValue& DecimalValue::operator+=(const DecimalValue& other) {$/;" f class:doris::DecimalValue +operator += src/runtime/decimalv2_value.cpp /^DecimalV2Value& DecimalV2Value::operator+=(const DecimalV2Value& other) {$/;" f class:doris::DecimalV2Value +operator += src/util/monotime.cpp /^MonoTime& MonoTime::operator+=(const MonoDelta& delta) {$/;" f class:doris::MonoTime +operator - src/gutil/int128.h /^inline uint128 operator-(const uint128& lhs, const uint128& rhs) {$/;" f +operator - src/gutil/int128.h /^inline uint128 operator-(const uint128& val) {$/;" f +operator - src/olap/short_key_index.h /^ ssize_t operator-(const ShortKeyIndexIterator& other) const {$/;" f class:doris::ShortKeyIndexIterator +operator - src/olap/utils.cpp /^int operator-(const BinarySearchIterator& left, const BinarySearchIterator& right) {$/;" f namespace:doris +operator - src/runtime/datetime_value.cpp /^std::size_t operator-(const DateTimeValue& v1, const DateTimeValue& v2) {$/;" f namespace:doris +operator - src/runtime/decimal_value.cpp /^DecimalValue operator-(const DecimalValue& v) {$/;" f namespace:doris +operator - src/runtime/decimal_value.cpp /^DecimalValue operator-(const DecimalValue& v1, const DecimalValue& v2) {$/;" f namespace:doris +operator - src/runtime/decimalv2_value.cpp /^DecimalV2Value operator-(const DecimalV2Value& v) {$/;" f namespace:doris +operator - src/runtime/decimalv2_value.cpp /^DecimalV2Value operator-(const DecimalV2Value& v1, const DecimalV2Value& v2) {$/;" f namespace:doris +operator - src/runtime/string_value.cpp /^std::size_t operator-(const StringValue& v1, const StringValue& v2) {$/;" f namespace:doris +operator - src/util/bitmap_value.h /^ Roaring64Map operator-(const Roaring64Map& o) const { return Roaring64Map(*this) -= o; }$/;" f class:doris::detail::Roaring64Map +operator - src/util/monotime.cpp /^MonoDelta operator-(const MonoTime& t_end, const MonoTime& t_beg) {$/;" f namespace:doris +operator - src/util/monotime.cpp /^MonoTime operator-(const MonoTime& t, const MonoDelta& delta) {$/;" f namespace:doris +operator -- src/common/atomic.h /^ AtomicInt& operator--() {$/;" f class:doris::AtomicInt +operator -- src/common/atomic.h /^ AtomicInt operator--(int) {$/;" f class:doris::AtomicInt +operator -- src/gutil/int128.h /^inline uint128 uint128::operator--(int) {$/;" f class:uint128 +operator -- src/gutil/int128.h /^inline uint128& uint128::operator--() {$/;" f class:uint128 +operator -- src/olap/short_key_index.h /^ ShortKeyIndexIterator& operator--() {$/;" f class:doris::ShortKeyIndexIterator +operator -- src/olap/utils.h /^ BinarySearchIterator& operator--() {$/;" f class:doris::BinarySearchIterator +operator -= src/common/atomic.h /^ AtomicInt& operator-=(T delta) {$/;" f class:doris::AtomicInt +operator -= src/gutil/int128.h /^inline uint128& uint128::operator-=(const uint128& b) {$/;" f class:uint128 +operator -= src/olap/short_key_index.h /^ ShortKeyIndexIterator& operator-=(ssize_t step) {$/;" f class:doris::ShortKeyIndexIterator +operator -= src/olap/utils.h /^ BinarySearchIterator& operator-=(size_t step) {$/;" f class:doris::BinarySearchIterator +operator -= src/util/bitmap_value.h /^ Roaring64Map& operator-=(const Roaring64Map& r) {$/;" f class:doris::detail::Roaring64Map +operator -= src/util/monotime.cpp /^MonoTime& MonoTime::operator-=(const MonoDelta& delta) {$/;" f class:doris::MonoTime +operator -> src/gutil/gscoped_ptr.h /^ C* operator->() const {$/;" f class:gscoped_ptr_malloc +operator -> src/gutil/gscoped_ptr.h /^ element_type* operator->() const {$/;" f class:gscoped_ptr +operator -> src/gutil/ref_counted.h /^ T* operator->() const {$/;" f class:scoped_refptr +operator -> src/gutil/strings/split_internal.h /^ StringPiece* operator->() { return &curr_piece_; }$/;" f class:strings::internal::SplitIterator +operator -> src/runtime/client_cache.h /^ T* operator->() const {$/;" f class:doris::ClientConnection +operator -> src/util/lru_cache.hpp /^ KeyValuePair* operator->() {$/;" f class:doris::LruCache::Iterator +operator / src/runtime/decimal_value.cpp /^DecimalValue operator\/(const DecimalValue& v1, const DecimalValue& v2){$/;" f namespace:doris +operator / src/runtime/decimalv2_value.cpp /^DecimalV2Value operator\/(const DecimalV2Value& v1, const DecimalV2Value& v2){$/;" f namespace:doris +operator < src/exprs/expr.cpp /^ bool operator<(const MemLayoutData& rhs) const {$/;" f struct:doris::MemLayoutData +operator < src/gutil/strings/stringpiece.h /^inline bool operator<(StringPiece x, StringPiece y) {$/;" f +operator < src/olap/decimal12.h /^ bool operator<(const decimal12_t& value) const {$/;" f struct:doris::decimal12_t +operator < src/olap/fs/block_id.h /^ bool operator<(const BlockId& other) const {$/;" f class:doris::BlockId +operator < src/olap/olap_common.h /^ bool operator<(const RowsetId& rhs) const {$/;" f struct:doris::RowsetId +operator < src/olap/olap_common.h /^ bool operator<(const TabletInfo& right) const {$/;" f struct:doris::TabletInfo +operator < src/olap/rowset/column_writer.h /^ bool operator < (const DictKey& other) const {$/;" f class:doris::VarStringColumnWriter::DictKey +operator < src/olap/schema_change.cpp /^ bool operator<(const MergeElement& other) const {$/;" f struct:doris::RowBlockMerger::MergeElement +operator < src/olap/stream_name.cpp /^bool StreamName::operator < (const StreamName& another) const {$/;" f class:doris::StreamName +operator < src/olap/tablet_sync_service.h /^ bool operator< (const FetchRowsetMetaTask& o) const {$/;" f struct:doris::FetchRowsetMetaTask +operator < src/olap/tablet_sync_service.h /^ bool operator< (const FetchTabletMetaTask& o) const {$/;" f struct:doris::FetchTabletMetaTask +operator < src/olap/tablet_sync_service.h /^ bool operator< (const PushRowsetMetaTask& o) const {$/;" f struct:doris::PushRowsetMetaTask +operator < src/olap/tablet_sync_service.h /^ bool operator< (const PushTabletMetaTask& o) const {$/;" f struct:doris::PushTabletMetaTask +operator < src/olap/uint24.h /^ bool operator<(const uint24_t& value) const {$/;" f struct:doris::uint24_t +operator < src/runtime/datetime_value.h /^ bool operator<(const DateTimeValue& other) const {$/;" f class:doris::DateTimeValue +operator < src/runtime/decimal_value.h /^ bool operator<(const DecimalValue& other) const {$/;" f class:doris::DecimalValue +operator < src/runtime/decimalv2_value.h /^ bool operator<(const DecimalV2Value& other) const {$/;" f class:doris::DecimalV2Value +operator < src/runtime/dpp_sink_internal.h /^ bool operator<(const PartRange& other) const {$/;" f class:doris::PartRange +operator < src/runtime/dpp_sink_internal.h /^ bool operator<(const PartRangeKey& other) const {$/;" f class:doris::PartRangeKey +operator < src/runtime/string_value.h /^ bool operator<(const StringValue& other) const {$/;" f struct:doris::StringValue +operator < src/util/bitmap_value.h /^ bool operator<(const type_of_iterator& o) {$/;" f class:doris::detail::final +operator < src/util/metrics.h /^ bool operator<(const MetricLabel& other) const {$/;" f struct:doris::MetricLabel +operator < src/util/metrics.h /^ bool operator<(const MetricLabels& other) const {$/;" f struct:doris::MetricLabels +operator < src/util/monotime.cpp /^bool operator<(const MonoDelta &lhs, const MonoDelta &rhs) {$/;" f namespace:doris +operator < src/util/monotime.cpp /^bool operator<(const MonoTime& lhs, const MonoTime& rhs) {$/;" f namespace:doris +operator < src/util/priority_thread_pool.hpp /^ bool operator< (const Task& o) const {$/;" f struct:doris::PriorityThreadPool::Task +operator < src/util/thrift_util.cpp /^bool TNetworkAddress::operator<(const TNetworkAddress& that) const {$/;" f class:doris::TNetworkAddress +operator < src/util/uid_util.h /^ bool operator<(const UniqueId& right) const {$/;" f struct:doris::UniqueId +operator << src/common/configbase.cpp /^std::ostream& operator<< (std::ostream& out, const std::vector& v) {$/;" f namespace:doris::config +operator << src/geo/geo_common.cpp /^std::ostream& operator<<(std::ostream& os, GeoParseStatus status) {$/;" f namespace:doris +operator << src/gutil/int128.cc /^std::ostream& operator<<(std::ostream& o, const uint128& b) {$/;" f +operator << src/gutil/int128.h /^inline uint128 operator<<(const uint128& val, int amount) {$/;" f +operator << src/gutil/port.h /^inline ostream& operator<< (ostream& os, const unsigned __int64& num ) {$/;" f +operator << src/gutil/strings/stringpiece.cc /^std::ostream& operator<<(std::ostream& o, StringPiece piece) {$/;" f +operator << src/http/http_parser.cpp /^std::ostream& operator<<(std::ostream& os, const HttpChunkParseCtx& ctx) {$/;" f namespace:doris +operator << src/olap/decimal12.h /^inline std::ostream& operator<<(std::ostream& os, const decimal12_t& val) {$/;" f namespace:doris +operator << src/olap/memtable.h /^inline std::ostream& operator<<(std::ostream& os, const MemTable& table) {$/;" f namespace:doris +operator << src/olap/memtable_flush_executor.cpp /^std::ostream& operator<<(std::ostream& os, const FlushStatistic& stat) {$/;" f namespace:doris +operator << src/olap/olap_common.h /^ friend std::ostream& operator<<(std::ostream& out, const RowsetId& rowset_id) {$/;" f struct:doris::RowsetId +operator << src/olap/olap_common.h /^inline std::ostream& operator<<(std::ostream& os, const Version& version) {$/;" f namespace:doris +operator << src/olap/tuple.h /^inline std::ostream& operator<<(std::ostream& os, const OlapTuple& tuple) {$/;" f namespace:doris +operator << src/olap/uint24.h /^inline std::ostream& operator<<(std::ostream& os, const uint24_t& val) {$/;" f namespace:doris +operator << src/runtime/datetime_value.cpp /^std::ostream& operator<<(std::ostream& os, const DateTimeValue& value) {$/;" f namespace:doris +operator << src/runtime/decimal_value.cpp /^std::ostream& operator<<(std::ostream& os, DecimalValue const& decimal_value) {$/;" f namespace:doris +operator << src/runtime/decimalv2_value.cpp /^std::ostream& operator<<(std::ostream& os, DecimalV2Value const& decimal_value) {$/;" f namespace:doris +operator << src/runtime/descriptors.cpp /^std::ostream& operator<<(std::ostream& os, const NullIndicatorOffset& null_indicator) {$/;" f namespace:doris +operator << src/runtime/large_int_value.cpp /^std::ostream& operator<<(std::ostream& os, __int128 const& value) {$/;" f namespace:doris +operator << src/runtime/load_channel.h /^inline std::ostream& operator<<(std::ostream& os, const LoadChannel& load_channel) {$/;" f namespace:doris +operator << src/runtime/string_value.cpp /^std::ostream& operator<<(std::ostream& os, const StringValue& string_value) {$/;" f namespace:doris +operator << src/runtime/tablets_channel.cpp /^std::ostream& operator<<(std::ostream& os, const TabletsChannelKey& key) {$/;" f namespace:doris +operator << src/runtime/types.cpp /^std::ostream& operator<<(std::ostream& os, const TypeDescriptor& type) {$/;" f namespace:doris +operator << src/testutil/desc_tbl_builder.h /^ TupleDescBuilder& operator<< (const TypeDescriptor& slot_type) {$/;" f class:doris::TupleDescBuilder +operator << src/util/metrics.cpp /^std::ostream& operator<<(std::ostream& os, MetricType type) {$/;" f namespace:doris +operator << src/util/slice.h /^inline std::ostream& operator<< (std::ostream& os, const Slice& slice) {$/;" f namespace:doris +operator << src/util/threadpool.cpp /^std::ostream& operator<<(std::ostream& o, ThreadPoolToken::State s) {$/;" f namespace:doris +operator << src/util/uid_util.cpp /^std::ostream& operator<<(std::ostream& os, const UniqueId& uid) {$/;" f namespace:doris +operator <<= src/gutil/int128.h /^inline uint128& uint128::operator<<=(int amount) {$/;" f class:uint128 +operator <= src/gutil/strings/stringpiece.h /^inline bool operator<=(StringPiece x, StringPiece y) {$/;" f +operator <= src/olap/decimal12.h /^ bool operator<=(const decimal12_t& value) const {$/;" f struct:doris::decimal12_t +operator <= src/olap/uint24.h /^ bool operator<=(const uint24_t& value) const {$/;" f struct:doris::uint24_t +operator <= src/runtime/datetime_value.h /^ bool operator<=(const DateTimeValue& other) const {$/;" f class:doris::DateTimeValue +operator <= src/runtime/decimal_value.h /^ bool operator<=(const DecimalValue& other) const {$/;" f class:doris::DecimalValue +operator <= src/runtime/decimalv2_value.h /^ bool operator<=(const DecimalV2Value& other) const {$/;" f class:doris::DecimalV2Value +operator <= src/runtime/dpp_sink_internal.h /^ bool operator<=(const PartRangeKey& other) const {$/;" f class:doris::PartRangeKey +operator <= src/runtime/string_value.h /^ bool operator<=(const StringValue& other) const {$/;" f struct:doris::StringValue +operator <= src/util/bitmap_value.h /^ bool operator<=(const type_of_iterator& o) {$/;" f class:doris::detail::final +operator <= src/util/monotime.cpp /^bool operator<=(const MonoDelta &lhs, const MonoDelta &rhs) {$/;" f namespace:doris +operator <= src/util/monotime.cpp /^bool operator<=(const MonoTime& lhs, const MonoTime& rhs) {$/;" f namespace:doris +operator = src/common/atomic.h /^ AtomicInt& operator=(T val) {$/;" f class:doris::AtomicInt +operator = src/common/atomic.h /^ AtomicInt& operator=(const AtomicInt& val) {$/;" f class:doris::AtomicInt +operator = src/common/status.h /^ Status& operator=(const Status& s) {$/;" f class:doris::Status +operator = src/gutil/gscoped_ptr.h /^ gscoped_array& operator=(RValue rhs) {$/;" f class:gscoped_array +operator = src/gutil/gscoped_ptr.h /^ gscoped_ptr& operator=(RValue rhs) {$/;" f class:gscoped_ptr +operator = src/gutil/gscoped_ptr.h /^ gscoped_ptr& operator=(gscoped_ptr rhs) {$/;" f class:gscoped_ptr +operator = src/gutil/gscoped_ptr.h /^ gscoped_ptr_malloc& operator=(RValue rhs) {$/;" f class:gscoped_ptr_malloc +operator = src/gutil/int128.h /^inline uint128& uint128::operator=(const uint128& b) {$/;" f class:uint128 +operator = src/gutil/ref_counted.h /^ scoped_refptr& operator=(T* p) {$/;" f class:scoped_refptr +operator = src/gutil/ref_counted.h /^ scoped_refptr& operator=(const scoped_refptr& r) {$/;" f class:scoped_refptr +operator = src/gutil/ref_counted.h /^ scoped_refptr& operator=(const scoped_refptr& r) {$/;" f class:scoped_refptr +operator = src/gutil/ref_counted.h /^ scoped_refptr& operator=(scoped_refptr&& r) {$/;" f class:scoped_refptr +operator = src/gutil/ref_counted.h /^ scoped_refptr& operator=(scoped_refptr&& r) {$/;" f class:scoped_refptr +operator = src/olap/decimal12.h /^ decimal12_t& operator=(const decimal12_t& value) {$/;" f struct:doris::decimal12_t +operator = src/olap/schema.cpp /^Schema& Schema::operator=(const Schema& other) {$/;" f class:doris::Schema +operator = src/olap/uint24.h /^ uint24_t& operator=(const int& value) {$/;" f struct:doris::uint24_t +operator = src/olap/uint24.h /^ uint24_t& operator=(const int64_t& value) {$/;" f struct:doris::uint24_t +operator = src/olap/uint24.h /^ uint24_t& operator=(const uint128_t& value) {$/;" f struct:doris::uint24_t +operator = src/olap/uint24.h /^ uint24_t& operator=(const uint32_t& value) {$/;" f struct:doris::uint24_t +operator = src/olap/uint24.h /^ uint24_t& operator=(const uint64_t& value) {$/;" f struct:doris::uint24_t +operator = src/runtime/bufferpool/buffer_pool.cc /^BufferPool::PageHandle& BufferPool::PageHandle::operator=(PageHandle&& src) {$/;" f class:doris::BufferPool::PageHandle +operator = src/runtime/bufferpool/buffer_pool.h /^inline BufferPool::BufferHandle& BufferPool::BufferHandle::operator=($/;" f class:doris::BufferPool::BufferHandle +operator = src/util/slice.h /^ OwnedSlice& operator= (OwnedSlice&& src) {$/;" f class:doris::OwnedSlice +operator = src/util/tdigest.h /^ TDigest& operator=(TDigest&& o) {$/;" f class:doris::TDigest +operator = src/util/types.h /^ PackedInt128& operator=(const PackedInt128& rhs) {$/;" f struct:doris::PackedInt128 +operator = src/util/types.h /^ PackedInt128& operator=(const __int128& value_) {$/;" f struct:doris::PackedInt128 +operator == output/udf/include/udf.h /^ bool operator==(const BigIntVal& other) const {$/;" f struct:doris_udf::BigIntVal +operator == output/udf/include/udf.h /^ bool operator==(const BooleanVal& other) const {$/;" f struct:doris_udf::BooleanVal +operator == output/udf/include/udf.h /^ bool operator==(const DateTimeVal& other) const {$/;" f struct:doris_udf::DateTimeVal +operator == output/udf/include/udf.h /^ bool operator==(const DecimalV2Val& other) const {$/;" f struct:doris_udf::DecimalV2Val +operator == output/udf/include/udf.h /^ bool operator==(const DoubleVal& other) const {$/;" f struct:doris_udf::DoubleVal +operator == output/udf/include/udf.h /^ bool operator==(const FloatVal& other) const {$/;" f struct:doris_udf::FloatVal +operator == output/udf/include/udf.h /^ bool operator==(const IntVal& other) const {$/;" f struct:doris_udf::IntVal +operator == output/udf/include/udf.h /^ bool operator==(const LargeIntVal& other) const {$/;" f struct:doris_udf::LargeIntVal +operator == output/udf/include/udf.h /^ bool operator==(const SmallIntVal& other) const {$/;" f struct:doris_udf::SmallIntVal +operator == output/udf/include/udf.h /^ bool operator==(const StringVal& other) const {$/;" f struct:doris_udf::StringVal +operator == output/udf/include/udf.h /^ bool operator==(const TinyIntVal& other) const {$/;" f struct:doris_udf::TinyIntVal +operator == src/exec/hash_table.h /^ bool operator==(const Iterator& rhs) {$/;" f class:doris::HashTable::Iterator +operator == src/gutil/gscoped_ptr.h /^ bool operator==(C* p) const { return array_ == p; }$/;" f class:gscoped_array +operator == src/gutil/gscoped_ptr.h /^ bool operator==(C* p) const {$/;" f class:gscoped_ptr_malloc +operator == src/gutil/gscoped_ptr.h /^ bool operator==(const element_type* p) const { return impl_.get() == p; }$/;" f class:gscoped_ptr +operator == src/gutil/gscoped_ptr.h /^ bool operator==(element_type* array) const { return impl_.get() == array; }$/;" f class:gscoped_ptr +operator == src/gutil/gscoped_ptr.h /^bool operator==(C* p, const gscoped_ptr_malloc& b) {$/;" f +operator == src/gutil/gscoped_ptr.h /^bool operator==(C* p1, const gscoped_array& p2) {$/;" f +operator == src/gutil/gscoped_ptr.h /^bool operator==(T* p1, const gscoped_ptr& p2) {$/;" f +operator == src/gutil/int128.h /^inline bool operator==(const uint128& lhs, const uint128& rhs) {$/;" f +operator == src/gutil/strings/split_internal.h /^ bool operator==(const SplitIterator& other) const {$/;" f class:strings::internal::SplitIterator +operator == src/gutil/strings/stringpiece.h /^inline bool operator==(StringPiece x, StringPiece y) {$/;" f +operator == src/olap/decimal12.h /^ bool operator==(const decimal12_t& value) const {$/;" f struct:doris::decimal12_t +operator == src/olap/fs/block_id.h /^ bool operator==(const BlockId& other) const {$/;" f class:doris::BlockId +operator == src/olap/lru_cache.h /^ inline bool operator==(const CacheKey& other) const {$/;" f class:doris::CacheKey +operator == src/olap/olap_common.h /^ bool operator==(const RowsetId& rhs) const {$/;" f struct:doris::RowsetId +operator == src/olap/olap_common.h /^ bool operator==(const Version& rhs) const {$/;" f struct:doris::Version +operator == src/olap/olap_index.h /^ bool operator==(const OLAPIndexOffset& other) const {$/;" f struct:doris::OLAPIndexOffset +operator == src/olap/olap_index.h /^ bool operator==(const RowBlockPosition& other) const {$/;" f struct:doris::RowBlockPosition +operator == src/olap/rowset/column_writer.h /^ bool operator == (const DictKey& other) const {$/;" f class:doris::VarStringColumnWriter::DictKey +operator == src/olap/rowset/segment_v2/page_pointer.h /^ bool operator==(const PagePointer& other) const {$/;" f struct:doris::segment_v2::PagePointer +operator == src/olap/short_key_index.h /^ bool operator==(const ShortKeyIndexIterator& other) {$/;" f class:doris::ShortKeyIndexIterator +operator == src/olap/stream_name.cpp /^bool StreamName::operator == (const StreamName& another) const {$/;" f class:doris::StreamName +operator == src/olap/uint24.h /^ bool operator==(const uint24_t& value) const {$/;" f struct:doris::uint24_t +operator == src/runtime/datetime_value.h /^ bool operator==(const DateTimeValue& other) const {$/;" f class:doris::DateTimeValue +operator == src/runtime/decimal_value.h /^ bool operator==(const DecimalValue& other) const {$/;" f class:doris::DecimalValue +operator == src/runtime/decimalv2_value.h /^ bool operator==(const DecimalV2Value& other) const {$/;" f class:doris::DecimalV2Value +operator == src/runtime/dpp_sink_internal.h /^ bool operator==(const PartRangeKey& other) const {$/;" f class:doris::PartRangeKey +operator == src/runtime/dpp_sink_internal.h /^ bool operator==(const TabletDesc& other) const {$/;" f struct:doris::TabletDesc +operator == src/runtime/string_value.h /^ bool operator==(const StringValue& other) const {$/;" f struct:doris::StringValue +operator == src/runtime/types.h /^ bool operator==(const TypeDescriptor& o) const {$/;" f struct:doris::TypeDescriptor +operator == src/udf/udf.cpp /^bool DecimalVal::operator==(const DecimalVal& other) const {$/;" f class:doris_udf::DecimalVal +operator == src/udf/udf.h /^ bool operator==(const BigIntVal& other) const {$/;" f struct:doris_udf::BigIntVal +operator == src/udf/udf.h /^ bool operator==(const BooleanVal& other) const {$/;" f struct:doris_udf::BooleanVal +operator == src/udf/udf.h /^ bool operator==(const DateTimeVal& other) const {$/;" f struct:doris_udf::DateTimeVal +operator == src/udf/udf.h /^ bool operator==(const DecimalV2Val& other) const {$/;" f struct:doris_udf::DecimalV2Val +operator == src/udf/udf.h /^ bool operator==(const DoubleVal& other) const {$/;" f struct:doris_udf::DoubleVal +operator == src/udf/udf.h /^ bool operator==(const FloatVal& other) const {$/;" f struct:doris_udf::FloatVal +operator == src/udf/udf.h /^ bool operator==(const IntVal& other) const {$/;" f struct:doris_udf::IntVal +operator == src/udf/udf.h /^ bool operator==(const LargeIntVal& other) const {$/;" f struct:doris_udf::LargeIntVal +operator == src/udf/udf.h /^ bool operator==(const SmallIntVal& other) const {$/;" f struct:doris_udf::SmallIntVal +operator == src/udf/udf.h /^ bool operator==(const StringVal& other) const {$/;" f struct:doris_udf::StringVal +operator == src/udf/udf.h /^ bool operator==(const TinyIntVal& other) const {$/;" f struct:doris_udf::TinyIntVal +operator == src/util/bitmap_value.h /^ bool operator==(const Roaring64Map& r) const {$/;" f class:doris::detail::Roaring64Map +operator == src/util/bitmap_value.h /^ bool operator==(const Roaring64MapSetBitForwardIterator& o) {$/;" f class:doris::detail::final +operator == src/util/lru_cache.hpp /^ bool operator==(const Iterator& rhs) const {$/;" f class:doris::LruCache::Iterator +operator == src/util/metrics.h /^ bool operator==(const MetricLabel& other) const {$/;" f struct:doris::MetricLabel +operator == src/util/metrics.h /^ bool operator==(const MetricLabels& other) const {$/;" f struct:doris::MetricLabels +operator == src/util/monotime.cpp /^bool operator==(const MonoDelta &lhs, const MonoDelta &rhs) {$/;" f namespace:doris +operator == src/util/monotime.cpp /^bool operator==(const MonoTime& lhs, const MonoTime& rhs) {$/;" f namespace:doris +operator == src/util/slice.h /^inline bool operator==(const Slice& x, const Slice& y) {$/;" f namespace:doris +operator == src/util/uid_util.h /^ bool operator==(const UniqueId& rhs) const {$/;" f struct:doris::UniqueId +operator > src/gutil/strings/stringpiece.h /^inline bool operator>(StringPiece x, StringPiece y) {$/;" f +operator > src/olap/decimal12.h /^ bool operator>(const decimal12_t& value) const {$/;" f struct:doris::decimal12_t +operator > src/olap/olap_index.h /^ bool operator>(const RowBlockPosition& other) const {$/;" f struct:doris::RowBlockPosition +operator > src/olap/uint24.h /^ bool operator>(const uint24_t& value) const {$/;" f struct:doris::uint24_t +operator > src/runtime/datetime_value.h /^ bool operator>(const DateTimeValue& other) const {$/;" f class:doris::DateTimeValue +operator > src/runtime/decimal_value.h /^ bool operator>(const DecimalValue& other) const {$/;" f class:doris::DecimalValue +operator > src/runtime/decimalv2_value.h /^ bool operator>(const DecimalV2Value& other) const {$/;" f class:doris::DecimalV2Value +operator > src/runtime/dpp_sink_internal.h /^ bool operator>(const PartRangeKey& other) const {$/;" f class:doris::PartRangeKey +operator > src/runtime/string_value.h /^ bool operator>(const StringValue& other) const {$/;" f struct:doris::StringValue +operator > src/util/bitmap_value.h /^ bool operator>(const type_of_iterator& o) {$/;" f class:doris::detail::final +operator > src/util/monotime.cpp /^bool operator>(const MonoDelta &lhs, const MonoDelta &rhs) {$/;" f namespace:doris +operator > src/util/monotime.cpp /^bool operator>(const MonoTime& lhs, const MonoTime& rhs) {$/;" f namespace:doris +operator >= src/gutil/strings/stringpiece.h /^inline bool operator>=(StringPiece x, StringPiece y) {$/;" f +operator >= src/olap/decimal12.h /^ bool operator>=(const decimal12_t& value) const {$/;" f struct:doris::decimal12_t +operator >= src/olap/olap_index.h /^ bool operator>=(const RowBlockPosition& other) const {$/;" f struct:doris::RowBlockPosition +operator >= src/olap/uint24.h /^ bool operator>=(const uint24_t& value) const {$/;" f struct:doris::uint24_t +operator >= src/runtime/datetime_value.h /^ bool operator>=(const DateTimeValue& other) const {$/;" f class:doris::DateTimeValue +operator >= src/runtime/decimal_value.h /^ bool operator>=(const DecimalValue& other) const {$/;" f class:doris::DecimalValue +operator >= src/runtime/decimalv2_value.h /^ bool operator>=(const DecimalV2Value& other) const {$/;" f class:doris::DecimalV2Value +operator >= src/runtime/dpp_sink_internal.h /^ bool operator>=(const PartRangeKey& other) const {$/;" f class:doris::PartRangeKey +operator >= src/runtime/string_value.h /^ bool operator>=(const StringValue& other) const {$/;" f struct:doris::StringValue +operator >= src/util/bitmap_value.h /^ bool operator>=(const type_of_iterator& o) {$/;" f class:doris::detail::final +operator >= src/util/monotime.cpp /^bool operator>=(const MonoDelta &lhs, const MonoDelta &rhs) {$/;" f namespace:doris +operator >= src/util/monotime.cpp /^bool operator>=(const MonoTime& lhs, const MonoTime& rhs) {$/;" f namespace:doris +operator >> src/gutil/int128.h /^inline uint128 operator>>(const uint128& val, int amount) {$/;" f +operator >> src/runtime/decimal_value.cpp /^std::istream& operator>>(std::istream& ism, DecimalValue& decimal_value) {$/;" f namespace:doris +operator >> src/runtime/decimalv2_value.cpp /^std::istream& operator>>(std::istream& ism, DecimalV2Value& decimal_value) {$/;" f namespace:doris +operator >> src/runtime/large_int_value.cpp /^std::istream& operator>>(std::istream& is, __int128 & value) {$/;" f namespace:doris +operator >>= src/gutil/int128.h /^inline uint128& uint128::operator>>=(int amount) {$/;" f class:uint128 +operator >>= src/olap/uint24.h /^ uint24_t& operator>>=(const int& bits) {$/;" f struct:doris::uint24_t +operator Container src/gutil/strings/split_internal.h /^ operator Container() {$/;" f class:strings::internal::Splitter +operator T src/common/atomic.h /^ operator T() const { return _value; }$/;" f class:doris::AtomicInt +operator T* src/gutil/ref_counted.h /^ operator T*() const { return ptr_; }$/;" f class:scoped_refptr +operator Testable src/gutil/gscoped_ptr.h /^ operator Testable() const { return array_ ? &gscoped_array::array_ : NULL; }$/;" f class:gscoped_array +operator Testable src/gutil/gscoped_ptr.h /^ operator Testable() const { return impl_.get() ? &gscoped_ptr::impl_ : NULL; }$/;" f class:gscoped_ptr +operator Testable src/gutil/gscoped_ptr.h /^ operator Testable() const { return ptr_ ? &gscoped_ptr_malloc::ptr_ : NULL; }$/;" f class:gscoped_ptr_malloc +operator Testable src/gutil/ref_counted.h /^ operator Testable() const { return ptr_ ? &scoped_refptr::ptr_ : NULL; }$/;" f class:scoped_refptr +operator [] src/gutil/gscoped_ptr.h /^ C& operator[](ptrdiff_t i) const {$/;" f class:gscoped_array +operator [] src/gutil/gscoped_ptr.h /^ element_type& operator[](size_t i) const {$/;" f class:gscoped_ptr +operator [] src/gutil/strings/stringpiece.h /^ char operator[](int i) const {$/;" f class:StringPiece +operator [] src/olap/lru_cache.h /^ char operator[](size_t n) const {$/;" f class:doris::CacheKey +operator [] src/util/faststring.h /^ const uint8_t &operator[](size_t i) const {$/;" f class:doris::faststring +operator [] src/util/faststring.h /^ uint8_t &operator[](size_t i) {$/;" f class:doris::faststring +operator [] src/util/slice.h /^ const char& operator[](size_t n) const {$/;" f struct:doris::Slice +operator ^ src/util/bitmap_value.h /^ Roaring64Map operator^(const Roaring64Map& o) const { return Roaring64Map(*this) ^= o; }$/;" f class:doris::detail::Roaring64Map +operator ^= src/util/bitmap_value.h /^ Roaring64Map& operator^=(const Roaring64Map& r) {$/;" f class:doris::detail::Roaring64Map +operator __int128 src/runtime/decimal_value.h /^ operator __int128() const {$/;" f class:doris::DecimalValue +operator bool src/runtime/decimal_value.h /^ operator bool() const {$/;" f class:doris::DecimalValue +operator bool src/runtime/decimalv2_value.h /^ operator bool() const {$/;" f class:doris::DecimalV2Value +operator double src/runtime/decimal_value.h /^ operator double() const {$/;" f class:doris::DecimalValue +operator double src/runtime/decimalv2_value.h /^ operator double() const {$/;" f class:doris::DecimalV2Value +operator float src/runtime/decimal_value.h /^ operator float() const {$/;" f class:doris::DecimalValue +operator float src/runtime/decimalv2_value.h /^ operator float() const {$/;" f class:doris::DecimalV2Value +operator int128_t src/runtime/decimalv2_value.h /^ operator int128_t() const {$/;" f class:doris::DecimalV2Value +operator int16_t src/runtime/decimal_value.h /^ operator int16_t() const {$/;" f class:doris::DecimalValue +operator int16_t src/runtime/decimalv2_value.h /^ operator int16_t() const {$/;" f class:doris::DecimalV2Value +operator int32_t src/runtime/decimal_value.h /^ operator int32_t() const {$/;" f class:doris::DecimalValue +operator int32_t src/runtime/decimalv2_value.h /^ operator int32_t() const {$/;" f class:doris::DecimalV2Value +operator int64_t src/runtime/datetime_value.h /^ operator int64_t() const {$/;" f class:doris::DateTimeValue +operator int64_t src/runtime/decimal_value.h /^ operator int64_t() const {$/;" f class:doris::DecimalValue +operator int64_t src/runtime/decimalv2_value.h /^ operator int64_t() const {$/;" f class:doris::DecimalV2Value +operator int8_t src/runtime/decimal_value.h /^ operator int8_t() const {$/;" f class:doris::DecimalValue +operator int8_t src/runtime/decimalv2_value.h /^ operator int8_t() const {$/;" f class:doris::DecimalV2Value +operator size_t src/runtime/decimal_value.h /^ operator size_t() const {$/;" f class:doris::DecimalValue +operator size_t src/runtime/decimalv2_value.h /^ operator size_t() const {$/;" f class:doris::DecimalV2Value +operator std::pair src/gutil/strings/split_internal.h /^ operator std::pair() {$/;" f class:strings::internal::Splitter +operator uint32_t src/olap/uint24.h /^ operator uint32_t() const {$/;" f struct:doris::uint24_t +operator | src/util/bitmap_value.h /^ Roaring64Map operator|(const Roaring64Map& o) const { return Roaring64Map(*this) |= o; }$/;" f class:doris::detail::Roaring64Map +operator |= src/common/atomic.h /^ AtomicInt& operator|=(T v) {$/;" f class:doris::AtomicInt +operator |= src/olap/uint24.h /^ uint24_t& operator|=(const uint24_t& value) {$/;" f struct:doris::uint24_t +operator |= src/util/bitmap_value.h /^ BitmapValue& operator|=(const BitmapValue& rhs) {$/;" f class:doris::BitmapValue +operator |= src/util/bitmap_value.h /^ Roaring64Map& operator|=(const Roaring64Map& r) {$/;" f class:doris::detail::Roaring64Map +operator ~ src/gutil/int128.h /^inline uint128 operator~(const uint128& val) {$/;" f +option src/exec/tablet_info.h /^ int64_t option;$/;" m struct:doris::NodeInfo +or_ src/gutil/template_util.h /^struct or_ : public integral_constant {$/;" s namespace:base +ordinal src/olap/short_key_index.h /^ ssize_t ordinal() const { return _ordinal; }$/;" f class:doris::ShortKeyIndexIterator +original_hardware_flags_ src/util/cpu_info.cpp /^int64_t CpuInfo::original_hardware_flags_;$/;" m class:doris::CpuInfo file: +original_hardware_flags_ src/util/cpu_info.h /^ static int64_t original_hardware_flags_;$/;" m class:doris::CpuInfo +os_version src/util/os_info.h /^ static const std::string os_version() {$/;" f class:doris::OsInfo +os_version_ src/util/os_info.cpp /^string OsInfo::os_version_ = "Unknown";$/;" m class:doris::OsInfo file: +os_version_ src/util/os_info.h /^ static std::string os_version_;$/;" m class:doris::OsInfo +other src/gutil/stl_util.h /^ rebind::other> other;$/;" t struct:STLCountingAllocator::rebind +other_core_alloc_count src/runtime/memory/chunk_allocator.cpp /^static IntCounter other_core_alloc_count;$/;" m namespace:doris file: +output_buf_read_remaining src/exec/plain_text_line_reader.h /^ inline size_t output_buf_read_remaining() {$/;" f class:doris::PlainTextLineReader +output_buffers src/olap/out_stream.h /^ const std::vector& output_buffers() {$/;" f class:doris::OutStream +output_checksum_type src/exec/decompressor.h /^ LzoChecksum output_checksum_type;$/;" m struct:doris::LzopDecompressor::HeaderInfo +output_column src/exprs/expr.h /^ int output_column() const {$/;" f class:doris::Expr +output_files src/runtime/runtime_state.h /^ std::vector& output_files() {$/;" f class:doris::RuntimeState +output_iterator_ src/exec/new_partitioned_aggregation_node.h /^ NewPartitionedHashTable::Iterator output_iterator_;$/;" m class:doris::NewPartitionedAggregationNode +output_partition_ src/exec/new_partitioned_aggregation_node.h /^ Partition* output_partition_;$/;" m class:doris::NewPartitionedAggregationNode +output_path src/runtime/dpp_sink.cpp /^ const std::string& output_path() const {$/;" f class:doris::Translator +output_rows src/olap/merger.h /^ int64_t output_rows = 0;$/;" m struct:doris::Merger::Statistics +output_scale src/exprs/expr.h /^ int output_scale() const {$/;" f class:doris::Expr +output_size src/olap/stream_index_writer.cpp /^int32_t PositionEntryWriter::output_size() const {$/;" f class:doris::PositionEntryWriter +output_size src/olap/stream_index_writer.cpp /^size_t StreamIndexWriter::output_size() {$/;" f class:doris::StreamIndexWriter +output_slot_desc src/exprs/agg_fn.h /^ const SlotDescriptor& output_slot_desc() const { return output_slot_desc_; }$/;" f class:doris::AggFn +output_slot_desc src/exprs/agg_fn_evaluator.h /^ const SlotDescriptor* output_slot_desc() const {$/;" f class:doris::AggFnEvaluator +output_slot_desc_ src/exprs/agg_fn.h /^ const SlotDescriptor& output_slot_desc_;$/;" m class:doris::AggFn +output_tuple_desc_ src/exec/new_partitioned_aggregation_node.h /^ TupleDescriptor* output_tuple_desc_;$/;" m class:doris::NewPartitionedAggregationNode +output_tuple_id_ src/exec/new_partitioned_aggregation_node.h /^ TupleId output_tuple_id_;$/;" m class:doris::NewPartitionedAggregationNode +output_type src/exec/decompressor.h /^ inline LzoChecksum output_type(int flags) {$/;" f class:doris::LzopDecompressor +override src/exec/broker_reader.h /^ virtual Status open() override;$/;" m class:doris::BrokerReader +override src/exec/broker_reader.h /^ virtual Status read(uint8_t* buf, size_t* buf_len, bool* eof) override;$/;" m class:doris::BrokerReader +override src/exec/broker_reader.h /^ virtual Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) override;$/;" m class:doris::BrokerReader +override src/exec/broker_reader.h /^ virtual Status seek(int64_t position) override;$/;" m class:doris::BrokerReader +override src/exec/broker_reader.h /^ virtual Status tell(int64_t* position) override;$/;" m class:doris::BrokerReader +override src/exec/broker_reader.h /^ virtual bool closed() override;$/;" m class:doris::BrokerReader +override src/exec/broker_reader.h /^ virtual int64_t size() override;$/;" m class:doris::BrokerReader +override src/exec/broker_reader.h /^ virtual void close() override;$/;" m class:doris::BrokerReader +override src/exec/broker_scan_node.h /^ virtual Status close(RuntimeState* state) override;$/;" m class:doris::BrokerScanNode +override src/exec/broker_scan_node.h /^ virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;$/;" m class:doris::BrokerScanNode +override src/exec/broker_scan_node.h /^ virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;$/;" m class:doris::BrokerScanNode +override src/exec/broker_scan_node.h /^ virtual Status open(RuntimeState* state) override;$/;" m class:doris::BrokerScanNode +override src/exec/broker_scan_node.h /^ virtual Status prepare(RuntimeState* state) override;$/;" m class:doris::BrokerScanNode +override src/exec/broker_scan_node.h /^ virtual Status set_scan_ranges(const std::vector& scan_ranges) override;$/;" m class:doris::BrokerScanNode +override src/exec/broker_scan_node.h /^ virtual void debug_string(int indentation_level, std::stringstream* out) const override;$/;" m class:doris::BrokerScanNode +override src/exec/broker_scanner.h /^ Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;$/;" m class:doris::BrokerScanner +override src/exec/broker_scanner.h /^ Status open() override;$/;" m class:doris::BrokerScanner +override src/exec/broker_scanner.h /^ void close() override;$/;" m class:doris::BrokerScanner +override src/exec/broker_writer.h /^ virtual Status close() override;$/;" m class:doris::BrokerWriter +override src/exec/broker_writer.h /^ virtual Status open() override;$/;" m class:doris::BrokerWriter +override src/exec/broker_writer.h /^ virtual Status write(const uint8_t* buf, size_t buf_len, size_t* written_len) override;$/;" m class:doris::BrokerWriter +override src/exec/decompressor.h /^ size_t* more_input_bytes, size_t* more_output_bytes) override;$/;" m class:doris::Bzip2Decompressor +override src/exec/decompressor.h /^ size_t* more_input_bytes, size_t* more_output_bytes) override;$/;" m class:doris::GzipDecompressor +override src/exec/decompressor.h /^ size_t* more_input_bytes, size_t* more_output_bytes) override;$/;" m class:doris::Lz4FrameDecompressor +override src/exec/decompressor.h /^ size_t* more_input_bytes, size_t* more_output_bytes) override;$/;" m class:doris::LzopDecompressor +override src/exec/decompressor.h /^ virtual Status init() override;$/;" m class:doris::Bzip2Decompressor +override src/exec/decompressor.h /^ virtual Status init() override;$/;" m class:doris::GzipDecompressor +override src/exec/decompressor.h /^ virtual Status init() override;$/;" m class:doris::Lz4FrameDecompressor +override src/exec/decompressor.h /^ virtual Status init() override;$/;" m class:doris::LzopDecompressor +override src/exec/decompressor.h /^ virtual std::string debug_info() override;$/;" m class:doris::Bzip2Decompressor +override src/exec/decompressor.h /^ virtual std::string debug_info() override;$/;" m class:doris::GzipDecompressor +override src/exec/decompressor.h /^ virtual std::string debug_info() override;$/;" m class:doris::Lz4FrameDecompressor +override src/exec/decompressor.h /^ virtual std::string debug_info() override;$/;" m class:doris::LzopDecompressor +override src/exec/empty_set_node.h /^ virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;$/;" m class:doris::EmptySetNode +override src/exec/es/es_query_builder.h /^ void to_json(rapidjson::Document* document, rapidjson::Value* query) override;$/;" m class:doris::BooleanQueryBuilder +override src/exec/es/es_query_builder.h /^ void to_json(rapidjson::Document* document, rapidjson::Value* query) override;$/;" m class:doris::ESQueryBuilder +override src/exec/es/es_query_builder.h /^ void to_json(rapidjson::Document* document, rapidjson::Value* query) override;$/;" m class:doris::ExistsQueryBuilder +override src/exec/es/es_query_builder.h /^ void to_json(rapidjson::Document* document, rapidjson::Value* query) override;$/;" m class:doris::MatchAllQueryBuilder +override src/exec/es/es_query_builder.h /^ void to_json(rapidjson::Document* document, rapidjson::Value* query) override;$/;" m class:doris::RangeQueryBuilder +override src/exec/es/es_query_builder.h /^ void to_json(rapidjson::Document* document, rapidjson::Value* query) override;$/;" m class:doris::TermQueryBuilder +override src/exec/es/es_query_builder.h /^ void to_json(rapidjson::Document* document, rapidjson::Value* query) override;$/;" m class:doris::TermsInSetQueryBuilder +override src/exec/es/es_query_builder.h /^ void to_json(rapidjson::Document* document, rapidjson::Value* query) override;$/;" m class:doris::WildCardQueryBuilder +override src/exec/es_http_scan_node.h /^ virtual Status close(RuntimeState* state) override;$/;" m class:doris::EsHttpScanNode +override src/exec/es_http_scan_node.h /^ virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;$/;" m class:doris::EsHttpScanNode +override src/exec/es_http_scan_node.h /^ virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;$/;" m class:doris::EsHttpScanNode +override src/exec/es_http_scan_node.h /^ virtual Status open(RuntimeState* state) override;$/;" m class:doris::EsHttpScanNode +override src/exec/es_http_scan_node.h /^ virtual Status prepare(RuntimeState* state) override;$/;" m class:doris::EsHttpScanNode +override src/exec/es_http_scan_node.h /^ virtual Status set_scan_ranges(const std::vector& scan_ranges) override;$/;" m class:doris::EsHttpScanNode +override src/exec/es_http_scan_node.h /^ virtual void debug_string(int indentation_level, std::stringstream* out) const override;$/;" m class:doris::EsHttpScanNode +override src/exec/es_scan_node.h /^ virtual Status close(RuntimeState* state) override;$/;" m class:doris::EsScanNode +override src/exec/es_scan_node.h /^ virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;$/;" m class:doris::EsScanNode +override src/exec/es_scan_node.h /^ virtual Status open(RuntimeState* state) override;$/;" m class:doris::EsScanNode +override src/exec/es_scan_node.h /^ virtual Status prepare(RuntimeState* state) override;$/;" m class:doris::EsScanNode +override src/exec/es_scan_node.h /^ virtual Status set_scan_ranges(const std::vector& scan_ranges) override;$/;" m class:doris::EsScanNode +override src/exec/exchange_node.h /^ Status collect_query_statistics(QueryStatistics* statistics) override;$/;" m class:doris::ExchangeNode +override src/exec/local_file_reader.h /^ virtual Status open() override;$/;" m class:doris::LocalFileReader +override src/exec/local_file_reader.h /^ virtual Status read(uint8_t* buf, size_t* buf_len, bool* eof) override;$/;" m class:doris::LocalFileReader +override src/exec/local_file_reader.h /^ virtual Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) override;$/;" m class:doris::LocalFileReader +override src/exec/local_file_reader.h /^ virtual Status seek(int64_t position) override;$/;" m class:doris::LocalFileReader +override src/exec/local_file_reader.h /^ virtual Status tell(int64_t* position) override;$/;" m class:doris::LocalFileReader +override src/exec/local_file_reader.h /^ virtual bool closed() override;$/;" m class:doris::LocalFileReader +override src/exec/local_file_reader.h /^ virtual int64_t size () override;$/;" m class:doris::LocalFileReader +override src/exec/local_file_reader.h /^ virtual void close() override;$/;" m class:doris::LocalFileReader +override src/exec/local_file_writer.h /^ Status open() override;$/;" m class:doris::LocalFileWriter +override src/exec/local_file_writer.h /^ virtual Status close() override;$/;" m class:doris::LocalFileWriter +override src/exec/local_file_writer.h /^ virtual Status write(const uint8_t* buf, size_t buf_len, size_t* written_len) override;$/;" m class:doris::LocalFileWriter +override src/exec/olap_scan_node.h /^ Status collect_query_statistics(QueryStatistics* statistics) override;$/;" m class:doris::OlapScanNode +override src/exec/orc_scanner.h /^ Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;$/;" m class:doris::ORCScanner +override src/exec/orc_scanner.h /^ Status open() override;$/;" m class:doris::ORCScanner +override src/exec/orc_scanner.h /^ void close() override;$/;" m class:doris::ORCScanner +override src/exec/orc_scanner.h /^ ~ORCScanner() override;$/;" m class:doris::ORCScanner +override src/exec/parquet_reader.h /^ void* out) override;$/;" m class:doris::ParquetFile +override src/exec/parquet_reader.h /^ arrow::Status Close() override;$/;" m class:doris::ParquetFile +override src/exec/parquet_reader.h /^ arrow::Status GetSize(int64_t* size) override;$/;" m class:doris::ParquetFile +override src/exec/parquet_reader.h /^ arrow::Status Read(int64_t nbytes, int64_t* bytes_read, void* buffer) override;$/;" m class:doris::ParquetFile +override src/exec/parquet_reader.h /^ arrow::Status Read(int64_t nbytes, std::shared_ptr* out) override;$/;" m class:doris::ParquetFile +override src/exec/parquet_reader.h /^ arrow::Status Seek(int64_t position) override;$/;" m class:doris::ParquetFile +override src/exec/parquet_reader.h /^ arrow::Status Tell(int64_t* position) const override;$/;" m class:doris::ParquetFile +override src/exec/parquet_reader.h /^ bool closed() const override;$/;" m class:doris::ParquetFile +override src/exec/plain_text_line_reader.h /^ virtual Status read_line(const uint8_t** ptr, size_t* size, bool* eof) override;$/;" m class:doris::PlainTextLineReader +override src/exec/plain_text_line_reader.h /^ virtual void close() override;$/;" m class:doris::PlainTextLineReader +override src/exec/repeat_node.h /^ virtual Status close(RuntimeState* state) override;$/;" m class:doris::RepeatNode +override src/exec/repeat_node.h /^ virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;$/;" m class:doris::RepeatNode +override src/exec/repeat_node.h /^ virtual Status open(RuntimeState* state) override;$/;" m class:doris::RepeatNode +override src/exec/repeat_node.h /^ virtual Status prepare(RuntimeState* state) override;$/;" m class:doris::RepeatNode +override src/exec/repeat_node.h /^ virtual void debug_string(int indentation_level, std::stringstream* out) const override;$/;" m class:doris::RepeatNode +override src/exec/schema_scan_node.h /^ Status close(RuntimeState* state) override;$/;" m class:doris::SchemaScanNode +override src/exec/schema_scan_node.h /^ Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;$/;" m class:doris::SchemaScanNode +override src/exec/schema_scan_node.h /^ Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;$/;" m class:doris::SchemaScanNode +override src/exec/schema_scan_node.h /^ Status open(RuntimeState* state) override;$/;" m class:doris::SchemaScanNode +override src/exec/schema_scan_node.h /^ Status prepare(RuntimeState* state) override;$/;" m class:doris::SchemaScanNode +override src/exec/schema_scan_node.h /^ Status set_scan_ranges(const std::vector& scan_ranges) override;$/;" m class:doris::SchemaScanNode +override src/exec/schema_scan_node.h /^ void debug_string(int indentation_level, std::stringstream* out) const override;$/;" m class:doris::SchemaScanNode +override src/exec/tablet_sink.h /^ Status close(RuntimeState* state, Status close_status) override;$/;" m class:doris::stream_load::OlapTableSink +override src/exec/tablet_sink.h /^ Status init(const TDataSink& sink) override;$/;" m class:doris::stream_load::OlapTableSink +override src/exec/tablet_sink.h /^ Status open(RuntimeState* state) override;$/;" m class:doris::stream_load::OlapTableSink +override src/exec/tablet_sink.h /^ Status prepare(RuntimeState* state) override;$/;" m class:doris::stream_load::OlapTableSink +override src/exec/tablet_sink.h /^ Status send(RuntimeState* state, RowBatch* batch) override;$/;" m class:doris::stream_load::OlapTableSink +override src/exec/tablet_sink.h /^ ~OlapTableSink() override;$/;" m class:doris::stream_load::OlapTableSink +override src/geo/geo_types.h /^ bool contains(const GeoShape* rhs) const override;$/;" m class:doris::GeoCircle +override src/geo/geo_types.h /^ bool contains(const GeoShape* rhs) const override;$/;" m class:doris::GeoPolygon +override src/geo/geo_types.h /^ bool decode(const void* data, size_t size) override;$/;" m class:doris::GeoCircle +override src/geo/geo_types.h /^ bool decode(const void* data, size_t size) override;$/;" m class:doris::GeoLine +override src/geo/geo_types.h /^ bool decode(const void* data, size_t size) override;$/;" m class:doris::GeoPoint +override src/geo/geo_types.h /^ bool decode(const void* data, size_t size) override;$/;" m class:doris::GeoPolygon +override src/geo/geo_types.h /^ std::string as_wkt() const override;$/;" m class:doris::GeoCircle +override src/geo/geo_types.h /^ std::string as_wkt() const override;$/;" m class:doris::GeoLine +override src/geo/geo_types.h /^ std::string as_wkt() const override;$/;" m class:doris::GeoPoint +override src/geo/geo_types.h /^ std::string as_wkt() const override;$/;" m class:doris::GeoPolygon +override src/geo/geo_types.h /^ std::string to_string() const override;$/;" m class:doris::GeoPoint +override src/geo/geo_types.h /^ void encode(std::string* buf) override;$/;" m class:doris::GeoCircle +override src/geo/geo_types.h /^ void encode(std::string* buf) override;$/;" m class:doris::GeoLine +override src/geo/geo_types.h /^ void encode(std::string* buf) override;$/;" m class:doris::GeoPoint +override src/geo/geo_types.h /^ void encode(std::string* buf) override;$/;" m class:doris::GeoPolygon +override src/gutil/threading/thread_collision_warner.h /^ void warn(int64_t previous_thread_id, int64_t current_thread_id) override;$/;" m struct:base::DCheckAsserter +override src/http/action/checksum_action.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::ChecksumAction +override src/http/action/compaction_action.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::CompactionAction +override src/http/action/health_action.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::HealthAction +override src/http/action/meta_action.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::MetaAction +override src/http/action/metrics_action.cpp /^ MetricCollector* collector) override;$/;" m class:doris::PrometheusMetricsVisitor file: +override src/http/action/metrics_action.cpp /^ MetricCollector* collector) override;$/;" m class:doris::SimpleCoreMetricsVisitor file: +override src/http/action/metrics_action.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::MetricsAction +override src/http/action/mini_load.h /^ int on_header(HttpRequest* req) override;$/;" m class:doris::MiniLoadAction +override src/http/action/mini_load.h /^ void free_handler_ctx(void* ctx) override;$/;" m class:doris::MiniLoadAction +override src/http/action/mini_load.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::MiniLoadAction +override src/http/action/mini_load.h /^ void on_chunk_data(HttpRequest* req) override;$/;" m class:doris::MiniLoadAction +override src/http/action/pprof_actions.cpp /^ virtual void handle(HttpRequest* req) override;$/;" m class:doris::CmdlineAction file: +override src/http/action/pprof_actions.cpp /^ virtual void handle(HttpRequest* req) override;$/;" m class:doris::GrowthAction file: +override src/http/action/pprof_actions.cpp /^ virtual void handle(HttpRequest* req) override;$/;" m class:doris::HeapAction file: +override src/http/action/pprof_actions.cpp /^ virtual void handle(HttpRequest* req) override;$/;" m class:doris::ProfileAction file: +override src/http/action/pprof_actions.cpp /^ virtual void handle(HttpRequest* req) override;$/;" m class:doris::SymbolAction file: +override src/http/action/reload_tablet_action.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::ReloadTabletAction +override src/http/action/restore_tablet_action.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::RestoreTabletAction +override src/http/action/snapshot_action.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::SnapshotAction +override src/http/action/stream_load.h /^ int on_header(HttpRequest* req) override;$/;" m class:doris::StreamLoadAction +override src/http/action/stream_load.h /^ void free_handler_ctx(void* ctx) override;$/;" m class:doris::StreamLoadAction +override src/http/action/stream_load.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::StreamLoadAction +override src/http/action/stream_load.h /^ void on_chunk_data(HttpRequest* req) override;$/;" m class:doris::StreamLoadAction +override src/http/action/stream_load.h /^ ~StreamLoadAction() override;$/;" m class:doris::StreamLoadAction +override src/http/download_action.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::DownloadAction +override src/http/monitor_action.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::MonitorAction +override src/http/web_page_handler.h /^ void handle(HttpRequest *req) override;$/;" m class:doris::WebPageHandler +override src/olap/base_compaction.h /^ OLAPStatus compact() override;$/;" m class:doris::BaseCompaction +override src/olap/base_compaction.h /^ OLAPStatus pick_rowsets_to_compact() override;$/;" m class:doris::BaseCompaction +override src/olap/base_compaction.h /^ ~BaseCompaction() override;$/;" m class:doris::BaseCompaction +override src/olap/cumulative_compaction.h /^ OLAPStatus compact() override;$/;" m class:doris::CumulativeCompaction +override src/olap/cumulative_compaction.h /^ OLAPStatus pick_rowsets_to_compact() override;$/;" m class:doris::CumulativeCompaction +override src/olap/cumulative_compaction.h /^ ~CumulativeCompaction() override;$/;" m class:doris::CumulativeCompaction +override src/olap/fs/file_block_manager.cpp /^ virtual BlockManager* block_manager() const override;$/;" m class:doris::fs::internal::FileReadableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual BlockManager* block_manager() const override;$/;" m class:doris::fs::internal::FileWritableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual State state() const override;$/;" m class:doris::fs::internal::FileWritableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual Status abort() override;$/;" m class:doris::fs::internal::FileWritableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual Status append(const Slice& data) override;$/;" m class:doris::fs::internal::FileWritableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual Status appendv(const Slice* data, size_t data_cnt) override;$/;" m class:doris::fs::internal::FileWritableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual Status close() override;$/;" m class:doris::fs::internal::FileReadableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual Status close() override;$/;" m class:doris::fs::internal::FileWritableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual Status finalize() override;$/;" m class:doris::fs::internal::FileWritableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual Status read(uint64_t offset, Slice result) const override;$/;" m class:doris::fs::internal::FileReadableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual Status readv(uint64_t offset, const Slice* results, size_t res_cnt) const override;$/;" m class:doris::fs::internal::FileReadableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual Status size(uint64_t* sz) const override;$/;" m class:doris::fs::internal::FileReadableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual const BlockId& id() const override;$/;" m class:doris::fs::internal::FileReadableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual const BlockId& id() const override;$/;" m class:doris::fs::internal::FileWritableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual const std::string& path() const override;$/;" m class:doris::fs::internal::FileReadableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual const std::string& path() const override;$/;" m class:doris::fs::internal::FileWritableBlock file: +override src/olap/fs/file_block_manager.cpp /^ virtual size_t bytes_appended() const override;$/;" m class:doris::fs::internal::FileWritableBlock file: +override src/olap/fs/file_block_manager.h /^ std::unique_ptr* block) override;$/;" m class:doris::fs::FileBlockManager +override src/olap/fs/file_block_manager.h /^ Status open() override;$/;" m class:doris::fs::FileBlockManager +override src/olap/fs/file_block_manager.h /^ Status open_block(const std::string& path, std::unique_ptr* block) override;$/;" m class:doris::fs::FileBlockManager +override src/olap/generic_iterators.cpp /^ Status init(const StorageReadOptions& opts) override;$/;" m class:doris::AutoIncrementIterator file: +override src/olap/generic_iterators.cpp /^ Status init(const StorageReadOptions& opts) override;$/;" m class:doris::MergeIterator file: +override src/olap/generic_iterators.cpp /^ Status init(const StorageReadOptions& opts) override;$/;" m class:doris::UnionIterator file: +override src/olap/generic_iterators.cpp /^ Status next_batch(RowBlockV2* block) override;$/;" m class:doris::AutoIncrementIterator file: +override src/olap/generic_iterators.cpp /^ Status next_batch(RowBlockV2* block) override;$/;" m class:doris::MergeIterator file: +override src/olap/generic_iterators.cpp /^ Status next_batch(RowBlockV2* block) override;$/;" m class:doris::UnionIterator file: +override src/olap/lru_cache.h /^ Slice value_slice(Handle* handle) override;$/;" m class:doris::CachePriority::ShardedLRUCache +override src/olap/null_predicate.h /^ uint32_t num_rows, Roaring* roaring) const override;$/;" m class:doris::NullPredicate +override src/olap/null_predicate.h /^ virtual void evaluate(VectorizedRowBatch* batch) const override;$/;" m class:doris::NullPredicate +override src/olap/null_predicate.h /^ void evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const override;$/;" m class:doris::NullPredicate +override src/olap/rowset/alpha_rowset.h /^ std::vector* ranges) override;$/;" m class:doris::AlphaRowset +override src/olap/rowset/alpha_rowset.h /^ OLAPStatus copy_files_to(const std::string& dir) override;$/;" m class:doris::AlphaRowset +override src/olap/rowset/alpha_rowset.h /^ OLAPStatus create_reader(std::shared_ptr* result) override;$/;" m class:doris::AlphaRowset +override src/olap/rowset/alpha_rowset.h /^ OLAPStatus do_load(bool use_cache) override;$/;" m class:doris::AlphaRowset +override src/olap/rowset/alpha_rowset.h /^ OLAPStatus init() override;$/;" m class:doris::AlphaRowset +override src/olap/rowset/alpha_rowset.h /^ OLAPStatus link_files_to(const std::string& dir, RowsetId new_rowset_id) override;$/;" m class:doris::AlphaRowset +override src/olap/rowset/alpha_rowset.h /^ OLAPStatus remove() override;$/;" m class:doris::AlphaRowset +override src/olap/rowset/alpha_rowset.h /^ OLAPStatus remove_old_files(std::vector* files_to_remove) override;$/;" m class:doris::AlphaRowset +override src/olap/rowset/alpha_rowset.h /^ bool check_path(const std::string& path) override;$/;" m class:doris::AlphaRowset +override src/olap/rowset/alpha_rowset.h /^ void make_visible_extra(Version version, VersionHash version_hash) override;$/;" m class:doris::AlphaRowset +override src/olap/rowset/alpha_rowset_reader.h /^ OLAPStatus init(RowsetReaderContext* read_context) override;$/;" m class:doris::AlphaRowsetReader +override src/olap/rowset/alpha_rowset_reader.h /^ OLAPStatus next_block(RowBlock** block) override;$/;" m class:doris::AlphaRowsetReader +override src/olap/rowset/alpha_rowset_reader.h /^ RowsetSharedPtr rowset() override;$/;" m class:doris::AlphaRowsetReader +override src/olap/rowset/alpha_rowset_reader.h /^ Version version() override;$/;" m class:doris::AlphaRowsetReader +override src/olap/rowset/alpha_rowset_reader.h /^ VersionHash version_hash() override;$/;" m class:doris::AlphaRowsetReader +override src/olap/rowset/alpha_rowset_reader.h /^ bool delete_flag() override;$/;" m class:doris::AlphaRowsetReader +override src/olap/rowset/alpha_rowset_reader.h /^ int64_t filtered_rows() override;$/;" m class:doris::AlphaRowsetReader +override src/olap/rowset/alpha_rowset_reader.h /^ ~AlphaRowsetReader() override;$/;" m class:doris::AlphaRowsetReader +override src/olap/rowset/alpha_rowset_writer.h /^ RowsetSharedPtr rowset, const SchemaMapping& schema_mapping) override;$/;" m class:doris::AlphaRowsetWriter +override src/olap/rowset/alpha_rowset_writer.h /^ OLAPStatus add_rowset(RowsetSharedPtr rowset) override;$/;" m class:doris::AlphaRowsetWriter +override src/olap/rowset/alpha_rowset_writer.h /^ OLAPStatus flush() override;$/;" m class:doris::AlphaRowsetWriter +override src/olap/rowset/alpha_rowset_writer.h /^ OLAPStatus init(const RowsetWriterContext& rowset_writer_context) override;$/;" m class:doris::AlphaRowsetWriter +override src/olap/rowset/alpha_rowset_writer.h /^ RowsetSharedPtr build() override;$/;" m class:doris::AlphaRowsetWriter +override src/olap/rowset/beta_rowset.h /^ std::vector* ranges) override;$/;" m class:doris::BetaRowset +override src/olap/rowset/beta_rowset.h /^ OLAPStatus copy_files_to(const std::string& dir) override;$/;" m class:doris::BetaRowset +override src/olap/rowset/beta_rowset.h /^ OLAPStatus create_reader(RowsetReaderSharedPtr* result) override;$/;" m class:doris::BetaRowset +override src/olap/rowset/beta_rowset.h /^ OLAPStatus do_load(bool use_cache) override;$/;" m class:doris::BetaRowset +override src/olap/rowset/beta_rowset.h /^ OLAPStatus init() override;$/;" m class:doris::BetaRowset +override src/olap/rowset/beta_rowset.h /^ OLAPStatus link_files_to(const std::string& dir, RowsetId new_rowset_id) override;$/;" m class:doris::BetaRowset +override src/olap/rowset/beta_rowset.h /^ OLAPStatus remove() override;$/;" m class:doris::BetaRowset +override src/olap/rowset/beta_rowset.h /^ bool check_path(const std::string& path) override;$/;" m class:doris::BetaRowset +override src/olap/rowset/beta_rowset.h /^ void do_close() override;$/;" m class:doris::BetaRowset +override src/olap/rowset/beta_rowset_reader.h /^ OLAPStatus init(RowsetReaderContext* read_context) override;$/;" m class:doris::BetaRowsetReader +override src/olap/rowset/beta_rowset_reader.h /^ OLAPStatus next_block(RowBlock** block) override;$/;" m class:doris::BetaRowsetReader +override src/olap/rowset/beta_rowset_writer.h /^ RowsetSharedPtr rowset, const SchemaMapping& schema_mapping) override;$/;" m class:doris::BetaRowsetWriter +override src/olap/rowset/beta_rowset_writer.h /^ OLAPStatus add_rowset(RowsetSharedPtr rowset) override;$/;" m class:doris::BetaRowsetWriter +override src/olap/rowset/beta_rowset_writer.h /^ OLAPStatus flush() override;$/;" m class:doris::BetaRowsetWriter +override src/olap/rowset/beta_rowset_writer.h /^ OLAPStatus init(const RowsetWriterContext& rowset_writer_context) override;$/;" m class:doris::BetaRowsetWriter +override src/olap/rowset/beta_rowset_writer.h /^ RowsetSharedPtr build() override;$/;" m class:doris::BetaRowsetWriter +override src/olap/rowset/beta_rowset_writer.h /^ ~BetaRowsetWriter() override;$/;" m class:doris::BetaRowsetWriter +override src/olap/rowset/column_reader.h /^ OlapReaderStatistics* stats) override;$/;" m class:doris::DecimalColumnReader +override src/olap/rowset/segment_v2/binary_dict_page.h /^ OwnedSlice finish() override;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ Status add(const uint8_t* vals, size_t* count) override;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ Status get_dictionary_page(OwnedSlice* dictionary_page) override;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ Status get_first_value(void* value) const override;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ Status get_last_value(void* value) const override;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ Status init() override;$/;" m class:doris::segment_v2::BinaryDictPageDecoder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ Status next_batch(size_t* n, ColumnBlockView* dst) override;$/;" m class:doris::segment_v2::BinaryDictPageDecoder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ Status seek_to_position_in_page(size_t pos) override;$/;" m class:doris::segment_v2::BinaryDictPageDecoder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ bool is_page_full() override;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ size_t count() const override;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ uint64_t size() const override;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +override src/olap/rowset/segment_v2/binary_dict_page.h /^ void reset() override;$/;" m class:doris::segment_v2::BinaryDictPageBuilder +override src/olap/rowset/segment_v2/binary_prefix_page.h /^ OwnedSlice finish() override;$/;" m class:doris::segment_v2::BinaryPrefixPageBuilder +override src/olap/rowset/segment_v2/binary_prefix_page.h /^ Status add(const uint8_t* vals, size_t* add_count) override;$/;" m class:doris::segment_v2::BinaryPrefixPageBuilder +override src/olap/rowset/segment_v2/binary_prefix_page.h /^ Status init() override;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +override src/olap/rowset/segment_v2/binary_prefix_page.h /^ Status next_batch(size_t* n, ColumnBlockView* dst) override;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +override src/olap/rowset/segment_v2/binary_prefix_page.h /^ Status seek_at_or_after_value(const void* value, bool* exact_match) override;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +override src/olap/rowset/segment_v2/binary_prefix_page.h /^ Status seek_to_position_in_page(size_t pos) override;$/;" m class:doris::segment_v2::BinaryPrefixPageDecoder +override src/olap/rowset/segment_v2/block_split_bloom_filter.h /^ bool test_hash(uint64_t hash) const override;$/;" m class:doris::segment_v2::BlockSplitBloomFilter +override src/olap/rowset/segment_v2/block_split_bloom_filter.h /^ void add_hash(uint64_t hash) override;$/;" m class:doris::segment_v2::BlockSplitBloomFilter +override src/olap/rowset/segment_v2/column_reader.h /^ RowRanges* row_ranges) override;$/;" m class:doris::segment_v2::FileColumnIterator +override src/olap/rowset/segment_v2/column_reader.h /^ Status get_row_ranges_by_bloom_filter(CondColumn* cond_column, RowRanges* row_ranges) override;$/;" m class:doris::segment_v2::FileColumnIterator +override src/olap/rowset/segment_v2/column_reader.h /^ Status init(const ColumnIteratorOptions& opts) override;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +override src/olap/rowset/segment_v2/column_reader.h /^ Status next_batch(size_t* n, ColumnBlockView* dst) override;$/;" m class:doris::segment_v2::DefaultValueColumnIterator +override src/olap/rowset/segment_v2/column_reader.h /^ Status next_batch(size_t* n, ColumnBlockView* dst) override;$/;" m class:doris::segment_v2::FileColumnIterator +override src/olap/rowset/segment_v2/column_reader.h /^ Status seek_to_first() override;$/;" m class:doris::segment_v2::FileColumnIterator +override src/olap/rowset/segment_v2/column_reader.h /^ Status seek_to_ordinal(ordinal_t ord) override;$/;" m class:doris::segment_v2::FileColumnIterator +override src/olap/rowset/segment_v2/column_reader.h /^ ~FileColumnIterator() override;$/;" m class:doris::segment_v2::FileColumnIterator +override src/olap/rowset/segment_v2/empty_segment_iterator.h /^ Status next_batch(RowBlockV2* row_block) override;$/;" m class:doris::segment_v2::EmptySegmentIterator +override src/olap/rowset/segment_v2/segment_iterator.h /^ Status init(const StorageReadOptions& opts) override;$/;" m class:doris::segment_v2::SegmentIterator +override src/olap/rowset/segment_v2/segment_iterator.h /^ Status next_batch(RowBlockV2* row_block) override;$/;" m class:doris::segment_v2::SegmentIterator +override src/olap/rowset/segment_v2/segment_iterator.h /^ ~SegmentIterator() override;$/;" m class:doris::segment_v2::SegmentIterator +override src/olap/rowset/unique_rowset_id_generator.h /^ RowsetId next_id() override;$/;" m class:doris::UniqueRowsetIdGenerator +override src/olap/rowset/unique_rowset_id_generator.h /^ bool id_in_use(const RowsetId& rowset_id) override;$/;" m class:doris::UniqueRowsetIdGenerator +override src/olap/rowset/unique_rowset_id_generator.h /^ void release_id(const RowsetId& rowset_id) override;$/;" m class:doris::UniqueRowsetIdGenerator +override src/olap/schema_change.h /^ TabletSharedPtr base_tablet) override;$/;" m class:doris::SchemaChangeDirectly +override src/olap/schema_change.h /^ TabletSharedPtr base_tablet) override;$/;" m class:doris::SchemaChangeWithSorting +override src/olap/schema_change.h /^ TabletSharedPtr base_tablet) override;$/;" m class:doris::LinkedSchemaChange +override src/olap/task/engine_publish_version_task.h /^ virtual OLAPStatus finish() override;$/;" m class:doris::EnginePublishVersionTask +override src/runtime/export_sink.h /^ virtual Status close(RuntimeState* state, Status exec_status) override;$/;" m class:doris::ExportSink +override src/runtime/export_sink.h /^ virtual Status init(const TDataSink& thrift_sink) override;$/;" m class:doris::ExportSink +override src/runtime/export_sink.h /^ virtual Status open(RuntimeState* state) override;$/;" m class:doris::ExportSink +override src/runtime/export_sink.h /^ virtual Status prepare(RuntimeState* state) override;$/;" m class:doris::ExportSink +override src/runtime/export_sink.h /^ virtual Status send(RuntimeState* state, RowBatch* batch) override;$/;" m class:doris::ExportSink +override src/runtime/message_body_sink.h /^ Status append(const char* data, size_t size) override;$/;" m class:doris::MessageBodyFileSink +override src/runtime/message_body_sink.h /^ Status finish() override;$/;" m class:doris::MessageBodyFileSink +override src/runtime/message_body_sink.h /^ void cancel() override;$/;" m class:doris::MessageBodyFileSink +override src/runtime/result_sink.h /^ void set_query_statistics(std::shared_ptr statistics) override;$/;" m class:doris::ResultSink +override src/runtime/routine_load/data_consumer.h /^ virtual Status cancel(StreamLoadContext* ctx) override;$/;" m class:doris::KafkaDataConsumer +override src/runtime/routine_load/data_consumer.h /^ virtual Status init(StreamLoadContext* ctx) override;$/;" m class:doris::KafkaDataConsumer +override src/runtime/routine_load/data_consumer.h /^ virtual Status reset() override;$/;" m class:doris::KafkaDataConsumer +override src/runtime/routine_load/data_consumer.h /^ virtual bool match(StreamLoadContext* ctx) override;$/;" m class:doris::KafkaDataConsumer +override src/runtime/routine_load/data_consumer_group.h /^ virtual Status start_all(StreamLoadContext* ctx) override;$/;" m class:doris::KafkaDataConsumerGroup +override src/service/backend_service.h /^ virtual void get_tablet_stat(TTabletStatResult& result) override;$/;" m class:doris::BackendService +override src/service/backend_service.h /^ virtual void submit_routine_load_task(TStatus& t_status, const std::vector& tasks) override;$/;" m class:doris::BackendService +override src/service/backend_service.h /^ void erase_export_task(TStatus& t_status, const TUniqueId& task_id) override;$/;" m class:doris::BackendService +override src/service/backend_service.h /^ void get_export_status(TExportStatusResult& result, const TUniqueId& task_id) override;$/;" m class:doris::BackendService +override src/service/backend_service.h /^ void submit_export_task(TStatus& t_status, const TExportTaskRequest& request) override;$/;" m class:doris::BackendService +override src/service/internal_service.h /^ google::protobuf::Closure* done) override;$/;" m class:doris::PInternalServiceImpl +override src/service/internal_service.h /^ google::protobuf::Closure* done) override;$/;" m class:doris::PInternalServiceImpl +override src/service/internal_service.h /^ google::protobuf::Closure* done) override;$/;" m class:doris::PInternalServiceImpl +override src/service/internal_service.h /^ ::google::protobuf::Closure* done) override;$/;" m class:doris::PInternalServiceImpl +override src/service/internal_service.h /^ google::protobuf::Closure* done) override;$/;" m class:doris::PInternalServiceImpl +own_lock src/util/mutex.h /^ bool own_lock() { return _locked; }$/;" f class:doris::ReadLock +own_lock src/util/mutex.h /^ bool own_lock() { return _locked; }$/;" f class:doris::WriteLock +own_lock src/util/mutex.h /^ inline bool own_lock() const { return _locked; }$/;" f class:doris::MutexLock +pack_vlen src/util/mysql_row_buffer.cpp /^static char* pack_vlen(char* packet, uint64_t length) {$/;" f namespace:doris +packed_time output/udf/include/udf.h /^ int64_t packed_time;$/;" m struct:doris_udf::DateTimeVal +packed_time src/udf/udf.h /^ int64_t packed_time;$/;" m struct:doris_udf::DateTimeVal +pad1 src/gutil/linux_syscall_support.h /^ short pad1;$/;" m struct:kernel_stat +pad2 src/gutil/linux_syscall_support.h /^ short pad2;$/;" m struct:kernel_stat +pad_extra_bits_wit_zeroes src/olap/selection_vector.h /^ void pad_extra_bits_wit_zeroes() {$/;" f class:doris::SelectionVector +page src/olap/rowset/segment_v2/ordinal_page_index.h /^ const PagePointer& page() const { return _index->_pages[_cur_idx]; };$/;" f class:doris::segment_v2::OrdinalPageIndexIterator +page_ src/runtime/bufferpool/buffer_pool.h /^ Page* page_;$/;" m class:doris::BufferPool::PageHandle +page_handle src/olap/rowset/segment_v2/parsed_page.h /^ PageHandle page_handle;$/;" m struct:doris::segment_v2::ParsedPage +page_index src/olap/rowset/segment_v2/ordinal_page_index.h /^ int32_t page_index() const { return _cur_idx; };$/;" f class:doris::segment_v2::OrdinalPageIndexIterator +page_index src/olap/rowset/segment_v2/parsed_page.h /^ uint32_t page_index = 0;$/;" m struct:doris::segment_v2::ParsedPage +page_pointer src/olap/rowset/segment_v2/page_io.h /^ PagePointer page_pointer;$/;" m struct:doris::segment_v2::PageReadOptions +page_pointer src/olap/rowset/segment_v2/parsed_page.h /^ PagePointer page_pointer;$/;" m struct:doris::segment_v2::ParsedPage +page_zone_maps src/olap/rowset/segment_v2/zone_map_index.h /^ const std::vector& page_zone_maps() const { return _page_zone_maps; }$/;" f class:doris::segment_v2::ZoneMapIndexReader +pages_ src/runtime/buffered_tuple_stream3.h /^ std::list pages_;$/;" m class:doris::BufferedTupleStream3 +pair src/olap/stream_index_common.h /^ std::pair pair() const {$/;" f class:doris::ColumnStatistics +param src/http/http_request.cpp /^const std::string& HttpRequest::param(const std::string& key) const {$/;" f class:doris::HttpRequest +params src/http/http_request.h /^ std::map* params() {$/;" f class:doris::HttpRequest +parent src/exec/new_partitioned_aggregation_node.h /^ NewPartitionedAggregationNode* parent;$/;" m struct:doris::NewPartitionedAggregationNode::Partition +parent src/exec/partitioned_aggregation_node.h /^ PartitionedAggregationNode* parent;$/;" m struct:doris::PartitionedAggregationNode::Partition +parent src/runtime/bufferpool/reservation_tracker.h /^ ReservationTracker* parent() const { return parent_; }$/;" f class:doris::ReservationTracker +parent src/runtime/descriptors.h /^ TupleId parent() const {$/;" f class:doris::SlotDescriptor +parent src/runtime/mem_tracker.h /^ MemTracker* parent() const {$/;" f class:doris::MemTracker +parent src/runtime/row_batch.h /^ RowBatch* parent() { return _parent; }$/;" f class:doris::RowBatch::Iterator +parent_ src/runtime/bufferpool/buffer_allocator.cc /^ BufferAllocator* const parent_;$/;" m class:doris::BufferPool::FreeBufferArena file: +parent_ src/runtime/bufferpool/reservation_tracker.h /^ ReservationTracker* parent_ = nullptr;$/;" m class:doris::ReservationTracker +parent_ src/runtime/bufferpool/suballocator.h /^ std::unique_ptr parent_;$/;" m class:doris::Suballocation +parent_mem_tracker src/olap/fs/block_manager.h /^ std::shared_ptr parent_mem_tracker;$/;" m struct:doris::fs::BlockManagerOptions +parent_queue src/util/internal_queue.h /^ InternalQueueBase* parent_queue;$/;" m struct:doris::InternalQueueBase::Node +parse src/exec/es/es_scroll_parser.cpp /^Status ScrollParser::parse(const std::string& scroll_result, bool exactly_once) {$/;" f class:doris::ScrollParser +parse src/olap/hll.cpp /^void HllSetResolver::parse() {$/;" f class:doris::HllSetResolver +parse src/olap/rowset/segment_v2/index_page.cpp /^Status IndexPageReader::parse(const Slice& body, const IndexPageFooterPB& footer) {$/;" f class:doris::segment_v2::IndexPageReader +parse src/olap/short_key_index.cpp /^Status ShortKeyIndexDecoder::parse(const Slice& body, const segment_v2::ShortKeyFooterPB& footer) {$/;" f class:doris::ShortKeyIndexDecoder +parse src/util/bfd_parser.cpp /^int BfdParser::parse() {$/;" f class:doris::BfdParser +parse_auth src/http/action/mini_load.cpp /^static bool parse_auth(const std::string& auth, std::string* user,$/;" f namespace:doris +parse_basic_auth src/http/utils.cpp /^bool parse_basic_auth(const HttpRequest& req, AuthInfo* auth) {$/;" f namespace:doris +parse_basic_auth src/http/utils.cpp /^bool parse_basic_auth(const HttpRequest& req, std::string* user, std::string* passwd) {$/;" f namespace:doris +parse_conf_store_paths src/olap/options.cpp /^OLAPStatus parse_conf_store_paths(const string& config_path, vector* paths) {$/;" f namespace:doris +parse_format src/http/action/stream_load.cpp /^static TFileFormatType::type parse_format(const std::string& format_str) {$/;" f namespace:doris +parse_from_str src/runtime/decimal_value.cpp /^int DecimalValue::parse_from_str(const char* decimal_str, int32_t length) {$/;" f class:doris::DecimalValue +parse_from_str src/runtime/decimalv2_value.cpp /^int DecimalV2Value::parse_from_str(const char* decimal_str, int32_t length) {$/;" f class:doris::DecimalV2Value +parse_header_info src/exec/lzo_decompressor.cpp /^Status LzopDecompressor::parse_header_info(uint8_t* input, size_t input_len,$/;" f class:doris::LzopDecompressor +parse_id src/util/uid_util.cpp /^bool parse_id(const std::string& s, TUniqueId* id) {$/;" f namespace:doris +parse_mem_spec src/util/parse_util.cpp /^int64_t ParseUtil::parse_mem_spec(const std::string& mem_spec_str, bool* is_percent) {$/;" f class:doris::ParseUtil +parse_op_type src/olap/olap_cond.cpp /^static CondOp parse_op_type(const string& op) {$/;" f namespace:doris +parse_root_path src/olap/options.cpp /^OLAPStatus parse_root_path(const string& root_path, StorePath* path) {$/;" f namespace:doris +parse_status src/geo/wkt_parse_ctx.h /^ doris::GeoParseStatus parse_status = doris::GEO_PARSE_OK;$/;" m struct:WktParseContext +parse_string src/exprs/expr.cpp /^bool parse_string(const std::string& str, T* val) {$/;" f namespace:doris +parse_url src/exprs/string_functions.cpp /^StringVal StringFunctions::parse_url($/;" f class:doris::StringFunctions +parse_url src/util/url_parser.cpp /^bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* result) {$/;" f class:doris::UrlParser +parse_url_close src/exprs/string_functions.cpp /^void StringFunctions::parse_url_close($/;" f class:doris::StringFunctions +parse_url_key src/exprs/string_functions.cpp /^StringVal StringFunctions::parse_url_key($/;" f class:doris::StringFunctions +parse_url_key src/util/url_parser.cpp /^bool UrlParser::parse_url_key($/;" f class:doris::UrlParser +parse_url_prepare src/exprs/string_functions.cpp /^void StringFunctions::parse_url_prepare($/;" f class:doris::StringFunctions +parse_wkt src/geo/wkt_parse.cpp /^GeoParseStatus WktParse::parse_wkt(const char* str, size_t len, GeoShape** shape) {$/;" f class:doris::WktParse +partition src/runtime/merge_sorter.cpp /^MergeSorter::TupleSorter::TupleIterator MergeSorter::TupleSorter::partition(TupleIterator first,$/;" f class:doris::MergeSorter::TupleSorter +partition src/runtime/spill_sorter.cc /^SpillSorter::TupleSorter::TupleIterator SpillSorter::TupleSorter::partition($/;" f class:doris::SpillSorter::TupleSorter +partition_eos_ src/exec/new_partitioned_aggregation_node.h /^ bool partition_eos_;$/;" m class:doris::NewPartitionedAggregationNode +partition_id src/olap/delta_writer.cpp /^int64_t DeltaWriter::partition_id() const {$/;" f class:doris::DeltaWriter +partition_id src/olap/delta_writer.h /^ int64_t partition_id;$/;" m struct:doris::WriteRequest +partition_id src/olap/rowset/rowset.h /^ int64_t partition_id() const { return rowset_meta()->partition_id(); }$/;" f class:doris::Rowset +partition_id src/olap/rowset/rowset_meta.h /^ int64_t partition_id() const {$/;" f class:doris::RowsetMeta +partition_id src/olap/rowset/rowset_writer_context.h /^ int64_t partition_id;$/;" m struct:doris::RowsetWriterContext +partition_id src/olap/rowset/segment_group.h /^ inline TPartitionId partition_id() const { return _partition_id; }$/;" f class:doris::SegmentGroup +partition_id src/olap/tablet.h /^inline int64_t Tablet::partition_id() const {$/;" f class:doris::Tablet +partition_id src/olap/tablet_meta.h /^inline int64_t TabletMeta::partition_id() const {$/;" f class:doris::TabletMeta +partition_id src/runtime/dpp_sink_internal.h /^ int64_t partition_id;$/;" m struct:doris::TabletDesc +partition_id test/olap/txn_manager_test.cpp /^ TPartitionId partition_id = 1123;$/;" m class:doris::TxnManagerTest file: +partition_id test/runtime/load_channel_mgr_test.cpp /^int64_t DeltaWriter::partition_id() const { return 1L; }$/;" f class:doris::DeltaWriter +partition_pool_ src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr partition_pool_;$/;" m class:doris::NewPartitionedAggregationNode +partitions_created_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* partitions_created_;$/;" m class:doris::NewPartitionedAggregationNode +passwd src/common/utils.h /^ std::string passwd;$/;" m struct:doris::AuthInfo +passwd src/exec/mysql_scanner.h /^ std::string passwd;$/;" m struct:doris::MysqlScannerParam +passwd src/runtime/descriptors.h /^ const std::string passwd() const {$/;" f class:doris::MySQLTableDescriptor +passwd src/runtime/mysql_table_writer.h /^ std::string passwd;$/;" m struct:doris::MysqlConnInfo +passwd src/util/mysql_load_error_hub.h /^ std::string passwd;$/;" m struct:doris::MysqlLoadErrorHub::MysqlInfo +patch_length src/olap/rowset/run_length_integer_writer.h /^ uint8_t patch_length: 5,$/;" m struct:doris::RunLengthIntegerWriter::PatchedBaseHead +patch_width src/olap/rowset/run_length_integer_writer.h /^ uint8_t patch_width: 5,$/;" m struct:doris::RunLengthIntegerWriter::PatchedBaseHead +path src/olap/data_dir.h /^ const std::string& path() const { return _path; }$/;" f class:doris::DataDir +path src/olap/fs/block_manager.h /^ const std::string path;$/;" m struct:doris::fs::CreateBlockOptions +path src/olap/fs/file_block_manager.cpp /^const string& FileReadableBlock::path() const {$/;" f class:doris::fs::internal::FileReadableBlock +path src/olap/fs/file_block_manager.cpp /^const string& FileWritableBlock::path() const {$/;" f class:doris::fs::internal::FileWritableBlock +path src/olap/olap_common.h /^ std::string path;$/;" m struct:doris::DataDirInfo +path src/olap/options.h /^ std::string path;$/;" m struct:doris::StorePath +path src/runtime/small_file_mgr.h /^ std::string path; \/\/ absolute path$/;" m struct:doris::CacheEntry +path src/runtime/tmp_file_mgr.h /^ const std::string& path() const {$/;" f class:doris::TmpFileMgr::Dir +path src/runtime/tmp_file_mgr.h /^ const std::string& path() const {$/;" f class:doris::TmpFileMgr::File +path src/util/broker_load_error_hub.h /^ std::string path;$/;" m struct:doris::BrokerLoadErrorHub::BrokerInfo +path_hash src/olap/data_dir.h /^ size_t path_hash() const { return _path_hash; }$/;" f class:doris::DataDir +path_hash src/olap/olap_common.h /^ size_t path_hash;$/;" m struct:doris::DataDirInfo +path_of_fd src/util/file_utils.cpp /^std::string FileUtils::path_of_fd(int fd) {$/;" f class:doris::FileUtils +path_util src/util/path_util.cpp /^namespace path_util {$/;" n namespace:doris file: +path_util src/util/path_util.h /^namespace path_util {$/;" n namespace:doris +pb_batch src/runtime/data_stream_sender.cpp /^ PRowBatch* pb_batch() { $/;" f class:doris::DataStreamSender::Channel +pc src/util/bfd_parser.cpp /^ bfd_vma pc;$/;" m struct:doris::BfdFindCtx file: +pc_finalize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::pc_finalize(FunctionContext* c, const StringVal& src) {$/;" f class:doris::AggregateFunctions +pc_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::pc_init(FunctionContext* c, StringVal* dst) {$/;" f class:doris::AggregateFunctions +pc_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::pc_merge(FunctionContext* c,$/;" f class:doris::AggregateFunctions +pc_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::pc_update(FunctionContext* c, const T& input, StringVal* dst) {$/;" f class:doris::AggregateFunctions +pcrc_32_tab src/util/minizip/unzip.c /^ const z_crc_t* pcrc_32_tab;$/;" m struct:__anon34 file: +pcsa_finalize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::pcsa_finalize(FunctionContext* c, const StringVal& src) {$/;" f class:doris::AggregateFunctions +pcsa_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::pcsa_update(FunctionContext* c, const T& input, StringVal* dst) {$/;" f class:doris::AggregateFunctions +peak_allocated_bytes src/runtime/mem_pool.h /^ int64_t peak_allocated_bytes() const { return peak_allocated_bytes_;}$/;" f class:doris::MemPool +peak_allocated_bytes_ src/runtime/mem_pool.h /^ int64_t peak_allocated_bytes_;$/;" m class:doris::MemPool +peak_consumption src/runtime/mem_tracker.h /^ int64_t peak_consumption() const { return _consumption->value(); }$/;" f class:doris::MemTracker +peak_reservation src/runtime/bufferpool/reservation_tracker_counters.h /^ RuntimeProfile::HighWaterMarkCounter* peak_reservation;$/;" m struct:doris::ReservationTrackerCounters +peak_unpinned_bytes src/runtime/bufferpool/buffer_pool_counters.h /^ RuntimeProfile::HighWaterMarkCounter* peak_unpinned_bytes;$/;" m struct:doris::BufferPoolClientCounters +peak_used_reservation src/runtime/bufferpool/reservation_tracker_counters.h /^ RuntimeProfile::HighWaterMarkCounter* peak_used_reservation;$/;" m struct:doris::ReservationTrackerCounters +per_core_arenas_ src/runtime/bufferpool/buffer_allocator.h /^ std::vector> per_core_arenas_;$/;" m struct:doris::BufferPool::BufferAllocator +per_fragment_instance_idx src/runtime/runtime_state.h /^ int per_fragment_instance_idx() const {$/;" f class:doris::RuntimeState +per_read_thread_throughput_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* per_read_thread_throughput_counter() const {$/;" f class:doris::ScanNode +percentile_approx_finalize src/exprs/aggregate_functions.cpp /^DoubleVal AggregateFunctions::percentile_approx_finalize(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::AggregateFunctions +percentile_approx_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::percentile_approx_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +percentile_approx_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::percentile_approx_merge(FunctionContext* ctx, const StringVal& src, StringVal* dst) {$/;" f class:doris::AggregateFunctions +percentile_approx_serialize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::percentile_approx_serialize(FunctionContext* ctx, const StringVal& src) {$/;" f class:doris::AggregateFunctions +percentile_approx_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::percentile_approx_update(FunctionContext* ctx, const T& src, const DoubleVal& quantile, StringVal* dst) {$/;" f class:doris::AggregateFunctions +percentile_approx_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::percentile_approx_update(FunctionContext* ctx, const T& src, const DoubleVal& quantile,$/;" f class:doris::AggregateFunctions +percentile_bits src/olap/serialize.cpp /^uint32_t percentile_bits(int64_t* data, uint16_t count, double p) {$/;" f namespace:doris::ser +percentile_bits_with_hist src/olap/serialize.h /^inline uint32_t percentile_bits_with_hist(uint16_t hists[65], uint16_t count, double p) {$/;" f namespace:doris::ser +perform_path_gc_by_rowsetid src/olap/data_dir.cpp /^void DataDir::perform_path_gc_by_rowsetid() {$/;" f class:doris::DataDir +perform_path_scan src/olap/data_dir.cpp /^void DataDir::perform_path_scan() {$/;" f class:doris::DataDir +period_ src/util/streaming_sampler.h /^ int period_;$/;" m class:doris::StreamingSampler +periodic_counter_update_loop src/util/runtime_profile.cpp /^void RuntimeProfile::periodic_counter_update_loop() {$/;" f class:doris::RuntimeProfile +pfile_in_zip_read src/util/minizip/unzip.c /^ file_in_zip64_read_info_s* pfile_in_zip_read; \/* structure about the current$/;" m struct:__anon34 file: +physical_mem src/util/mem_info.h /^ static int64_t physical_mem() {$/;" f class:doris::MemInfo +pi src/exprs/math_functions.cpp /^DoubleVal MathFunctions::pi(FunctionContext* ctx) {$/;" f class:doris::MathFunctions +pi src/exprs/udf_builtins.cpp /^DoubleVal UdfBuiltins::pi(FunctionContext* context) {$/;" f class:doris::UdfBuiltins +pick_candicate_rowsets_to_base_compaction src/olap/tablet.cpp /^void Tablet::pick_candicate_rowsets_to_base_compaction(vector* candidate_rowsets) {$/;" f class:doris::Tablet +pick_candicate_rowsets_to_cumulative_compaction src/olap/tablet.cpp /^void Tablet::pick_candicate_rowsets_to_cumulative_compaction($/;" f class:doris::Tablet +pick_rowsets_to_compact src/olap/base_compaction.cpp /^OLAPStatus BaseCompaction::pick_rowsets_to_compact() {$/;" f class:doris::BaseCompaction +pick_rowsets_to_compact src/olap/cumulative_compaction.cpp /^OLAPStatus CumulativeCompaction::pick_rowsets_to_compact() {$/;" f class:doris::CumulativeCompaction +pid_t src/gutil/port.h /^typedef int pid_t;$/;" t +piece src/gutil/strings/strcat.h /^ StringPiece piece;$/;" m struct:AlphaNum +pin src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::Block::pin(bool* pinned, Block* release_block, bool unpin) {$/;" f class:doris::BufferedBlockMgr2::Block +pin_block src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::pin_block(Block* block, bool* pinned, Block* release_block, bool unpin) {$/;" f class:doris::BufferedBlockMgr2 +pin_buffer src/runtime/buffered_block_mgr2.cc /^ void pin_buffer(BufferDescriptor* buffer) {$/;" f class:doris::BufferedBlockMgr2::Client +pin_count src/runtime/buffered_tuple_stream3.h /^ inline int pin_count() const { return handle.pin_count(); }$/;" f struct:doris::BufferedTupleStream3::Page +pin_count src/runtime/bufferpool/buffer_pool.cc /^int BufferPool::PageHandle::pin_count() const {$/;" f class:doris::BufferPool::PageHandle +pin_count src/runtime/bufferpool/buffer_pool_internal.h /^ int pin_count;$/;" m class:doris::BufferPool::Page +pin_in_flight src/runtime/bufferpool/buffer_pool_internal.h /^ bool pin_in_flight;$/;" m class:doris::BufferPool::Page +pin_stream src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::pin_stream(bool already_reserved, bool* pinned) {$/;" f class:doris::BufferedTupleStream2 +ping src/runtime/broker_mgr.cpp /^void BrokerMgr::ping(const TNetworkAddress& addr) {$/;" f class:doris::BrokerMgr +ping_worker src/runtime/broker_mgr.cpp /^void BrokerMgr::ping_worker() {$/;" f class:doris::BrokerMgr +pinned_ src/runtime/buffered_tuple_stream3.h /^ bool pinned_;$/;" m class:doris::BufferedTupleStream3 +pinned_pages_ src/runtime/bufferpool/buffer_pool_internal.h /^ PageList pinned_pages_;$/;" m class:doris::BufferPool::Client +pipe src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(pipe)( int * p) {$/;" f +pipe src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(pipe)(int *p) {$/;" f +pmod_bigint src/exprs/math_functions.cpp /^BigIntVal MathFunctions::pmod_bigint($/;" f class:doris::MathFunctions +pmod_double src/exprs/math_functions.cpp /^DoubleVal MathFunctions::pmod_double($/;" f class:doris::MathFunctions +point src/geo/geo_types.h /^ const S2Point& point() const { return _point; }$/;" f class:doris::GeoPoint +point src/geo/wkt_yacc.y /^point:$/;" l +pointer src/gutil/stl_util.h /^ typedef typename Alloc::pointer pointer;$/;" t class:STLCountingAllocator +pointer src/gutil/strings/stringpiece.h /^ typedef const char* pointer;$/;" t class:StringPiece +pointer src/util/bitmap_value.h /^ typedef uint64_t* pointer;$/;" t class:doris::detail::final +polygon src/geo/geo_types.h /^ const S2Polygon* polygon() const { return _polygon.get(); }$/;" f class:doris::GeoPolygon +polygon src/geo/wkt_yacc.y /^polygon:$/;" l +polyline src/geo/geo_types.h /^ const S2Polyline* polyline() const { return _polyline.get(); }$/;" f class:doris::GeoLine +pool src/olap/column_block.h /^ MemPool* pool() const { return _block->pool(); }$/;" f class:doris::ColumnBlockView +pool src/olap/column_block.h /^ MemPool* pool() const { return _pool; }$/;" f class:doris::ColumnBlock +pool src/olap/row_block2.h /^ MemPool* pool() const { return _pool.get(); }$/;" f class:doris::RowBlockV2 +pool_ src/runtime/bufferpool/buffer_allocator.h /^ BufferPool* const pool_;$/;" m struct:doris::BufferPool::BufferAllocator +pool_ src/runtime/bufferpool/buffer_pool_internal.h /^ BufferPool* const pool_;$/;" m class:doris::BufferPool::Client +pool_ src/runtime/bufferpool/suballocator.h /^ BufferPool* pool_;$/;" m class:doris::Suballocator +pool_mem_trackers src/runtime/exec_env.h /^ PoolMemTrackerRegistry* pool_mem_trackers() { return _pool_mem_trackers; }$/;" f class:doris::ExecEnv +pop_back src/runtime/bufferpool/buffer_pool_internal.h /^ Page* pop_back() {$/;" f class:doris::BufferPool::PageList +pop_back src/util/internal_queue.h /^ T* pop_back() {$/;" f class:doris::InternalQueueBase +pop_free_chunk src/runtime/memory/chunk_allocator.cpp /^ bool pop_free_chunk(size_t size, uint8_t** ptr) {$/;" f class:doris::ChunkArena +popcount src/util/bit_util.h /^ static inline int popcount(uint64_t x) {$/;" f class:doris::BitUtil +popcount_no_hw src/util/bit_util.h /^ static inline int popcount_no_hw(uint64_t x) {$/;" f class:doris::BitUtil +port src/exec/mysql_scanner.h /^ std::string port;$/;" m struct:doris::MysqlScannerParam +port src/exec/schema_scanner.h /^ int32_t port; \/\/ frontend thrift port$/;" m struct:doris::SchemaScannerParam +port src/runtime/descriptors.h /^ const std::string port() const {$/;" f class:doris::MySQLTableDescriptor +port src/runtime/mysql_table_writer.h /^ int port;$/;" m struct:doris::MysqlConnInfo +port src/util/mysql_load_error_hub.h /^ int32_t port;$/;" m struct:doris::MysqlLoadErrorHub::MysqlInfo +port src/util/thrift_client.h /^ int port() {$/;" f class:doris::ThriftClientImpl +port src/util/thrift_server.h /^ int port() const {$/;" f class:doris::ThriftServer +pos src/olap/row_block.h /^ size_t pos() const { return _pos; }$/;" f class:doris::RowBlock +pos src/util/byte_buffer.h /^ size_t pos;$/;" m struct:doris::ByteBuffer +pos src/util/mysql_row_buffer.h /^ const char* pos() const {$/;" f class:doris::MysqlRowBuffer +pos test/util/utf8_check_test.cpp /^ std::vector pos = {$/;" m class:doris::Utf8CheckTest file: +pos_in_central_dir src/util/minizip/unzip.c /^ ZPOS64_T pos_in_central_dir; \/* pos of the current file in the central dir*\/$/;" m struct:__anon34 file: +pos_in_zip_directory src/util/minizip/unzip.h /^ ZPOS64_T pos_in_zip_directory; \/* offset in minizip file directory *\/$/;" m struct:unz64_file_pos_s +pos_in_zip_directory src/util/minizip/unzip.h /^ uLong pos_in_zip_directory; \/* offset in minizip file directory *\/$/;" m struct:unz_file_pos_s +pos_in_zipfile src/util/minizip/unzip.c /^ ZPOS64_T pos_in_zipfile; \/* position in byte on the zipfile, for fseek*\/$/;" m struct:__anon33 file: +pos_inc src/olap/row_block.h /^ void pos_inc() { _pos++; }$/;" f class:doris::RowBlock +pos_infinite src/runtime/dpp_sink_internal.h /^ static const PartRangeKey& pos_infinite() {$/;" f class:doris::PartRangeKey +pos_local_extrafield src/util/minizip/unzip.c /^ ZPOS64_T pos_local_extrafield; \/* position in the local extra field in read*\/$/;" m struct:__anon33 file: +position src/olap/byte_buffer.h /^ inline uint64_t position() const {$/;" f class:doris::StorageByteBuffer +position src/olap/file_stream.h /^ size_t position() {$/;" f class:doris::ReadOnlyFileStream::FileCursor +position src/util/bit_stream_utils.h /^ int position() const { return byte_offset_ * 8 + bit_offset_; }$/;" f class:doris::BitReader +position_format src/olap/stream_index_common.h /^ uint32_t position_format; \/\/ position的个数,每个长度为sizeof(uint32_t)$/;" m struct:doris::StreamIndexHeader +positions src/olap/stream_index_reader.cpp /^int64_t PositionEntryReader::positions(size_t index) const {$/;" f class:doris::PositionEntryReader +positions src/olap/stream_index_writer.cpp /^int64_t PositionEntryWriter::positions(size_t index) const {$/;" f class:doris::PositionEntryWriter +positions_count src/olap/stream_index_reader.cpp /^int32_t PositionEntryReader::positions_count() const {$/;" f class:doris::PositionEntryReader +positions_count src/olap/stream_index_writer.cpp /^int32_t PositionEntryWriter::positions_count() const {$/;" f class:doris::PositionEntryWriter +positive_bigint src/exprs/math_functions.cpp /^BigIntVal MathFunctions::positive_bigint($/;" f class:doris::MathFunctions +positive_decimal src/exprs/math_functions.cpp /^DecimalV2Val MathFunctions::positive_decimal($/;" f class:doris::MathFunctions +positive_decimal src/exprs/math_functions.cpp /^DecimalVal MathFunctions::positive_decimal($/;" f class:doris::MathFunctions +positive_double src/exprs/math_functions.cpp /^DoubleVal MathFunctions::positive_double($/;" f class:doris::MathFunctions +pow src/exprs/math_functions.cpp /^DoubleVal MathFunctions::pow($/;" f class:doris::MathFunctions +powers10 src/runtime/decimal_value.h /^static const int32_t powers10[DIG_PER_DEC1 + 1] =$/;" m namespace:doris +preServe src/util/thrift_server.cpp /^void ThriftServer::ThriftServerEventProcessor::preServe() {$/;" f class:doris::ThriftServer::ThriftServerEventProcessor +pread src/olap/file_helper.cpp /^OLAPStatus FileHandler::pread(void* buf, size_t size, size_t offset) {$/;" f class:doris::FileHandler +pread src/olap/file_helper.cpp /^OLAPStatus FileHandlerWithBuf::pread(void* buf, size_t size, size_t offset) {$/;" f class:doris::FileHandlerWithBuf +preagg_estimated_reduction_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* preagg_estimated_reduction_;$/;" m class:doris::NewPartitionedAggregationNode +preagg_streaming_ht_min_reduction_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* preagg_streaming_ht_min_reduction_;$/;" m class:doris::NewPartitionedAggregationNode +precise_code src/common/status.h /^ int16_t precise_code() const {$/;" f class:doris::Status +precision output/udf/include/udf.h /^ int precision;$/;" m struct:doris_udf::FunctionContext::TypeDesc +precision src/olap/tablet_schema.h /^ int precision() const { return _precision; }$/;" f class:doris::TabletColumn +precision src/runtime/decimal_value.h /^ int32_t precision() const {$/;" f class:doris::DecimalValue +precision src/runtime/decimalv2_value.h /^ int32_t precision() const {$/;" f class:doris::DecimalV2Value +precision src/runtime/types.h /^ int precision;$/;" m struct:doris::TypeDescriptor +precision src/udf/udf.h /^ int precision;$/;" m struct:doris_udf::FunctionContext::TypeDesc +predicate_ src/gutil/strings/split_internal.h /^ Predicate predicate_;$/;" m class:strings::internal::SplitIterator +predicates src/olap/rowset/rowset_reader_context.h /^ const std::vector* predicates = nullptr;$/;" m struct:doris::RowsetReaderContext +preferred_rowset_type src/olap/tablet_meta.h /^ RowsetTypePB preferred_rowset_type() const {$/;" f class:doris::TabletMeta +prefetch src/gutil/port.h /^extern inline void prefetch(const char *x) {$/;" f +prefetch src/gutil/port.h /^extern inline void prefetch(const char *x, int hint) {$/;" f +prepare src/exec/aggregation_node.cpp /^Status AggregationNode::prepare(RuntimeState* state) {$/;" f class:doris::AggregationNode +prepare src/exec/analytic_eval_node.cpp /^Status AnalyticEvalNode::prepare(RuntimeState* state) {$/;" f class:doris::AnalyticEvalNode +prepare src/exec/assert_num_rows_node.cpp /^Status AssertNumRowsNode::prepare(RuntimeState* state) {$/;" f class:doris::AssertNumRowsNode +prepare src/exec/blocking_join_node.cpp /^Status BlockingJoinNode::prepare(RuntimeState* state) {$/;" f class:doris::BlockingJoinNode +prepare src/exec/broker_scan_node.cpp /^Status BrokerScanNode::prepare(RuntimeState* state) {$/;" f class:doris::BrokerScanNode +prepare src/exec/cross_join_node.cpp /^Status CrossJoinNode::prepare(RuntimeState* state) {$/;" f class:doris::CrossJoinNode +prepare src/exec/csv_scan_node.cpp /^Status CsvScanNode::prepare(RuntimeState* state) {$/;" f class:doris::CsvScanNode +prepare src/exec/data_sink.cpp /^Status DataSink::prepare(RuntimeState* state) {$/;" f class:doris::DataSink +prepare src/exec/es_http_scan_node.cpp /^Status EsHttpScanNode::prepare(RuntimeState* state) {$/;" f class:doris::EsHttpScanNode +prepare src/exec/es_scan_node.cpp /^Status EsScanNode::prepare(RuntimeState* state) {$/;" f class:doris::EsScanNode +prepare src/exec/exchange_node.cpp /^Status ExchangeNode::prepare(RuntimeState* state) {$/;" f class:doris::ExchangeNode +prepare src/exec/exec_node.cpp /^Status ExecNode::prepare(RuntimeState* state) {$/;" f class:doris::ExecNode +prepare src/exec/hash_join_node.cpp /^Status HashJoinNode::prepare(RuntimeState* state) {$/;" f class:doris::HashJoinNode +prepare src/exec/intersect_node.cpp /^Status IntersectNode::prepare(RuntimeState* state) {$/;" f class:doris::IntersectNode +prepare src/exec/kudu_scan_node.cpp /^Status KuduScanNode::prepare(RuntimeState* state) {$/;" f class:doris::KuduScanNode +prepare src/exec/merge_join_node.cpp /^Status MergeJoinNode::prepare(RuntimeState* state) {$/;" f class:doris::MergeJoinNode +prepare src/exec/merge_node.cpp /^Status MergeNode::prepare(RuntimeState* state) {$/;" f class:doris::MergeNode +prepare src/exec/mysql_scan_node.cpp /^Status MysqlScanNode::prepare(RuntimeState* state) {$/;" f class:doris::MysqlScanNode +prepare src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::prepare(RuntimeState* state) {$/;" f class:doris::NewPartitionedAggregationNode +prepare src/exec/olap_rewrite_node.cpp /^Status OlapRewriteNode::prepare(RuntimeState* state) {$/;" f class:doris::OlapRewriteNode +prepare src/exec/olap_scan_node.cpp /^Status OlapScanNode::prepare(RuntimeState* state) {$/;" f class:doris::OlapScanNode +prepare src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::prepare(RuntimeState* state) {$/;" f class:doris::PartitionedAggregationNode +prepare src/exec/pl_task_root.cpp /^Status ExchangeNode::prepare(RuntimeState* state) {$/;" f class:doris::ExchangeNode +prepare src/exec/pre_aggregation_node.cpp /^Status PreAggregationNode::prepare(RuntimeState* state) {$/;" f class:doris::PreAggregationNode +prepare src/exec/repeat_node.cpp /^Status RepeatNode::prepare(RuntimeState* state) {$/;" f class:doris::RepeatNode +prepare src/exec/scan_node.cpp /^Status ScanNode::prepare(RuntimeState* state) {$/;" f class:doris::ScanNode +prepare src/exec/schema_scan_node.cpp /^Status SchemaScanNode::prepare(RuntimeState* state) {$/;" f class:doris::SchemaScanNode +prepare src/exec/select_node.cpp /^Status SelectNode::prepare(RuntimeState* state) {$/;" f class:doris::SelectNode +prepare src/exec/sort_exec_exprs.cpp /^Status SortExecExprs::prepare(RuntimeState* state, const RowDescriptor& child_row_desc,$/;" f class:doris::SortExecExprs +prepare src/exec/sort_node.cpp /^Status SortNode::prepare(RuntimeState* state) {$/;" f class:doris::SortNode +prepare src/exec/spill_sort_node.cc /^Status SpillSortNode::prepare(RuntimeState* state) {$/;" f class:doris::SpillSortNode +prepare src/exec/tablet_sink.cpp /^Status OlapTableSink::prepare(RuntimeState* state) {$/;" f class:doris::stream_load::OlapTableSink +prepare src/exec/topn_node.cpp /^Status TopNNode::prepare(RuntimeState* state) {$/;" f class:doris::TopNNode +prepare src/exec/union_node.cpp /^Status UnionNode::prepare(RuntimeState* state) {$/;" f class:doris::UnionNode +prepare src/exprs/agg_fn_evaluator.cpp /^Status AggFnEvaluator::prepare($/;" f class:doris::AggFnEvaluator +prepare src/exprs/case_expr.cpp /^Status CaseExpr::prepare($/;" f class:doris::CaseExpr +prepare src/exprs/expr.cpp /^Status Expr::prepare($/;" f class:doris::Expr +prepare src/exprs/expr.cpp /^Status Expr::prepare(RuntimeState* state, const RowDescriptor& row_desc,$/;" f class:doris::Expr +prepare src/exprs/expr_context.cpp /^Status ExprContext::prepare(RuntimeState* state, const RowDescriptor& row_desc,$/;" f class:doris::ExprContext +prepare src/exprs/in_predicate.cpp /^Status InPredicate::prepare($/;" f class:doris::InPredicate +prepare src/exprs/in_predicate.cpp /^Status InPredicate::prepare(RuntimeState* state, const TypeDescriptor& type) {$/;" f class:doris::InPredicate +prepare src/exprs/scalar_fn_call.cpp /^Status ScalarFnCall::prepare($/;" f class:doris::ScalarFnCall +prepare src/exprs/slot_ref.cpp /^Status SlotRef::prepare($/;" f class:doris::SlotRef +prepare src/exprs/slot_ref.cpp /^Status SlotRef::prepare(const SlotDescriptor* slot_desc,$/;" f class:doris::SlotRef +prepare src/exprs/tuple_is_null_predicate.cpp /^Status TupleIsNullPredicate::prepare($/;" f class:doris::TupleIsNullPredicate +prepare src/olap/file_helper.h /^OLAPStatus FileHeader::prepare($/;" f class:doris::FileHeader +prepare src/olap/task/engine_task.h /^ virtual OLAPStatus prepare() { return OLAP_SUCCESS; }$/;" f class:doris::EngineTask +prepare src/runtime/data_spliter.cpp /^Status DataSpliter::prepare(RuntimeState* state) {$/;" f class:doris::DataSpliter +prepare src/runtime/data_stream_sender.cpp /^Status DataStreamSender::prepare(RuntimeState* state) {$/;" f class:doris::DataStreamSender +prepare src/runtime/dpp_sink.cpp /^Status Translator::prepare(RuntimeState* state) {$/;" f class:doris::Translator +prepare src/runtime/dpp_sink.cpp /^void HllDppSinkMerge::prepare(int count, MemPool* pool) {$/;" f class:doris::HllDppSinkMerge +prepare src/runtime/dpp_sink_internal.cpp /^Status PartitionInfo::prepare($/;" f class:doris::PartitionInfo +prepare src/runtime/dpp_sink_internal.cpp /^Status RollupSchema::prepare($/;" f class:doris::RollupSchema +prepare src/runtime/export_sink.cpp /^Status ExportSink::prepare(RuntimeState* state) {$/;" f class:doris::ExportSink +prepare src/runtime/fragment_mgr.cpp /^Status FragmentExecState::prepare(const TExecPlanFragmentParams& params) {$/;" f class:doris::FragmentExecState +prepare src/runtime/memory_scratch_sink.cpp /^Status MemoryScratchSink::prepare(RuntimeState* state) {$/;" f class:doris::MemoryScratchSink +prepare src/runtime/mysql_table_sink.cpp /^Status MysqlTableSink::prepare(RuntimeState* state) {$/;" f class:doris::MysqlTableSink +prepare src/runtime/plan_fragment_executor.cpp /^Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request) {$/;" f class:doris::PlanFragmentExecutor +prepare src/runtime/qsorter.cpp /^Status QSorter::prepare(RuntimeState* state) {$/;" f class:doris::QSorter +prepare src/runtime/result_sink.cpp /^Status ResultSink::prepare(RuntimeState* state) {$/;" f class:doris::ResultSink +prepare src/runtime/sorted_run_merger.cc /^Status SortedRunMerger::prepare(const vector& input_runs) {$/;" f class:doris::SortedRunMerger +prepare src/runtime/sorter.h /^ virtual Status prepare(RuntimeState* state) {$/;" f class:doris::Sorter +prepare src/util/broker_load_error_hub.cpp /^Status BrokerLoadErrorHub::prepare() {$/;" f class:doris::BrokerLoadErrorHub +prepare src/util/mysql_load_error_hub.cpp /^Status MysqlLoadErrorHub::prepare() {$/;" f class:doris::MysqlLoadErrorHub +prepare src/util/null_load_error_hub.cpp /^Status NullLoadErrorHub::prepare() {$/;" f class:doris::NullLoadErrorHub +prepare test/runtime/fragment_mgr_test.cpp /^Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request) {$/;" f class:doris::PlanFragmentExecutor +prepare_block_read src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::prepare_block_read($/;" f class:doris::ColumnData +prepare_bucket_for_insert src/exec/partitioned_hash_table.inline.h /^inline void PartitionedHashTable::prepare_bucket_for_insert(int64_t bucket_idx, uint32_t hash) {$/;" f class:doris::PartitionedHashTable +prepare_exprs src/runtime/memory_scratch_sink.cpp /^Status MemoryScratchSink::prepare_exprs(RuntimeState* state) {$/;" f class:doris::MemoryScratchSink +prepare_exprs src/runtime/result_sink.cpp /^Status ResultSink::prepare_exprs(RuntimeState* state) {$/;" f class:doris::ResultSink +prepare_for_output src/exec/topn_node.cpp /^void TopNNode::prepare_for_output() {$/;" f class:doris::TopNNode +prepare_for_read src/runtime/buffered_tuple_stream.cpp /^Status BufferedTupleStream::prepare_for_read(bool* got_buffer) {$/;" f class:doris::BufferedTupleStream +prepare_for_read src/runtime/buffered_tuple_stream.h /^ Status prepare_for_read() {$/;" f class:doris::BufferedTupleStream +prepare_for_read src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::prepare_for_read(bool delete_on_read, bool* got_buffer) {$/;" f class:doris::BufferedTupleStream2 +prepare_read src/runtime/merge_sorter.cpp /^Status MergeSorter::Run::prepare_read() {$/;" f class:doris::MergeSorter::Run +prepare_read src/runtime/spill_sorter.cc /^Status SpillSorter::Run::prepare_read() {$/;" f class:doris::SpillSorter::Run +prepare_txn src/olap/txn_manager.cpp /^OLAPStatus TxnManager::prepare_txn($/;" f class:doris::TxnManager +prepare_txn src/olap/txn_manager.cpp /^OLAPStatus TxnManager::prepare_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, TTransactionId transaction_id, $/;" f class:doris::TxnManager +pretty_print src/util/perf_counters.cpp /^void PerfCounters::pretty_print(ostream* s) const {$/;" f class:doris::PerfCounters +pretty_print src/util/runtime_profile.cpp /^void RuntimeProfile::pretty_print(std::ostream* s, const std::string& prefix) const {$/;" f class:doris::RuntimeProfile +prev src/olap/lru_cache.h /^ LRUHandle* prev;$/;" m struct:doris::CachePriority::LRUHandle +prev src/olap/olap_index.cpp /^const OLAPIndexOffset MemIndex::prev(const OLAPIndexOffset& pos) const {$/;" f class:doris::MemIndex +prev src/olap/skiplist.h /^ Node* prev[kMaxHeight];$/;" m struct:doris::SkipList::Hint +prev src/runtime/merge_sorter.cpp /^ void prev() {$/;" f class:doris::MergeSorter::TupleSorter::TupleIterator +prev src/runtime/spill_sorter.cc /^ void prev() {$/;" f class:doris::SpillSorter::TupleSorter::TupleIterator +prev src/util/internal_queue.h /^ T* prev() const {$/;" f struct:doris::InternalQueueBase::Node +prev_free_ src/runtime/bufferpool/suballocator.h /^ Suballocation* prev_free_;$/;" m class:doris::Suballocation +prev_node src/util/internal_queue.h /^ Node* prev_node;$/;" m struct:doris::InternalQueueBase::Node +prev_row_compare src/exec/analytic_eval_node.cpp /^inline bool AnalyticEvalNode::prev_row_compare(ExprContext* pred_ctx) {$/;" f class:doris::AnalyticEvalNode +print src/util/pretty_printer.h /^ print(T value, TUnit::type unit, bool verbose = false) {$/;" f class:doris::PrettyPrinter +print src/util/pretty_printer.h /^ print(const T& value, TUnit::type unit) {$/;" f class:doris::PrettyPrinter +print src/util/pretty_printer.h /^ static std::string print(bool value, TUnit::type ignored, bool verbose = false) {$/;" f class:doris::PrettyPrinter +print_agent_status src/agent/utils.cpp /^std::string AgentUtils::print_agent_status(AgentStatus status) {$/;" f class:doris::AgentUtils +print_bytes src/util/pretty_printer.h /^ static std::string print_bytes(int64_t value) {$/;" f class:doris::PrettyPrinter +print_child_counters src/util/runtime_profile.cpp /^void RuntimeProfile::print_child_counters($/;" f class:doris::RuntimeProfile +print_id src/util/uid_util.cpp /^std::string print_id(const PUniqueId& id) {$/;" f namespace:doris +print_id src/util/uid_util.cpp /^std::string print_id(const TUniqueId& id) {$/;" f namespace:doris +print_plan_node_type src/util/debug_util.cpp /^std::string print_plan_node_type(const TPlanNodeType::type& type) {$/;" f namespace:doris +print_position_debug_info src/olap/out_stream.h /^ void print_position_debug_info() {$/;" f class:doris::OutStream +print_position_debug_info src/olap/rowset/run_length_integer_writer.h /^ void print_position_debug_info() {$/;" f class:doris::RunLengthIntegerWriter +print_row src/olap/row.h /^std::string print_row(const RowType& row) {$/;" f namespace:doris +print_s2point src/geo/geo_types.cpp /^void print_s2point(std::ostream& os, const S2Point& point) {$/;" f namespace:doris +print_stats src/exec/partitioned_hash_table.cc /^string PartitionedHashTable::print_stats() const {$/;" f class:doris::PartitionedHashTable +print_stringList src/util/pretty_printer.h /^ static void print_stringList(const I& iterable, TUnit::type unit,$/;" f class:doris::PrettyPrinter +print_timems src/util/pretty_printer.h /^ static void print_timems(T value, std::stringstream* ss) {$/;" f class:doris::PrettyPrinter +print_value src/exprs/expr_context.cpp /^void ExprContext::print_value(TupleRow* row, std::string* str) {$/;" f class:doris::ExprContext +print_value src/exprs/expr_context.cpp /^void ExprContext::print_value(TupleRow* row, std::stringstream* stream) {$/;" f class:doris::ExprContext +print_value src/exprs/expr_context.cpp /^void ExprContext::print_value(void* value, std::string* str) {$/;" f class:doris::ExprContext +print_value src/exprs/expr_context.cpp /^void ExprContext::print_value(void* value, std::stringstream* stream) {$/;" f class:doris::ExprContext +print_value src/runtime/raw_value.cpp /^void RawValue::print_value(const void* value, const TypeDescriptor& type, int scale,$/;" f class:doris::RawValue +print_value_as_bytes src/runtime/raw_value.cpp /^void RawValue::print_value_as_bytes(const void* value, const TypeDescriptor& type,$/;" f class:doris::RawValue +print_volume_ids src/runtime/plan_fragment_executor.cpp /^void PlanFragmentExecutor::print_volume_ids($/;" f class:doris::PlanFragmentExecutor +printf src/util/bitmap_value.h /^ void printf() const {$/;" f class:doris::detail::Roaring64Map +priority src/olap/lru_cache.h /^ CachePriority priority = CachePriority::NORMAL;$/;" m struct:doris::CachePriority::LRUHandle +priority src/olap/tablet_sync_service.h /^ int priority;$/;" m struct:doris::FetchRowsetMetaTask +priority src/olap/tablet_sync_service.h /^ int priority;$/;" m struct:doris::FetchTabletMetaTask +priority src/olap/tablet_sync_service.h /^ int priority;$/;" m struct:doris::PushRowsetMetaTask +priority src/olap/tablet_sync_service.h /^ int priority;$/;" m struct:doris::PushTabletMetaTask +priority src/util/priority_thread_pool.hpp /^ int priority;$/;" m struct:doris::PriorityThreadPool::Task +private test/exec/olap_common_test.cpp 24;" d file: +private test/runtime/snapshot_loader_test.cpp 24;" d file: +pro src/olap/tablet_sync_service.h /^ std::shared_ptr> pro;$/;" m struct:doris::FetchRowsetMetaTask +pro src/olap/tablet_sync_service.h /^ std::shared_ptr> pro;$/;" m struct:doris::FetchTabletMetaTask +pro src/olap/tablet_sync_service.h /^ std::shared_ptr> pro;$/;" m struct:doris::PushRowsetMetaTask +pro src/olap/tablet_sync_service.h /^ std::shared_ptr> pro;$/;" m struct:doris::PushTabletMetaTask +probe src/exec/partitioned_hash_table.inline.h /^inline int64_t PartitionedHashTable::probe(Bucket* buckets, int64_t num_buckets,$/;" f class:doris::PartitionedHashTable +probe_expr_evals src/exec/new_partitioned_hash_table_ir.cc /^ExprContext* const* NewPartitionedHashTableCtx::probe_expr_evals() const {$/;" f class:NewPartitionedHashTableCtx +probe_expr_evals_ src/exec/new_partitioned_hash_table.h /^ std::vector probe_expr_evals_;$/;" m class:doris::NewPartitionedHashTableCtx +probe_exprs_ src/exec/new_partitioned_hash_table.h /^ const std::vector& probe_exprs_;$/;" m class:doris::NewPartitionedHashTableCtx +probe_row test/exec/hash_table_test.cpp /^ TupleRow* probe_row;$/;" m struct:doris::HashTableTest::ProbeTestData file: +probe_row test/exec/partitioned_hash_table_test.cpp /^ TupleRow* probe_row;$/;" m struct:doris::PartitionedHashTableTest::ProbeTestData file: +probe_test test/exec/hash_table_test.cpp /^ void probe_test(HashTable* table, ProbeTestData* data, int num_data, bool scan) {$/;" f class:doris::HashTableTest +proc_io_line_number src/util/perf_counters.h /^ int proc_io_line_number;$/;" m union:doris::PerfCounters::CounterData::__anon32 +proc_status_field src/util/perf_counters.h /^ std::string proc_status_field;$/;" m struct:doris::PerfCounters::CounterData +process src/agent/pusher.cpp /^AgentStatus Pusher::process(vector* tablet_infos) {$/;" f class:doris::Pusher +process src/olap/schema_change.cpp /^bool LinkedSchemaChange::process($/;" f class:doris::LinkedSchemaChange +process src/olap/schema_change.cpp /^bool SchemaChangeDirectly::process(RowsetReaderSharedPtr rowset_reader, RowsetWriter* rowset_writer,$/;" f class:doris::SchemaChangeDirectly +process src/olap/schema_change.cpp /^bool SchemaChangeWithSorting::process($/;" f class:doris::SchemaChangeWithSorting +process src/runtime/dpp_sink.cpp /^Status Translator::process(RuntimeState* state) {$/;" f class:doris::Translator +process src/runtime/dpp_sink.cpp /^void DppSink::process(RuntimeState* state, Translator* trans, CountDownLatch* latch) {$/;" f class:doris::DppSink +process src/util/tdigest.h /^ inline void process() {$/;" f class:doris::TDigest +process test/olap/rowset/rowset_converter_test.cpp /^void RowsetConverterTest::process(RowsetTypePB src_type, RowsetTypePB dst_type) {$/;" f class:doris::RowsetConverterTest +processContext src/util/thrift_server.cpp /^void ThriftServer::ThriftServerEventProcessor::processContext($/;" f class:doris::ThriftServer::ThriftServerEventProcessor +processIfNecessary src/util/tdigest.h /^ inline void processIfNecessary() {$/;" f class:doris::TDigest +process_abnormal_rows src/runtime/etl_job_mgr.h /^ int64_t process_abnormal_rows;$/;" m struct:doris::EtlJobResult +process_alter_tablet_v2 src/olap/schema_change.cpp /^OLAPStatus SchemaChangeHandler::process_alter_tablet_v2(const TAlterTabletReqV2& request) {$/;" f class:doris::SchemaChangeHandler +process_batch src/exec/partitioned_aggregation_node_ir.cc /^Status PartitionedAggregationNode::process_batch(RowBatch* batch, PartitionedHashTableCtx* ht_ctx) {$/;" f class:doris::PartitionedAggregationNode +process_batch_fn_ src/exec/new_partitioned_aggregation_node.h /^ ProcessBatchFn process_batch_fn_;$/;" m class:doris::NewPartitionedAggregationNode +process_batch_no_grouping src/exec/partitioned_aggregation_node_ir.cc /^Status PartitionedAggregationNode::process_batch_no_grouping($/;" f class:doris::PartitionedAggregationNode +process_batch_no_grouping_fn_ src/exec/new_partitioned_aggregation_node.h /^ ProcessBatchNoGroupingFn process_batch_no_grouping_fn_;$/;" m class:doris::NewPartitionedAggregationNode +process_batch_status_ src/exec/new_partitioned_aggregation_node.h /^ Status process_batch_status_;$/;" m class:doris::NewPartitionedAggregationNode +process_batch_streaming_fn_ src/exec/new_partitioned_aggregation_node.h /^ ProcessBatchStreamingFn process_batch_streaming_fn_;$/;" m class:doris::NewPartitionedAggregationNode +process_build_batch src/exec/hash_join_node_ir.cpp /^void HashJoinNode::process_build_batch(RowBatch* build_batch) {$/;" f class:doris::HashJoinNode +process_child_batch src/exec/analytic_eval_node.cpp /^Status AnalyticEvalNode::process_child_batch(RuntimeState* state) {$/;" f class:doris::AnalyticEvalNode +process_child_batches src/exec/analytic_eval_node.cpp /^Status AnalyticEvalNode::process_child_batches(RuntimeState* state) {$/;" f class:doris::AnalyticEvalNode +process_distribute src/runtime/data_spliter.cpp /^Status DataSpliter::process_distribute($/;" f class:doris::DataSpliter +process_distribute src/runtime/data_stream_sender.cpp /^Status DataStreamSender::process_distribute($/;" f class:doris::DataStreamSender +process_fd_num_limit_hard src/util/doris_metrics.cpp /^IntGauge DorisMetrics::process_fd_num_limit_hard;$/;" m class:doris::DorisMetrics file: +process_fd_num_limit_hard src/util/doris_metrics.h /^ static IntGauge process_fd_num_limit_hard;$/;" m class:doris::DorisMetrics +process_fd_num_limit_soft src/util/doris_metrics.cpp /^IntGauge DorisMetrics::process_fd_num_limit_soft;$/;" m class:doris::DorisMetrics file: +process_fd_num_limit_soft src/util/doris_metrics.h /^ static IntGauge process_fd_num_limit_soft;$/;" m class:doris::DorisMetrics +process_fd_num_used src/util/doris_metrics.cpp /^IntGauge DorisMetrics::process_fd_num_used;$/;" m class:doris::DorisMetrics file: +process_fd_num_used src/util/doris_metrics.h /^ static IntGauge process_fd_num_used;$/;" m class:doris::DorisMetrics +process_left_child_batch src/exec/cross_join_node.cpp /^int CrossJoinNode::process_left_child_batch(RowBatch* output_batch, RowBatch* batch,$/;" f class:doris::CrossJoinNode +process_mem_tracker src/runtime/exec_env.h /^ MemTracker* process_mem_tracker() { return _mem_tracker; }$/;" f class:doris::ExecEnv +process_normal_rows src/runtime/etl_job_mgr.h /^ int64_t process_normal_rows;$/;" m struct:doris::EtlJobResult +process_one_row src/runtime/data_spliter.cpp /^Status DataSpliter::process_one_row(RuntimeState* state, TupleRow* row) {$/;" f class:doris::DataSpliter +process_one_row src/runtime/dpp_sink.cpp /^Status Translator::process_one_row(TupleRow* row) {$/;" f class:doris::Translator +process_partition src/runtime/data_spliter.cpp /^Status DataSpliter::process_partition($/;" f class:doris::DataSpliter +process_path src/runtime/load_path_mgr.cpp /^void LoadPathMgr::process_path(time_t now, const std::string& path, int64_t reserve_hours) {$/;" f class:doris::LoadPathMgr +process_probe_batch src/exec/hash_join_node_ir.cpp /^int HashJoinNode::process_probe_batch(RowBatch* out_batch, RowBatch* probe_batch,$/;" f class:doris::HashJoinNode +process_row src/exec/partitioned_aggregation_node_ir.cc /^Status PartitionedAggregationNode::process_row(TupleRow* row, PartitionedHashTableCtx* ht_ctx) {$/;" f class:doris::PartitionedAggregationNode +process_row_batch_no_grouping src/exec/aggregation_node_ir.cpp /^void AggregationNode::process_row_batch_no_grouping(RowBatch* batch, MemPool* pool) {$/;" f class:doris::AggregationNode +process_row_batch_no_grouping src/exec/pre_aggregation_node.cpp /^Status PreAggregationNode::process_row_batch_no_grouping(RowBatch* batch) {$/;" f class:doris::PreAggregationNode +process_row_batch_with_grouping src/exec/aggregation_node_ir.cpp /^void AggregationNode::process_row_batch_with_grouping(RowBatch* batch, MemPool* pool) {$/;" f class:doris::AggregationNode +process_row_batch_with_grouping src/exec/pre_aggregation_node.cpp /^Status PreAggregationNode::process_row_batch_with_grouping(RowBatch* batch) {$/;" f class:doris::PreAggregationNode +process_scan_token src/exec/kudu_scan_node.cpp /^Status KuduScanNode::process_scan_token(KuduScanner* scanner, const string& scan_token) {$/;" f class:doris::KuduScanNode +process_stream src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::process_stream(BufferedTupleStream2* input_stream) {$/;" f class:doris::PartitionedAggregationNode +process_streaming_ingestion src/olap/push_handler.cpp /^OLAPStatus PushHandler::process_streaming_ingestion($/;" f class:doris::PushHandler +process_thread_num src/util/doris_metrics.cpp /^IntGauge DorisMetrics::process_thread_num;$/;" m class:doris::DorisMetrics file: +process_thread_num src/util/doris_metrics.h /^ static IntGauge process_thread_num;$/;" m class:doris::DorisMetrics +processed src/util/tdigest.h /^ const std::vector& processed() const { return _processed; }$/;" f class:doris::TDigest +processedWeight src/util/tdigest.h /^ Weight processedWeight() const { return _processed_weight; }$/;" f class:doris::TDigest +processed_utf_bytes src/util/simdutf8check.h /^struct processed_utf_bytes {$/;" s +profile src/olap/reader.h /^ RuntimeProfile* profile;$/;" m struct:doris::ReaderParams +profile src/runtime/buffered_block_mgr2.h /^ RuntimeProfile* profile() { {$/;" f class:doris::BufferedBlockMgr2 +profile src/runtime/data_spliter.h /^ virtual RuntimeProfile* profile() {$/;" f class:doris::DataSpliter +profile src/runtime/data_stream_recvr.hpp /^ RuntimeProfile* profile() {$/;" f class:doris::DataStreamRecvr +profile src/runtime/data_stream_sender.h /^ virtual RuntimeProfile* profile() {$/;" f class:doris::DataStreamSender +profile src/runtime/dpp_sink.cpp /^ RuntimeProfile* profile() {$/;" f class:doris::Translator +profile src/runtime/dpp_sink.h /^ RuntimeProfile* profile() {$/;" f class:doris::DppSink +profile src/runtime/export_sink.h /^ virtual RuntimeProfile* profile() {$/;" f class:doris::ExportSink +profile src/runtime/memory_scratch_sink.h /^ virtual RuntimeProfile* profile() {$/;" f class:doris::MemoryScratchSink +profile src/runtime/mysql_table_sink.h /^ virtual RuntimeProfile* profile() {$/;" f class:doris::MysqlTableSink +profile src/runtime/plan_fragment_executor.cpp /^RuntimeProfile* PlanFragmentExecutor::profile() {$/;" f class:doris::PlanFragmentExecutor +profile src/runtime/result_sink.h /^ virtual RuntimeProfile* profile() {$/;" f class:doris::ResultSink +profile src/util/debug_counters.h /^ static RuntimeProfile& profile() {$/;" f class:doris::DebugRuntimeProfile +profile src/util/dummy_runtime_profile.h /^ RuntimeProfile* profile() { return _profile; }$/;" f class:doris::DummyProfile +promise src/runtime/stream_load/stream_load_context.h /^ std::promise promise;$/;" m class:doris::StreamLoadContext +properties src/runtime/stream_load/stream_load_context.h /^ std::map properties;$/;" m class:doris::KafkaLoadInfo +propmap src/common/configbase.h /^ std::map propmap;$/;" m class:doris::config::Properties +props src/common/configbase.cpp /^Properties props;$/;" m namespace:doris::config file: +props src/util/broker_load_error_hub.h /^ std::map props;$/;" m struct:doris::BrokerLoadErrorHub::BrokerInfo +protected test/exec/olap_common_test.cpp 23;" d file: +protected test/runtime/snapshot_loader_test.cpp 25;" d file: +protobuf src/runtime/buffer_control_block.h /^namespace protobuf {$/;" n namespace:google +protobuf src/runtime/data_stream_mgr.h /^namespace protobuf {$/;" n namespace:google +protobuf src/runtime/data_stream_recvr.h /^namespace protobuf {$/;" n namespace:google +protobuf_checksum src/olap/file_helper.h /^ uint32_t protobuf_checksum;$/;" m struct:doris::_FixedFileHeader +protobuf_checksum src/olap/file_helper.h /^ uint32_t protobuf_checksum;$/;" m struct:doris::_FixedFileHeaderV2 +protobuf_length src/olap/file_helper.h /^ uint32_t protobuf_length;$/;" m struct:doris::_FixedFileHeader +protobuf_length src/olap/file_helper.h /^ uint64_t protobuf_length;$/;" m struct:doris::_FixedFileHeaderV2 +prune src/olap/lru_cache.cpp /^int LRUCache::prune() {$/;" f class:doris::LRUCache +prune src/olap/lru_cache.cpp /^void ShardedLRUCache::prune() {$/;" f class:doris::ShardedLRUCache +prune src/olap/lru_cache.h /^ virtual void prune() {}$/;" f class:doris::CachePriority::Cache +pthread_id src/util/thread.cpp /^pthread_t Thread::pthread_id() const {$/;" f class:doris::Thread +ptr output/udf/include/udf.h /^ uint8_t* ptr;$/;" m struct:doris_udf::StringVal +ptr src/gutil/gscoped_ptr.h /^ T* ptr;$/;" m struct:doris::internal::gscoped_ptr_impl::Data +ptr src/olap/wrapper_field.h /^ char* ptr() const { return _field_buf + 1; }$/;" f class:doris::WrapperField +ptr src/runtime/string_value.h /^ char* ptr;$/;" m struct:doris::StringValue +ptr src/udf/udf.h /^ uint8_t* ptr;$/;" m struct:doris_udf::StringVal +ptr src/util/byte_buffer.h /^ char* const ptr;$/;" m struct:doris::ByteBuffer +ptr_ src/gutil/gscoped_ptr.h /^ C* ptr_;$/;" m class:gscoped_ptr_malloc +ptr_ src/gutil/ref_counted.h /^ T* ptr_;$/;" m class:scoped_refptr +ptr_ src/gutil/ref_counted.h /^ scoped_refptr(scoped_refptr&& r) noexcept : ptr_(r.get()) { \/\/ NOLINT$/;" f class:scoped_refptr +ptr_ src/gutil/ref_counted.h /^ scoped_refptr(scoped_refptr&& r) noexcept : ptr_(r.get()) { \/\/ NOLINT$/;" f class:scoped_refptr +ptr_ src/gutil/strings/stringpiece.h /^ const char* ptr_;$/;" m class:StringPiece +ptrace_detach src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(ptrace_detach)(pid_t pid) {$/;" f +publish_cluster_state src/agent/agent_server.cpp /^void AgentServer::publish_cluster_state(TAgentResult& t_agent_result,$/;" f class:doris::AgentServer +publish_cluster_state src/service/backend_service.h /^ virtual void publish_cluster_state(TAgentResult& result,$/;" f class:doris::BackendService +publish_cluster_state test/runtime/data_stream_test.cpp /^ virtual void publish_cluster_state($/;" f class:doris::DorisTestBackend +publish_task_failed_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::publish_task_failed_total;$/;" m class:doris::DorisMetrics file: +publish_task_failed_total src/util/doris_metrics.h /^ static IntCounter publish_task_failed_total;$/;" m class:doris::DorisMetrics +publish_task_request_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::publish_task_request_total;$/;" m class:doris::DorisMetrics file: +publish_task_request_total src/util/doris_metrics.h /^ static IntCounter publish_task_request_total;$/;" m class:doris::DorisMetrics +publish_txn src/olap/txn_manager.cpp /^OLAPStatus TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id,$/;" f class:doris::TxnManager +publish_txn src/olap/txn_manager.cpp /^OLAPStatus TxnManager::publish_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, TTransactionId transaction_id,$/;" f class:doris::TxnManager +push_back src/util/faststring.h /^ void push_back(const char byte) {$/;" f class:doris::faststring +push_bigint src/util/mysql_row_buffer.cpp /^int MysqlRowBuffer::push_bigint(int64_t data) {$/;" f class:doris::MysqlRowBuffer +push_double src/util/mysql_row_buffer.cpp /^int MysqlRowBuffer::push_double(double data) {$/;" f class:doris::MysqlRowBuffer +push_down_predicate src/exec/aggregation_node.cpp /^void AggregationNode::push_down_predicate(RuntimeState *state,$/;" f class:doris::AggregationNode +push_down_predicate src/exec/exec_node.cpp /^void ExecNode::push_down_predicate($/;" f class:doris::ExecNode +push_down_predicate src/exec/topn_node.cpp /^void TopNNode::push_down_predicate($/;" f class:doris::TopNNode +push_float src/util/mysql_row_buffer.cpp /^int MysqlRowBuffer::push_float(float data) {$/;" f class:doris::MysqlRowBuffer +push_free_chunk src/runtime/memory/chunk_allocator.cpp /^ void push_free_chunk(uint8_t* ptr, size_t size) {$/;" f class:doris::ChunkArena +push_int src/util/mysql_row_buffer.cpp /^int MysqlRowBuffer::push_int(int32_t data) {$/;" f class:doris::MysqlRowBuffer +push_last_2bytes_of_a_to_b src/util/simdutf8check.h /^static inline __m256i push_last_2bytes_of_a_to_b(__m256i a, __m256i b) {$/;" f +push_last_byte_of_a_to_b src/util/simdutf8check.h /^static inline __m256i push_last_byte_of_a_to_b(__m256i a, __m256i b) {$/;" f +push_null src/util/mysql_row_buffer.cpp /^int MysqlRowBuffer::push_null() {$/;" f class:doris::MysqlRowBuffer +push_request_duration_us src/util/doris_metrics.cpp /^IntCounter DorisMetrics::push_request_duration_us;$/;" m class:doris::DorisMetrics file: +push_request_duration_us src/util/doris_metrics.h /^ static IntCounter push_request_duration_us;$/;" m class:doris::DorisMetrics +push_request_write_bytes src/util/doris_metrics.cpp /^IntCounter DorisMetrics::push_request_write_bytes;$/;" m class:doris::DorisMetrics file: +push_request_write_bytes src/util/doris_metrics.h /^ static IntCounter push_request_write_bytes;$/;" m class:doris::DorisMetrics +push_request_write_bytes_per_second src/util/doris_metrics.cpp /^IntGauge DorisMetrics::push_request_write_bytes_per_second;$/;" m class:doris::DorisMetrics file: +push_request_write_bytes_per_second src/util/doris_metrics.h /^ static IntGauge push_request_write_bytes_per_second;$/;" m class:doris::DorisMetrics +push_request_write_rows src/util/doris_metrics.cpp /^IntCounter DorisMetrics::push_request_write_rows;$/;" m class:doris::DorisMetrics file: +push_request_write_rows src/util/doris_metrics.h /^ static IntCounter push_request_write_rows;$/;" m class:doris::DorisMetrics +push_requests_fail_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::push_requests_fail_total;$/;" m class:doris::DorisMetrics file: +push_requests_fail_total src/util/doris_metrics.h /^ static IntCounter push_requests_fail_total;$/;" m class:doris::DorisMetrics +push_requests_success_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::push_requests_success_total;$/;" m class:doris::DorisMetrics file: +push_requests_success_total src/util/doris_metrics.h /^ static IntCounter push_requests_success_total;$/;" m class:doris::DorisMetrics +push_rowset_meta src/olap/tablet_sync_service.cpp /^std::future TabletSyncService::push_rowset_meta(RowsetMetaPB& rowset_meta) {$/;" f class:doris::TabletSyncService +push_smallint src/util/mysql_row_buffer.cpp /^int MysqlRowBuffer::push_smallint(int16_t data) {$/;" f class:doris::MysqlRowBuffer +push_string src/util/mysql_row_buffer.cpp /^int MysqlRowBuffer::push_string(const char* str, int length) {$/;" f class:doris::MysqlRowBuffer +push_tablet_meta src/olap/tablet_sync_service.cpp /^std::future TabletSyncService::push_tablet_meta(TabletMetaPB& tablet_meta) {$/;" f class:doris::TabletSyncService +push_tinyint src/util/mysql_row_buffer.cpp /^int MysqlRowBuffer::push_tinyint(int8_t data) {$/;" f class:doris::MysqlRowBuffer +push_unsigned_bigint src/util/mysql_row_buffer.cpp /^int MysqlRowBuffer::push_unsigned_bigint(uint64_t data) {$/;" f class:doris::MysqlRowBuffer +put src/olap/byte_buffer.cpp /^OLAPStatus StorageByteBuffer::put(char src) {$/;" f class:doris::StorageByteBuffer +put src/olap/byte_buffer.cpp /^OLAPStatus StorageByteBuffer::put(const char* src, uint64_t src_size, uint64_t offset,$/;" f class:doris::StorageByteBuffer +put src/olap/byte_buffer.cpp /^OLAPStatus StorageByteBuffer::put(uint64_t index, char src) {$/;" f class:doris::StorageByteBuffer +put src/olap/byte_buffer.h /^ OLAPStatus put(const char* src, uint64_t src_size) {$/;" f class:doris::StorageByteBuffer +put src/olap/olap_meta.cpp /^OLAPStatus OlapMeta::put(int column_family_index, const std::string& key, const std::string& value) {$/;" f class:doris::OlapMeta +put src/runtime/stream_load/load_stream_mgr.h /^ Status put(const UniqueId& id,$/;" f class:doris::LoadStreamMgr +put src/util/frame_of_reference_coding.h /^ void put(const T value) {$/;" f class:doris::ForEncoder +put src/util/lru_cache.hpp /^ void put(const Key& key, const Value& value) {$/;" f class:doris::LruCache +put src/util/path_trie.hpp /^ void put(std::map* params, $/;" f class:doris::PathTrie::TrieNode +put_batch src/util/frame_of_reference_coding.cpp /^void ForEncoder::put_batch(const T *in_data, size_t count) {$/;" f class:doris::ForEncoder +put_bytes src/util/byte_buffer.h /^ void put_bytes(const char* data, size_t size) {$/;" f struct:doris::ByteBuffer +put_fixed128_le src/util/coding.h /^inline void put_fixed128_le(T* dst, uint128_t val) {$/;" f namespace:doris +put_fixed32_le src/util/coding.h /^inline void put_fixed32_le(T* dst, uint32_t val) {$/;" f namespace:doris +put_fixed64_le src/util/coding.h /^inline void put_fixed64_le(T* dst, uint64_t val) {$/;" f namespace:doris +put_int src/exec/read_write_util.h /^inline void ReadWriteUtil::put_int(uint8_t* buf, int32_t integer) {$/;" f class:doris::ReadWriteUtil +put_length_prefixed_slice src/util/coding.h /^inline void put_length_prefixed_slice(T* dst, const Slice& value) {$/;" f namespace:doris +put_result src/runtime/stream_load/stream_load_context.h /^ TStreamLoadPutResult put_result;$/;" m class:doris::StreamLoadContext +put_varint32 src/util/coding.h /^inline void put_varint32(T* dst, uint32_t v) {$/;" f namespace:doris +put_varint64 src/util/coding.h /^inline void put_varint64(T* dst, uint64_t v) {$/;" f namespace:doris +put_varint64_varint32 src/util/coding.h /^inline void put_varint64_varint32(T* dst, uint64_t v1, uint32_t v2) {$/;" f namespace:doris +put_zint src/exec/read_write_util.cpp /^int ReadWriteUtil::put_zint(int32_t integer, uint8_t* buf) {$/;" f class:doris::ReadWriteUtil +put_zlong src/exec/read_write_util.cpp /^int ReadWriteUtil::put_zlong(int64_t longint, uint8_t* buf) {$/;" f class:doris::ReadWriteUtil +putenv src/gutil/port.h 846;" d +pwrite src/olap/file_helper.cpp /^OLAPStatus FileHandler::pwrite(const void* buf, size_t buf_size, size_t offset) {$/;" f class:doris::FileHandler +pwrite src/olap/file_helper.cpp /^OLAPStatus FileHandlerWithBuf::pwrite(const void* buf, size_t buf_size, size_t offset) {$/;" f class:doris::FileHandlerWithBuf +qnxnto build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c /^char const* qnxnto = "INFO" ":" "qnxnto[]";$/;" v +qnxnto build/CMakeFiles/3.12.3/CompilerIdCXX/CMakeCXXCompilerId.cpp /^char const* qnxnto = "INFO" ":" "qnxnto[]";$/;" v +quadratic_probing src/exec/new_partitioned_hash_table.h /^ int quadratic_probing;$/;" m struct:doris::NewPartitionedHashTableCtx::HashTableReplacedConstants +quadratic_probing src/exec/new_partitioned_hash_table.h /^ bool IR_NO_INLINE quadratic_probing() const { return quadratic_probing_; }$/;" f class:doris::NewPartitionedHashTable +quadratic_probing_ src/exec/new_partitioned_hash_table.h /^ const bool quadratic_probing_;$/;" m class:doris::NewPartitionedHashTable +quantile src/util/tdigest.h /^ Value quantile(Value q) {$/;" f class:doris::TDigest +quantile src/util/tdigest.h /^ static Value quantile(Value index, Value previousIndex, Value nextIndex, Value previousMean, Value nextMean) {$/;" f class:doris::TDigest +quantile test/util/tdigest_test.cpp /^static double quantile(const double q, const std::vector& values) {$/;" f namespace:doris +quantileProcessed src/util/tdigest.h /^ Value quantileProcessed(Value q) const {$/;" f class:doris::TDigest +quarter src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::quarter($/;" f class:doris::TimestampFunctions +query src/exec/mysql_scanner.cpp /^Status MysqlScanner::query(const std::string& query) {$/;" f class:doris::MysqlScanner +query src/exec/mysql_scanner.cpp /^Status MysqlScanner::query(const std::string& table, const std::vector& fields,$/;" f class:doris::MysqlScanner +query_id src/runtime/runtime_state.h /^ const TUniqueId& query_id() const {$/;" f class:doris::RuntimeState +query_id src/udf/udf.cpp /^FunctionContext::UniqueId FunctionContext::query_id() const {$/;" f class:doris_udf::FunctionContext +query_mem_tracker src/runtime/runtime_state.h /^ MemTracker* query_mem_tracker() { {$/;" f class:doris::RuntimeState +query_options src/runtime/runtime_state.h /^ const TQueryOptions& query_options() const {$/;" f class:doris::RuntimeState +query_params src/http/http_request.h /^ const std::map& query_params() const {$/;" f class:doris::HttpRequest +query_scan_bytes src/util/doris_metrics.cpp /^IntCounter DorisMetrics::query_scan_bytes;$/;" m class:doris::DorisMetrics file: +query_scan_bytes src/util/doris_metrics.h /^ static IntCounter query_scan_bytes;$/;" m class:doris::DorisMetrics +query_scan_bytes_per_second src/util/doris_metrics.cpp /^IntGauge DorisMetrics::query_scan_bytes_per_second;$/;" m class:doris::DorisMetrics file: +query_scan_bytes_per_second src/util/doris_metrics.h /^ static IntGauge query_scan_bytes_per_second;$/;" m class:doris::DorisMetrics +query_scan_rows src/util/doris_metrics.cpp /^IntCounter DorisMetrics::query_scan_rows;$/;" m class:doris::DorisMetrics file: +query_scan_rows src/util/doris_metrics.h /^ static IntCounter query_scan_rows;$/;" m class:doris::DorisMetrics +query_status src/runtime/runtime_state.h /^ Status query_status() {$/;" f class:doris::RuntimeState +queue_size src/runtime/disk_io_mgr.cc /^int64_t DiskIoMgr::queue_size(RequestContext* reader) const {$/;" f class:doris::DiskIoMgr +quota src/runtime/thread_resource_mgr.h /^ int quota() const {$/;" f class:doris::ThreadResourceMgr::ResourcePool +r2 src/gutil/valgrind.h /^ unsigned int r2; \/* what tocptr do we need? *\/$/;" m struct:__anon13 +r2 src/gutil/valgrind.h /^ unsigned long long int r2; \/* what tocptr do we need? *\/$/;" m struct:__anon12 +r2 src/gutil/valgrind.h /^ unsigned long long int r2; \/* what tocptr do we need? *\/$/;" m struct:__anon14 +radians src/exprs/math_functions.cpp /^DoubleVal MathFunctions::radians($/;" f class:doris::MathFunctions +radixSortLSD src/util/radix_sort.h /^void radixSortLSD(T *arr, size_t size) {$/;" f namespace:doris +rand src/exprs/math_functions.cpp /^DoubleVal MathFunctions::rand(FunctionContext* ctx) {$/;" f class:doris::MathFunctions +rand_prepare src/exprs/math_functions.cpp /^void MathFunctions::rand_prepare($/;" f class:doris::MathFunctions +rand_seed src/exprs/math_functions.cpp /^DoubleVal MathFunctions::rand_seed(FunctionContext* ctx, const BigIntVal& seed) {$/;" f class:doris::MathFunctions +random src/gutil/port.h 850;" d +random_seed test/olap/skiplist_test.cpp /^const int random_seed = 301;$/;" m namespace:doris file: +random_target test/olap/skiplist_test.cpp /^ static Key random_target(Random* rnd) {$/;" f class:doris::ConcurrentTest file: +range src/olap/olap_index.h /^ IDRange range;$/;" m struct:doris::SegmentMetaInfo +range src/olap/reader.h /^ std::string range;$/;" m struct:doris::Reader::KeysParam +range src/olap/reader.h /^ std::string range;$/;" m struct:doris::ReaderParams +range src/runtime/dpp_sink_internal.h /^ const PartRange& range() const {$/;" f class:doris::PartitionInfo +range_intersection src/olap/rowset/segment_v2/row_ranges.h /^ static bool range_intersection(const RowRange& left, const RowRange& right, RowRange* range) {$/;" f class:doris::segment_v2::RowRange +range_size src/olap/rowset/segment_v2/row_ranges.h /^ size_t range_size() {$/;" f class:doris::segment_v2::RowRanges +range_union src/olap/rowset/segment_v2/row_ranges.h /^ static bool range_union(const RowRange& left, const RowRange& right, RowRange* range) {$/;" f class:doris::segment_v2::RowRange +ranges_intersection src/olap/rowset/segment_v2/row_ranges.h /^ static void ranges_intersection(const RowRanges& left, const RowRanges& right, RowRanges* result) {$/;" f class:doris::segment_v2::RowRanges +ranges_processed_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::ranges_processed_total;$/;" m class:doris::DorisMetrics file: +ranges_processed_total src/util/doris_metrics.h /^ static IntCounter ranges_processed_total;$/;" m class:doris::DorisMetrics +ranges_to_roaring src/olap/rowset/segment_v2/row_ranges.h /^ static Roaring ranges_to_roaring(const RowRanges& ranges) {$/;" f class:doris::segment_v2::RowRanges +ranges_union src/olap/rowset/segment_v2/row_ranges.h /^ static void ranges_union(const RowRanges& left, const RowRanges& right, RowRanges* result) {$/;" f class:doris::segment_v2::RowRanges +rank src/exprs/aggregate_functions.cpp /^ int64_t rank;$/;" m struct:doris::RankState file: +rank src/util/bitmap_value.h /^ uint64_t rank(uint64_t x) const {$/;" f class:doris::detail::Roaring64Map +rank_finalize src/exprs/aggregate_functions.cpp /^BigIntVal AggregateFunctions::rank_finalize(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +rank_get_value src/exprs/aggregate_functions.cpp /^BigIntVal AggregateFunctions::rank_get_value(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +rank_init src/exprs/aggregate_functions.cpp /^void AggregateFunctions::rank_init(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +rank_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::rank_update(FunctionContext* ctx, StringVal* dst) {$/;" f class:doris::AggregateFunctions +rate_counters src/util/runtime_profile.h /^ RateCounterMap rate_counters;$/;" m struct:doris::RuntimeProfile::PeriodicCounterUpdateState +raw src/util/minizip/unzip.c /^ int raw;$/;" m struct:__anon33 file: +raw_path src/http/http_request.h /^ const std::string& raw_path() const {$/;" f class:doris::HttpRequest +raw_rows_read src/exec/olap_scanner.h /^ int64_t raw_rows_read() const { return _raw_rows_read; }$/;" f class:doris::OlapScanner +raw_rows_read src/olap/olap_common.h /^ int64_t raw_rows_read = 0;$/;" m struct:doris::OlapReaderStatistics +rawbytes src/util/simdutf8check.h /^ __m128i rawbytes;$/;" m struct:processed_utf_bytes +rawbytes src/util/simdutf8check.h /^ __m256i rawbytes;$/;" m struct:avx_processed_utf_bytes +rbegin src/gutil/strings/stringpiece.h /^ const_reverse_iterator rbegin() const {$/;" f class:StringPiece +rdlock src/util/mutex.cpp /^OLAPStatus RWMutex::rdlock() {$/;" f class:doris::RWMutex +reach_capacity_limit src/olap/data_dir.cpp /^bool DataDir::reach_capacity_limit(int64_t incoming_data_size) {$/;" f class:doris::DataDir +reached_limit src/exec/exec_node.h /^ bool reached_limit() {$/;" f class:doris::ExecNode +read src/common/atomic.h /^ T read() {$/;" f class:doris::AtomicInt +read src/exec/broker_reader.cpp /^Status BrokerReader::read(uint8_t* buf, size_t* buf_len, bool* eof) {$/;" f class:doris::BrokerReader +read src/exec/local_file_reader.cpp /^Status LocalFileReader::read(uint8_t* buf, size_t* buf_len, bool* eof) {$/;" f class:doris::LocalFileReader +read src/exec/parquet_reader.cpp /^Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& tuple_slot_descs, MemPool* mem_pool, bool* eof) {$/;" f class:doris::ParquetReaderWrap +read src/exec/read_write_util.h /^inline bool ReadWriteUtil::read(uint8_t** buf, int* buf_len, T* val, Status* status) {$/;" f class:doris::ReadWriteUtil +read src/olap/file_helper.cpp /^OLAPStatus FileHandlerWithBuf::read(void* buf, size_t size) {$/;" f class:doris::FileHandlerWithBuf +read src/olap/file_stream.h /^ OLAPStatus read(char* out_buffer, size_t length) {$/;" f class:doris::ReadOnlyFileStream::FileCursor +read src/olap/file_stream.h /^inline OLAPStatus ReadOnlyFileStream::read(char* buffer, uint64_t* buf_size) {$/;" f class:doris::ReadOnlyFileStream +read src/olap/file_stream.h /^inline OLAPStatus ReadOnlyFileStream::read(char* byte) {$/;" f class:doris::ReadOnlyFileStream +read src/olap/fs/file_block_manager.cpp /^Status FileReadableBlock::read(uint64_t offset, Slice result) const {$/;" f class:doris::fs::internal::FileReadableBlock +read src/olap/in_stream.h /^inline OLAPStatus InStream::read(char* buffer, uint64_t* buf_size) {$/;" f class:doris::InStream +read src/olap/in_stream.h /^inline OLAPStatus InStream::read(char* byte) {$/;" f class:doris::InStream +read src/runtime/disk_io_mgr.cc /^Status DiskIoMgr::read(RequestContext* reader, ScanRange* range, BufferDescriptor** buffer) {$/;" f class:doris::DiskIoMgr +read src/runtime/disk_io_mgr_scan_range.cc /^Status DiskIoMgr::ScanRange::read(char* buffer, int64_t* bytes_read, bool* eosr) {$/;" f class:doris::DiskIoMgr::ScanRange +read src/util/bitmap_value.h /^ static Roaring64Map read(const char* buf) {$/;" f class:doris::detail::Roaring64Map +read_all src/olap/file_stream.h /^inline OLAPStatus ReadOnlyFileStream::read_all(char* buffer, uint64_t* buffer_size) {$/;" f class:doris::ReadOnlyFileStream +read_and_decompress_page src/olap/rowset/segment_v2/page_io.cpp /^Status PageIO::read_and_decompress_page(const PageReadOptions& opts,$/;" f class:doris::segment_v2::PageIO +read_bitmap src/olap/rowset/segment_v2/bitmap_index_reader.cpp /^Status BitmapIndexIterator::read_bitmap(rowid_t ordinal, Roaring* result) {$/;" f class:doris::segment_v2::BitmapIndexIterator +read_bloom_filter src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp /^Status BloomFilterIndexIterator::read_bloom_filter(rowid_t ordinal, std::unique_ptr* bf) {$/;" f class:doris::segment_v2::BloomFilterIndexIterator +read_buffer src/util/minizip/unzip.c /^ char *read_buffer; \/* internal buffer for compressed data *\/$/;" m struct:__anon33 file: +read_cgroup_value src/util/cgroup_util.cpp /^static Status read_cgroup_value(const string& limit_file_path, int64_t* val) {$/;" f namespace:doris +read_data test/exec/new_olap_scan_node_test.cpp /^ void read_data(int version, vector* data) {$/;" f class:doris::TestOlapScanNode +read_end_ptr_ src/runtime/buffered_tuple_stream3.h /^ const uint8_t* read_end_ptr_;$/;" m class:doris::BufferedTupleStream3 +read_from src/exprs/bitmap_function.cpp /^void read_from(const char** src, DateTimeValue* result) {$/;" f namespace:doris::detail +read_from src/exprs/bitmap_function.cpp /^void read_from(const char** src, DecimalV2Value* result) {$/;" f namespace:doris::detail +read_from src/exprs/bitmap_function.cpp /^void read_from(const char** src, StringValue* result) {$/;" f namespace:doris::detail +read_from src/exprs/bitmap_function.cpp /^void read_from(const char** src, T* result) {$/;" f namespace:doris::detail +read_ints src/olap/serialize.cpp /^OLAPStatus read_ints(ReadOnlyFileStream* input, int64_t* data, uint32_t count, uint32_t bit_width) {$/;" f namespace:doris::ser +read_io_ops src/runtime/bufferpool/buffer_pool_counters.h /^ RuntimeProfile::Counter* read_io_ops;$/;" m struct:doris::BufferPoolClientCounters +read_line src/exec/plain_text_line_reader.cpp /^Status PlainTextLineReader::read_line(const uint8_t** ptr, size_t* size, bool* eof) {$/;" f class:doris::PlainTextLineReader +read_null_bitmap src/olap/rowset/segment_v2/bitmap_index_reader.h /^ Status read_null_bitmap(Roaring* result) {$/;" f class:doris::segment_v2::BitmapIndexIterator +read_only src/olap/fs/block_manager.h /^ bool read_only;$/;" m struct:doris::fs::BlockManagerOptions +read_page src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::read_page(const ColumnIteratorOptions& iter_opts, const PagePointer& pp,$/;" f class:doris::segment_v2::ColumnReader +read_page src/olap/rowset/segment_v2/indexed_column_reader.cpp /^Status IndexedColumnReader::read_page(RandomAccessFile* file, const PagePointer& pp,$/;" f class:doris::segment_v2::IndexedColumnReader +read_page_ src/runtime/buffered_tuple_stream3.h /^ std::list::iterator read_page_;$/;" m class:doris::BufferedTupleStream3 +read_page_reservation_ src/runtime/buffered_tuple_stream3.h /^ BufferPool::SubReservation read_page_reservation_;$/;" m class:doris::BufferedTupleStream3 +read_page_rows_returned_ src/runtime/buffered_tuple_stream3.h /^ uint32_t read_page_rows_returned_;$/;" m class:doris::BufferedTupleStream3 +read_ptr_ src/runtime/buffered_tuple_stream3.h /^ uint8_t* read_ptr_;$/;" m class:doris::BufferedTupleStream3 +read_range src/runtime/disk_io_mgr.cc /^void DiskIoMgr::read_range(DiskQueue* disk_queue, RequestContext* reader, ScanRange* range) {$/;" f class:doris::DiskIoMgr +read_record_batch src/exec/parquet_reader.cpp /^Status ParquetReaderWrap::read_record_batch(const std::vector& tuple_slot_descs, bool* eof) {$/;" f class:doris::ParquetReaderWrap +read_step test/olap/skiplist_test.cpp /^ void read_step(Random* rnd) {$/;" f class:doris::ConcurrentTest +read_stream test/runtime/data_stream_test.cpp /^ void read_stream(ReceiverInfo* info) {$/;" f class:doris::DataStreamTest +read_stream_merging test/runtime/data_stream_test.cpp /^ void read_stream_merging(ReceiverInfo* info, RuntimeProfile* profile) {$/;" f class:doris::DataStreamTest +read_strings src/runtime/buffered_tuple_stream2.cc /^void BufferedTupleStream2::read_strings(const vector& string_slots,$/;" f class:doris::BufferedTupleStream2 +read_time_ms src/util/system_metrics.cpp /^ IntLockCounter read_time_ms;$/;" m struct:doris::DiskMetrics file: +read_timer src/exec/scan_node.h /^ RuntimeProfile::Counter* read_timer() const {$/;" f class:doris::ScanNode +read_union_bitmap src/olap/rowset/segment_v2/bitmap_index_reader.cpp /^Status BitmapIndexIterator::read_union_bitmap(rowid_t from, rowid_t to, Roaring* result) {$/;" f class:doris::segment_v2::BitmapIndexIterator +read_var_signed src/olap/serialize.h /^inline OLAPStatus read_var_signed(ReadOnlyFileStream* stream, int64_t* value) {$/;" f namespace:doris::ser +read_var_unsigned src/olap/serialize.cpp /^OLAPStatus read_var_unsigned(ReadOnlyFileStream* stream, int64_t* value) {$/;" f namespace:doris::ser +read_wait_time src/runtime/bufferpool/buffer_pool_counters.h /^ RuntimeProfile::Counter* read_wait_time;$/;" m struct:doris::BufferPoolClientCounters +read_zint src/exec/read_write_util.h /^inline bool ReadWriteUtil::read_zint(uint8_t** buf, int* buf_len, int32_t* val,$/;" f class:doris::ReadWriteUtil +read_zlong src/exec/read_write_util.h /^inline bool ReadWriteUtil::read_zlong(uint8_t** buf, int* buf_len, int64_t* val,$/;" f class:doris::ReadWriteUtil +readable_blocks_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::readable_blocks_total;$/;" m class:doris::DorisMetrics file: +readable_blocks_total src/util/doris_metrics.h /^ static IntCounter readable_blocks_total;$/;" m class:doris::DorisMetrics +readat src/exec/broker_reader.cpp /^Status BrokerReader::readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) {$/;" f class:doris::BrokerReader +readat src/exec/local_file_reader.cpp /^Status LocalFileReader::readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) {$/;" f class:doris::LocalFileReader +readat src/runtime/stream_load/stream_load_pipe.h /^ Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) {$/;" f class:doris::StreamLoadPipe +reader_ src/olap/fs/file_block_manager.cpp /^ shared_ptr reader_;$/;" m class:doris::fs::internal::FileReadableBlock file: +reader_type src/olap/reader.h /^ ReaderType reader_type;$/;" m struct:doris::ReaderParams +reader_type src/olap/rowset/rowset_reader_context.h /^ ReaderType reader_type = READER_QUERY;$/;" m struct:doris::RowsetReaderContext +reads_completed src/util/system_metrics.cpp /^ IntLockCounter reads_completed;$/;" m struct:doris::DiskMetrics file: +readv src/olap/fs/file_block_manager.cpp /^Status FileReadableBlock::readv(uint64_t offset, const Slice* results, size_t res_cnt) const {$/;" f class:doris::fs::internal::FileReadableBlock +ready_buffers_capacity src/runtime/disk_io_mgr.h /^ int ready_buffers_capacity() const { return _ready_buffers_capacity; }$/;" f class:doris::DiskIoMgr::ScanRange +real_port test/exec/es_scan_reader_test.cpp /^static int real_port = 0;$/;" m namespace:doris file: +real_port test/http/http_client_test.cpp /^static int real_port = 0;$/;" m namespace:doris file: +real_port test/runtime/small_file_mgr_test.cpp /^static int real_port = 0;$/;" m namespace:doris file: +real_port test/runtime/user_function_cache_test.cpp /^static int real_port = 0;$/;" m namespace:doris file: +reallocate src/runtime/free_pool.hpp /^ uint8_t* reallocate(uint8_t* ptr, int size) {$/;" f class:doris::FreePool +reallocate src/udf/udf.cpp /^ uint8_t* reallocate(uint8_t* ptr, int byte_size) {$/;" f class:doris::FreePool +reallocate src/udf/udf.cpp /^uint8_t* FunctionContext::reallocate(uint8_t* ptr, int byte_size) {$/;" f class:doris_udf::FunctionContext +rebind src/gutil/stl_util.h /^ template struct rebind {$/;" s class:STLCountingAllocator +rebuild_pool_with_builder test/util/threadpool_test.cpp /^ Status rebuild_pool_with_builder(const ThreadPoolBuilder& builder) {$/;" f class:doris::ThreadPoolTest +rebuild_pool_with_min_max test/util/threadpool_test.cpp /^ Status rebuild_pool_with_min_max(int min_threads, int max_threads) {$/;" f class:doris::ThreadPoolTest +receive_bytes src/runtime/stream_load/stream_load_context.h /^ size_t receive_bytes = 0;$/;" m class:doris::StreamLoadContext +receive_bytes src/util/system_metrics.cpp /^ IntLockCounter receive_bytes;$/;" m struct:doris::NetMetrics file: +receive_packets src/util/system_metrics.cpp /^ IntLockCounter receive_packets;$/;" m struct:doris::NetMetrics file: +receiver_num test/runtime/data_stream_test.cpp /^ int receiver_num;$/;" m struct:doris::DataStreamTest::ReceiverInfo file: +reclaim_id src/util/core_local.h /^ void reclaim_id(int id) {$/;" f class:doris::CoreLocalValueController +reconstruct_rowset_graph src/olap/rowset_graph.cpp /^OLAPStatus RowsetGraph::reconstruct_rowset_graph(const std::vector& rs_metas) {$/;" f class:doris::RowsetGraph +record_position src/olap/rowset/column_writer.cpp /^void ByteColumnWriter::record_position() {$/;" f class:doris::ByteColumnWriter +record_position src/olap/rowset/column_writer.cpp /^void ColumnWriter::record_position() {$/;" f class:doris::ColumnWriter +record_position src/olap/rowset/column_writer.cpp /^void DecimalColumnWriter::record_position() {$/;" f class:doris::DecimalColumnWriter +record_position src/olap/rowset/column_writer.cpp /^void LargeIntColumnWriter::record_position() {$/;" f class:doris::LargeIntColumnWriter +record_position src/olap/rowset/column_writer.cpp /^void VarStringColumnWriter::record_position() {$/;" f class:doris::VarStringColumnWriter +record_position src/olap/rowset/column_writer.h /^ virtual void record_position() {$/;" f class:doris::DoubleColumnWriterBase +record_position src/olap/rowset/column_writer.h /^ virtual void record_position() {$/;" f class:doris::IntegerColumnWriterWrapper +record_position src/olap/rowset/column_writer.h /^ void record_position(PositionEntryWriter* index_entry) {$/;" f class:doris::IntegerColumnWriter +recover_tablet_until_specfic_version src/olap/storage_engine.cpp /^OLAPStatus StorageEngine::recover_tablet_until_specfic_version($/;" f class:doris::StorageEngine +recover_tablet_until_specfic_version src/olap/tablet.cpp /^OLAPStatus Tablet::recover_tablet_until_specfic_version(const int64_t& spec_version,$/;" f class:doris::Tablet +reduce_mem_usage src/runtime/tablets_channel.cpp /^Status TabletsChannel::reduce_mem_usage() {$/;" f class:doris::TabletsChannel +reenable_ src/util/cpu_info.h /^ bool reenable_;$/;" m struct:doris::CpuInfo::TempDisable +ref src/runtime/stream_load/stream_load_context.h /^ void ref() { _refs.fetch_add(1); }$/;" f class:doris::StreamLoadContext +ref src/runtime/user_function_cache.cpp /^ void ref() { _refs.fetch_add(1); }$/;" f struct:doris::UserFunctionCacheEntry +ref src/util/ref_count_closure.h /^ void ref() { _refs.fetch_add(1); }$/;" f class:doris::RefCountClosure +ref_column src/olap/column_mapping.h /^ int32_t ref_column;$/;" m struct:doris::ColumnMapping +ref_count src/olap/rowset/segment_group.cpp /^int64_t SegmentGroup::ref_count() {$/;" f class:doris::SegmentGroup +ref_count_ src/gutil/ref_counted.h /^ mutable AtomicRefCount ref_count_;$/;" m class:doris::subtle::RefCountedThreadSafeBase +ref_count_ src/gutil/ref_counted.h /^ mutable int ref_count_;$/;" m class:doris::subtle::RefCountedBase +ref_rowset_readers src/olap/schema_change.h /^ std::vector ref_rowset_readers;$/;" m struct:doris::SchemaChangeHandler::SchemaChangeParams +reference src/gutil/strings/stringpiece.h /^ typedef const char& reference;$/;" t class:StringPiece +reference_buffer src/olap/byte_buffer.cpp /^StorageByteBuffer* StorageByteBuffer::reference_buffer(StorageByteBuffer* reference,$/;" f class:doris::StorageByteBuffer +reference_type src/util/bitmap_value.h /^ typedef uint64_t& reference_type;$/;" t class:doris::detail::final +referenced_column src/olap/tablet_schema.h /^ std::string referenced_column() const { return _referenced_column; }$/;" f class:doris::TabletColumn +referenced_column_id src/olap/tablet_schema.h /^ int32_t referenced_column_id() const { return _referenced_column_id; }$/;" f class:doris::TabletColumn +refs src/olap/lru_cache.h /^ uint32_t refs;$/;" m struct:doris::CachePriority::LRUHandle +regex src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::regex($/;" f class:doris::LikePredicate +regex src/exprs/like_predicate.h /^ std::unique_ptr regex;$/;" m struct:doris::LikePredicate::LikePredicateState +regex_close src/exprs/like_predicate.cpp /^void LikePredicate::regex_close($/;" f class:doris::LikePredicate +regex_fn src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::regex_fn($/;" f class:doris::LikePredicate +regex_match src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::regex_match($/;" f class:doris::LikePredicate +regex_prepare src/exprs/like_predicate.cpp /^void LikePredicate::regex_prepare($/;" f class:doris::LikePredicate +regexp_close src/exprs/string_functions.cpp /^void StringFunctions::regexp_close($/;" f class:doris::StringFunctions +regexp_extract src/exprs/string_functions.cpp /^StringVal StringFunctions::regexp_extract($/;" f class:doris::StringFunctions +regexp_like src/exprs/like_predicate.cpp /^BooleanVal LikePredicate::regexp_like($/;" f class:doris::LikePredicate +regexp_like_prepare src/exprs/like_predicate.cpp /^void LikePredicate::regexp_like_prepare($/;" f class:doris::LikePredicate +regexp_prepare src/exprs/string_functions.cpp /^void StringFunctions::regexp_prepare($/;" f class:doris::StringFunctions +regexp_replace src/exprs/string_functions.cpp /^StringVal StringFunctions::regexp_replace($/;" f class:doris::StringFunctions +register_cgroups src/runtime/fragment_mgr.cpp /^static void register_cgroups(const std::string& user, const std::string& group) {$/;" f namespace:doris +register_client src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::register_client($/;" f class:doris::BufferedBlockMgr2 +register_context src/runtime/disk_io_mgr.cc /^Status DiskIoMgr::register_context(RequestContext** request_context, MemTracker* mem_tracker) {$/;" f class:doris::DiskIoMgr +register_func src/exprs/expr_context.cpp /^int ExprContext::register_func($/;" f class:doris::ExprContext +register_function_context src/exprs/expr.cpp /^FunctionContext* Expr::register_function_context($/;" f class:doris::Expr +register_handler src/http/ev_http_server.cpp /^bool EvHttpServer::register_handler($/;" f class:doris::EvHttpServer +register_hook src/util/metrics.cpp /^bool MetricRegistry::register_hook(const std::string& name, const std::function& hook) {$/;" f class:doris::MetricRegistry +register_listener src/agent/topic_subscriber.cpp /^void TopicSubscriber::register_listener(TTopicType::type topic_type, TopicListener* listener) {$/;" f class:doris::TopicSubscriber +register_metric src/util/metrics.cpp /^bool MetricRegistry::register_metric(const std::string& name,$/;" f class:doris::MetricRegistry +register_metric src/util/metrics.h /^ bool register_metric(const std::string& name, Metric* metric) {$/;" f class:doris::MetricRegistry +register_module src/http/monitor_action.cpp /^void MonitorAction::register_module(const std::string& name, RestMonitorIface* module) {$/;" f class:doris::MonitorAction +register_page src/http/web_page_handler.cpp /^void WebPageHandler::register_page($/;" f class:doris::WebPageHandler +register_periodic_counter src/util/runtime_profile.cpp /^void RuntimeProfile::register_periodic_counter(Counter* src_counter, SampleFn sample_fn,$/;" f class:doris::RuntimeProfile +register_pool src/runtime/thread_resource_mgr.cpp /^ThreadResourceMgr::ResourcePool* ThreadResourceMgr::register_pool() {$/;" f class:doris::ThreadResourceMgr +register_pull_load_task test/runtime/data_stream_test.cpp /^ virtual void register_pull_load_task($/;" f class:doris::DorisTestBackend +register_tablet src/olap/data_dir.cpp /^OLAPStatus DataDir::register_tablet(Tablet* tablet) {$/;" f class:doris::DataDir +register_tablet_into_dir src/olap/tablet.h /^inline OLAPStatus Tablet::register_tablet_into_dir() {$/;" f class:doris::Tablet +related_schema_hash src/olap/tablet_meta.h /^ inline int32_t related_schema_hash() const { return _related_schema_hash; }$/;" f class:doris::AlterTabletTask +related_tablet_id src/olap/tablet_meta.h /^ inline int64_t related_tablet_id() const { return _related_tablet_id; }$/;" f class:doris::AlterTabletTask +release src/gutil/gscoped_ptr.h /^ T* release() {$/;" f class:doris::internal::gscoped_ptr_impl +release src/olap/file_helper.cpp /^OLAPStatus FileHandler::release() {$/;" f class:doris::FileHandler +release src/olap/lru_cache.cpp /^void LRUCache::release(Cache::Handle* handle) {$/;" f class:doris::LRUCache +release src/olap/lru_cache.cpp /^void ShardedLRUCache::release(Handle* handle) {$/;" f class:doris::ShardedLRUCache +release src/olap/rowset/rowset.h /^ void release() {$/;" f class:doris::Rowset +release src/olap/rowset/segment_group.cpp /^void SegmentGroup::release() {$/;" f class:doris::SegmentGroup +release src/olap/schema_change.cpp /^void RowBlockAllocator::release(RowBlock* row_block) {$/;" f class:doris::RowBlockAllocator +release src/runtime/mem_tracker.h /^ void release(int64_t bytes) {$/;" f class:doris::MemTracker +release_base_compaction_lock src/olap/tablet.h /^ inline void release_base_compaction_lock() { _base_lock.unlock(); }$/;" f class:doris::Tablet +release_client src/runtime/client_cache.cpp /^void ClientCacheHelper::release_client(void** client_key) {$/;" f class:doris::ClientCacheHelper +release_client src/runtime/client_cache.h /^ void release_client(T** client) {$/;" f class:doris::ClientCache +release_cumulative_lock src/olap/tablet.h /^ inline void release_cumulative_lock() { _cumulative_lock.unlock(); }$/;" f class:doris::Tablet +release_entry src/runtime/user_function_cache.cpp /^void UserFunctionCache::release_entry(UserFunctionCacheEntry* entry) {$/;" f class:doris::UserFunctionCache +release_header_lock src/olap/tablet.h /^ inline void release_header_lock() { _meta_lock.unlock(); }$/;" f class:doris::Tablet +release_id src/olap/rowset/unique_rowset_id_generator.cpp /^void UniqueRowsetIdGenerator::release_id(const RowsetId& rowset_id) {$/;" f class:doris::UniqueRowsetIdGenerator +release_local src/runtime/mem_tracker.h /^ void release_local(int64_t bytes, MemTracker* end_tracker) {$/;" f class:doris::MemTracker +release_memory src/runtime/buffered_block_mgr2.cc /^void BufferedBlockMgr2::release_memory(Client* client, int64_t size) {$/;" f class:doris::BufferedBlockMgr2 +release_ptr src/gutil/stl_util.h /^template T* release_ptr(T **ptr) {$/;" f +release_push_lock src/olap/tablet.h /^ inline void release_push_lock() { _ingest_lock.unlock(); }$/;" f class:doris::Tablet +release_rowset_id src/olap/storage_engine.h /^ void release_rowset_id(const RowsetId& rowset_id) {$/;" f class:doris::StorageEngine +release_schema_change_lock src/olap/tablet_manager.cpp /^void TabletManager::release_schema_change_lock(TTabletId tablet_id) {$/;" f class:doris::TabletManager +release_snapshot src/agent/agent_server.cpp /^void AgentServer::release_snapshot(TAgentResult& t_agent_result, const std::string& snapshot_path) {$/;" f class:doris::AgentServer +release_snapshot src/olap/snapshot_manager.cpp /^OLAPStatus SnapshotManager::release_snapshot(const string& snapshot_path) {$/;" f class:doris::SnapshotManager +release_snapshot src/service/backend_service.h /^ virtual void release_snapshot(TAgentResult& return_value, const std::string& snapshot_path) {$/;" f class:doris::BackendService +release_snapshot test/runtime/data_stream_test.cpp /^ virtual void release_snapshot(TAgentResult& return_val, const std::string& snapshot_path) {}$/;" f class:doris::DorisTestBackend +release_string src/runtime/tuple.cpp /^int64_t Tuple::release_string(const TupleDescriptor& desc) {$/;" f class:doris::Tuple +release_thread_token src/runtime/plan_fragment_executor.cpp /^void PlanFragmentExecutor::release_thread_token() {$/;" f class:doris::PlanFragmentExecutor +release_thread_token src/runtime/thread_resource_mgr.h /^inline void ThreadResourceMgr::ResourcePool::release_thread_token(bool required) {$/;" f class:doris::ThreadResourceMgr::ResourcePool +release_token src/util/threadpool.cpp /^void ThreadPool::release_token(ThreadPoolToken* t) {$/;" f class:doris::ThreadPool +release_tuples src/runtime/tuple_row.h /^ int64_t release_tuples(const std::vector& descs) {$/;" f class:doris::TupleRow +release_unused_reservation src/exec/exec_node.cpp /^Status ExecNode::release_unused_reservation() {$/;" f class:doris::ExecNode +reload src/http/action/reload_tablet_action.cpp /^void ReloadTabletAction::reload($/;" f class:doris::ReloadTabletAction +relocate src/util/slice.h /^ void relocate(char* d) {$/;" f struct:doris::Slice +relocate_tasks src/agent/cgroups_mgr.cpp /^AgentStatus CgroupsMgr::relocate_tasks(const string& src_cgroups, const string& dest_cgroups) {$/;" f class:doris::CgroupsMgr +remain src/olap/file_stream.h /^ size_t remain() {$/;" f class:doris::ReadOnlyFileStream::FileCursor +remaining src/olap/byte_buffer.h /^ inline uint64_t remaining() const {$/;" f class:doris::StorageByteBuffer +remaining src/olap/file_stream.h /^ inline int remaining() {$/;" f class:doris::ReadOnlyFileStream +remaining src/olap/row_block.h /^ size_t remaining() const { return _limit - _pos; }$/;" f class:doris::RowBlock +remaining src/olap/rowset/segment_v2/parsed_page.h /^ size_t remaining() const { return num_rows - offset_in_page; }$/;" f struct:doris::segment_v2::ParsedPage +remaining src/util/byte_buffer.h /^ size_t remaining() const { return limit - pos; }$/;" f struct:doris::ByteBuffer +remaining_initial_reservation_claims_ src/runtime/initial_reservations.h /^ int64_t remaining_initial_reservation_claims_;$/;" m class:doris::InitialReservations +remaining_unreserved_buffers src/runtime/buffered_block_mgr2.cc /^int64_t BufferedBlockMgr2::remaining_unreserved_buffers() const {$/;" f class:doris::BufferedBlockMgr2 +remote_host src/http/http_request.cpp /^const char* HttpRequest::remote_host() const {$/;" f class:doris::HttpRequest +remove src/exprs/agg_fn_evaluator.h /^inline void AggFnEvaluator::remove($/;" f class:doris::AggFnEvaluator +remove src/exprs/agg_fn_evaluator.h /^inline void AggFnEvaluator::remove(const std::vector& evaluators,$/;" f class:doris::AggFnEvaluator +remove src/olap/lru_cache.cpp /^LRUHandle* HandleTable::remove(const CacheKey& key, uint32_t hash) {$/;" f class:doris::HandleTable +remove src/olap/olap_meta.cpp /^OLAPStatus OlapMeta::remove(int column_family_index, const std::string& key) {$/;" f class:doris::OlapMeta +remove src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::remove() {$/;" f class:doris::AlphaRowset +remove src/olap/rowset/beta_rowset.cpp /^OLAPStatus BetaRowset::remove() {$/;" f class:doris::BetaRowset +remove src/olap/rowset/rowset_meta_manager.cpp /^OLAPStatus RowsetMetaManager::remove(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id) {$/;" f class:doris::RowsetMetaManager +remove src/olap/tablet_meta_manager.cpp /^OLAPStatus TabletMetaManager::remove(DataDir* store, TTabletId tablet_id, TSchemaHash schema_hash,$/;" f class:doris::TabletMetaManager +remove src/runtime/bufferpool/buffer_pool_internal.h /^ bool remove(Page* page) {$/;" f class:doris::BufferPool::PageList +remove src/runtime/stream_load/load_stream_mgr.h /^ void remove(const UniqueId& id) {$/;" f class:doris::LoadStreamMgr +remove src/runtime/tmp_file_mgr.cc /^Status TmpFileMgr::File::remove() {$/;" f class:doris::TmpFileMgr::File +remove src/util/bitmap_value.h /^ void remove(uint32_t x) { roarings[0].remove(x); }$/;" f class:doris::detail::Roaring64Map +remove src/util/bitmap_value.h /^ void remove(uint64_t x) {$/;" f class:doris::detail::Roaring64Map +remove src/util/file_utils.cpp /^Status FileUtils::remove(const std::string& path) {$/;" f class:doris::FileUtils +remove src/util/file_utils.cpp /^Status FileUtils::remove(const std::string& path, doris::Env* env) {$/;" f class:doris::FileUtils +remove src/util/internal_queue.h /^ bool remove(T* n) {$/;" f class:doris::InternalQueueBase +removeChecked src/util/bitmap_value.h /^ bool removeChecked(uint32_t x) { return roarings[0].removeChecked(x); }$/;" f class:doris::detail::Roaring64Map +removeChecked src/util/bitmap_value.h /^ bool removeChecked(uint64_t x) {$/;" f class:doris::detail::Roaring64Map +removeRunCompression src/util/bitmap_value.h /^ bool removeRunCompression() {$/;" f class:doris::detail::Roaring64Map +remove_all src/util/file_utils.cpp /^Status FileUtils::remove_all(const std::string& file_path) {$/;" f class:doris::FileUtils +remove_const src/gutil/type_traits.h /^template struct remove_const { typedef T type; };$/;" s namespace:base +remove_const src/gutil/type_traits.h /^template struct remove_const { typedef T type; };$/;" s namespace:base +remove_cv src/gutil/type_traits.h /^template struct remove_cv {$/;" s namespace:base +remove_delete_predicate_by_version src/olap/tablet_meta.cpp /^void TabletMeta::remove_delete_predicate_by_version(const Version& version) {$/;" f class:doris::TabletMeta +remove_duplicate_points src/geo/geo_types.cpp /^static void remove_duplicate_points(std::vector* points) {$/;" f namespace:doris +remove_escape_character src/exprs/like_predicate.cpp /^void LikePredicate::remove_escape_character(std::string* search_string) {$/;" f class:doris::LikePredicate +remove_fn src/exprs/agg_fn.h /^ void* remove_fn() const { return remove_fn_; }$/;" f class:doris::AggFn +remove_fn_ src/exprs/agg_fn.h /^ void* remove_fn_ = nullptr;$/;" m class:doris::AggFn +remove_metric src/util/metrics.cpp /^void MetricCollector::remove_metric(Metric* metric) {$/;" f class:doris::MetricCollector +remove_old_files src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::remove_old_files(std::vector* files_to_remove) {$/;" f class:doris::AlphaRowset +remove_old_files src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::remove_old_files(std::vector* links_to_remove) {$/;" f class:doris::SegmentGroup +remove_old_meta_and_files src/olap/data_dir.cpp /^OLAPStatus DataDir::remove_old_meta_and_files() {$/;" f class:doris::DataDir +remove_paths src/util/file_utils.cpp /^Status FileUtils::remove_paths(const std::vector& paths) {$/;" f class:doris::FileUtils +remove_paths src/util/filesystem_util.cc /^Status FileSystemUtil::remove_paths(const vector& directories) {$/;" f class:doris::FileSystemUtil +remove_pending_ids src/olap/data_dir.cpp /^void DataDir::remove_pending_ids(const std::string& id) {$/;" f class:doris::DataDir +remove_pointer src/gutil/type_traits.h /^template struct remove_pointer { typedef T type; };$/;" s namespace:base +remove_pointer src/gutil/type_traits.h /^template struct remove_pointer {$/;" s namespace:base +remove_pointer src/gutil/type_traits.h /^template struct remove_pointer { typedef T type; };$/;" s namespace:base +remove_pointer src/gutil/type_traits.h /^template struct remove_pointer { typedef T type; };$/;" s namespace:base +remove_pointer src/gutil/type_traits.h /^template struct remove_pointer { typedef T type; };$/;" s namespace:base +remove_prefix src/gutil/strings/stringpiece.h /^ void remove_prefix(int n) {$/;" f class:StringPiece +remove_prefix src/olap/lru_cache.h /^ void remove_prefix(size_t n) {$/;" f class:doris::CacheKey +remove_prefix src/util/slice.h /^ void remove_prefix(size_t n) {$/;" f struct:doris::Slice +remove_reference src/gutil/type_traits.h /^template struct remove_reference { typedef T type; };$/;" s namespace:base +remove_reference src/gutil/type_traits.h /^template struct remove_reference { typedef T type; };$/;" s namespace:base +remove_sender src/runtime/data_stream_recvr.cc /^void DataStreamRecvr::remove_sender(int sender_id, int be_number) {$/;" f class:doris::DataStreamRecvr +remove_suffix src/gutil/strings/stringpiece.h /^ void remove_suffix(int n) {$/;" f class:StringPiece +remove_thread src/util/thread.cpp /^void ThreadMgr::remove_thread(const pthread_t& pthread_id, const std::string& category) {$/;" f class:doris::ThreadMgr +remove_volatile src/gutil/type_traits.h /^template struct remove_volatile { typedef T type; };$/;" s namespace:base +remove_volatile src/gutil/type_traits.h /^template struct remove_volatile { typedef T type; };$/;" s namespace:base +remove_written_position src/olap/stream_index_writer.cpp /^OLAPStatus PositionEntryWriter::remove_written_position(uint32_t from, size_t count) {$/;" f class:doris::PositionEntryWriter +rend src/gutil/strings/stringpiece.h /^ const_reverse_iterator rend() const {$/;" f class:StringPiece +reopen src/runtime/client_cache.h /^ Status reopen() {$/;" f class:doris::ClientConnection +reopen src/runtime/client_cache.h /^ Status reopen(int timeout_ms) {$/;" f class:doris::ClientConnection +reopen_client src/runtime/client_cache.cpp /^Status ClientCacheHelper::reopen_client(client_factory factory_method, void** client_key,$/;" f class:doris::ClientCacheHelper +reopen_client src/runtime/client_cache.h /^ Status reopen_client(T** client, int timeout_ms) {$/;" f class:doris::ClientCache +repeat src/exprs/string_functions.cpp /^StringVal StringFunctions::repeat($/;" f class:doris::StringFunctions +repeat_count_ src/util/rle_encoding.h /^ int repeat_count_;$/;" m class:doris::RleEncoder +repeat_count_ src/util/rle_encoding.h /^ uint32_t repeat_count_;$/;" m class:doris::RleDecoder +replace src/runtime/string_value.h /^ void replace(char* ptr, int len) {$/;" f struct:doris::StringValue +replaceenv src/common/configbase.cpp /^bool Properties::replaceenv(std::string& s) {$/;" f class:doris::config::Properties +report src/agent/utils.cpp /^AgentStatus MasterServerClient::report(const TReportRequest& request, TMasterResult* result) {$/;" f class:doris::MasterServerClient +report_all_tablets_info src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::report_all_tablets_info(std::map* tablets_info) {$/;" f class:doris::TabletManager +report_all_tablets_requests_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::report_all_tablets_requests_failed;$/;" m class:doris::DorisMetrics file: +report_all_tablets_requests_failed src/util/doris_metrics.h /^ static IntCounter report_all_tablets_requests_failed;$/;" m class:doris::DorisMetrics +report_all_tablets_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::report_all_tablets_requests_total;$/;" m class:doris::DorisMetrics file: +report_all_tablets_requests_total src/util/doris_metrics.h /^ static IntCounter report_all_tablets_requests_total;$/;" m class:doris::DorisMetrics +report_bad_format src/exprs/timestamp_functions.cpp /^void TimestampFunctions::report_bad_format(const StringVal* format) {$/;" f class:doris::TimestampFunctions +report_disk_requests_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::report_disk_requests_failed;$/;" m class:doris::DorisMetrics file: +report_disk_requests_failed src/util/doris_metrics.h /^ static IntCounter report_disk_requests_failed;$/;" m class:doris::DorisMetrics +report_disk_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::report_disk_requests_total;$/;" m class:doris::DorisMetrics file: +report_disk_requests_total src/util/doris_metrics.h /^ static IntCounter report_disk_requests_total;$/;" m class:doris::DorisMetrics +report_io_error src/runtime/tmp_file_mgr.cc /^void TmpFileMgr::File::report_io_error(const std::string& error_msg) {$/;" f class:doris::TmpFileMgr::File +report_profile src/runtime/plan_fragment_executor.cpp /^void PlanFragmentExecutor::report_profile() {$/;" f class:doris::PlanFragmentExecutor +report_profile_once src/runtime/plan_fragment_executor.h /^ void report_profile_once() {$/;" f class:doris::PlanFragmentExecutor +report_pull_load_sub_task_info test/runtime/data_stream_test.cpp /^ virtual void report_pull_load_sub_task_info($/;" f class:doris::DorisTestBackend +report_status_callback src/runtime/plan_fragment_executor.h /^ report_status_callback;$/;" t class:doris::PlanFragmentExecutor +report_tablet_info src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::report_tablet_info(TTabletInfo* tablet_info) {$/;" f class:doris::TabletManager +report_tablet_requests_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::report_tablet_requests_failed;$/;" m class:doris::DorisMetrics file: +report_tablet_requests_failed src/util/doris_metrics.h /^ static IntCounter report_tablet_requests_failed;$/;" m class:doris::DorisMetrics +report_tablet_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::report_tablet_requests_total;$/;" m class:doris::DorisMetrics file: +report_tablet_requests_total src/util/doris_metrics.h /^ static IntCounter report_tablet_requests_total;$/;" m class:doris::DorisMetrics +report_task_requests_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::report_task_requests_failed;$/;" m class:doris::DorisMetrics file: +report_task_requests_failed src/util/doris_metrics.h /^ static IntCounter report_task_requests_failed;$/;" m class:doris::DorisMetrics +report_task_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::report_task_requests_total;$/;" m class:doris::DorisMetrics file: +report_task_requests_total src/util/doris_metrics.h /^ static IntCounter report_task_requests_total;$/;" m class:doris::DorisMetrics +report_to_master src/runtime/etl_job_mgr.cpp /^void EtlJobMgr::report_to_master(PlanFragmentExecutor* executor) {$/;" f class:doris::EtlJobMgr +report_to_master src/runtime/export_task_mgr.cpp /^void ExportTaskMgr::report_to_master(PlanFragmentExecutor* executor) {$/;" f class:doris::ExportTaskMgr +request_contexts src/runtime/disk_io_mgr_internal.h /^ std::list request_contexts;$/;" m struct:doris::DiskIoMgr::DiskQueue +request_type src/runtime/disk_io_mgr.h /^ RequestType::type request_type() const { return _request_type; }$/;" f class:doris::DiskIoMgr::RequestRange +request_will_be_read_progressively src/http/http_handler.h /^ virtual bool request_will_be_read_progressively() { return false; }$/;" f class:doris::HttpHandler +require_separator src/gutil/strings/numbers.h /^ bool require_separator;$/;" m struct:DoubleRangeOptions +res_buf test/exec/olap_scanner_test.cpp /^static char res_buf[RES_BUF_SIZE];$/;" m namespace:doris file: +reservation src/runtime/bufferpool/buffer_pool_internal.h /^ ReservationTracker* reservation() { return &reservation_; }$/;" f class:doris::BufferPool::Client +reservation_ src/runtime/bufferpool/buffer_pool_internal.h /^ ReservationTracker reservation_;$/;" m class:doris::BufferPool::Client +reservation_ src/runtime/bufferpool/reservation_tracker.h /^ int64_t reservation_;$/;" m class:doris::ReservationTracker +reservation_limit src/runtime/bufferpool/reservation_tracker_counters.h /^ RuntimeProfile::Counter* reservation_limit;$/;" m struct:doris::ReservationTrackerCounters +reservation_limit_ src/runtime/bufferpool/reservation_tracker.h /^ int64_t reservation_limit_;$/;" m class:doris::ReservationTracker +reserve src/olap/tuple.h /^ void reserve(size_t size) {$/;" f class:doris::OlapTuple +reserve src/util/faststring.h /^ void reserve(size_t newcapacity) {$/;" f class:doris::faststring +reserve src/util/mysql_row_buffer.cpp /^int MysqlRowBuffer::reserve(int size) {$/;" f class:doris::MysqlRowBuffer +reserve_optional_tokens src/runtime/thread_resource_mgr.cpp /^void ThreadResourceMgr::ResourcePool::reserve_optional_tokens(int num) {$/;" f class:doris::ThreadResourceMgr::ResourcePool +reserved src/util/mysql_row_buffer.cpp /^char* MysqlRowBuffer::reserved(int size) {$/;" f class:doris::MysqlRowBuffer +reset src/exec/exec_node.cpp /^Status ExecNode::reset(RuntimeState* state) {$/;" f class:doris::ExecNode +reset src/exec/new_partitioned_aggregation_node.cc /^Status NewPartitionedAggregationNode::reset(RuntimeState* state) {$/;" f class:doris::NewPartitionedAggregationNode +reset src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::reset(RuntimeState* state) {$/;" f class:doris::PartitionedAggregationNode +reset src/exec/row_batch_list.h /^ void reset() {$/;" f class:doris::RowBatchList +reset src/exec/spill_sort_node.cc /^Status SpillSortNode::reset(RuntimeState* state) {$/;" f class:doris::SpillSortNode +reset src/gutil/gscoped_ptr.h /^ void reset(C* p = NULL) {$/;" f class:gscoped_array +reset src/gutil/gscoped_ptr.h /^ void reset(C* p = NULL) {$/;" f class:gscoped_ptr_malloc +reset src/gutil/gscoped_ptr.h /^ void reset(T* p) {$/;" f class:doris::internal::gscoped_ptr_impl +reset src/gutil/gscoped_ptr.h /^ void reset(element_type* array = NULL) { impl_.reset(array); }$/;" f class:gscoped_ptr +reset src/gutil/gscoped_ptr.h /^ void reset(element_type* p = NULL) { impl_.reset(p); }$/;" f class:gscoped_ptr +reset src/gutil/ref_counted.h /^ void reset(T* p = NULL) {$/;" f class:scoped_refptr +reset src/olap/bloom_filter.hpp /^ void reset() {$/;" f class:doris::BitSet +reset src/olap/bloom_filter.hpp /^ void reset() {$/;" f class:doris::BloomFilter +reset src/olap/file_stream.h /^ void reset(size_t offset, size_t length) {$/;" f class:doris::ReadOnlyFileStream::FileCursor +reset src/olap/file_stream.h /^ inline void reset(uint64_t offset, uint64_t length) {$/;" f class:doris::ReadOnlyFileStream +reset src/olap/rowset/segment_v2/binary_dict_page.cpp /^void BinaryDictPageBuilder::reset() {$/;" f class:doris::segment_v2::BinaryDictPageBuilder +reset src/olap/rowset/segment_v2/bloom_filter.h /^ void reset() {$/;" f class:doris::segment_v2::BloomFilter +reset src/olap/rowset/segment_v2/column_writer.cpp /^ void reset() {$/;" f class:doris::segment_v2::NullBitmapBuilder +reset src/olap/rowset/segment_v2/index_page.h /^ void reset() {$/;" f class:doris::segment_v2::IndexPageBuilder +reset src/olap/rowset/segment_v2/page_pointer.h /^ void reset() {$/;" f struct:doris::segment_v2::PagePointer +reset src/olap/stream_index_common.cpp /^void ColumnStatistics::reset() {$/;" f class:doris::ColumnStatistics +reset src/olap/stream_index_writer.cpp /^OLAPStatus StreamIndexWriter::reset() {$/;" f class:doris::StreamIndexWriter +reset src/olap/tuple.h /^ void reset() {$/;" f class:doris::OlapTuple +reset src/olap/utils.h /^ void reset() {$/;" f class:doris::OlapStopWatch +reset src/runtime/disk_io_mgr.cc /^void DiskIoMgr::BufferDescriptor::reset(RequestContext* reader,$/;" f class:doris::DiskIoMgr::BufferDescriptor +reset src/runtime/disk_io_mgr_internal.h /^ void reset() {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +reset src/runtime/disk_io_mgr_reader_context.cc /^void DiskIoMgr::RequestContext::reset(MemTracker* tracker) {$/;" f class:doris::DiskIoMgr::RequestContext +reset src/runtime/disk_io_mgr_scan_range.cc /^void DiskIoMgr::ScanRange::reset($/;" f class:doris::DiskIoMgr::ScanRange +reset src/runtime/free_list.hpp /^ void reset() {$/;" f class:doris::FreeList +reset src/runtime/routine_load/data_consumer.cpp /^Status KafkaDataConsumer::reset() {$/;" f class:doris::KafkaDataConsumer +reset src/runtime/row_batch.cpp /^void RowBatch::reset() {$/;" f class:doris::RowBatch +reset src/runtime/spill_sorter.cc /^Status SpillSorter::reset() {$/;" f class:doris::SpillSorter +reset src/runtime/string_buffer.hpp /^ void reset() {$/;" f class:doris::StringBuffer +reset src/runtime/thread_resource_mgr.cpp /^void ThreadResourceMgr::ResourcePool::reset() {$/;" f class:doris::ThreadResourceMgr::ResourcePool +reset src/util/cidr.cpp /^bool CIDR::reset(const std::string& cidr_str) {$/;" f class:doris::CIDR +reset src/util/cidr.cpp /^void CIDR::reset() {$/;" f class:doris::CIDR +reset src/util/countdown_latch.h /^ void reset(uint64_t count) {$/;" f class:doris::CountDownLatch +reset src/util/mysql_row_buffer.h /^ void reset() {$/;" f class:doris::MysqlRowBuffer +reset src/util/stopwatch.hpp /^ uint64_t reset() {$/;" f class:doris::MonotonicStopWatch +reset test/runtime/data_stream_test.cpp /^ void reset() {$/;" f class:doris::DataStreamTest +reset_buf src/runtime/dpp_writer.cpp /^void DppWriter::reset_buf() {$/;" f class:doris::DppWriter +reset_filtered_rows src/olap/schema_change.h /^ void reset_filtered_rows() {$/;" f class:doris::SchemaChange +reset_merged_rows src/olap/schema_change.h /^ void reset_merged_rows() {$/;" f class:doris::SchemaChange +reset_offset src/runtime/stream_load/stream_load_context.h /^ void reset_offset() {$/;" f class:doris::KafkaLoadInfo +reset_sizeinfo src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::reset_sizeinfo() {$/;" f class:doris::AlphaRowset +reset_tablet_uid src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::reset_tablet_uid(const string& file_path) {$/;" f class:doris::TabletMeta +reset_write_offset src/olap/stream_index_writer.cpp /^void PositionEntryWriter::reset_write_offset() {$/;" f class:doris::PositionEntryWriter +resize src/udf/udf.cpp /^bool StringVal::resize(FunctionContext* ctx, int new_len) {$/;" f class:doris_udf::StringVal +resize src/util/faststring.h /^ void resize(size_t newsize) {$/;" f class:doris::faststring +resize_and_allocate_tuple_buffer src/runtime/row_batch.cpp /^Status RowBatch::resize_and_allocate_tuple_buffer(RuntimeState* state,$/;" f class:doris::RowBatch +resize_buckets src/exec/hash_table.cpp /^void HashTable::resize_buckets(int64_t num_buckets) {$/;" f class:doris::HashTable +resize_buckets src/exec/partitioned_hash_table.cc /^bool PartitionedHashTable::resize_buckets(int64_t num_buckets, PartitionedHashTableCtx* ht_ctx) {$/;" f class:doris::PartitionedHashTable +resize_file src/util/filesystem_util.cc /^Status FileSystemUtil::resize_file(const string& file_path, int64_t trunc_len) {$/;" f class:doris::FileSystemUtil +resize_table test/exec/hash_table_test.cpp /^ void resize_table(HashTable* table, int64_t new_size) {$/;" f class:doris::HashTableTest +resource_destructor src/common/resource_tls.cpp /^static void resource_destructor(void* value) {$/;" f namespace:doris +resource_pool src/runtime/runtime_state.h /^ ThreadResourceMgr::ResourcePool* resource_pool() {$/;" f class:doris::RuntimeState +rest_clear_scroll_action test/exec/es_scan_reader_test.cpp /^static RestClearScrollAction rest_clear_scroll_action = RestClearScrollAction();$/;" m namespace:doris file: +rest_read_compressed src/util/minizip/unzip.c /^ ZPOS64_T rest_read_compressed; \/* number of byte to be decompressed *\/$/;" m struct:__anon33 file: +rest_read_uncompressed src/util/minizip/unzip.c /^ ZPOS64_T rest_read_uncompressed;\/*number of byte to be obtained after decomp*\/$/;" m struct:__anon33 file: +rest_search_action test/exec/es_scan_reader_test.cpp /^static RestSearchAction rest_search_action = RestSearchAction();$/;" m namespace:doris file: +rest_search_scroll_action test/exec/es_scan_reader_test.cpp /^static RestSearchScrollAction rest_search_scroll_action = RestSearchScrollAction();$/;" m namespace:doris file: +result src/gutil/port.h /^struct AlignType { typedef char result[Size]; };$/;" t struct:AlignType +result src/gutil/port.h /^template struct AlignType<0, size> { typedef char result[size]; };$/;" t struct:AlignType +result src/runtime/buffer_control_block.h /^ PFetchDataResult* result = nullptr;$/;" m struct:doris::GetResultBatchCtx +result src/runtime/etl_job_mgr.h /^ EtlJobResult result;$/;" m struct:doris::EtlJobCtx +result src/runtime/export_task_mgr.h /^ ExportTaskResult result;$/;" m struct:doris::ExportTaskCtx +result src/util/ref_count_closure.h /^ T result;$/;" m class:doris::RefCountClosure +result_mgr src/runtime/exec_env.h /^ ResultBufferMgr* result_mgr() { return _result_mgr; }$/;" f class:doris::ExecEnv +result_queue_mgr src/runtime/exec_env.h /^ ResultQueueMgr* result_queue_mgr() {return _result_queue_mgr;}$/;" f class:doris::ExecEnv +results_buffer_size src/exec/partitioned_hash_table.h /^ int results_buffer_size() const { return _results_buffer_size; }$/;" f class:doris::PartitionedHashTableCtx +retrieve src/util/path_trie.hpp /^ bool retrieve(const std::vector path, int index, $/;" f class:doris::PathTrie::TrieNode +retrieve src/util/path_trie.hpp /^ bool retrieve(const std::string& path, T* value) {$/;" f class:doris::PathTrie +retrieve src/util/path_trie.hpp /^ bool retrieve(const std::string& path, T* value, $/;" f class:doris::PathTrie +retrieved_buffer src/runtime/buffered_tuple_stream3.h /^ bool retrieved_buffer;$/;" m struct:doris::BufferedTupleStream3::Page +return_allocation src/runtime/buffered_block_mgr.h /^ void return_allocation(int size) {$/;" f class:doris::BufferedBlockMgr::Block +return_allocation src/runtime/buffered_block_mgr2.h /^ void return_allocation(int size) {$/;" f class:doris::BufferedBlockMgr2::Block +return_buffer src/runtime/disk_io_mgr.cc /^void DiskIoMgr::BufferDescriptor::return_buffer() {$/;" f class:doris::DiskIoMgr::BufferDescriptor +return_buffer src/runtime/disk_io_mgr.cc /^void DiskIoMgr::return_buffer(BufferDescriptor* buffer_desc) {$/;" f class:doris::DiskIoMgr +return_buffer_desc src/runtime/disk_io_mgr.cc /^void DiskIoMgr::return_buffer_desc(BufferDescriptor* desc) {$/;" f class:doris::DiskIoMgr +return_columns src/olap/reader.h /^ std::vector return_columns;$/;" m struct:doris::ReaderParams +return_columns src/olap/rowset/rowset_reader_context.h /^ const std::vector* return_columns = nullptr;$/;" m struct:doris::RowsetReaderContext +return_consumer src/runtime/routine_load/data_consumer_pool.cpp /^void DataConsumerPool::return_consumer(std::shared_ptr consumer) {$/;" f class:doris::DataConsumerPool +return_consumers src/runtime/routine_load/data_consumer_pool.cpp /^void DataConsumerPool::return_consumers(DataConsumerGroup* grp) {$/;" f class:doris::DataConsumerPool +return_context src/runtime/disk_io_mgr.cc /^ void return_context(RequestContext* reader) {$/;" f class:doris::DiskIoMgr::RequestContextCache +return_free_buffer src/runtime/disk_io_mgr.cc /^void DiskIoMgr::return_free_buffer(BufferDescriptor* desc) {$/;" f class:doris::DiskIoMgr +return_free_buffer src/runtime/disk_io_mgr.cc /^void DiskIoMgr::return_free_buffer(char* buffer, int64_t buffer_size) {$/;" f class:doris::DiskIoMgr +return_unused_block src/runtime/buffered_block_mgr2.cc /^void BufferedBlockMgr2::return_unused_block(Block* block) {$/;" f class:doris::BufferedBlockMgr2 +reverse src/exprs/string_functions.cpp /^StringVal StringFunctions::reverse(FunctionContext* context, const StringVal& str) {$/;" f class:doris::StringFunctions +reverse_iterator src/gutil/strings/stringpiece.h /^ typedef std::reverse_iterator reverse_iterator;$/;" t class:StringPiece +revise_inc_rs_metas src/olap/tablet_meta.cpp /^void TabletMeta::revise_inc_rs_metas(const std::vector& rs_metas) {$/;" f class:doris::TabletMeta +revise_rs_metas src/olap/tablet_meta.cpp /^void TabletMeta::revise_rs_metas(const std::vector& rs_metas) {$/;" f class:doris::TabletMeta +revise_tablet_meta src/olap/tablet.cpp /^OLAPStatus Tablet::revise_tablet_meta($/;" f class:doris::Tablet +rewind_state_ src/util/rle_encoding.h /^ RewindState rewind_state_;$/;" m class:doris::RleDecoder +rfind src/gutil/strings/stringpiece.cc /^int StringPiece::rfind(StringPiece s, size_type pos) const {$/;" f class:StringPiece +rfind src/gutil/strings/stringpiece.cc /^int StringPiece::rfind(char c, size_type pos) const {$/;" f class:StringPiece +rhs_ordering_expr_ctxs src/exec/sort_exec_exprs.h /^ const std::vector& rhs_ordering_expr_ctxs() const {$/;" f class:doris::SortExecExprs +right src/exprs/string_functions.cpp /^StringVal StringFunctions::right($/;" f class:doris::StringFunctions +rnd_ src/olap/skiplist.h /^ Random rnd_;$/;" m class:doris::SkipList +roarings src/util/bitmap_value.h /^ std::map roarings;$/;" m class:doris::detail::Roaring64Map +rollback_txn src/olap/txn_manager.cpp /^OLAPStatus TxnManager::rollback_txn(TPartitionId partition_id, TTransactionId transaction_id,$/;" f class:doris::TxnManager +rollback_txn src/olap/txn_manager.cpp /^OLAPStatus TxnManager::rollback_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, TTransactionId transaction_id) {$/;" f class:doris::TxnManager +rollback_txn src/runtime/stream_load/stream_load_executor.cpp /^void StreamLoadExecutor::rollback_txn(StreamLoadContext* ctx) {$/;" f class:doris::StreamLoadExecutor +root src/exprs/expr_context.h /^ Expr* root() { $/;" f class:doris::ExprContext +root_handler src/http/web_page_handler.cpp /^void WebPageHandler::root_handler($/;" f class:doris::WebPageHandler +rotl32 src/util/hash_util.hpp /^ ALWAYS_INLINE static uint32_t rotl32(uint32_t x, int8_t r) {$/;" f class:doris::HashUtil +rotl32 src/util/murmur_hash3.cpp /^inline uint32_t rotl32(uint32_t x, int8_t r)$/;" f +rotl64 src/util/murmur_hash3.cpp /^inline uint64_t rotl64(uint64_t x, int8_t r)$/;" f +round src/exprs/math_functions.cpp /^BigIntVal MathFunctions::round($/;" f class:doris::MathFunctions +round src/runtime/decimal_value.cpp /^int DecimalValue::round(DecimalValue *to, int scale, DecimalRoundMode mode) {$/;" f class:doris::DecimalValue +round src/runtime/decimalv2_value.cpp /^int DecimalV2Value::round(DecimalV2Value *to, int rounding_scale, DecimalRoundMode op) {$/;" f class:doris::DecimalV2Value +round_up src/runtime/decimal_value.h /^template inline T round_up(T length) {$/;" f namespace:doris +round_up src/util/bit_util.h /^ static inline int64_t round_up(int64_t value, int64_t factor) {$/;" f class:doris::BitUtil +round_up_numi64 src/util/bit_util.h /^ static inline uint32_t round_up_numi64(uint32_t bits) {$/;" f class:doris::BitUtil +round_up_numi_64 src/util/bit_util.h /^ static inline uint32_t round_up_numi_64(uint32_t bits) {$/;" f class:doris::BitUtil +round_up_to src/exprs/math_functions.cpp /^DoubleVal MathFunctions::round_up_to($/;" f class:doris::MathFunctions +routine_load_task_executor src/runtime/exec_env.h /^ RoutineLoadTaskExecutor* routine_load_task_executor() { return _routine_load_task_executor; }$/;" f class:doris::ExecEnv +row src/exec/partitioned_hash_table.h /^ TupleRow* row() const { return _row; }$/;" f class:doris::PartitionedHashTableCtx +row src/olap/row_block2.h /^inline RowBlockRow RowBlockV2::row(size_t row_idx) const {$/;" f class:doris::RowBlockV2 +row_block src/olap/row_block2.h /^ const RowBlockV2* row_block() const { return _block; }$/;" f class:doris::RowBlockRow +row_block src/olap/rowset/alpha_rowset_reader.h /^ RowBlock* row_block = nullptr;$/;" m struct:doris::AlphaMergeContext +row_block src/olap/schema_change.cpp /^ const RowBlock* row_block;$/;" m struct:doris::RowBlockMerger::MergeElement file: +row_block_index src/olap/schema_change.cpp /^ uint32_t row_block_index;$/;" m struct:doris::RowBlockMerger::MergeElement file: +row_block_info src/olap/row_block.h /^ const RowBlockInfo& row_block_info() const { return _info; }$/;" f class:doris::RowBlock +row_byte_size src/runtime/row_batch.h /^ int row_byte_size() {$/;" f class:doris::RowBatch +row_counters test/exec/tablet_sink_test.cpp /^ int64_t row_counters = 0;$/;" m class:doris::stream_load::TestInternalService file: +row_cursor src/olap/rowset/alpha_rowset_reader.h /^ std::unique_ptr row_cursor = nullptr;$/;" m struct:doris::AlphaMergeContext +row_cursor src/olap/schema_change.cpp /^ RowCursor* row_cursor;$/;" m struct:doris::RowBlockMerger::MergeElement file: +row_desc src/exec/exec_node.h /^ const RowDescriptor& row_desc() const {$/;" f class:doris::ExecNode +row_desc src/runtime/data_stream_recvr.h /^ const RowDescriptor& row_desc() const { return _row_desc; }$/;" f class:doris::DataStreamRecvr +row_desc src/runtime/plan_fragment_executor.cpp /^const RowDescriptor& PlanFragmentExecutor::row_desc() {$/;" f class:doris::PlanFragmentExecutor +row_desc src/runtime/row_batch.h /^ const RowDescriptor& row_desc() const {$/;" f class:doris::RowBatch +row_desc test/exprs/binary_predicate_test.cpp /^ RowDescriptor* row_desc() {$/;" f class:doris::BinaryOpTest +row_idx src/exec/merge_join_node.h /^ int row_idx;$/;" m struct:doris::MergeJoinNode::ChildReaderContext +row_index src/olap/row_block2.h /^ size_t row_index() const { return _row_index; }$/;" f class:doris::RowBlockRow +row_num src/olap/row_block.h /^ const uint32_t row_num() const { return _info.row_num; }$/;" f class:doris::RowBlock +row_num src/olap/row_block.h /^ uint32_t row_num; \/\/ block最大数据行数$/;" m struct:doris::RowBlockInfo +row_ptr src/olap/row.h /^ void* row_ptr() const { return _row; }$/;" f struct:doris::ContiguousRow +row_ptr src/olap/row_cursor.h /^ char* row_ptr() const { return _fixed_buf; }$/;" f class:doris::RowCursor +row_size src/olap/tablet.h /^inline size_t Tablet::row_size() const {$/;" f class:doris::Tablet +row_size src/olap/tablet_schema.cpp /^size_t TabletSchema::row_size() const {$/;" f class:doris::TabletSchema +rows_bf_filtered src/olap/olap_common.h /^ int64_t rows_bf_filtered = 0;$/;" m struct:doris::OlapReaderStatistics +rows_del_filtered src/olap/olap_common.h /^ int64_t rows_del_filtered = 0;$/;" m struct:doris::OlapReaderStatistics +rows_read_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* rows_read_counter() const {$/;" f class:doris::ScanNode +rows_returned src/exec/exec_node.h /^ int64_t rows_returned() const {$/;" f class:doris::ExecNode +rows_returned src/runtime/buffered_tuple_stream.h /^ int64_t rows_returned() const {$/;" f class:doris::BufferedTupleStream +rows_returned src/runtime/buffered_tuple_stream2.h /^ int64_t rows_returned() const { return _rows_returned; }$/;" f class:doris::BufferedTupleStream2 +rows_returned src/runtime/buffered_tuple_stream3.h /^ int64_t rows_returned() const { return rows_returned_; }$/;" f class:doris::BufferedTupleStream3 +rows_returned_ src/runtime/buffered_tuple_stream3.h /^ int64_t rows_returned_;$/;" m class:doris::BufferedTupleStream3 +rows_stats_filtered src/olap/olap_common.h /^ int64_t rows_stats_filtered = 0;$/;" m struct:doris::OlapReaderStatistics +rows_vec_cond_filtered src/olap/olap_common.h /^ int64_t rows_vec_cond_filtered = 0;$/;" m struct:doris::OlapReaderStatistics +rowset src/olap/rowset/alpha_rowset_reader.cpp /^RowsetSharedPtr AlphaRowsetReader::rowset() {$/;" f class:doris::AlphaRowsetReader +rowset src/olap/txn_manager.h /^ RowsetSharedPtr rowset;$/;" m struct:doris::TabletTxnInfo +rowset_id src/olap/rowset/rowset.h /^ RowsetId rowset_id() const { return rowset_meta()->rowset_id(); }$/;" f class:doris::Rowset +rowset_id src/olap/rowset/rowset_meta.h /^ RowsetId rowset_id() const {$/;" f class:doris::RowsetMeta +rowset_id src/olap/rowset/rowset_writer_context.h /^ RowsetId rowset_id;$/;" m struct:doris::RowsetWriterContext +rowset_id src/olap/rowset/segment_group.h /^ const RowsetId& rowset_id() {$/;" f class:doris::SegmentGroup +rowset_id_in_use src/olap/storage_engine.h /^ bool rowset_id_in_use(const RowsetId& rowset_id) {$/;" f class:doris::StorageEngine +rowset_meta src/olap/rowset/rowset.h /^ const RowsetMetaSharedPtr& rowset_meta() const { return _rowset_meta; }$/;" f class:doris::Rowset +rowset_meta_is_useful src/olap/tablet.cpp /^bool Tablet::rowset_meta_is_useful(RowsetMetaSharedPtr rowset_meta) {$/;" f class:doris::Tablet +rowset_meta_path test/olap/rowset/rowset_meta_manager_test.cpp /^const std::string rowset_meta_path = ".\/be\/test\/olap\/test_data\/rowset_meta.json";$/;" m namespace:doris file: +rowset_meta_path test/olap/rowset/rowset_meta_test.cpp /^const std::string rowset_meta_path = ".\/be\/test\/olap\/test_data\/rowset.json";$/;" m namespace:doris file: +rowset_meta_path test/olap/txn_manager_test.cpp /^const std::string rowset_meta_path = ".\/be\/test\/olap\/test_data\/rowset_meta.json";$/;" m namespace:doris file: +rowset_meta_path_2 test/olap/txn_manager_test.cpp /^const std::string rowset_meta_path_2 = ".\/be\/test\/olap\/test_data\/rowset_meta2.json";$/;" m namespace:doris file: +rowset_meta_pb src/olap/tablet_sync_service.h /^ RowsetMetaPB rowset_meta_pb;$/;" m struct:doris::PushRowsetMetaTask +rowset_path_prefix src/olap/rowset/rowset_writer_context.h /^ std::string rowset_path_prefix;$/;" m struct:doris::RowsetWriterContext +rowset_path_prefix src/olap/rowset/segment_group.cpp /^std::string SegmentGroup::rowset_path_prefix() {$/;" f class:doris::SegmentGroup +rowset_pruning_filter src/olap/olap_cond.cpp /^bool Conditions::rowset_pruning_filter(const std::vector& zone_maps) const {$/;" f class:doris::Conditions +rowset_pruning_filter src/olap/rowset/column_data.cpp /^bool ColumnData::rowset_pruning_filter() {$/;" f class:doris::ColumnData +rowset_state src/olap/rowset/rowset.h /^ RowsetState rowset_state() {$/;" f class:doris::RowsetStateMachine +rowset_state src/olap/rowset/rowset_meta.h /^ RowsetStatePB rowset_state() const {$/;" f class:doris::RowsetMeta +rowset_state src/olap/rowset/rowset_writer_context.h /^ RowsetStatePB rowset_state;$/;" m struct:doris::RowsetWriterContext +rowset_to_add src/olap/push_handler.h /^ RowsetSharedPtr rowset_to_add;$/;" m struct:doris::TabletVars +rowset_type src/olap/rowset/rowset_meta.h /^ RowsetTypePB rowset_type() const {$/;" f class:doris::RowsetMeta +rowset_type src/olap/rowset/rowset_writer_context.h /^ RowsetTypePB rowset_type;$/;" m struct:doris::RowsetWriterContext +rowset_with_max_version src/olap/tablet.cpp /^const RowsetSharedPtr Tablet::rowset_with_max_version() const {$/;" f class:doris::Tablet +rpad src/exprs/string_functions.cpp /^StringVal StringFunctions::rpad($/;" f class:doris::StringFunctions +rpc src/util/thrift_rpc_helper.cpp /^Status ThriftRpcHelper::rpc($/;" f class:doris::ThriftRpcHelper +rpc src/util/thrift_rpc_helper.h /^ static Status rpc($/;" f class:doris::ThriftRpcHelper +rs_readers src/olap/reader.h /^ std::vector rs_readers;$/;" m struct:doris::ReaderParams +rsync_from_remote src/agent/utils.cpp /^AgentStatus AgentUtils::rsync_from_remote($/;" f class:doris::AgentUtils +rtrim src/exprs/string_functions.cpp /^StringVal StringFunctions::rtrim(FunctionContext* context, const StringVal& str) {$/;" f class:doris::StringFunctions +ru_idrss src/gutil/linux_syscall_support.h /^ long ru_idrss;$/;" m struct:kernel_rusage +ru_inblock src/gutil/linux_syscall_support.h /^ long ru_inblock;$/;" m struct:kernel_rusage +ru_isrss src/gutil/linux_syscall_support.h /^ long ru_isrss;$/;" m struct:kernel_rusage +ru_ixrss src/gutil/linux_syscall_support.h /^ long ru_ixrss;$/;" m struct:kernel_rusage +ru_majflt src/gutil/linux_syscall_support.h /^ long ru_majflt;$/;" m struct:kernel_rusage +ru_maxrss src/gutil/linux_syscall_support.h /^ long ru_maxrss;$/;" m struct:kernel_rusage +ru_minflt src/gutil/linux_syscall_support.h /^ long ru_minflt;$/;" m struct:kernel_rusage +ru_msgrcv src/gutil/linux_syscall_support.h /^ long ru_msgrcv;$/;" m struct:kernel_rusage +ru_msgsnd src/gutil/linux_syscall_support.h /^ long ru_msgsnd;$/;" m struct:kernel_rusage +ru_nivcsw src/gutil/linux_syscall_support.h /^ long ru_nivcsw;$/;" m struct:kernel_rusage +ru_nsignals src/gutil/linux_syscall_support.h /^ long ru_nsignals;$/;" m struct:kernel_rusage +ru_nswap src/gutil/linux_syscall_support.h /^ long ru_nswap;$/;" m struct:kernel_rusage +ru_nvcsw src/gutil/linux_syscall_support.h /^ long ru_nvcsw;$/;" m struct:kernel_rusage +ru_oublock src/gutil/linux_syscall_support.h /^ long ru_oublock;$/;" m struct:kernel_rusage +ru_stime src/gutil/linux_syscall_support.h /^ struct kernel_timeval ru_stime;$/;" m struct:kernel_rusage typeref:struct:kernel_rusage::kernel_timeval +ru_utime src/gutil/linux_syscall_support.h /^ struct kernel_timeval ru_utime;$/;" m struct:kernel_rusage typeref:struct:kernel_rusage::kernel_timeval +runOptimize src/util/bitmap_value.h /^ bool runOptimize() {$/;" f class:doris::detail::Roaring64Map +run_concurrent test/olap/skiplist_test.cpp /^static void run_concurrent(int run) {$/;" f namespace:doris +run_length src/olap/rowset/run_length_integer_writer.h /^ uint8_t run_length: 3,$/;" m struct:doris::RunLengthIntegerWriter::ShortRepeatHead +run_scanner src/exec/kudu_scan_node.cpp /^void KuduScanNode::run_scanner(const string& name, const string* initial_token) {$/;" f class:doris::KuduScanNode +run_scanner_thread src/exec/kudu_scan_node.cpp /^void KuduScanNode::run_scanner_thread(KuduScanNode *scanNode, const string& name, const string* initial_token) {$/;" f class:doris::KuduScanNode +runelen src/gutil/utf/rune.c /^runelen(Rune rune)$/;" f +runenlen src/gutil/utf/rune.c /^runenlen(const Rune *r, int nrune)$/;" f +runetochar src/gutil/utf/rune.c /^runetochar(char *str, const Rune *rune)$/;" f +runnable src/util/threadpool.h /^ std::shared_ptr runnable;$/;" m struct:doris::ThreadPool::Task +runtime_profile src/exec/exec_node.h /^ RuntimeProfile* runtime_profile() {$/;" f class:doris::ExecNode +runtime_profile src/runtime/runtime_state.h /^ RuntimeProfile* runtime_profile() {$/;" f class:doris::RuntimeState +runtime_state src/exec/olap_scanner.h /^ RuntimeState* runtime_state() {$/;" f class:doris::OlapScanner +runtime_state src/olap/reader.h /^ RuntimeState* runtime_state;$/;" m struct:doris::ReaderParams +runtime_state src/olap/rowset/rowset_reader_context.h /^ RuntimeState* runtime_state = nullptr;$/;" m struct:doris::RowsetReaderContext +runtime_state src/runtime/plan_fragment_executor.h /^ RuntimeState* runtime_state() {$/;" f class:doris::PlanFragmentExecutor +runtime_state test/exprs/binary_predicate_test.cpp /^ RuntimeState* runtime_state() {$/;" f class:doris::BinaryOpTest +s_ab_day_name src/runtime/datetime_value.cpp /^static const char* s_ab_day_name[] = $/;" m namespace:doris file: +s_ab_month_name src/runtime/datetime_value.cpp /^static const char* s_ab_month_name[] = $/;" m namespace:doris file: +s_day_name src/runtime/datetime_value.cpp /^static const char* s_day_name[] = $/;" m namespace:doris file: +s_days_in_month src/runtime/datetime_value.cpp /^static int s_days_in_month[13] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31};$/;" m namespace:doris file: +s_db_result test/exec/schema_scanner/schema_columns_scanner_test.cpp /^Status s_db_result;$/;" m namespace:doris file: +s_db_result test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^Status s_db_result;$/;" m namespace:doris file: +s_db_result test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^Status s_db_result;$/;" m namespace:doris file: +s_db_result test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^Status s_db_result;$/;" m namespace:doris file: +s_db_result test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^Status s_db_result;$/;" m namespace:doris file: +s_db_result test/exec/schema_scanner/schema_tables_scanner_test.cpp /^Status s_db_result;$/;" m namespace:doris file: +s_decoding_table src/exprs/base64.cpp /^static short s_decoding_table[256] = {$/;" v file: +s_desc_by_method src/http/http_method.cpp /^static std::map s_desc_by_method = $/;" m namespace:doris file: +s_desc_result test/exec/schema_scanner/schema_columns_scanner_test.cpp /^Status s_desc_result;$/;" m namespace:doris file: +s_desc_result test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^Status s_desc_result;$/;" m namespace:doris file: +s_desc_result test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^Status s_desc_result;$/;" m namespace:doris file: +s_desc_result test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^Status s_desc_result;$/;" m namespace:doris file: +s_desc_result test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^Status s_desc_result;$/;" m namespace:doris file: +s_desc_result test/exec/schema_scanner/schema_tables_scanner_test.cpp /^Status s_desc_result;$/;" m namespace:doris file: +s_empty src/http/http_request.cpp /^static std::string s_empty = "";$/;" m namespace:doris file: +s_encoding_info_resolver src/olap/rowset/segment_v2/encoding_info.cpp /^static EncodingInfoResolver s_encoding_info_resolver;$/;" m namespace:doris::segment_v2 file: +s_encoding_table src/exprs/base64.cpp /^static char s_encoding_table[] = {$/;" v file: +s_expect_response test/http/metrics_action_test.cpp /^const char* s_expect_response = nullptr;$/;" m namespace:doris file: +s_global_cg_mgr src/agent/cgroups_mgr.cpp /^static CgroupsMgr *s_global_cg_mgr;$/;" m namespace:doris file: +s_html_content_type src/http/web_page_handler.cpp /^static std::string s_html_content_type = "text\/html";$/;" m namespace:doris file: +s_is_init src/common/resource_tls.cpp /^static bool s_is_init = false;$/;" m namespace:doris file: +s_method_by_desc src/http/http_method.cpp /^static std::map s_method_by_desc = $/;" m namespace:doris file: +s_mod_table src/exprs/base64.cpp /^static int s_mod_table[] = {0, 2, 1};$/;" v file: +s_month_name src/runtime/datetime_value.cpp /^static const char* s_month_name[] = $/;" m namespace:doris file: +s_mutex src/runtime/memory/chunk_allocator.cpp /^static std::mutex s_mutex;$/;" m namespace:doris file: +s_open_status test/runtime/fragment_mgr_test.cpp /^static Status s_open_status;$/;" m namespace:doris file: +s_prepare_status test/runtime/fragment_mgr_test.cpp /^static Status s_prepare_status;$/;" m namespace:doris file: +s_reason_map src/http/http_status.cpp /^static std::map s_reason_map = {$/;" m namespace:doris file: +s_resource_key src/common/resource_tls.cpp /^static pthread_key_t s_resource_key;$/;" m namespace:doris file: +s_server test/http/http_client_test.cpp /^static EvHttpServer* s_server = nullptr;$/;" m namespace:doris file: +s_server test/runtime/small_file_mgr_test.cpp /^static EvHttpServer* s_server = nullptr;$/;" m namespace:doris file: +s_server test/runtime/user_function_cache_test.cpp /^static EvHttpServer* s_server = nullptr;$/;" m namespace:doris file: +s_simple_get_handler test/http/http_client_test.cpp /^static HttpClientTestSimpleGetHandler s_simple_get_handler = HttpClientTestSimpleGetHandler();$/;" m namespace:doris file: +s_simple_post_handler test/http/http_client_test.cpp /^static HttpClientTestSimplePostHandler s_simple_post_handler = HttpClientTestSimplePostHandler();$/;" m namespace:doris file: +s_table_result test/exec/schema_scanner/schema_columns_scanner_test.cpp /^Status s_table_result;$/;" m namespace:doris file: +s_table_result test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^Status s_table_result;$/;" m namespace:doris file: +s_table_result test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^Status s_table_result;$/;" m namespace:doris file: +s_table_result test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^Status s_table_result;$/;" m namespace:doris file: +s_table_result test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^Status s_table_result;$/;" m namespace:doris file: +s_table_result test/exec/schema_scanner/schema_tables_scanner_test.cpp /^Status s_table_result;$/;" m namespace:doris file: +s_test_columns test/exec/schema_scanner_test.cpp /^SchemaScanner::ColumnDesc s_test_columns[] = {$/;" m namespace:doris file: +s_test_handler test/runtime/small_file_mgr_test.cpp /^static SmallFileMgrTestHandler s_test_handler = SmallFileMgrTestHandler();$/;" m namespace:doris file: +s_test_handler test/runtime/user_function_cache_test.cpp /^static UserFunctionTestHandler s_test_handler = UserFunctionTestHandler();$/;" m namespace:doris file: +s_text_content_type src/http/http_response.cpp /^static std::string s_text_content_type = "text\/plain; charset=UTF-8";$/;" m namespace:doris file: +sa_flags src/gutil/linux_syscall_support.h /^ unsigned long sa_flags;$/;" m struct:kernel_old_sigaction +sa_flags src/gutil/linux_syscall_support.h /^ unsigned long sa_flags;$/;" m struct:kernel_sigaction +sa_handler_ src/gutil/linux_syscall_support.h /^ void (*sa_handler_)(int);$/;" m union:kernel_old_sigaction::__anon17 +sa_handler_ src/gutil/linux_syscall_support.h /^ void (*sa_handler_)(int);$/;" m union:kernel_sigaction::__anon18 +sa_mask src/gutil/linux_syscall_support.h /^ struct kernel_sigset_t sa_mask;$/;" m struct:kernel_sigaction typeref:struct:kernel_sigaction::kernel_sigset_t +sa_mask src/gutil/linux_syscall_support.h /^ unsigned long sa_mask;$/;" m struct:kernel_old_sigaction +sa_restorer src/gutil/linux_syscall_support.h /^ void (*sa_restorer)(void);$/;" m struct:kernel_old_sigaction +sa_sigaction_ src/gutil/linux_syscall_support.h /^ void (*sa_sigaction_)(int, siginfo_t *, void *);$/;" m union:kernel_old_sigaction::__anon17 +sa_sigaction_ src/gutil/linux_syscall_support.h /^ void (*sa_sigaction_)(int, siginfo_t *, void *);$/;" m union:kernel_sigaction::__anon18 +safe_int_internal src/gutil/strings/numbers.cc /^bool safe_int_internal(const char* start, const char* end, int base,$/;" f namespace:__anon28 +safe_strto32 src/gutil/strings/numbers.cc /^bool safe_strto32(const char* startptr, const int buffer_size, int32* value) {$/;" f +safe_strto32_base src/gutil/strings/numbers.cc /^bool safe_strto32_base(const char* startptr, const int buffer_size,$/;" f +safe_strto32_base src/gutil/strings/numbers.cc /^bool safe_strto32_base(const char* str, int32* value, int base) {$/;" f +safe_strto64 src/gutil/strings/numbers.cc /^bool safe_strto64(const char* startptr, const int buffer_size, int64* value) {$/;" f +safe_strto64_base src/gutil/strings/numbers.cc /^bool safe_strto64_base(const char* startptr, const int buffer_size,$/;" f +safe_strto64_base src/gutil/strings/numbers.cc /^bool safe_strto64_base(const char* str, int64* value, int base) {$/;" f +safe_strtod src/gutil/strings/numbers.cc /^bool safe_strtod(const char* str, double* value) {$/;" f +safe_strtod src/gutil/strings/numbers.cc /^bool safe_strtod(const string& str, double* value) {$/;" f +safe_strtof src/gutil/strings/numbers.cc /^bool safe_strtof(const char* str, float* value) {$/;" f +safe_strtof src/gutil/strings/numbers.cc /^bool safe_strtof(const string& str, float* value) {$/;" f +safe_strtou32_base src/gutil/strings/numbers.cc /^bool safe_strtou32_base(const char* str, uint32* value, int base) {$/;" f +safe_strtou64_base src/gutil/strings/numbers.cc /^bool safe_strtou64_base(const char* str, uint64* value, int base) {$/;" f +safestrncpy src/gutil/strings/util.h /^inline char* safestrncpy(char* dest, const char* src, size_t n) {$/;" f +sample_fn src/util/runtime_profile.h /^ SampleFn sample_fn;$/;" m struct:doris::RuntimeProfile::RateCounterInfo +sample_fn src/util/runtime_profile.h /^ SampleFn sample_fn;$/;" m struct:doris::RuntimeProfile::SamplingCounterInfo +samples_ src/util/streaming_sampler.h /^ T samples_[MAX_SAMPLES];$/;" m class:doris::StreamingSampler +samples_collected_ src/util/streaming_sampler.h /^ int samples_collected_;$/;" m class:doris::StreamingSampler +sampling_counters src/util/runtime_profile.h /^ SamplingCounterMap sampling_counters;$/;" m struct:doris::RuntimeProfile::PeriodicCounterUpdateState +sanity_check src/olap/rowset/segment_v2/column_reader.h /^ void sanity_check() const {$/;" f struct:doris::segment_v2::ColumnIteratorOptions +sanity_check src/olap/rowset/segment_v2/page_io.h /^ void sanity_check() const {$/;" f struct:doris::segment_v2::PageReadOptions +save src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::save(const string& file_path, const OLAPHeaderMessage& olap_header) {$/;" f class:doris::OlapSnapshotConverter +save src/olap/rowset/rowset_meta_manager.cpp /^OLAPStatus RowsetMetaManager::save(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id, const RowsetMetaPB& rowset_meta_pb) {$/;" f class:doris::RowsetMetaManager +save src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::save(const string& file_path) {$/;" f class:doris::TabletMeta +save src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::save(const string& file_path, const TabletMetaPB& tablet_meta_pb) {$/;" f class:doris::TabletMeta +save src/olap/tablet_meta_manager.cpp /^OLAPStatus TabletMetaManager::save(DataDir* store,$/;" f class:doris::TabletMetaManager +save_encoding src/olap/rowset/column_writer.cpp /^void ColumnWriter::save_encoding(ColumnEncodingMessage* encoding) {$/;" f class:doris::ColumnWriter +save_encoding src/olap/rowset/column_writer.cpp /^void VarStringColumnWriter::save_encoding(ColumnEncodingMessage* encoding) {$/;" f class:doris::VarStringColumnWriter +save_meta src/olap/tablet.cpp /^OLAPStatus Tablet::save_meta() {$/;" f class:doris::Tablet +save_meta src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::save_meta(DataDir* data_dir) {$/;" f class:doris::TabletMeta +scale output/udf/include/udf.h /^ int scale;$/;" m struct:doris_udf::FunctionContext::TypeDesc +scale src/runtime/decimal_value.h /^ int32_t scale() const {$/;" f class:doris::DecimalValue +scale src/runtime/decimalv2_value.h /^ int32_t scale() const {$/;" f class:doris::DecimalV2Value +scale src/runtime/types.h /^ int scale;$/;" m struct:doris::TypeDescriptor +scale src/udf/udf.h /^ int scale;$/;" m struct:doris_udf::FunctionContext::TypeDesc +scan_bytes src/runtime/query_statistics.h /^ int64_t scan_bytes;$/;" m class:doris::QueryStatistics +scan_range src/runtime/disk_io_mgr.h /^ ScanRange* scan_range() { return _scan_range; }$/;" f class:doris::DiskIoMgr::BufferDescriptor +scan_range_offset src/runtime/disk_io_mgr.h /^ int64_t scan_range_offset() const { return _scan_range_offset; }$/;" f class:doris::DiskIoMgr::BufferDescriptor +scan_range_thread test/runtime/disk_io_mgr_test.cpp /^ static void scan_range_thread(DiskIoMgr* io_mgr, DiskIoMgr::RequestContext* reader,$/;" f class:doris::DiskIoMgrTest +scan_ranges_complete_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* scan_ranges_complete_counter() const {$/;" f class:doris::ScanNode +scan_rows src/runtime/query_statistics.h /^ int64_t scan_rows;$/;" m class:doris::QueryStatistics +scaninfo src/geo/wkt_parse_ctx.h /^ yyscan_t scaninfo;$/;" m struct:WktParseContext +scanner_id src/runtime/row_batch.h /^ int scanner_id() {$/;" f class:doris::RowBatch +scanner_scan src/exec/broker_scan_node.cpp /^Status BrokerScanNode::scanner_scan($/;" f class:doris::BrokerScanNode +scanner_scan src/exec/es_http_scan_node.cpp /^Status EsHttpScanNode::scanner_scan($/;" f class:doris::EsHttpScanNode +scanner_thread src/exec/olap_scan_node.cpp /^void OlapScanNode::scanner_thread(OlapScanner* scanner) {$/;" f class:doris::OlapScanNode +scanner_thread_counters src/exec/scan_node.h /^ RuntimeProfile::ThreadCounters* scanner_thread_counters() const {$/;" f class:doris::ScanNode +scanner_worker src/exec/broker_scan_node.cpp /^void BrokerScanNode::scanner_worker(int start_idx, int length) {$/;" f class:doris::BrokerScanNode +scanner_worker src/exec/es_http_scan_node.cpp /^void EsHttpScanNode::scanner_worker(int start_idx, int length, std::promise& p_status) {$/;" f class:doris::EsHttpScanNode +schar src/gutil/integral_types.h /^typedef int8_t schar;$/;" t +schedule_context src/runtime/disk_io_mgr_internal.h /^ void schedule_context(RequestContext* context, int disk_id) {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +schedule_scan_range src/runtime/disk_io_mgr_internal.h /^ void schedule_scan_range(DiskIoMgr::ScanRange* range) {$/;" f class:doris::DiskIoMgr::RequestContext +schema src/olap/row.h /^ const Schema* schema() const { return _schema; }$/;" f struct:doris::ContiguousRow +schema src/olap/row_block2.h /^ const Schema* schema() const { return &_schema; }$/;" f class:doris::RowBlockV2 +schema src/olap/row_block2.h /^ const Schema* schema() const { return _block->schema(); }$/;" f class:doris::RowBlockRow +schema src/olap/row_cursor.h /^ const Schema* schema() const { return _schema.get(); }$/;" f class:doris::RowCursor +schema_change_init src/olap/rowset/column_data.cpp /^OLAPStatus ColumnData::schema_change_init() {$/;" f class:doris::ColumnData +schema_change_lock src/olap/tablet_manager.h /^ Mutex schema_change_lock;$/;" m struct:doris::TabletManager::TableInstances +schema_change_requests_failed src/util/doris_metrics.cpp /^IntCounter DorisMetrics::schema_change_requests_failed;$/;" m class:doris::DorisMetrics file: +schema_change_requests_failed src/util/doris_metrics.h /^ static IntCounter schema_change_requests_failed;$/;" m class:doris::DorisMetrics +schema_change_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::schema_change_requests_total;$/;" m class:doris::DorisMetrics file: +schema_change_requests_total src/util/doris_metrics.h /^ static IntCounter schema_change_requests_total;$/;" m class:doris::DorisMetrics +schema_hash src/exec/tablet_info.h /^ int32_t schema_hash;$/;" m struct:doris::OlapTableIndexSchema +schema_hash src/olap/delta_writer.h /^ int32_t schema_hash;$/;" m struct:doris::WriteRequest +schema_hash src/olap/olap_common.h /^ TSchemaHash schema_hash;$/;" m struct:doris::TabletInfo +schema_hash src/olap/push_handler.h /^ SchemaHash schema_hash() const {$/;" f class:doris::BinaryFile +schema_hash src/olap/tablet.h /^inline int32_t Tablet::schema_hash() const {$/;" f class:doris::Tablet +schema_hash src/olap/tablet_meta.h /^inline int32_t TabletMeta::schema_hash() const {$/;" f class:doris::TabletMeta +schema_hash test/olap/txn_manager_test.cpp /^ SchemaHash schema_hash = 333;$/;" m class:doris::TxnManagerTest file: +schema_size src/olap/schema.h /^ size_t schema_size() const {$/;" f class:doris::Schema +schema_table_type src/runtime/descriptors.h /^ TSchemaTableType::type schema_table_type() const {$/;" f class:doris::SchemaTableDescriptor +schema_version_convert src/olap/schema_change.cpp /^OLAPStatus SchemaChangeHandler::schema_version_convert($/;" f class:doris::SchemaChangeHandler +scoped_refptr src/gutil/ref_counted.h /^ scoped_refptr() : ptr_(NULL) {$/;" f class:scoped_refptr +scoped_refptr src/gutil/ref_counted.h /^ scoped_refptr(T* p) : ptr_(p) {$/;" f class:scoped_refptr +scoped_refptr src/gutil/ref_counted.h /^ scoped_refptr(const scoped_refptr& r) : ptr_(r.ptr_) {$/;" f class:scoped_refptr +scoped_refptr src/gutil/ref_counted.h /^ scoped_refptr(const scoped_refptr& r) : ptr_(r.get()) {$/;" f class:scoped_refptr +scoped_refptr src/gutil/ref_counted.h /^class scoped_refptr {$/;" c +scratch_ src/gutil/strings/substitute.h /^ char scratch_[kFastToBufferSize];$/;" m class:strings::internal::SubstituteArg +scratch_row src/exec/new_partitioned_hash_table.h /^ TupleRow* ALWAYS_INLINE scratch_row() const { return scratch_row_; }$/;" f class:doris::NewPartitionedHashTableCtx +scratch_row_ src/exec/new_partitioned_hash_table.h /^ TupleRow* scratch_row_;$/;" m class:doris::NewPartitionedHashTable::Iterator +scratch_row_ src/exec/new_partitioned_hash_table.h /^ TupleRow* scratch_row_;$/;" m class:doris::NewPartitionedHashTableCtx +search src/runtime/string_search.hpp /^ int search(const StringValue* str) const {$/;" f class:doris::StringSearch +search_string src/exprs/like_predicate.h /^ std::string search_string;$/;" m struct:doris::LikePredicate::LikePredicateState +search_string_sv src/exprs/like_predicate.h /^ StringValue search_string_sv;$/;" m struct:doris::LikePredicate::LikePredicateState +second src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::second($/;" f class:doris::TimestampFunctions +second src/olap/olap_common.h /^ int64_t second;$/;" m struct:doris::Version +second src/runtime/datetime_value.h /^ int second() const {$/;" f class:doris::DateTimeValue +second src/runtime/datetime_value.h /^ int32_t second;$/;" m struct:doris::TimeInterval +second_diff src/runtime/datetime_value.h /^ int64_t second_diff(const DateTimeValue& rhs) const {$/;" f class:doris::DateTimeValue +seconds_add src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::seconds_add($/;" f class:doris::TimestampFunctions +seconds_diff src/exprs/timestamp_functions.cpp /^BigIntVal TimestampFunctions::seconds_diff($/;" f class:doris::TimestampFunctions +seconds_sub src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::seconds_sub($/;" f class:doris::TimestampFunctions +section_print src/util/bfd_parser.cpp /^static void section_print(bfd* bfd, asection* sec, void* arg) {$/;" f namespace:doris +seed src/exec/new_partitioned_hash_table.h /^ uint32_t ALWAYS_INLINE seed(int level) { return seeds_.at(level); }$/;" f class:doris::NewPartitionedHashTableCtx +seed src/exec/partitioned_hash_table.h /^ uint32_t seed(int level) { return _seeds.at(level); }$/;" f class:doris::PartitionedHashTableCtx +seed_ src/util/random.h /^ uint32_t seed_;$/;" m class:doris::Random +seeds_ src/exec/new_partitioned_hash_table.h /^ std::vector seeds_;$/;" m class:doris::NewPartitionedHashTableCtx +seek src/exec/broker_reader.cpp /^Status BrokerReader::seek(int64_t position) {$/;" f class:doris::BrokerReader +seek src/exec/local_file_reader.cpp /^Status LocalFileReader::seek(int64_t position) {$/;" f class:doris::LocalFileReader +seek src/olap/file_helper.h /^ off_t seek(off_t offset, int whence) {$/;" f class:doris::FileHandler +seek src/olap/file_helper.h /^ off_t seek(off_t offset, int whence) {$/;" f class:doris::FileHandlerWithBuf +seek src/olap/file_stream.cpp /^OLAPStatus ReadOnlyFileStream::seek(PositionProvider* position) {$/;" f class:doris::ReadOnlyFileStream +seek src/olap/file_stream.h /^ OLAPStatus seek(size_t offset) {$/;" f class:doris::ReadOnlyFileStream::FileCursor +seek src/olap/in_stream.cpp /^OLAPStatus InStream::seek(PositionProvider* position) {$/;" f class:doris::InStream +seek src/olap/rowset/bit_field_reader.cpp /^OLAPStatus BitFieldReader::seek(PositionProvider* position) {$/;" f class:doris::BitFieldReader +seek src/olap/rowset/column_reader.cpp /^OLAPStatus ColumnReader::seek(PositionProvider* position) {$/;" f class:doris::ColumnReader +seek src/olap/rowset/column_reader.cpp /^OLAPStatus DecimalColumnReader::seek(PositionProvider* positions) {$/;" f class:doris::DecimalColumnReader +seek src/olap/rowset/column_reader.cpp /^OLAPStatus IntegerColumnReader::seek(PositionProvider* position) {$/;" f class:doris::IntegerColumnReader +seek src/olap/rowset/column_reader.cpp /^OLAPStatus LargeIntColumnReader::seek(PositionProvider* positions) {$/;" f class:doris::LargeIntColumnReader +seek src/olap/rowset/column_reader.cpp /^OLAPStatus StringColumnDictionaryReader::seek(PositionProvider* position) {$/;" f class:doris::StringColumnDictionaryReader +seek src/olap/rowset/column_reader.cpp /^OLAPStatus StringColumnDirectReader::seek(PositionProvider* position) {$/;" f class:doris::StringColumnDirectReader +seek src/olap/rowset/column_reader.cpp /^OLAPStatus TinyColumnReader::seek(PositionProvider* positions) {$/;" f class:doris::TinyColumnReader +seek src/olap/rowset/column_reader.h /^ virtual OLAPStatus seek(PositionProvider* position) {$/;" f class:doris::FloatintPointColumnReader +seek src/olap/rowset/column_reader.h /^ virtual OLAPStatus seek(PositionProvider* position) {$/;" f class:doris::VarStringColumnReader +seek src/olap/rowset/column_reader.h /^ virtual OLAPStatus seek(PositionProvider* positions) {$/;" f class:doris::DefaultValueReader +seek src/olap/rowset/column_reader.h /^ virtual OLAPStatus seek(PositionProvider* positions) {$/;" f class:doris::FixLengthStringColumnReader +seek src/olap/rowset/column_reader.h /^ virtual OLAPStatus seek(PositionProvider* positions) {$/;" f class:doris::IntegerColumnReaderWrapper +seek src/olap/rowset/column_reader.h /^ virtual OLAPStatus seek(PositionProvider* positions) {$/;" f class:doris::NullValueReader +seek src/olap/rowset/run_length_byte_reader.cpp /^OLAPStatus RunLengthByteReader::seek(PositionProvider* position) {$/;" f class:doris::RunLengthByteReader +seek src/olap/rowset/run_length_integer_reader.cpp /^OLAPStatus RunLengthIntegerReader::seek(PositionProvider* position) {$/;" f class:doris::RunLengthIntegerReader +seek src/olap/short_key_index.h /^ ShortKeyIndexIterator seek(const Slice& key) const {$/;" f class:doris::ShortKeyIndexDecoder +seek64_file_func src/util/minizip/ioapi.h /^typedef long (ZCALLBACK *seek64_file_func) OF((voidpf opaque, voidpf stream, ZPOS64_T offset, int origin));$/;" t +seek_and_get_current_row src/olap/rowset/column_data.cpp /^const RowCursor* ColumnData::seek_and_get_current_row(const RowBlockPosition& position) {$/;" f class:doris::ColumnData +seek_at_or_after src/olap/rowset/segment_v2/indexed_column_reader.cpp /^Status IndexedColumnIterator::seek_at_or_after(const void* key, bool* exact_match) {$/;" f class:doris::segment_v2::IndexedColumnIterator +seek_at_or_after_value src/olap/rowset/segment_v2/binary_prefix_page.cpp /^Status BinaryPrefixPageDecoder::seek_at_or_after_value(const void* value, bool* exact_match) {$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +seek_at_or_after_value src/olap/rowset/segment_v2/page_decoder.h /^ virtual Status seek_at_or_after_value(const void* value, bool* exact_match) {$/;" f class:doris::segment_v2::PageDecoder +seek_at_or_after_value src/util/frame_of_reference_coding.cpp /^bool ForDecoder::seek_at_or_after_value(const void* value, bool* exact_match) {$/;" f class:doris::ForDecoder +seek_at_or_before src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::seek_at_or_before(ordinal_t ordinal, OrdinalPageIndexIterator* iter) {$/;" f class:doris::segment_v2::ColumnReader +seek_at_or_before src/olap/rowset/segment_v2/index_page.cpp /^Status IndexPageIterator::seek_at_or_before(const Slice& search_key) {$/;" f class:doris::segment_v2::IndexPageIterator +seek_at_or_before src/olap/rowset/segment_v2/ordinal_page_index.cpp /^OrdinalPageIndexIterator OrdinalIndexReader::seek_at_or_before(ordinal_t ordinal) {$/;" f class:doris::segment_v2::OrdinalIndexReader +seek_columns src/olap/rowset/rowset_reader_context.h /^ const std::vector* seek_columns = nullptr;$/;" m struct:doris::RowsetReaderContext +seek_dictionary src/olap/rowset/segment_v2/bitmap_index_reader.cpp /^Status BitmapIndexIterator::seek_dictionary(const void* value, bool* exact_match) {$/;" f class:doris::segment_v2::BitmapIndexIterator +seek_file_func src/util/minizip/ioapi.h /^typedef long (ZCALLBACK *seek_file_func) OF((voidpf opaque, voidpf stream, uLong offset, int origin));$/;" t +seek_forward src/olap/rowset/segment_v2/page_decoder.h /^ virtual size_t seek_forward(size_t n) {$/;" f class:doris::segment_v2::PageDecoder +seek_last_frame_before_value src/util/frame_of_reference_coding.cpp /^uint32_t ForDecoder::seek_last_frame_before_value(T target) {$/;" f class:doris::ForDecoder +seek_lower_bound_inside_frame src/util/frame_of_reference_coding.cpp /^bool ForDecoder::seek_lower_bound_inside_frame(uint32_t frame_index, T target, bool* exact_match) {$/;" f class:doris::ForDecoder +seek_to_block src/olap/rowset/segment_reader.cpp /^OLAPStatus SegmentReader::seek_to_block($/;" f class:doris::SegmentReader +seek_to_first src/olap/rowset/segment_v2/column_reader.cpp /^Status ColumnReader::seek_to_first(OrdinalPageIndexIterator* iter) {$/;" f class:doris::segment_v2::ColumnReader +seek_to_first src/olap/rowset/segment_v2/column_reader.cpp /^Status FileColumnIterator::seek_to_first() {$/;" f class:doris::segment_v2::FileColumnIterator +seek_to_ordinal src/olap/rowset/segment_v2/column_reader.cpp /^Status FileColumnIterator::seek_to_ordinal(ordinal_t ord) {$/;" f class:doris::segment_v2::FileColumnIterator +seek_to_ordinal src/olap/rowset/segment_v2/indexed_column_reader.cpp /^Status IndexedColumnIterator::seek_to_ordinal(ordinal_t idx) {$/;" f class:doris::segment_v2::IndexedColumnIterator +seek_to_position_in_page src/olap/rowset/segment_v2/binary_dict_page.cpp /^Status BinaryDictPageDecoder::seek_to_position_in_page(size_t pos) {$/;" f class:doris::segment_v2::BinaryDictPageDecoder +seek_to_position_in_page src/olap/rowset/segment_v2/binary_prefix_page.cpp /^Status BinaryPrefixPageDecoder::seek_to_position_in_page(size_t pos) {$/;" f class:doris::segment_v2::BinaryPrefixPageDecoder +segment src/olap/olap_index.h /^ iterator_offset_t segment;$/;" m struct:doris::OLAPIndexOffset +segment src/olap/olap_index.h /^ uint32_t segment;$/;" m struct:doris::RowBlockPosition +segment_count src/olap/olap_index.h /^ size_t segment_count() const {$/;" f class:doris::MemIndex +segment_file_path src/olap/rowset/beta_rowset.cpp /^std::string BetaRowset::segment_file_path($/;" f class:doris::BetaRowset +segment_group src/olap/rowset/column_data.h /^ SegmentGroup* segment_group() const { return _segment_group; }$/;" f class:doris::ColumnData +segment_group_id src/olap/rowset/segment_group.h /^ inline int32_t segment_group_id() const { return _segment_group_id; }$/;" f class:doris::SegmentGroup +segment_id src/olap/rowset/segment_v2/segment_iterator.h /^ uint32_t segment_id() const { return _segment->id(); }$/;" f class:doris::segment_v2::SegmentIterator +segment_read_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::segment_read_total;$/;" m class:doris::DorisMetrics file: +segment_read_total src/util/doris_metrics.h /^ static IntCounter segment_read_total;$/;" m class:doris::DorisMetrics +segment_row_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::segment_row_total;$/;" m class:doris::DorisMetrics file: +segment_row_total src/util/doris_metrics.h /^ static IntCounter segment_row_total;$/;" m class:doris::DorisMetrics +segment_rows_by_short_key src/util/doris_metrics.cpp /^IntCounter DorisMetrics::segment_rows_by_short_key;$/;" m class:doris::DorisMetrics file: +segment_rows_by_short_key src/util/doris_metrics.h /^ static IntCounter segment_rows_by_short_key;$/;" m class:doris::DorisMetrics +segment_rows_read_by_zone_map src/util/doris_metrics.cpp /^IntCounter DorisMetrics::segment_rows_read_by_zone_map;$/;" m class:doris::DorisMetrics file: +segment_rows_read_by_zone_map src/util/doris_metrics.h /^ static IntCounter segment_rows_read_by_zone_map;$/;" m class:doris::DorisMetrics +segment_statistics src/olap/rowset/column_writer.h /^ ColumnStatistics* segment_statistics() {$/;" f class:doris::ColumnWriter +segment_v2 src/olap/rowset/beta_rowset_writer.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/binary_dict_page.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/binary_dict_page.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/binary_plain_page.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/binary_prefix_page.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/binary_prefix_page.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/bitmap_index_reader.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/bitmap_index_reader.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/bitmap_index_writer.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/bitmap_index_writer.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/bitshuffle_page.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/bitshuffle_page.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/block_split_bloom_filter.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/block_split_bloom_filter.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/bloom_filter.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/bloom_filter.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/bloom_filter_index_writer.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/column_reader.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/column_reader.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/column_writer.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/column_writer.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/common.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/empty_segment_iterator.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/empty_segment_iterator.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/encoding_info.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/encoding_info.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/frame_of_reference_page.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/index_page.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/index_page.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/indexed_column_reader.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/indexed_column_reader.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/indexed_column_writer.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/indexed_column_writer.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/options.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/ordinal_page_index.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/ordinal_page_index.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/page_builder.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/page_decoder.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/page_handle.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/page_io.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/page_io.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/page_pointer.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/parsed_page.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/plain_page.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/rle_page.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/row_ranges.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/segment.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/segment.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/segment_iterator.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/segment_iterator.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/segment_writer.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/segment_writer.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 src/olap/rowset/segment_v2/zone_map_index.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 src/olap/rowset/segment_v2/zone_map_index.h /^namespace segment_v2 {$/;" n namespace:doris +segment_v2 test/olap/rowset/segment_v2/binary_dict_page_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/binary_plain_page_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/binary_prefix_page_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/bitmap_index_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/block_bloom_filter_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/bloom_filter_page_test.cpp /^namespace segment_v2{$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/encoding_info_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/ordinal_page_index_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/plain_page_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/row_ranges_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/segment_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segment_v2 test/olap/rowset/segment_v2/zone_map_index_test.cpp /^namespace segment_v2 {$/;" n namespace:doris file: +segments_overlap src/olap/rowset/rowset_meta.h /^ SegmentsOverlapPB segments_overlap() const {$/;" f class:doris::RowsetMeta +segments_overlap src/olap/rowset/rowset_writer_context.h /^ SegmentsOverlapPB segments_overlap;$/;" m struct:doris::RowsetWriterContext +select src/util/bitmap_value.h /^ bool select(uint64_t rnk, uint64_t* element) const {$/;" f class:doris::detail::Roaring64Map +selected src/runtime/vectorized_row_batch.h /^ uint16_t* selected() const {$/;" f class:doris::VectorizedRowBatch +selected_in_use src/runtime/vectorized_row_batch.h /^ bool selected_in_use() const {$/;" f class:doris::VectorizedRowBatch +selected_size src/olap/row_block2.h /^ uint16_t selected_size() const {$/;" f class:doris::RowBlockV2 +selection_vector src/olap/row_block2.h /^ uint16_t* selection_vector() const {$/;" f class:doris::RowBlockV2 +send src/exec/tablet_sink.cpp /^Status OlapTableSink::send(RuntimeState* state, RowBatch* input_batch) {$/;" f class:doris::stream_load::OlapTableSink +send src/runtime/data_spliter.cpp /^Status DataSpliter::send(RuntimeState* state, RowBatch* batch) {$/;" f class:doris::DataSpliter +send src/runtime/data_stream_sender.cpp /^Status DataStreamSender::send(RuntimeState* state, RowBatch* batch) {$/;" f class:doris::DataStreamSender +send src/runtime/export_sink.cpp /^Status ExportSink::send(RuntimeState* state, RowBatch* batch) {$/;" f class:doris::ExportSink +send src/runtime/memory_scratch_sink.cpp /^Status MemoryScratchSink::send(RuntimeState* state, RowBatch* batch) {$/;" f class:doris::MemoryScratchSink +send src/runtime/mysql_table_sink.cpp /^Status MysqlTableSink::send(RuntimeState* state, RowBatch* batch) {$/;" f class:doris::MysqlTableSink +send src/runtime/result_sink.cpp /^Status ResultSink::send(RuntimeState* state, RowBatch* batch) {$/;" f class:doris::ResultSink +send_basic_challenge src/http/http_channel.cpp /^void HttpChannel::send_basic_challenge(HttpRequest* req, const std::string& realm) {$/;" f class:doris::HttpChannel +send_basic_challenge test/http/stream_load_test.cpp /^void HttpChannel::send_basic_challenge(HttpRequest* req, const std::string& realm) {$/;" f class:doris::HttpChannel +send_batch src/runtime/data_stream_sender.cpp /^Status DataStreamSender::Channel::send_batch(PRowBatch* batch, bool eos) {$/;" f class:doris::DataStreamSender::Channel +send_bytes src/util/system_metrics.cpp /^ IntLockCounter send_bytes;$/;" m struct:doris::NetMetrics file: +send_current_batch src/runtime/data_stream_sender.cpp /^Status DataStreamSender::Channel::send_current_batch(bool eos) {$/;" f class:doris::DataStreamSender::Channel +send_error src/http/http_channel.cpp /^void HttpChannel::send_error(HttpRequest* request, HttpStatus status) {$/;" f class:doris::HttpChannel +send_error test/http/stream_load_test.cpp /^void HttpChannel::send_error(HttpRequest* request, HttpStatus status) {$/;" f class:doris::HttpChannel +send_file src/http/http_channel.cpp /^void HttpChannel::send_file(HttpRequest* request, int fd, size_t off, size_t size) {$/;" f class:doris::HttpChannel +send_file test/http/stream_load_test.cpp /^void HttpChannel::send_file(HttpRequest* request, int fd, size_t off, size_t size) {$/;" f class:doris::HttpChannel +send_packets src/util/system_metrics.cpp /^ IntLockCounter send_packets;$/;" m struct:doris::NetMetrics file: +send_reply src/http/http_channel.cpp /^void HttpChannel::send_reply($/;" f class:doris::HttpChannel +send_reply src/http/http_channel.cpp /^void HttpChannel::send_reply(HttpRequest* request, HttpStatus status) {$/;" f class:doris::HttpChannel +send_reply src/http/http_channel.h /^ static inline void send_reply(HttpRequest* request, const std::string& content) {$/;" f class:doris::HttpChannel +send_reply test/http/metrics_action_test.cpp /^void HttpChannel::send_reply($/;" f class:doris::HttpChannel +send_reply test/http/stream_load_test.cpp /^void HttpChannel::send_reply($/;" f class:doris::HttpChannel +send_reply test/http/stream_load_test.cpp /^void HttpChannel::send_reply(HttpRequest* request, HttpStatus status) {$/;" f class:doris::HttpChannel +send_report src/runtime/plan_fragment_executor.cpp /^void PlanFragmentExecutor::send_report(bool done) {$/;" f class:doris::PlanFragmentExecutor +send_row src/runtime/data_spliter.cpp /^Status DataSpliter::send_row($/;" f class:doris::DataSpliter +sender test/runtime/data_stream_test.cpp /^ void sender(int sender_num, int channel_buffer_size,$/;" f class:doris::DataStreamTest +separators src/gutil/strings/numbers.h /^ const char* separators;$/;" m struct:DoubleRangeOptions +ser src/olap/serialize.cpp /^namespace ser {$/;" n namespace:doris file: +ser src/olap/serialize.h /^namespace ser {$/;" n namespace:doris +ser test/olap/serialize_test.cpp /^namespace ser {$/;" n namespace:doris file: +serialize src/exprs/agg_fn_evaluator.cpp /^void AggFnEvaluator::serialize(FunctionContext* agg_fn_ctx, Tuple* tuple) {$/;" f class:doris::AggFnEvaluator +serialize src/exprs/agg_fn_evaluator.h /^inline void AggFnEvaluator::serialize(const std::vector& evaluators,$/;" f class:doris::AggFnEvaluator +serialize src/exprs/aggregate_functions.cpp /^ StringVal serialize(FunctionContext* ctx) {$/;" f class:doris::MultiDistinctCountDateState +serialize src/exprs/aggregate_functions.cpp /^ StringVal serialize(FunctionContext* ctx) {$/;" f class:doris::MultiDistinctDecimalState +serialize src/exprs/aggregate_functions.cpp /^ StringVal serialize(FunctionContext* ctx) {$/;" f class:doris::MultiDistinctDecimalV2State +serialize src/exprs/aggregate_functions.cpp /^ StringVal serialize(FunctionContext* ctx) {$/;" f class:doris::MultiDistinctNumericState +serialize src/exprs/aggregate_functions.cpp /^ StringVal serialize(FunctionContext* ctx) {$/;" f class:doris::MultiDistinctStringCountState +serialize src/exprs/bitmap_function.cpp /^ void serialize(char* dest) {$/;" f struct:doris::BitmapIntersect +serialize src/exprs/bitmap_function.cpp /^static StringVal serialize(FunctionContext* ctx, BitmapValue* value) {$/;" f namespace:doris +serialize src/olap/file_helper.h /^OLAPStatus FileHeader::serialize($/;" f class:doris::FileHeader +serialize src/olap/hll.cpp /^size_t HyperLogLog::serialize(uint8_t* dst) const {$/;" f class:doris::HyperLogLog +serialize src/olap/rowset/rowset_meta.h /^ virtual bool serialize(std::string* value) {$/;" f class:doris::RowsetMeta +serialize src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::serialize(string* meta_binary) {$/;" f class:doris::TabletMeta +serialize src/runtime/row_batch.cpp /^int RowBatch::serialize(PRowBatch* output_batch) {$/;" f class:doris::RowBatch +serialize src/runtime/row_batch.cpp /^int RowBatch::serialize(TRowBatch* output_batch) {$/;" f class:doris::RowBatch +serialize src/util/tdigest.h /^ void serialize(uint8_t* writer) {$/;" f class:doris::TDigest +serialize src/util/thrift_util.h /^ Status serialize(T* obj) {$/;" f class:doris::ThriftSerializer +serialize src/util/thrift_util.h /^ Status serialize(T* obj, std::string* result) {$/;" f class:doris::ThriftSerializer +serialize src/util/thrift_util.h /^ Status serialize(T* obj, std::vector* result) {$/;" f class:doris::ThriftSerializer +serialize src/util/thrift_util.h /^ Status serialize(T* obj, uint32_t* len, uint8_t** buffer) {$/;" f class:doris::ThriftSerializer +serialize_batch src/runtime/data_stream_sender.cpp /^Status DataStreamSender::serialize_batch(RowBatch* src, T* dest, int num_receivers) {$/;" f class:doris::DataStreamSender +serialize_fn src/exprs/agg_fn.h /^ void* serialize_fn() const { return serialize_fn_; }$/;" f class:doris::AggFn +serialize_fn_ src/exprs/agg_fn.h /^ void* serialize_fn_ = nullptr;$/;" m class:doris::AggFn +serialize_or_finalize src/exprs/agg_fn_evaluator.cpp /^void AggFnEvaluator::serialize_or_finalize(FunctionContext* agg_fn_ctx, Tuple* src,$/;" f class:doris::AggFnEvaluator +serialize_record_batch src/util/arrow/row_batch.cpp /^Status serialize_record_batch(const arrow::RecordBatch& record_batch, std::string* result) {$/;" f namespace:doris +serialize_size src/exprs/bitmap_function.cpp /^int32_t serialize_size(const DateTimeValue& v) {$/;" f namespace:doris::detail +serialize_size src/exprs/bitmap_function.cpp /^int32_t serialize_size(const DecimalV2Value& v) {$/;" f namespace:doris::detail +serialize_size src/exprs/bitmap_function.cpp /^int32_t serialize_size(const StringValue& v) {$/;" f namespace:doris::detail +serialize_size src/exprs/bitmap_function.cpp /^int32_t serialize_size(const T& v) {$/;" f namespace:doris::detail +serialize_stream_ src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr serialize_stream_;$/;" m class:doris::NewPartitionedAggregationNode +serialized_size src/util/tdigest.h /^ uint32_t serialized_size() {$/;" f class:doris::TDigest +set src/gutil/strings/stringpiece.h /^ void set(const char* data, int len) {$/;" f class:StringPiece +set src/gutil/strings/stringpiece.h /^ void set(const char* str) {$/;" f class:StringPiece +set src/gutil/strings/stringpiece.h /^ void set(const void* data, int len) {$/;" f class:StringPiece +set src/olap/bloom_filter.hpp /^ void set(uint32_t index) {$/;" f class:doris::BitSet +set src/runtime/buffered_tuple_stream.h /^ uint64_t set(uint64_t block, uint64_t offset, uint64_t idx) {$/;" f struct:doris::BufferedTupleStream::RowIdx +set src/runtime/buffered_tuple_stream2.h /^ uint64_t set(uint64_t block, uint64_t offset, uint64_t idx) {$/;" f struct:doris::BufferedTupleStream2::RowIdx +set src/util/runtime_profile.h /^ virtual void set(double value) {$/;" f class:doris::RuntimeProfile::Counter +set src/util/runtime_profile.h /^ virtual void set(int value) { _value.store(value); }$/;" f class:doris::RuntimeProfile::Counter +set src/util/runtime_profile.h /^ virtual void set(int64_t v) {$/;" f class:doris::RuntimeProfile::HighWaterMarkCounter +set src/util/runtime_profile.h /^ virtual void set(int64_t value) {$/;" f class:doris::RuntimeProfile::Counter +set test/olap/skiplist_test.cpp /^ void set(int k, int v) {$/;" f struct:doris::ConcurrentTest::State +set test/udf/uda_test.cpp /^ void set(FunctionContext* context, const StringVal& val) {$/;" f struct:doris_udf::MinState +setCopyOnWrite src/util/bitmap_value.h /^ void setCopyOnWrite(bool val) {$/;" f class:doris::detail::Roaring64Map +set_active_read_thread_counter src/runtime/disk_io_mgr.cc /^void DiskIoMgr::set_active_read_thread_counter(RequestContext* r,$/;" f class:doris::DiskIoMgr +set_all_false src/olap/selection_vector.h /^ void set_all_false() {$/;" f class:doris::SelectionVector +set_all_true src/olap/selection_vector.h /^ void set_all_true() {$/;" f class:doris::SelectionVector +set_alter_state src/olap/tablet.cpp /^OLAPStatus Tablet::set_alter_state(AlterTabletState state) {$/;" f class:doris::Tablet +set_alter_state src/olap/tablet_meta.cpp /^OLAPStatus AlterTabletTask::set_alter_state(AlterTabletState alter_state) {$/;" f class:doris::AlterTabletTask +set_alter_state src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::set_alter_state(AlterTabletState alter_state) {$/;" f class:doris::TabletMeta +set_alter_type src/olap/tablet_meta.h /^ inline void set_alter_type(AlterTabletType alter_type) { _alter_type = alter_type; }$/;" f class:doris::AlterTabletTask +set_any_val src/exprs/agg_fn_evaluator.cpp /^inline void AggFnEvaluator::set_any_val($/;" f class:doris::AggFnEvaluator +set_any_val src/exprs/anyval_util.h /^ static void set_any_val(const void* slot, const TypeDescriptor& type, doris_udf::AnyVal* dst) {$/;" f class:doris::AnyValUtil +set_any_val src/exprs/new_agg_fn_evaluator.cc /^inline void NewAggFnEvaluator::set_any_val($/;" f class:NewAggFnEvaluator +set_any_val2 src/exprs/agg_fn_evaluator.cpp /^static void set_any_val2(const SlotDescriptor* desc, Tuple* tuple, AnyVal* dst) {$/;" f namespace:doris +set_at_end src/exec/partitioned_hash_table.inline.h /^inline void PartitionedHashTable::Iterator::set_at_end() {$/;" f class:doris::PartitionedHashTable::Iterator +set_bad src/olap/tablet.h /^ void set_bad(bool is_bad) { _is_bad = is_bad; }$/;" f class:doris::Tablet +set_basic_auth src/http/http_client.h /^ void set_basic_auth(const std::string& user, const std::string& passwd) {$/;" f class:doris::HttpClient +set_be_number src/runtime/runtime_state.h /^ void set_be_number(int be_number) {$/;" f class:doris::RuntimeState +set_begin_include src/exec/olap_common.h /^ void set_begin_include(bool begin_include) {$/;" f class:doris::OlapScanKeys +set_bit src/olap/selection_vector.h /^ void set_bit(size_t row_idx) {$/;" f class:doris::SelectionVectorView +set_block_mgr src/runtime/runtime_state.h /^ void set_block_mgr(const boost::shared_ptr& block_mgr) {$/;" f class:doris::RuntimeState +set_block_mgr2 src/runtime/runtime_state.h /^ void set_block_mgr2(const boost::shared_ptr& block_mgr) {$/;" f class:doris::RuntimeState +set_block_status src/olap/row_block.h /^ void set_block_status(uint8_t status) { _block_status = status; }$/;" f class:doris::RowBlock +set_block_status src/runtime/vectorized_row_batch.h /^ void set_block_status(uint8_t status) { _block_status = status; }$/;" f class:doris::VectorizedRowBatch +set_bytes_read_counter src/runtime/disk_io_mgr.cc /^void DiskIoMgr::set_bytes_read_counter(RequestContext* r, RuntimeProfile::Counter* c) {$/;" f class:doris::DiskIoMgr +set_capacity src/olap/lru_cache.h /^ void set_capacity(size_t capacity) {$/;" f class:doris::CachePriority::LRUCache +set_checksum src/olap/file_helper.h /^ void set_checksum(uint32_t checksum) {$/;" f class:doris::FileHeader +set_cluster_id src/olap/data_dir.cpp /^Status DataDir::set_cluster_id(int32_t cluster_id) {$/;" f class:doris::DataDir +set_cluster_id src/olap/storage_engine.cpp /^Status StorageEngine::set_cluster_id(int32_t cluster_id) {$/;" f class:doris::StorageEngine +set_col_data src/runtime/vectorized_row_batch.h /^ void set_col_data(void* data) {$/;" f class:doris::ColumnVector +set_column_value_by_type test/olap/tablet_schema_helper.h /^void set_column_value_by_type(FieldType fieldType, int src, char* target, MemPool* pool, size_t _length = 0) {$/;" f namespace:doris +set_conn_timeout src/util/thrift_client.h /^ void set_conn_timeout(int ms) {$/;" f class:doris::ThriftClientImpl +set_constant_args src/udf/udf.cpp /^void FunctionContextImpl::set_constant_args(const std::vector& constant_args) {$/;" f class:doris::FunctionContextImpl +set_content_type src/http/http_client.h /^ void set_content_type(const std::string content_type) {$/;" f class:doris::HttpClient +set_convert_finished src/olap/data_dir.cpp /^OLAPStatus DataDir::set_convert_finished() {$/;" f class:doris::DataDir +set_count src/util/semaphore.hpp /^ void set_count(int count) { _count = count; }$/;" f class:__anon35::Semaphore +set_creation_time src/olap/rowset/rowset_meta.h /^ void set_creation_time(int64_t creation_time) {$/;" f class:doris::RowsetMeta +set_creation_time src/olap/tablet.h /^inline void Tablet::set_creation_time(int64_t creation_time) {$/;" f class:doris::Tablet +set_creation_time src/olap/tablet_meta.h /^inline void TabletMeta::set_creation_time(int64_t creation_time) {$/;" f class:doris::TabletMeta +set_cumulative_layer_point src/olap/tablet.h /^inline void Tablet::set_cumulative_layer_point(int64_t new_point) {$/;" f class:doris::Tablet +set_cumulative_layer_point src/olap/tablet_meta.h /^inline void TabletMeta::set_cumulative_layer_point(int64_t new_point) {$/;" f class:doris::TabletMeta +set_data src/runtime/disk_io_mgr.cc /^void DiskIoMgr::WriteRange::set_data(const uint8_t* buffer, int64_t len) {$/;" f class:doris::DiskIoMgr::WriteRange +set_data_disk_size src/olap/rowset/rowset_meta.h /^ void set_data_disk_size(size_t data_disk_size) {$/;" f class:doris::RowsetMeta +set_db_name src/runtime/runtime_state.h /^ void set_db_name(const std::string& db_name) {$/;" f class:doris::RuntimeState +set_debug_options src/exec/exec_node.cpp /^void ExecNode::set_debug_options($/;" f class:doris::ExecNode +set_debug_write_delay_ms src/runtime/bufferpool/buffer_pool_internal.h /^ void set_debug_write_delay_ms(int val) { debug_write_delay_ms_ = val; }$/;" f class:doris::BufferPool::Client +set_default_create_tablet_request test/olap/delete_handler_test.cpp /^void set_default_create_tablet_request(TCreateTabletReq* request) {$/;" f namespace:doris +set_default_create_tablet_request test/olap/olap_reader_test.cpp /^void set_default_create_tablet_request(TCreateTabletReq* request) {$/;" f namespace:doris +set_default_create_tablet_request test/olap/vectorized_olap_reader_test.cpp /^void set_default_create_tablet_request(TCreateTabletReq* request) {$/;" f namespace:doris +set_default_push_request test/olap/delete_handler_test.cpp /^void set_default_push_request(TPushReq* request) {$/;" f namespace:doris +set_default_push_request test/olap/olap_reader_test.cpp /^void set_default_push_request(TPushReq* request) {$/;" f namespace:doris +set_default_push_request test/olap/vectorized_olap_reader_test.cpp /^void set_default_push_request(TPushReq* request) {$/;" f namespace:doris +set_delete_flag src/olap/rowset/rowset_meta.h /^ void set_delete_flag(bool delete_flag) {$/;" f class:doris::RowsetMeta +set_delete_handler src/olap/rowset/column_data.h /^ void set_delete_handler(const DeleteHandler* delete_handler) {$/;" f class:doris::ColumnData +set_delete_predicate src/olap/rowset/rowset_meta.h /^ void set_delete_predicate(const DeletePredicatePB& delete_predicate) {$/;" f class:doris::RowsetMeta +set_delete_state src/olap/column_block.h /^ void set_delete_state(DelCondSatisfied delete_state) {$/;" f class:doris::ColumnBlock +set_delete_state src/olap/row_block2.h /^ void set_delete_state(DelCondSatisfied delete_state) {$/;" f class:doris::RowBlockV2 +set_delete_status src/olap/rowset/column_data.h /^ void set_delete_status(const DelCondSatisfied delete_status) {$/;" f class:doris::ColumnData +set_desc_tbl src/runtime/runtime_state.h /^ void set_desc_tbl(DescriptorTbl* desc_tbl) {$/;" f class:doris::RuntimeState +set_dict_decoder src/olap/rowset/segment_v2/binary_dict_page.cpp /^void BinaryDictPageDecoder::set_dict_decoder(PageDecoder* dict_decoder){$/;" f class:doris::segment_v2::BinaryDictPageDecoder +set_disks_access_bitmap src/runtime/disk_io_mgr.cc /^void DiskIoMgr::set_disks_access_bitmap(RequestContext* r,$/;" f class:doris::DiskIoMgr +set_distinct_estimate_bit src/exprs/aggregate_functions.cpp /^static inline void set_distinct_estimate_bit(uint8_t* bitmap,$/;" f namespace:doris +set_done src/runtime/disk_io_mgr_internal.h /^ void set_done(bool b) { _done = b; }$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +set_doris_server src/exec/schema_scanner.h /^ static void set_doris_server(DorisServer* doris_server) {$/;" f class:doris::SchemaScanner +set_empty src/olap/rowset/rowset_meta.h /^ void set_empty(bool empty) {$/;" f class:doris::RowsetMeta +set_empty src/olap/rowset/segment_group.h /^ void set_empty(bool empty) {$/;" f class:doris::SegmentGroup +set_empty_value_range src/exec/olap_common.h /^ void set_empty_value_range() {$/;" f class:doris::ColumnValueRange +set_end_include src/exec/olap_common.h /^ void set_end_include(bool end_include) {$/;" f class:doris::OlapScanKeys +set_end_version src/olap/rowset/rowset_meta.h /^ void set_end_version(int64_t end_version) {$/;" f class:doris::RowsetMeta +set_eof src/olap/rowset/column_data.h /^ void set_eof(bool eof) { _eof = eof; }$/;" f class:doris::ColumnData +set_error src/udf/udf.cpp /^void FunctionContext::set_error(const char* error_msg) {$/;" f class:doris_udf::FunctionContext +set_error_row_number src/runtime/runtime_state.h /^ const void set_error_row_number(int64_t number) {$/;" f class:doris::RuntimeState +set_explicit src/olap/hll.cpp /^void HllSetHelper::set_explicit(char* result, const std::set& hash_value_set, int& len) {$/;" f class:doris::HllSetHelper +set_failed src/exec/tablet_sink.h /^ void set_failed() { _already_failed = true; }$/;" f class:doris::stream_load::NodeChannel +set_fd_cache src/olap/file_helper.h /^ static void set_fd_cache(Cache* cache) {$/;" f class:doris::FileHandler +set_field_content src/olap/row_cursor.h /^ void set_field_content(size_t index, const char* buf, MemPool* mem_pool) {$/;" f class:doris::RowCursor +set_field_content_shallow src/olap/row_cursor.h /^ void set_field_content_shallow(size_t index, const char* buf) {$/;" f class:doris::RowCursor +set_field_null src/exec/parquet_reader.cpp /^inline Status ParquetReaderWrap::set_field_null(Tuple* tuple, const SlotDescriptor* slot_desc) {$/;" f class:doris::ParquetReaderWrap +set_file_length src/olap/file_helper.h /^ void set_file_length(uint64_t file_length) {$/;" f class:doris::FileHeader +set_fragment_mem_tracker src/runtime/runtime_state.h /^ void set_fragment_mem_tracker(MemTracker* limit) {$/;" f class:doris::RuntimeState +set_fragment_root_id src/runtime/runtime_state.h /^ void set_fragment_root_id(PlanNodeId id) {$/;" f class:doris::RuntimeState +set_full src/olap/hll.cpp /^void HllSetHelper::set_full(char* result,$/;" f class:doris::HllSetHelper +set_function_state src/udf/udf.cpp /^void FunctionContext::set_function_state(FunctionStateScope scope, void* ptr) {$/;" f class:doris_udf::FunctionContext +set_group src/runtime/fragment_mgr.cpp /^ void set_group(const TResourceInfo& info) {$/;" f class:doris::FragmentExecState +set_grp src/runtime/routine_load/data_consumer.h /^ void set_grp(const UniqueId& grp_id) {$/;" f class:doris::DataConsumer +set_handler src/http/http_request.h /^ void set_handler(HttpHandler* handler) { _handler = handler; }$/;" f class:doris::HttpRequest +set_handler_ctx src/http/http_request.h /^ void set_handler_ctx(void* ctx) { _handler_ctx = ctx; }$/;" f class:doris::HttpRequest +set_has_null src/olap/rowset/segment_v2/bloom_filter.h /^ void set_has_null(bool has_null) {$/;" f class:doris::segment_v2::BloomFilter +set_heartbeat_flags src/olap/storage_engine.h /^ void set_heartbeat_flags(HeartbeatFlags* heartbeat_flags) {$/;" f class:doris::StorageEngine +set_id src/exec/olap_scanner.h /^ void set_id(int id) { _id = id; }$/;" f class:doris::OlapScanner +set_id src/olap/fs/block_id.h /^ void set_id(uint64_t id) {$/;" f class:doris::BlockId +set_idle_timeout src/util/threadpool.cpp /^ThreadPoolBuilder& ThreadPoolBuilder::set_idle_timeout(const MonoDelta& idle_timeout) {$/;" f class:doris::ThreadPoolBuilder +set_if_bigger src/runtime/decimal_value.cpp /^template inline void set_if_bigger(T* num1_ptr, const T num2) {$/;" f namespace:doris +set_if_smaller src/runtime/decimal_value.cpp /^template inline void set_if_smaller(T* num1_ptr, const T num2) {$/;" f namespace:doris +set_import_label src/runtime/runtime_state.h /^ void set_import_label(const std::string& import_label) {$/;" f class:doris::RuntimeState +set_in_restore_mode src/olap/tablet_meta.h /^inline void TabletMeta::set_in_restore_mode(bool in_restore_mode) {$/;" f class:doris::TabletMeta +set_index_disk_size src/olap/rowset/rowset_meta.h /^ void set_index_disk_size(size_t index_disk_size) {$/;" f class:doris::RowsetMeta +set_intermediate_size output/udf/include/uda_test_harness.h /^ void set_intermediate_size(int byte_size) {$/;" f class:doris_udf::UdaTestHarnessBase +set_intermediate_size src/udf/uda_test_harness.h /^ void set_intermediate_size(int byte_size) {$/;" f class:doris_udf::UdaTestHarnessBase +set_is_cancelled src/runtime/runtime_state.h /^ void set_is_cancelled(bool v) {$/;" f class:doris::RuntimeState +set_is_convertible src/exec/olap_common.h /^ void set_is_convertible(bool is_convertible) {$/;" f class:doris::OlapScanKeys +set_is_in_memory src/olap/tablet_schema.h /^ inline void set_is_in_memory (bool is_in_memory) {$/;" f class:doris::TabletSchema +set_is_null src/olap/column_block.h /^ void set_is_null(bool is_null) const { return _block.set_is_null(_idx, is_null); }$/;" f struct:doris::ColumnBlockCell +set_is_null src/olap/column_block.h /^ void set_is_null(size_t idx, bool is_null) const {$/;" f class:doris::ColumnBlock +set_is_null src/olap/row.h /^ void set_is_null(uint32_t cid, bool is_null) const {$/;" f struct:doris::ContiguousRow +set_is_null src/olap/row_block2.h /^ void set_is_null(size_t col_idx, bool is_null) {$/;" f class:doris::RowBlockRow +set_is_null src/olap/row_cursor_cell.h /^ void set_is_null(bool is_null) const { *reinterpret_cast(_ptr) = is_null; }$/;" f struct:doris::RowCursorCell +set_is_null src/olap/schema.h /^ void set_is_null(void* row, uint32_t cid, bool is_null) const {$/;" f class:doris::Schema +set_is_null src/olap/wrapper_field.h /^ void set_is_null(bool is_null) { *reinterpret_cast(_field_buf) = is_null; }$/;" f class:doris::WrapperField +set_is_null src/runtime/vectorized_row_batch.h /^ void set_is_null(bool* is_null) {$/;" f class:doris::ColumnVector +set_is_report_on_cancel src/runtime/plan_fragment_executor.h /^ void set_is_report_on_cancel(bool val) {$/;" f class:doris::PlanFragmentExecutor +set_is_used src/olap/data_dir.h /^ void set_is_used(bool is_used) { _is_used = is_used; }$/;" f class:doris::DataDir +set_is_using_mmap src/olap/rowset/segment_reader.h /^ void set_is_using_mmap(bool is_using_mmap) {$/;" f class:doris::SegmentReader +set_key src/util/doris_metrics.h /^ IntGauge* set_key(const std::string& key) {$/;" f class:doris::IntGaugeMetricsMap +set_last_base_compaction_failure_time src/olap/tablet.h /^ void set_last_base_compaction_failure_time(int64_t millis) { _last_base_compaction_failure_millis = millis; }$/;" f class:doris::Tablet +set_last_base_compaction_success_time src/olap/tablet.h /^ void set_last_base_compaction_success_time(int64_t millis) { _last_base_compaction_success_millis = millis; }$/;" f class:doris::Tablet +set_last_cumu_compaction_failure_time src/olap/tablet.h /^ void set_last_cumu_compaction_failure_time(int64_t millis) { _last_cumu_compaction_failure_millis = millis; }$/;" f class:doris::Tablet +set_last_cumu_compaction_success_time src/olap/tablet.h /^ void set_last_cumu_compaction_success_time(int64_t millis) { _last_cumu_compaction_success_millis = millis; }$/;" f class:doris::Tablet +set_length src/olap/rowset/run_length_integer_writer.h /^ inline void set_length(uint16_t length) {$/;" f struct:doris::RunLengthIntegerWriter::DeltaHead +set_length src/olap/rowset/run_length_integer_writer.h /^ inline void set_length(uint16_t length) {$/;" f struct:doris::RunLengthIntegerWriter::DirectHead +set_length src/olap/rowset/run_length_integer_writer.h /^ inline void set_length(uint16_t length) {$/;" f struct:doris::RunLengthIntegerWriter::PatchedBaseHead +set_level src/exec/new_partitioned_hash_table.inline.h /^inline void NewPartitionedHashTableCtx::set_level(int level) {$/;" f class:doris::NewPartitionedHashTableCtx +set_level src/exec/partitioned_hash_table.inline.h /^inline void PartitionedHashTableCtx::set_level(int level) {$/;" f class:doris::PartitionedHashTableCtx +set_limit src/olap/byte_buffer.h /^ OLAPStatus set_limit(uint64_t new_limit) {$/;" f class:doris::StorageByteBuffer +set_limit src/olap/row_block.h /^ void set_limit(size_t limit) { _limit = limit; }$/;" f class:doris::RowBlock +set_limit src/runtime/vectorized_row_batch.h /^ void set_limit(uint16_t limit) { _limit = limit; }$/;" f class:doris::VectorizedRowBatch +set_load_error_hub_info src/runtime/runtime_state.h /^ void set_load_error_hub_info(const TLoadErrorHubInfo& hub_info) {$/;" f class:doris::RuntimeState +set_load_id src/olap/rowset/rowset_meta.h /^ void set_load_id(PUniqueId load_id) {$/;" f class:doris::RowsetMeta +set_load_id src/olap/rowset/segment_group.h /^ inline void set_load_id(const PUniqueId& load_id) { _load_id = load_id; }$/;" f class:doris::SegmentGroup +set_load_job_id src/runtime/runtime_state.h /^ void set_load_job_id(int64_t job_id) {$/;" f class:doris::RuntimeState +set_logging_level src/util/progress_updater.h /^ void set_logging_level(int level) {$/;" f class:doris::ProgressUpdater +set_lookup src/exprs/new_in_predicate.cpp /^BooleanVal InPredicate::set_lookup($/;" f class:doris::InPredicate +set_lookup_close src/exprs/new_in_predicate.cpp /^void InPredicate::set_lookup_close($/;" f class:doris::InPredicate +set_lookup_prepare src/exprs/new_in_predicate.cpp /^void InPredicate::set_lookup_prepare($/;" f class:doris::InPredicate +set_matched src/exec/hash_table.h /^ void set_matched() {$/;" f class:doris::HashTable::Iterator +set_matched src/exec/partitioned_hash_table.inline.h /^inline void PartitionedHashTable::Iterator::set_matched() {$/;" f class:doris::PartitionedHashTable::Iterator +set_max_queue_size src/util/threadpool.cpp /^ThreadPoolBuilder& ThreadPoolBuilder::set_max_queue_size(int max_queue_size) {$/;" f class:doris::ThreadPoolBuilder +set_max_quota src/runtime/thread_resource_mgr.h /^ void set_max_quota(int quota) {$/;" f class:doris::ThreadResourceMgr::ResourcePool +set_max_scavenge_attempts src/runtime/bufferpool/buffer_allocator.h /^ void set_max_scavenge_attempts(int val) {$/;" f struct:doris::BufferPool::BufferAllocator +set_max_threads src/util/threadpool.cpp /^ThreadPoolBuilder& ThreadPoolBuilder::set_max_threads(int max_threads) {$/;" f class:doris::ThreadPoolBuilder +set_max_time src/runtime/datetime_value.cpp /^void DateTimeValue::set_max_time(bool neg) {$/;" f class:doris::DateTimeValue +set_mem_limit_exceeded src/runtime/runtime_state.cpp /^Status RuntimeState::set_mem_limit_exceeded($/;" f class:doris::RuntimeState +set_mem_limit_exceeded src/runtime/runtime_state.h /^ Status set_mem_limit_exceeded(const std::string& msg) {$/;" f class:doris::RuntimeState +set_mem_tracker src/runtime/disk_io_mgr.cc /^void DiskIoMgr::BufferDescriptor::set_mem_tracker(MemTracker* tracker) {$/;" f class:doris::DiskIoMgr::BufferDescriptor +set_metadata src/util/runtime_profile.h /^ void set_metadata(int64_t md) {$/;" f class:doris::RuntimeProfile +set_method src/http/http_client.cpp /^void HttpClient::set_method(HttpMethod method) {$/;" f class:doris::HttpClient +set_metric src/util/doris_metrics.h /^ void set_metric(const std::string& key, int64_t val) {$/;" f class:doris::IntGaugeMetricsMap +set_min_threads src/util/threadpool.cpp /^ThreadPoolBuilder& ThreadPoolBuilder::set_min_threads(int min_threads) {$/;" f class:doris::ThreadPoolBuilder +set_mmap src/olap/byte_buffer.cpp /^void StorageByteBuffer::BufDeleter::set_mmap(size_t mmap_length) {$/;" f class:doris::StorageByteBuffer::BufDeleter +set_name src/util/runtime_profile.h /^ void set_name(const std::string& name) {$/;" f class:doris::RuntimeProfile +set_need_delete_file src/olap/rowset/rowset.h /^ void set_need_delete_file() {$/;" f class:doris::Rowset +set_next_scan_range_to_start src/runtime/disk_io_mgr_internal.h /^ void set_next_scan_range_to_start(ScanRange* range) {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +set_no_agg_finalize src/exec/olap_scan_node.h /^ inline void set_no_agg_finalize() {$/;" f class:doris::OlapScanNode +set_no_nulls src/runtime/vectorized_row_batch.h /^ void set_no_nulls(bool no_nulls) {$/;" f class:doris::ColumnVector +set_normal_row_number src/runtime/runtime_state.h /^ const void set_normal_row_number(int64_t number) {$/;" f class:doris::RuntimeState +set_not_null src/olap/row_cursor.h /^ inline void set_not_null(size_t index) const {$/;" f class:doris::RowCursor +set_not_null src/olap/row_cursor_cell.h /^ void set_not_null() const { *reinterpret_cast(_ptr) = false; }$/;" f struct:doris::RowCursorCell +set_not_null src/olap/wrapper_field.h /^ void set_not_null() { *reinterpret_cast(_field_buf) = false; }$/;" f class:doris::WrapperField +set_not_null src/runtime/tuple.h /^ void set_not_null(const NullIndicatorOffset& offset) {$/;" f class:doris::Tuple +set_null src/olap/row_cursor.h /^ inline void set_null(size_t index) const {$/;" f class:doris::RowCursor +set_null src/olap/row_cursor_cell.h /^ void set_null() const { *reinterpret_cast(_ptr) = true; }$/;" f struct:doris::RowCursorCell +set_null src/olap/wrapper_field.h /^ void set_null() { *reinterpret_cast(_field_buf) = true; }$/;" f class:doris::WrapperField +set_null src/runtime/tuple.h /^ void set_null(const NullIndicatorOffset& offset) {$/;" f class:doris::Tuple +set_null_bits src/olap/column_block.h /^ void set_null_bits(size_t num_rows, bool val) {$/;" f class:doris::ColumnBlockView +set_num_per_fragment_instances src/runtime/runtime_state.h /^ void set_num_per_fragment_instances(int num_instances) {$/;" f class:doris::RuntimeState +set_num_removes src/udf/udf_internal.h /^ void set_num_removes(int64_t n) { $/;" f class:doris::FunctionContextImpl +set_num_rows src/olap/row_block2.h /^ void set_num_rows(size_t num_rows) { _num_rows = num_rows; }$/;" f class:doris::RowBlockV2 +set_num_rows src/olap/rowset/rowset_meta.h /^ void set_num_rows(int64_t num_rows) {$/;" f class:doris::RowsetMeta +set_num_rows src/runtime/row_batch.h /^ void set_num_rows(int num_rows) {$/;" f class:doris::RowBatch +set_num_rows_load_total src/runtime/runtime_state.h /^ void set_num_rows_load_total(int64_t num_rows) {$/;" f class:doris::RuntimeState +set_num_segments src/olap/rowset/rowset_meta.h /^ void set_num_segments(int64_t num_segments) {$/;" f class:doris::RowsetMeta +set_num_segments src/olap/rowset/segment_group.h /^ inline void set_num_segments(int32_t num_segments) { _num_segments = num_segments; }$/;" f class:doris::SegmentGroup +set_num_senders src/exec/exchange_node.h /^ void set_num_senders(int num_senders) {$/;" f class:doris::ExchangeNode +set_num_senders src/exec/pl_task_root.h /^ void set_num_senders(int num_senders) {$/;" f class:doris::PlTaskRoot +set_num_updates src/udf/udf_internal.h /^ void set_num_updates(int64_t n) {$/;" f class:doris::FunctionContextImpl +set_opened src/exec/olap_scanner.h /^ void set_opened() { _is_open = true; }$/;" f class:doris::OlapScanner +set_output_slot src/exprs/agg_fn_evaluator.cpp /^inline void AggFnEvaluator::set_output_slot(const AnyVal* src,$/;" f class:doris::AggFnEvaluator +set_partition_id src/olap/rowset/rowset_meta.h /^ void set_partition_id(int64_t partition_id) {$/;" f class:doris::RowsetMeta +set_partition_id src/olap/tablet.cpp /^OLAPStatus Tablet::set_partition_id(int64_t partition_id) {$/;" f class:doris::Tablet +set_partition_id src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::set_partition_id(int64_t partition_id) {$/;" f class:doris::TabletMeta +set_payload src/http/http_client.h /^ void set_payload(const std::string& post_body) {$/;" f class:doris::HttpClient +set_pending_finished src/olap/rowset/segment_group.h /^ inline void set_pending_finished() { _is_pending = false; }$/;" f class:doris::SegmentGroup +set_per_fragment_instance_idx src/runtime/runtime_state.h /^ void set_per_fragment_instance_idx(int idx) {$/;" f class:doris::RuntimeState +set_pos src/olap/row_block.h /^ void set_pos(size_t pos) { _pos = pos; }$/;" f class:doris::RowBlock +set_position src/olap/byte_buffer.h /^ OLAPStatus set_position(uint64_t new_position) {$/;" f class:doris::StorageByteBuffer +set_position src/olap/file_stream.h /^ inline void set_position(uint32_t pos) {$/;" f class:doris::ReadOnlyFileStream +set_positions_count src/olap/stream_index_reader.cpp /^void PositionEntryReader::set_positions_count(size_t count) {$/;" f class:doris::PositionEntryReader +set_preferred_rowset_type src/olap/tablet_meta.h /^ void set_preferred_rowset_type(RowsetTypePB preferred_rowset_type) {$/;" f class:doris::TabletMeta +set_process_status src/runtime/runtime_state.h /^ void set_process_status(const Status& status) {$/;" f class:doris::RuntimeState +set_process_status src/runtime/runtime_state.h /^ void set_process_status(const std::string& err_msg) {$/;" f class:doris::RuntimeState +set_process_status src/udf/udf.cpp /^ void set_process_status(const std::string& error_msg) {$/;" f class:doris::RuntimeState +set_query_statistics src/exec/data_sink.h /^ virtual void set_query_statistics(std::shared_ptr statistics) {$/;" f class:doris::DataSink +set_query_statistics src/runtime/buffer_control_block.h /^ void set_query_statistics(std::shared_ptr statistics) {$/;" f class:doris::BufferControlBlock +set_query_statistics src/runtime/result_sink.cpp /^void ResultSink::set_query_statistics(std::shared_ptr statistics) {$/;" f class:doris::ResultSink +set_re2_options src/exprs/string_functions.cpp /^bool StringFunctions::set_re2_options($/;" f class:doris::StringFunctions +set_read_params src/olap/rowset/column_data.cpp /^void ColumnData::set_read_params($/;" f class:doris::ColumnData +set_read_timer src/runtime/disk_io_mgr.cc /^void DiskIoMgr::set_read_timer(RequestContext* r, RuntimeProfile::Counter* c) {$/;" f class:doris::DiskIoMgr +set_recv_timeout src/util/thrift_client.h /^ void set_recv_timeout(int ms) {$/;" f class:doris::ThriftClientImpl +set_related_schema_hash src/olap/tablet_meta.h /^ inline void set_related_schema_hash(int32_t schema_hash) { _related_schema_hash = schema_hash; }$/;" f class:doris::AlterTabletTask +set_related_tablet_id src/olap/tablet_meta.h /^ inline void set_related_tablet_id(int64_t related_tablet_id) { _related_tablet_id = related_tablet_id; }$/;" f class:doris::AlterTabletTask +set_remove_from_rowset_meta src/olap/rowset/rowset_meta.h /^ void set_remove_from_rowset_meta() {$/;" f class:doris::RowsetMeta +set_request_auth src/common/utils.h /^void set_request_auth(T* req, const AuthInfo& auth) {$/;" f namespace:doris +set_resource_tls src/common/resource_tls.cpp /^int ResourceTls::set_resource_tls(TResourceInfo* info) {$/;" f class:doris::ResourceTls +set_result_comparator output/udf/include/uda_test_harness.h /^ void set_result_comparator(ResultComparator fn) {$/;" f class:doris_udf::UdaTestHarnessBase +set_result_comparator src/udf/uda_test_harness.h /^ void set_result_comparator(ResultComparator fn) {$/;" f class:doris_udf::UdaTestHarnessBase +set_row src/olap/row_block.h /^ inline void set_row(uint32_t row_index, const RowType& row) const {$/;" f class:doris::RowBlock +set_row_selected src/olap/selection_vector.h /^ void set_row_selected(size_t row) {$/;" f class:doris::SelectionVector +set_rowset_id src/olap/rowset/rowset_meta.h /^ void set_rowset_id(const RowsetId& rowset_id) {$/;" f class:doris::RowsetMeta +set_rowset_state src/olap/rowset/rowset_meta.h /^ void set_rowset_state(RowsetStatePB rowset_state) {$/;" f class:doris::RowsetMeta +set_rowset_type src/olap/rowset/rowset_meta.h /^ void set_rowset_type(RowsetTypePB rowset_type) {$/;" f class:doris::RowsetMeta +set_scan_ranges src/exec/broker_scan_node.cpp /^Status BrokerScanNode::set_scan_ranges(const std::vector& scan_ranges) {$/;" f class:doris::BrokerScanNode +set_scan_ranges src/exec/csv_scan_node.cpp /^Status CsvScanNode::set_scan_ranges(const vector& scan_ranges) {$/;" f class:doris::CsvScanNode +set_scan_ranges src/exec/es_http_scan_node.cpp /^Status EsHttpScanNode::set_scan_ranges(const std::vector& scan_ranges) {$/;" f class:doris::EsHttpScanNode +set_scan_ranges src/exec/es_scan_node.cpp /^Status EsScanNode::set_scan_ranges(const vector& scan_ranges) {$/;" f class:doris::EsScanNode +set_scan_ranges src/exec/kudu_scan_node.cpp /^Status KuduScanNode::set_scan_ranges(const std::vector& scan_ranges) {$/;" f class:doris::KuduScanNode +set_scan_ranges src/exec/mysql_scan_node.cpp /^Status MysqlScanNode::set_scan_ranges(const vector& scan_ranges) {$/;" f class:doris::MysqlScanNode +set_scan_ranges src/exec/olap_scan_node.cpp /^Status OlapScanNode::set_scan_ranges(const std::vector& scan_ranges) {$/;" f class:doris::OlapScanNode +set_scan_ranges src/exec/schema_scan_node.cpp /^Status SchemaScanNode::set_scan_ranges(const vector& scan_ranges) {$/;" f class:doris::SchemaScanNode +set_scanner_id src/runtime/row_batch.h /^ void set_scanner_id(int id) {$/;" f class:doris::RowBatch +set_search_string src/exprs/like_predicate.h /^ void set_search_string(const std::string& search_string_arg) {$/;" f struct:doris::LikePredicate::LikePredicateState +set_segment_group src/olap/rowset/column_data.h /^ void set_segment_group(SegmentGroup* segment_group) { _segment_group = segment_group; }$/;" f class:doris::ColumnData +set_segment_group_id src/olap/rowset/segment_group.h /^ inline void set_segment_group_id(int32_t segment_group_id) { _segment_group_id = segment_group_id; }$/;" f class:doris::SegmentGroup +set_segment_id src/olap/olap_index.h /^ OLAPStatus set_segment_id(const iterator_offset_t& segment_id) {$/;" f class:doris::IndexComparator +set_segments_overlap src/olap/rowset/rowset_meta.h /^ void set_segments_overlap(SegmentsOverlapPB segments_overlap) {$/;" f class:doris::RowsetMeta +set_selected_in_use src/runtime/vectorized_row_batch.h /^ void set_selected_in_use(bool selected_in_use) {$/;" f class:doris::VectorizedRowBatch +set_selected_size src/olap/row_block2.h /^ void set_selected_size(uint16_t selected_size) {$/;" f class:doris::RowBlockV2 +set_send_timeout src/util/thrift_client.h /^ void set_send_timeout(int ms) {$/;" f class:doris::ThriftClientImpl +set_session_handler src/util/thrift_server.h /^ void set_session_handler(SessionHandlerIf* session) {$/;" f class:doris::ThriftServer +set_shard_id src/olap/tablet_meta.h /^inline void TabletMeta::set_shard_id(int32_t shard_id) {$/;" f class:doris::TabletMeta +set_size src/runtime/vectorized_row_batch.h /^ void set_size(int size) {$/;" f class:doris::VectorizedRowBatch +set_sparse src/olap/hll.cpp /^void HllSetHelper::set_sparse($/;" f class:doris::HllSetHelper +set_start_version src/olap/rowset/rowset_meta.h /^ void set_start_version(int64_t start_version) {$/;" f class:doris::RowsetMeta +set_statistic src/olap/stream_index_writer.cpp /^OLAPStatus PositionEntryWriter::set_statistic(ColumnStatistics* statistic) {$/;" f class:doris::PositionEntryWriter +set_stats src/olap/rowset/column_data.h /^ void set_stats(OlapReaderStatistics* stats) {$/;" f class:doris::ColumnData +set_storage_engine src/runtime/exec_env.h /^ void set_storage_engine(StorageEngine* storage_engine) { _storage_engine = storage_engine; }$/;" f class:doris::ExecEnv +set_store_paths src/runtime/exec_env.h /^ void set_store_paths(const std::vector& paths) { _store_paths = paths; }$/;" f class:doris::ExecEnv +set_store_used_flag src/olap/storage_engine.cpp /^void StorageEngine::set_store_used_flag(const string& path, bool is_used) {$/;" f class:doris::StorageEngine +set_string_val src/exprs/expr_value.h /^ void set_string_val(const StringValue& str) {$/;" f struct:doris::ExprValue +set_string_val src/exprs/expr_value.h /^ void set_string_val(const std::string& str) {$/;" f struct:doris::ExprValue +set_t_status src/common/status.h /^ void set_t_status(T* status_container) const {$/;" f class:doris::Status +set_tablet_convert_finished src/olap/olap_meta.cpp /^OLAPStatus OlapMeta::set_tablet_convert_finished() {$/;" f class:doris::OlapMeta +set_tablet_id src/olap/rowset/rowset_meta.h /^ void set_tablet_id(int64_t tablet_id) {$/;" f class:doris::RowsetMeta +set_tablet_schema src/olap/olap_cond.h /^ void set_tablet_schema(const TabletSchema* schema) {$/;" f class:doris::Conditions +set_tablet_schema_for_cmp_and_aggregate test/olap/row_cursor_test.cpp /^void set_tablet_schema_for_cmp_and_aggregate(TabletSchema* tablet_schema) {$/;" f namespace:doris +set_tablet_schema_for_init test/olap/row_cursor_test.cpp /^void set_tablet_schema_for_init(TabletSchema* tablet_schema) {$/;" f namespace:doris +set_tablet_schema_for_scan_key test/olap/row_cursor_test.cpp /^void set_tablet_schema_for_scan_key(TabletSchema* tablet_schema) {$/;" f namespace:doris +set_tablet_schema_hash src/olap/rowset/rowset_meta.h /^ void set_tablet_schema_hash(int64_t tablet_schema_hash) {$/;" f class:doris::RowsetMeta +set_tablet_state src/olap/tablet.cpp /^OLAPStatus Tablet::set_tablet_state(TabletState state) {$/;" f class:doris::Tablet +set_tablet_state src/olap/tablet_meta.h /^inline void TabletMeta::set_tablet_state(TabletState state) {$/;" f class:doris::TabletMeta +set_tablet_uid src/olap/rowset/rowset_meta.h /^ void set_tablet_uid(TabletUid tablet_uid) {$/;" f class:doris::RowsetMeta +set_thread_available_cb src/runtime/thread_resource_mgr.cpp /^void ThreadResourceMgr::ResourcePool::set_thread_available_cb(thread_available_cb fn) {$/;" f class:doris::ThreadResourceMgr::ResourcePool +set_thread_name src/util/thread.cpp /^void ThreadMgr::set_thread_name(const std::string& name, int64_t tid) {$/;" f class:doris::ThreadMgr +set_timeout_ms src/http/http_client.h /^ void set_timeout_ms(int64_t timeout_ms) {$/;" f class:doris::HttpClient +set_to_abs_value output/udf/include/udf.h /^ void set_to_abs_value() {$/;" f struct:doris_udf::DecimalV2Val +set_to_abs_value output/udf/include/udf.h /^ void set_to_abs_value() {$/;" f struct:doris_udf::DecimalVal +set_to_abs_value src/udf/udf.h /^ void set_to_abs_value() {$/;" f struct:doris_udf::DecimalV2Val +set_to_abs_value src/udf/udf.h /^ void set_to_abs_value() {$/;" f struct:doris_udf::DecimalVal +set_to_max src/exprs/expr_value.h /^ void* set_to_max(const TypeDescriptor& type) {$/;" f struct:doris::ExprValue +set_to_max src/olap/field.h /^ virtual inline void set_to_max(char* buf) const { return _type_info->set_to_max(buf); }$/;" f class:doris::Field +set_to_max src/olap/types.cpp /^void (*FieldTypeTraits::set_to_max)(void*) = nullptr;$/;" m namespace:doris file: +set_to_max src/olap/types.h /^ inline void set_to_max(void* buf) const { _set_to_max(buf); }$/;" f class:doris::TypeInfo +set_to_max src/olap/types.h /^ static inline void set_to_max(void* buf) {$/;" f struct:doris::BaseFieldtypeTraits +set_to_max src/olap/types.h /^ static void (*set_to_max)(void*);$/;" m struct:doris::FieldTypeTraits +set_to_max src/olap/types.h /^ static void set_to_max(void* buf) {$/;" f struct:doris::FieldTypeTraits +set_to_max src/olap/wrapper_field.h /^ void set_to_max() { _rep->set_to_max(_field_buf + 1); }$/;" f class:doris::WrapperField +set_to_min src/exprs/expr_value.h /^ void* set_to_min(const TypeDescriptor& type) {$/;" f struct:doris::ExprValue +set_to_min src/olap/field.h /^ inline void set_to_min(char* buf) const { return _type_info->set_to_min(buf); }$/;" f class:doris::Field +set_to_min src/olap/types.h /^ inline void set_to_min(void* buf) const { _set_to_min(buf); }$/;" f class:doris::TypeInfo +set_to_min src/olap/types.h /^ static inline void set_to_min(void* buf) {$/;" f struct:doris::BaseFieldtypeTraits +set_to_min src/olap/types.h /^ static void set_to_min(void* buf) {$/;" f struct:doris::FieldTypeTraits +set_to_min src/olap/wrapper_field.h /^ void set_to_min() { _rep->set_to_min(_field_buf + 1); }$/;" f class:doris::WrapperField +set_to_zero output/udf/include/udf.h /^ void set_to_zero() {$/;" f struct:doris_udf::DecimalV2Val +set_to_zero output/udf/include/udf.h /^ void set_to_zero() {$/;" f struct:doris_udf::DecimalVal +set_to_zero src/exprs/expr_value.h /^ void* set_to_zero(const TypeDescriptor& type) {$/;" f struct:doris::ExprValue +set_to_zero src/runtime/decimal_value.h /^ void set_to_zero() {$/;" f class:doris::DecimalValue +set_to_zero src/runtime/decimalv2_value.h /^ void set_to_zero() {$/;" f class:doris::DecimalV2Value +set_to_zero src/udf/udf.h /^ void set_to_zero() {$/;" f struct:doris_udf::DecimalV2Val +set_to_zero src/udf/udf.h /^ void set_to_zero() {$/;" f struct:doris_udf::DecimalVal +set_total_disk_size src/olap/rowset/rowset_meta.h /^ void set_total_disk_size(size_t total_disk_size) {$/;" f class:doris::RowsetMeta +set_tuple src/exec/partitioned_hash_table.inline.h /^inline void PartitionedHashTable::Iterator::set_tuple(Tuple* tuple, uint32_t hash) {$/;" f class:doris::PartitionedHashTable::Iterator +set_tuple src/runtime/tuple_row.h /^ void set_tuple(int tuple_idx, Tuple* tuple) {$/;" f class:doris::TupleRow +set_txn_id src/olap/rowset/rowset_meta.h /^ void set_txn_id(int64_t txn_id) {$/;" f class:doris::RowsetMeta +set_type src/exprs/aggregate_functions.cpp /^ FunctionContext::Type set_type() {$/;" f class:doris::MultiDistinctCountDateState +set_type src/exprs/aggregate_functions.cpp /^ FunctionContext::Type set_type() {$/;" f class:doris::MultiDistinctDecimalState +set_type src/exprs/aggregate_functions.cpp /^ FunctionContext::Type set_type() {$/;" f class:doris::MultiDistinctDecimalV2State +set_type src/exprs/aggregate_functions.cpp /^ FunctionContext::Type set_type() {$/;" f class:doris::MultiDistinctNumericState +set_type src/exprs/aggregate_functions.cpp /^ FunctionContext::Type set_type() {$/;" f class:doris::MultiDistinctStringCountState +set_type src/runtime/datetime_value.cpp /^void DateTimeValue::set_type(int type) {$/;" f class:doris::DateTimeValue +set_unrestricted_auth src/http/http_client.h /^ void set_unrestricted_auth(int gohead) {$/;" f class:doris::HttpClient +set_up test/olap/delete_handler_test.cpp /^void set_up() {$/;" f namespace:doris +set_up test/olap/delta_writer_test.cpp /^void set_up() {$/;" f namespace:doris +set_up test/olap/memtable_flush_executor_test.cpp /^void set_up() {$/;" f namespace:doris +set_up test/olap/olap_reader_test.cpp /^void set_up() {$/;" f namespace:doris +set_up test/olap/options_test.cpp /^void set_up() {$/;" f namespace:doris +set_up test/olap/rowset/alpha_rowset_test.cpp /^void set_up() {$/;" f namespace:doris +set_up test/olap/vectorized_olap_reader_test.cpp /^void set_up() {$/;" f namespace:doris +set_using_cache src/olap/rowset/column_data.h /^ void set_using_cache(bool is_using_cache) {$/;" f class:doris::ColumnData +set_value src/olap/tuple.h /^ void set_value(size_t i, const std::string& value, bool is_null = false) {$/;" f class:doris::OlapTuple +set_value src/runtime/decimalv2_value.h /^ void set_value(int128_t value) {$/;" f class:doris::DecimalV2Value +set_value src/util/metrics.h /^ void set_value(const T& value) {$/;" f class:doris::LockSimpleMetric +set_version src/olap/rowset/rowset_meta.h /^ void set_version(Version version) {$/;" f class:doris::RowsetMeta +set_version src/olap/rowset/segment_group.h /^ inline void set_version(Version version) { _version = version; }$/;" f class:doris::SegmentGroup +set_version_hash src/olap/rowset/rowset_meta.h /^ void set_version_hash(VersionHash version_hash) {$/;" f class:doris::RowsetMeta +set_version_hash src/olap/rowset/segment_group.h /^ inline void set_version_hash(VersionHash version_hash) { _version_hash = version_hash; }$/;" f class:doris::SegmentGroup +set_zero src/runtime/datetime_value.cpp /^void DateTimeValue::set_zero(int type) {$/;" f class:doris::DateTimeValue +set_zone_maps src/olap/rowset/rowset_meta.h /^ void set_zone_maps(const std::vector& zone_maps) {$/;" f class:doris::RowsetMeta +setup src/http/action/pprof_actions.cpp /^Status PprofActions::setup(ExecEnv* exec_env, EvHttpServer* http_server) {$/;" f class:doris::PprofActions +setup src/util/thrift_rpc_helper.cpp /^void ThriftRpcHelper::setup(ExecEnv* exec_env) {$/;" f class:doris::ThriftRpcHelper +shallow_copy src/olap/types.h /^ inline void shallow_copy(void* dest, const void* src) const {$/;" f class:doris::TypeInfo +shallow_copy src/olap/types.h /^ static inline void shallow_copy(void* dest, const void* src) {$/;" f struct:doris::BaseFieldtypeTraits +shallow_copy src/olap/types.h /^ static void shallow_copy(void* dest, const void* src) {$/;" f struct:doris::FieldTypeTraits +shallow_copy_content src/olap/field.h /^ inline void shallow_copy_content(char* dst, const char* src) const {$/;" f class:doris::Field +shape src/geo/wkt_parse_ctx.h /^ doris::GeoShape* shape = nullptr;$/;" m struct:WktParseContext +shape src/geo/wkt_yacc.y /^shape:$/;" l +shapes src/geo/geo_functions.cpp /^ GeoShape* shapes[2];$/;" m struct:doris::StContainsState file: +shard_id src/olap/tablet.h /^inline int16_t Tablet::shard_id() {$/;" f class:doris::Tablet +shard_id src/olap/tablet_meta.h /^inline int16_t TabletMeta::shard_id() const {$/;" f class:doris::TabletMeta +shift src/runtime/decimal_value.h /^ int shift (int32_t shift) {$/;" f class:doris::DecimalValue +short_key_columns src/olap/olap_index.h /^ const std::vector& short_key_columns() const {$/;" f class:doris::MemIndex +short_key_columns src/olap/rowset/segment_group.h /^ const std::vector& short_key_columns() const {$/;" f class:doris::SegmentGroup +short_key_length src/olap/olap_index.h /^ const size_t short_key_length() const {$/;" f class:doris::MemIndex +short_key_length src/olap/rowset/segment_group.h /^ const size_t short_key_length() const {$/;" f class:doris::SegmentGroup +short_key_num src/olap/olap_index.h /^ const size_t short_key_num() const {$/;" f class:doris::MemIndex +should src/exec/es/es_query_builder.cpp /^void BooleanQueryBuilder::should(QueryBuilder* filter) {$/;" f class:doris::BooleanQueryBuilder +show_meta src/tools/meta_tool.cpp /^void show_meta() {$/;" f +show_varialbes src/exec/schema_scanner/schema_helper.cpp /^Status SchemaHelper::show_varialbes($/;" f class:doris::SchemaHelper +shrinkToFit src/util/bitmap_value.h /^ size_t shrinkToFit() {$/;" f class:doris::detail::Roaring64Map +shrink_to_fit src/util/faststring.h /^ void shrink_to_fit() {$/;" f class:doris::faststring +shutdown src/runtime/record_batch_queue.cpp /^void RecordBatchQueue::shutdown() {$/;" f class:doris::RecordBatchQueue +shutdown src/util/batch_process_thread_pool.hpp /^ void shutdown() {$/;" f class:doris::BatchProcessThreadPool +shutdown src/util/blocking_priority_queue.hpp /^ void shutdown() {$/;" f class:doris::BlockingPriorityQueue +shutdown src/util/blocking_queue.hpp /^ void shutdown() {$/;" f class:doris::BlockingQueue +shutdown src/util/priority_thread_pool.hpp /^ void shutdown() {$/;" f class:doris::PriorityThreadPool +shutdown src/util/threadpool.cpp /^void ThreadPool::shutdown() {$/;" f class:doris::ThreadPool +shutdown src/util/threadpool.cpp /^void ThreadPoolToken::shutdown() {$/;" f class:doris::ThreadPoolToken +shutdown_logging src/common/logconfig.cpp /^void shutdown_logging() {$/;" f namespace:doris +sig src/gutil/linux_syscall_support.h /^ unsigned long sig[(KERNEL_NSIG + 8*sizeof(unsigned long) - 1)\/$/;" m struct:kernel_sigset_t +sig_t src/gutil/port.h /^typedef void (*sig_t)(int);$/;" t +sigaction src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(sigaction)(int signum,$/;" f +sigaddset src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(sigaddset)(struct kernel_sigset_t *set,$/;" f +sigdelset src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(sigdelset)(struct kernel_sigset_t *set,$/;" f +sigemptyset src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(sigemptyset)(struct kernel_sigset_t *set) {$/;" f +sigfillset src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(sigfillset)(struct kernel_sigset_t *set) {$/;" f +sign output/udf/include/udf.h /^ bool sign;$/;" m struct:doris_udf::DecimalVal +sign src/exprs/math_functions.cpp /^FloatVal MathFunctions::sign($/;" f class:doris::MathFunctions +sign src/udf/udf.h /^ bool sign;$/;" m struct:doris_udf::DecimalVal +signal src/util/semaphore.hpp /^ void signal() {$/;" f class:__anon35::Semaphore +signature src/gutil/cpu.h /^ int signature() const { return signature_; }$/;" f class:base::final +signature_ src/gutil/cpu.h /^ int signature_; \/\/ raw form of type, family, model, and stepping$/;" m class:base::final +sigprocmask src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(sigprocmask)(int how,$/;" f +sigterm_handler src/common/daemon.cpp /^void sigterm_handler(int signo) {$/;" f namespace:doris +simple_task_method test/util/threadpool_test.cpp /^static void simple_task_method(int n, std::atomic* counter) {$/;" f namespace:doris +single_version src/olap/olap_common.h /^ bool single_version() const {$/;" f struct:doris::Version +singleton_output_tuple_ src/exec/new_partitioned_aggregation_node.h /^ Tuple* singleton_output_tuple_;$/;" m class:doris::NewPartitionedAggregationNode +singleton_output_tuple_returned_ src/exec/new_partitioned_aggregation_node.h /^ bool singleton_output_tuple_returned_;$/;" m class:doris::NewPartitionedAggregationNode +sixty_four_bit src/gutil/hash/string_hash.h /^enum { x86_64 = true, sixty_four_bit = true };$/;" e enum:hash_internal::__anon21 +size src/exec/broker_reader.cpp /^int64_t BrokerReader::size() {$/;" f class:doris::BrokerReader +size src/exec/csv_scan_node.cpp /^ int size() const {$/;" f class:doris::StringRef +size src/exec/hash_table.h /^ int64_t size() {$/;" f class:doris::HashTable +size src/exec/local_file_reader.cpp /^int64_t LocalFileReader::size () {$/;" f class:doris::LocalFileReader +size src/exec/new_partitioned_hash_table.h /^ int64_t size() const {$/;" f class:doris::NewPartitionedHashTable +size src/exec/olap_common.h /^ size_t size() {$/;" f class:doris::OlapScanKeys +size src/exec/parquet_reader.cpp /^Status ParquetReaderWrap::size(int64_t* size) {$/;" f class:doris::ParquetReaderWrap +size src/exec/partitioned_hash_table.h /^ int64_t size() const {$/;" f class:doris::PartitionedHashTable +size src/exec/schema_scanner.h /^ int size;$/;" m struct:doris::SchemaScanner::ColumnDesc +size src/exprs/bitmap_function.cpp /^ size_t size() {$/;" f struct:doris::BitmapIntersect +size src/exprs/hybird_set.h /^ virtual int size() {$/;" f class:doris::HybirdSet +size src/exprs/hybird_set.h /^ virtual int size() {$/;" f class:doris::StringValueSet +size src/gutil/strings/strcat.h /^ StringPiece::size_type size() const { return piece.size(); }$/;" f struct:AlphaNum +size src/gutil/strings/stringpiece.h /^ int size() const { return length_; }$/;" f class:StringPiece +size src/gutil/strings/substitute.h /^ inline int size() const { return size_; }$/;" f class:strings::internal::SubstituteArg +size src/http/http_parser.h /^ size_t size; \/\/ Chunk size$/;" m struct:doris::HttpChunkParseCtx +size src/olap/field.h /^ inline size_t size() const { return _type_info->size(); }$/;" f class:doris::Field +size src/olap/file_helper.h /^ uint64_t size() const {$/;" f class:doris::FileHeader +size src/olap/fs/file_block_manager.cpp /^Status FileReadableBlock::size(uint64_t* sz) const {$/;" f class:doris::fs::internal::FileReadableBlock +size src/olap/lru_cache.h /^ size_t size() const {$/;" f class:doris::CacheKey +size src/olap/rowset/segment_v2/binary_dict_page.cpp /^uint64_t BinaryDictPageBuilder::size() const {$/;" f class:doris::segment_v2::BinaryDictPageBuilder +size src/olap/rowset/segment_v2/bloom_filter.h /^ uint32_t size() const { return _size; }$/;" f class:doris::segment_v2::BloomFilter +size src/olap/rowset/segment_v2/column_writer.cpp /^ uint64_t size() {$/;" f class:doris::segment_v2::NullBitmapBuilder +size src/olap/rowset/segment_v2/index_page.h /^ uint64_t size() {$/;" f class:doris::segment_v2::IndexPageBuilder +size src/olap/rowset/segment_v2/ordinal_page_index.h /^ uint64_t size() { return _page_builder->size(); }$/;" f class:doris::segment_v2::OrdinalIndexWriter +size src/olap/rowset/segment_v2/page_pointer.h /^ uint32_t size;$/;" m struct:doris::segment_v2::PagePointer +size src/olap/rowset/segment_v2/zone_map_index.h /^ uint64_t size() { return _estimated_size; }$/;" f class:doris::segment_v2::ZoneMapIndexWriter +size src/olap/short_key_index.h /^ uint64_t size() {$/;" f class:doris::ShortKeyIndexBuilder +size src/olap/stream_index_common.cpp /^size_t ColumnStatistics::size() const {$/;" f class:doris::ColumnStatistics +size src/olap/tuple.h /^ size_t size() const { return _values.size(); }$/;" f class:doris::OlapTuple +size src/olap/types.h /^ inline const size_t size() const { return _size; }$/;" f class:doris::TypeInfo +size src/olap/types.h /^ static const int32_t size = sizeof(CppType);$/;" m struct:doris::TypeTraits +size src/olap/wrapper_field.h /^ size_t size() const { return _rep->size(); }$/;" f class:doris::WrapperField +size src/runtime/bufferpool/buffer_pool_internal.h /^ int size() const { return list_.size(); }$/;" f class:doris::BufferPool::PageList +size src/runtime/free_list.hpp /^ int size;$/;" m struct:doris::FreeList::FreeListNode +size src/runtime/memory/chunk.h /^ size_t size = 0;$/;" m struct:doris::Chunk +size src/runtime/snapshot_loader.h /^ int64_t size;$/;" m struct:doris::FileStat +size src/runtime/string_buffer.hpp /^ int size() const {$/;" f class:doris::StringBuffer +size src/runtime/vectorized_row_batch.h /^ uint16_t size() {$/;" f class:doris::VectorizedRowBatch +size src/util/buffer_builder.hpp /^ int size() const {$/;" f class:doris::BufferBuilder +size src/util/core_local.h /^ inline size_t size() const { return _size; }$/;" f class:doris::CoreLocalValue +size src/util/core_local.h /^ size_t size() const { return _size; }$/;" f class:doris::CoreLocalValueController +size src/util/faststring.h /^ size_t size() const {$/;" f class:doris::faststring +size src/util/internal_queue.h /^ int size() const { return size_; }$/;" f class:doris::InternalQueueBase +size src/util/lru_cache.hpp /^ size_t size() const {$/;" f class:doris::LruCache +size src/util/slice.h /^ size_t size;$/;" m struct:doris::Slice +size_ src/gutil/strings/substitute.h /^ int size_;$/;" m class:strings::internal::SubstituteArg +size_ src/util/internal_queue.h /^ int size_;$/;" m class:doris::InternalQueueBase +size_central_dir src/util/minizip/unzip.c /^ ZPOS64_T size_central_dir; \/* size of the central directory *\/$/;" m struct:__anon34 file: +size_comment src/util/minizip/unzip.h /^ uLong size_comment; \/* size of the global comment of the zipfile *\/$/;" m struct:unz_global_info64_s +size_comment src/util/minizip/unzip.h /^ uLong size_comment; \/* size of the global comment of the zipfile *\/$/;" m struct:unz_global_info_s +size_file_comment src/util/minizip/unzip.h /^ uLong size_file_comment; \/* file comment length 2 bytes *\/$/;" m struct:unz_file_info64_s +size_file_comment src/util/minizip/unzip.h /^ uLong size_file_comment; \/* file comment length 2 bytes *\/$/;" m struct:unz_file_info_s +size_file_extra src/util/minizip/unzip.h /^ uLong size_file_extra; \/* extra field length 2 bytes *\/$/;" m struct:unz_file_info64_s +size_file_extra src/util/minizip/unzip.h /^ uLong size_file_extra; \/* extra field length 2 bytes *\/$/;" m struct:unz_file_info_s +size_filename src/util/minizip/unzip.h /^ uLong size_filename; \/* filename length 2 bytes *\/$/;" m struct:unz_file_info64_s +size_filename src/util/minizip/unzip.h /^ uLong size_filename; \/* filename length 2 bytes *\/$/;" m struct:unz_file_info_s +size_local_extrafield src/util/minizip/unzip.c /^ uInt size_local_extrafield;\/* size of the local extra field *\/$/;" m struct:__anon33 file: +size_type src/gutil/stl_util.h /^ typedef typename Alloc::size_type size_type;$/;" t class:STLCountingAllocator +size_type src/gutil/strings/stringpiece.h /^ typedef size_t size_type;$/;" t class:StringPiece +skip src/olap/file_stream.cpp /^OLAPStatus ReadOnlyFileStream::skip(uint64_t skip_length) {$/;" f class:doris::ReadOnlyFileStream +skip src/olap/in_stream.cpp /^OLAPStatus InStream::skip(uint64_t skip_length) {$/;" f class:doris::InStream +skip src/olap/rowset/bit_field_reader.cpp /^OLAPStatus BitFieldReader::skip(uint64_t num_values) {$/;" f class:doris::BitFieldReader +skip src/olap/rowset/column_reader.cpp /^OLAPStatus ColumnReader::skip(uint64_t row_count) {$/;" f class:doris::ColumnReader +skip src/olap/rowset/column_reader.cpp /^OLAPStatus DecimalColumnReader::skip(uint64_t row_count) {$/;" f class:doris::DecimalColumnReader +skip src/olap/rowset/column_reader.cpp /^OLAPStatus IntegerColumnReader::skip(uint64_t row_count) {$/;" f class:doris::IntegerColumnReader +skip src/olap/rowset/column_reader.cpp /^OLAPStatus LargeIntColumnReader::skip(uint64_t row_count) {$/;" f class:doris::LargeIntColumnReader +skip src/olap/rowset/column_reader.cpp /^OLAPStatus StringColumnDictionaryReader::skip(uint64_t row_count) {$/;" f class:doris::StringColumnDictionaryReader +skip src/olap/rowset/column_reader.cpp /^OLAPStatus StringColumnDirectReader::skip(uint64_t row_count) {$/;" f class:doris::StringColumnDirectReader +skip src/olap/rowset/column_reader.cpp /^OLAPStatus TinyColumnReader::skip(uint64_t row_count) {$/;" f class:doris::TinyColumnReader +skip src/olap/rowset/column_reader.h /^ virtual OLAPStatus skip(uint64_t row_count) {$/;" f class:doris::DefaultValueReader +skip src/olap/rowset/column_reader.h /^ virtual OLAPStatus skip(uint64_t row_count) {$/;" f class:doris::FixLengthStringColumnReader +skip src/olap/rowset/column_reader.h /^ virtual OLAPStatus skip(uint64_t row_count) {$/;" f class:doris::FloatintPointColumnReader +skip src/olap/rowset/column_reader.h /^ virtual OLAPStatus skip(uint64_t row_count) {$/;" f class:doris::IntegerColumnReaderWrapper +skip src/olap/rowset/column_reader.h /^ virtual OLAPStatus skip(uint64_t row_count) {$/;" f class:doris::NullValueReader +skip src/olap/rowset/column_reader.h /^ virtual OLAPStatus skip(uint64_t row_count) {$/;" f class:doris::VarStringColumnReader +skip src/olap/rowset/run_length_byte_reader.cpp /^OLAPStatus RunLengthByteReader::skip(uint64_t num_values) {$/;" f class:doris::RunLengthByteReader +skip src/olap/rowset/run_length_integer_reader.cpp /^OLAPStatus RunLengthIntegerReader::skip(uint64_t num_values) {$/;" f class:doris::RunLengthIntegerReader +skip src/util/frame_of_reference_coding.cpp /^bool ForDecoder::skip(int32_t skip_num) {$/;" f class:doris::ForDecoder +skip_bytes src/exec/read_write_util.h /^inline bool ReadWriteUtil::skip_bytes(uint8_t** buf, int* buf_len, int num_bytes,$/;" f class:doris::ReadWriteUtil +skip_leading_whitespace src/util/string_parser.hpp /^ static inline int skip_leading_whitespace(const char* s, int len) {$/;" f class:doris::StringParser +sleep src/exprs/utility_functions.cpp /^BooleanVal UtilityFunctions::sleep($/;" f class:doris::UtilityFunctions +slice src/util/slice.h /^ const Slice& slice() const {$/;" f class:doris::OwnedSlice +slot_id src/exprs/slot_ref.h /^ SlotId slot_id() const {$/;" f class:doris::SlotRef +slot_size src/runtime/descriptors.h /^ int slot_size() const {$/;" f class:doris::SlotDescriptor +slot_types src/testutil/desc_tbl_builder.h /^ std::vector slot_types() const { return _slot_types; }$/;" f class:doris::TupleDescBuilder +slots src/exec/tablet_info.h /^ std::vector slots;$/;" m struct:doris::OlapTableIndexSchema +slots src/olap/delta_writer.h /^ const std::vector* slots;$/;" m struct:doris::WriteRequest +slots src/runtime/descriptors.h /^ const std::vector& slots() const {$/;" f class:doris::TupleDescriptor +slots_ordered_by_idx src/runtime/descriptors.cpp /^std::vector TupleDescriptor::slots_ordered_by_idx() const {$/;" f class:doris::TupleDescriptor +slow_acquire src/util/spinlock.cc /^void SpinLock::slow_acquire() {$/;" f class:doris::SpinLock +small_ src/gutil/template_util.h /^typedef char small_;$/;" t namespace:base +small_file_mgr src/runtime/exec_env.h /^ SmallFileMgr* small_file_mgr() { return _small_file_mgr; }$/;" f class:doris::ExecEnv +small_val test/runtime/dpp_writer_test.cpp /^ int16_t small_val;$/;" m struct:doris::TestDataTuple file: +small_val test/runtime/mysql_table_writer_test.cpp /^ int16_t small_val;$/;" m struct:doris::TestDataTuple file: +smaller src/runtime/decimal_value.h /^ bool smaller(const DecimalValue& other) const {$/;" f class:doris::DecimalValue +smallint_val src/exprs/expr_value.h /^ int16_t smallint_val;$/;" m struct:doris::ExprValue +snapshot src/util/perf_counters.cpp /^void PerfCounters::snapshot(const string& name) {$/;" f class:doris::PerfCounters +snprintf src/gutil/port.h 833;" d +socket src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(socket)(int domain, int type, int protocol) {$/;" f +sort src/olap/schema_change.cpp /^bool RowBlockSorter::sort(RowBlock** row_block) {$/;" f class:doris::RowBlockSorter +sort src/runtime/merge_sorter.cpp /^ void sort(Run* run) {$/;" f class:doris::MergeSorter::TupleSorter +sort src/runtime/spill_sorter.cc /^void SpillSorter::TupleSorter::sort(Run* run) {$/;" f class:doris::SpillSorter::TupleSorter +sort_childer src/util/runtime_profile.h /^ void sort_childer(const Compare& cmp) {$/;" f class:doris::RuntimeProfile +sort_helper src/runtime/merge_sorter.cpp /^void MergeSorter::TupleSorter::sort_helper(TupleIterator first, TupleIterator last) {$/;" f class:doris::MergeSorter::TupleSorter +sort_helper src/runtime/spill_sorter.cc /^void SpillSorter::TupleSorter::sort_helper(TupleIterator first, TupleIterator last) {$/;" f class:doris::SpillSorter::TupleSorter +sort_input src/exec/sort_node.cpp /^Status SortNode::sort_input(RuntimeState* state) {$/;" f class:doris::SortNode +sort_input src/exec/spill_sort_node.cc /^Status SpillSortNode::sort_input(RuntimeState* state) {$/;" f class:doris::SpillSortNode +sort_run src/runtime/merge_sorter.cpp /^Status MergeSorter::sort_run() {$/;" f class:doris::MergeSorter +sort_run src/runtime/spill_sorter.cc /^Status SpillSorter::sort_run() {$/;" f class:doris::SpillSorter +sort_tuple_slot_expr_ctxs src/exec/sort_exec_exprs.h /^ const std::vector& sort_tuple_slot_expr_ctxs() const {$/;" f class:doris::SortExecExprs +sortlen src/exec/schema_scanner/schema_collations_scanner.h /^ int64_t sortlen;$/;" m struct:doris::SchemaCollationsScanner::CollationStruct +source src/util/perf_counters.h /^ DataSource source;$/;" m struct:doris::PerfCounters::CounterData +space src/exprs/string_functions.cpp /^StringVal StringFunctions::space(FunctionContext* context, const IntVal& len) {$/;" f class:doris::StringFunctions +space test/util/string_parser_test.cpp /^std::string space[] = {"", " ", "\\t\\t\\t", "\\n\\n\\n", "\\v\\v\\v", "\\f\\f\\f", "\\r\\r\\r"};$/;" m namespace:doris file: +space_len test/util/string_parser_test.cpp /^const int space_len = 7;$/;" m namespace:doris file: +spare_capacity src/runtime/mem_tracker.h /^ int64_t spare_capacity() const {$/;" f class:doris::MemTracker +spill src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::Partition::spill() {$/;" f class:doris::PartitionedAggregationNode::Partition +spill_partition src/exec/partitioned_aggregation_node.cc /^Status PartitionedAggregationNode::spill_partition() {$/;" f class:doris::PartitionedAggregationNode +spilled_partitions_ src/exec/new_partitioned_aggregation_node.h /^ std::deque spilled_partitions_;$/;" m class:doris::NewPartitionedAggregationNode +split src/util/path_trie.hpp /^ void split(const std::string& path, std::vector* array) {$/;" f class:doris::PathTrie +split_check_fill src/exec/csv_scan_node.cpp /^bool CsvScanNode::split_check_fill(const std::string& line, RuntimeState* state) {$/;" f class:doris::CsvScanNode +split_line src/exec/broker_scanner.cpp /^void BrokerScanner::split_line($/;" f class:doris::BrokerScanner +split_line src/exec/csv_scan_node.cpp /^void split_line(const std::string& str, char delimiter, std::vector& result) {$/;" f namespace:doris +split_part src/exprs/string_functions.cpp /^StringVal StringFunctions::split_part(FunctionContext* context, const StringVal& content,$/;" f class:doris::StringFunctions +split_path src/util/path_util.cpp /^vector split_path(const string& path) {$/;" f namespace:doris::path_util +split_pathes src/util/file_utils.cpp /^Status FileUtils::split_pathes(const char* path, std::vector* path_vec) {$/;" f class:doris::FileUtils +split_range src/olap/rowset/alpha_rowset.cpp /^OLAPStatus AlphaRowset::split_range($/;" f class:doris::AlphaRowset +split_range src/olap/rowset/beta_rowset.cpp /^OLAPStatus BetaRowset::split_range(const RowCursor& start_key,$/;" f class:doris::BetaRowset +split_range src/olap/tablet.cpp /^OLAPStatus Tablet::split_range(const OlapTuple& start_key_strings,$/;" f class:doris::Tablet +split_string src/olap/utils.h /^OLAPStatus split_string(const std::string& base,$/;" f namespace:doris +split_string_to_map src/util/string_parser.hpp /^ static Status split_string_to_map(const std::string& base, const T element_separator,$/;" f class:doris::StringParser +splitkv src/common/configbase.cpp /^void Properties::splitkv(const std::string& s, std::string& k, std::string& v) {$/;" f class:doris::config::Properties +srandom src/gutil/port.h 851;" d +src_counter src/util/runtime_profile.h /^ Counter* src_counter; \/\/ the counter to be sampled$/;" m struct:doris::RuntimeProfile::BucketCountersInfo +src_counter src/util/runtime_profile.h /^ Counter* src_counter; \/\/ the counter to be sampled$/;" m struct:doris::RuntimeProfile::SamplingCounterInfo +src_counter src/util/runtime_profile.h /^ Counter* src_counter;$/;" m struct:doris::RuntimeProfile::RateCounterInfo +src_file test/runtime/small_file_mgr_test.cpp /^std::string src_file = "small_file.txt";$/;" m namespace:doris file: +sse_util src/util/sse_util.hpp /^namespace sse_util {$/;" n namespace:doris +ssize_t src/gutil/port.h /^typedef int ssize_t;$/;" t +st_as_wkt src/geo/geo_functions.cpp /^StringVal GeoFunctions::st_as_wkt(doris_udf::FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_atim src/gutil/linux_syscall_support.h /^ kernel_timespec st_atim;$/;" m struct:kernel_stat +st_atim src/gutil/linux_syscall_support.h /^ kernel_timespec st_atim;$/;" m struct:kernel_stat64 +st_atime_ src/gutil/linux_syscall_support.h /^ long st_atime_;$/;" m struct:kernel_stat +st_atime_ src/gutil/linux_syscall_support.h /^ uint64_t st_atime_;$/;" m struct:kernel_stat +st_atime_ src/gutil/linux_syscall_support.h /^ unsigned st_atime_;$/;" m struct:kernel_stat +st_atime_ src/gutil/linux_syscall_support.h /^ unsigned st_atime_;$/;" m struct:kernel_stat64 +st_atime_ src/gutil/linux_syscall_support.h /^ unsigned long st_atime_;$/;" m struct:kernel_stat +st_atime_nsec_ src/gutil/linux_syscall_support.h /^ long st_atime_nsec_;$/;" m struct:kernel_stat +st_atime_nsec_ src/gutil/linux_syscall_support.h /^ uint64_t st_atime_nsec_;$/;" m struct:kernel_stat +st_atime_nsec_ src/gutil/linux_syscall_support.h /^ unsigned st_atime_nsec_;$/;" m struct:kernel_stat +st_atime_nsec_ src/gutil/linux_syscall_support.h /^ unsigned st_atime_nsec_;$/;" m struct:kernel_stat64 +st_atime_nsec_ src/gutil/linux_syscall_support.h /^ unsigned long st_atime_nsec_;$/;" m struct:kernel_stat +st_blksize src/gutil/linux_syscall_support.h /^ int st_blksize;$/;" m struct:kernel_stat +st_blksize src/gutil/linux_syscall_support.h /^ int64_t st_blksize;$/;" m struct:kernel_stat +st_blksize src/gutil/linux_syscall_support.h /^ long long st_blksize;$/;" m struct:kernel_stat64 +st_blksize src/gutil/linux_syscall_support.h /^ unsigned st_blksize;$/;" m struct:kernel_stat +st_blksize src/gutil/linux_syscall_support.h /^ unsigned st_blksize;$/;" m struct:kernel_stat64 +st_blksize src/gutil/linux_syscall_support.h /^ unsigned long st_blksize;$/;" m struct:kernel_stat +st_blocks src/gutil/linux_syscall_support.h /^ int st_blocks;$/;" m struct:kernel_stat +st_blocks src/gutil/linux_syscall_support.h /^ int64_t st_blocks;$/;" m struct:kernel_stat +st_blocks src/gutil/linux_syscall_support.h /^ long st_blocks;$/;" m struct:kernel_stat +st_blocks src/gutil/linux_syscall_support.h /^ long long st_blocks;$/;" m struct:kernel_stat64 +st_blocks src/gutil/linux_syscall_support.h /^ unsigned st_blocks;$/;" m struct:kernel_stat +st_blocks src/gutil/linux_syscall_support.h /^ unsigned long st_blocks;$/;" m struct:kernel_stat +st_blocks src/gutil/linux_syscall_support.h /^ unsigned long long st_blocks;$/;" m struct:kernel_stat +st_blocks src/gutil/linux_syscall_support.h /^ unsigned long long st_blocks;$/;" m struct:kernel_stat64 +st_circle src/geo/geo_functions.cpp /^doris_udf::StringVal GeoFunctions::st_circle(FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_circle_prepare src/geo/geo_functions.cpp /^void GeoFunctions::st_circle_prepare(doris_udf::FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_contains src/geo/geo_functions.cpp /^doris_udf::BooleanVal GeoFunctions::st_contains($/;" f class:doris::GeoFunctions +st_contains_close src/geo/geo_functions.cpp /^void GeoFunctions::st_contains_close(doris_udf::FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_contains_prepare src/geo/geo_functions.cpp /^void GeoFunctions::st_contains_prepare(doris_udf::FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_ctim src/gutil/linux_syscall_support.h /^ kernel_timespec st_ctim;$/;" m struct:kernel_stat +st_ctim src/gutil/linux_syscall_support.h /^ kernel_timespec st_ctim;$/;" m struct:kernel_stat64 +st_ctime_ src/gutil/linux_syscall_support.h /^ long st_ctime_;$/;" m struct:kernel_stat +st_ctime_ src/gutil/linux_syscall_support.h /^ uint64_t st_ctime_;$/;" m struct:kernel_stat +st_ctime_ src/gutil/linux_syscall_support.h /^ unsigned st_ctime_;$/;" m struct:kernel_stat +st_ctime_ src/gutil/linux_syscall_support.h /^ unsigned st_ctime_;$/;" m struct:kernel_stat64 +st_ctime_ src/gutil/linux_syscall_support.h /^ unsigned long st_ctime_;$/;" m struct:kernel_stat +st_ctime_nsec_ src/gutil/linux_syscall_support.h /^ long st_ctime_nsec_;$/;" m struct:kernel_stat +st_ctime_nsec_ src/gutil/linux_syscall_support.h /^ uint64_t st_ctime_nsec_;$/;" m struct:kernel_stat +st_ctime_nsec_ src/gutil/linux_syscall_support.h /^ unsigned st_ctime_nsec_;$/;" m struct:kernel_stat +st_ctime_nsec_ src/gutil/linux_syscall_support.h /^ unsigned st_ctime_nsec_;$/;" m struct:kernel_stat64 +st_ctime_nsec_ src/gutil/linux_syscall_support.h /^ unsigned long st_ctime_nsec_;$/;" m struct:kernel_stat +st_dev src/gutil/linux_syscall_support.h /^ uint64_t st_dev;$/;" m struct:kernel_stat +st_dev src/gutil/linux_syscall_support.h /^ unsigned st_dev;$/;" m struct:kernel_stat +st_dev src/gutil/linux_syscall_support.h /^ unsigned long st_dev;$/;" m struct:kernel_stat +st_dev src/gutil/linux_syscall_support.h /^ unsigned long long st_dev;$/;" m struct:kernel_stat +st_dev src/gutil/linux_syscall_support.h /^ unsigned long long st_dev;$/;" m struct:kernel_stat64 +st_dev src/gutil/linux_syscall_support.h /^ unsigned short st_dev;$/;" m struct:kernel_stat +st_distance_sphere src/geo/geo_functions.cpp /^DoubleVal GeoFunctions::st_distance_sphere(FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_from_wkt src/geo/geo_functions.h /^ static doris_udf::StringVal st_from_wkt($/;" f class:doris::GeoFunctions +st_from_wkt_close src/geo/geo_functions.cpp /^void GeoFunctions::st_from_wkt_close(FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_from_wkt_common src/geo/geo_functions.cpp /^StringVal GeoFunctions::st_from_wkt_common(FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_from_wkt_prepare src/geo/geo_functions.h /^ static void st_from_wkt_prepare(doris_udf::FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_from_wkt_prepare_common src/geo/geo_functions.cpp /^void GeoFunctions::st_from_wkt_prepare_common(FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_gid src/gutil/linux_syscall_support.h /^ unsigned st_gid;$/;" m struct:kernel_stat +st_gid src/gutil/linux_syscall_support.h /^ unsigned st_gid;$/;" m struct:kernel_stat64 +st_gid src/gutil/linux_syscall_support.h /^ unsigned int st_gid;$/;" m struct:kernel_stat +st_gid src/gutil/linux_syscall_support.h /^ unsigned short st_gid;$/;" m struct:kernel_stat +st_ino src/gutil/linux_syscall_support.h /^ uint64_t st_ino;$/;" m struct:kernel_stat +st_ino src/gutil/linux_syscall_support.h /^ unsigned st_ino;$/;" m struct:kernel_stat +st_ino src/gutil/linux_syscall_support.h /^ unsigned long st_ino;$/;" m struct:kernel_stat +st_ino src/gutil/linux_syscall_support.h /^ unsigned long long st_ino;$/;" m struct:kernel_stat +st_ino src/gutil/linux_syscall_support.h /^ unsigned long long st_ino;$/;" m struct:kernel_stat64 +st_line src/geo/geo_functions.h /^ static doris_udf::StringVal st_line($/;" f class:doris::GeoFunctions +st_line_prepare src/geo/geo_functions.h /^ static void st_line_prepare($/;" f class:doris::GeoFunctions +st_mode src/gutil/linux_syscall_support.h /^ unsigned st_mode;$/;" m struct:kernel_stat +st_mode src/gutil/linux_syscall_support.h /^ unsigned st_mode;$/;" m struct:kernel_stat64 +st_mode src/gutil/linux_syscall_support.h /^ unsigned int st_mode;$/;" m struct:kernel_stat +st_mode src/gutil/linux_syscall_support.h /^ unsigned long st_mode;$/;" m struct:kernel_stat +st_mode src/gutil/linux_syscall_support.h /^ unsigned short st_mode;$/;" m struct:kernel_stat +st_mtim src/gutil/linux_syscall_support.h /^ kernel_timespec st_mtim;$/;" m struct:kernel_stat +st_mtim src/gutil/linux_syscall_support.h /^ kernel_timespec st_mtim;$/;" m struct:kernel_stat64 +st_mtime_ src/gutil/linux_syscall_support.h /^ long st_mtime_;$/;" m struct:kernel_stat +st_mtime_ src/gutil/linux_syscall_support.h /^ uint64_t st_mtime_;$/;" m struct:kernel_stat +st_mtime_ src/gutil/linux_syscall_support.h /^ unsigned st_mtime_;$/;" m struct:kernel_stat +st_mtime_ src/gutil/linux_syscall_support.h /^ unsigned st_mtime_;$/;" m struct:kernel_stat64 +st_mtime_ src/gutil/linux_syscall_support.h /^ unsigned long st_mtime_;$/;" m struct:kernel_stat +st_mtime_nsec_ src/gutil/linux_syscall_support.h /^ long st_mtime_nsec_;$/;" m struct:kernel_stat +st_mtime_nsec_ src/gutil/linux_syscall_support.h /^ uint64_t st_mtime_nsec_;$/;" m struct:kernel_stat +st_mtime_nsec_ src/gutil/linux_syscall_support.h /^ unsigned st_mtime_nsec_;$/;" m struct:kernel_stat +st_mtime_nsec_ src/gutil/linux_syscall_support.h /^ unsigned st_mtime_nsec_;$/;" m struct:kernel_stat64 +st_mtime_nsec_ src/gutil/linux_syscall_support.h /^ unsigned long st_mtime_nsec_;$/;" m struct:kernel_stat +st_nlink src/gutil/linux_syscall_support.h /^ uint64_t st_nlink;$/;" m struct:kernel_stat +st_nlink src/gutil/linux_syscall_support.h /^ unsigned st_nlink;$/;" m struct:kernel_stat +st_nlink src/gutil/linux_syscall_support.h /^ unsigned st_nlink;$/;" m struct:kernel_stat64 +st_nlink src/gutil/linux_syscall_support.h /^ unsigned int st_nlink;$/;" m struct:kernel_stat +st_nlink src/gutil/linux_syscall_support.h /^ unsigned long st_nlink;$/;" m struct:kernel_stat +st_nlink src/gutil/linux_syscall_support.h /^ unsigned short st_nlink;$/;" m struct:kernel_stat +st_pad1 src/gutil/linux_syscall_support.h /^ int st_pad1[3];$/;" m struct:kernel_stat +st_pad2 src/gutil/linux_syscall_support.h /^ int st_pad2[2];$/;" m struct:kernel_stat +st_pad3 src/gutil/linux_syscall_support.h /^ int st_pad3;$/;" m struct:kernel_stat +st_pad4 src/gutil/linux_syscall_support.h /^ int st_pad4[14];$/;" m struct:kernel_stat +st_point src/geo/geo_functions.cpp /^doris_udf::StringVal GeoFunctions::st_point(doris_udf::FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_polygon src/geo/geo_functions.h /^ static doris_udf::StringVal st_polygon($/;" f class:doris::GeoFunctions +st_polygon_prepare src/geo/geo_functions.h /^ static void st_polygon_prepare($/;" f class:doris::GeoFunctions +st_rdev src/gutil/linux_syscall_support.h /^ uint64_t st_rdev;$/;" m struct:kernel_stat +st_rdev src/gutil/linux_syscall_support.h /^ unsigned st_rdev;$/;" m struct:kernel_stat +st_rdev src/gutil/linux_syscall_support.h /^ unsigned long st_rdev;$/;" m struct:kernel_stat +st_rdev src/gutil/linux_syscall_support.h /^ unsigned long long st_rdev;$/;" m struct:kernel_stat +st_rdev src/gutil/linux_syscall_support.h /^ unsigned long long st_rdev;$/;" m struct:kernel_stat64 +st_rdev src/gutil/linux_syscall_support.h /^ unsigned short st_rdev;$/;" m struct:kernel_stat +st_size src/gutil/linux_syscall_support.h /^ int64_t st_size;$/;" m struct:kernel_stat +st_size src/gutil/linux_syscall_support.h /^ long st_size;$/;" m struct:kernel_stat +st_size src/gutil/linux_syscall_support.h /^ long long st_size;$/;" m struct:kernel_stat +st_size src/gutil/linux_syscall_support.h /^ long long st_size;$/;" m struct:kernel_stat64 +st_size src/gutil/linux_syscall_support.h /^ unsigned st_size;$/;" m struct:kernel_stat +st_size src/gutil/linux_syscall_support.h /^ unsigned long st_size;$/;" m struct:kernel_stat +st_uid src/gutil/linux_syscall_support.h /^ unsigned st_uid;$/;" m struct:kernel_stat +st_uid src/gutil/linux_syscall_support.h /^ unsigned st_uid;$/;" m struct:kernel_stat64 +st_uid src/gutil/linux_syscall_support.h /^ unsigned int st_uid;$/;" m struct:kernel_stat +st_uid src/gutil/linux_syscall_support.h /^ unsigned short st_uid;$/;" m struct:kernel_stat +st_x src/geo/geo_functions.cpp /^DoubleVal GeoFunctions::st_x(doris_udf::FunctionContext* ctx,$/;" f class:doris::GeoFunctions +st_y src/geo/geo_functions.cpp /^DoubleVal GeoFunctions::st_y(doris_udf::FunctionContext* ctx,$/;" f class:doris::GeoFunctions +staging_input_vals src/udf/udf_internal.h /^ std::vector* staging_input_vals() { $/;" f class:doris::FunctionContextImpl +staging_input_vals_ src/exprs/new_agg_fn_evaluator.h /^ std::vector staging_input_vals_;$/;" m class:doris::NewAggFnEvaluator +staging_intermediate_val_ src/exprs/new_agg_fn_evaluator.h /^ doris_udf::AnyVal* staging_intermediate_val_ = nullptr;$/;" m class:doris::NewAggFnEvaluator +staging_merge_input_val_ src/exprs/new_agg_fn_evaluator.h /^ doris_udf::AnyVal* staging_merge_input_val_ = nullptr;$/;" m class:doris::NewAggFnEvaluator +standardlize_timevalue src/runtime/datetime_value.cpp /^int64_t DateTimeValue::standardlize_timevalue(int64_t value) {$/;" f class:doris::DateTimeValue +start src/agent/task_worker_pool.cpp /^void TaskWorkerPool::start() {$/;" f class:doris::TaskWorkerPool +start src/exec/schema_scanner.cpp /^Status SchemaScanner::start(RuntimeState* state) {$/;" f class:doris::SchemaScanner +start src/exec/schema_scanner/schema_columns_scanner.cpp /^Status SchemaColumnsScanner::start(RuntimeState *state) {$/;" f class:doris::SchemaColumnsScanner +start src/exec/schema_scanner/schema_dummy_scanner.cpp /^Status SchemaDummyScanner::start() {$/;" f class:doris::SchemaDummyScanner +start src/exec/schema_scanner/schema_schemata_scanner.cpp /^Status SchemaSchemataScanner::start(RuntimeState *state) {$/;" f class:doris::SchemaSchemataScanner +start src/exec/schema_scanner/schema_tables_scanner.cpp /^Status SchemaTablesScanner::start(RuntimeState *state) {$/;" f class:doris::SchemaTablesScanner +start src/exec/schema_scanner/schema_variables_scanner.cpp /^Status SchemaVariablesScanner::start(RuntimeState *state) {$/;" f class:doris::SchemaVariablesScanner +start src/http/ev_http_server.cpp /^Status EvHttpServer::start() {$/;" f class:doris::EvHttpServer +start src/service/brpc_service.cpp /^Status BRpcService::start(int port) {$/;" f class:doris::BRpcService +start src/service/http_service.cpp /^Status HttpService::start() {$/;" f class:doris::HttpService +start src/util/runtime_profile.h /^ void start() {$/;" f class:doris::RuntimeProfile::EventSequence +start src/util/runtime_profile.h /^ void start() {$/;" f class:doris::ScopedTimer +start src/util/stopwatch.hpp /^ void start() {$/;" f class:doris::MonotonicStopWatch +start src/util/thrift_server.cpp /^Status ThriftServer::start() {$/;" f class:doris::ThriftServer +start_all src/runtime/routine_load/data_consumer_group.cpp /^Status KafkaDataConsumerGroup::start_all(StreamLoadContext* ctx) {$/;" f class:doris::KafkaDataConsumerGroup +start_all src/runtime/routine_load/data_consumer_group.h /^ virtual Status start_all(StreamLoadContext* ctx) { return Status::OK(); }$/;" f class:doris::DataConsumerGroup +start_and_wait_for_server src/util/thrift_server.cpp /^Status ThriftServer::ThriftServerEventProcessor::start_and_wait_for_server() {$/;" f class:doris::ThriftServer::ThriftServerEventProcessor +start_backend test/runtime/data_stream_test.cpp /^ void start_backend() {$/;" f class:doris::DataStreamTest +start_bg_worker src/runtime/routine_load/data_consumer_pool.cpp /^Status DataConsumerPool::start_bg_worker() {$/;" f class:doris::DataConsumerPool +start_delete_unused_rowset src/olap/storage_engine.cpp /^void StorageEngine::start_delete_unused_rowset() {$/;" f class:doris::StorageEngine +start_job src/runtime/etl_job_mgr.cpp /^Status EtlJobMgr::start_job(const TMiniLoadEtlTaskRequest& req) {$/;" f class:doris::EtlJobMgr +start_key src/exec/tablet_info.h /^ Tuple* start_key = nullptr;$/;" m struct:doris::OlapTablePartition +start_key src/olap/reader.h /^ std::vector start_key;$/;" m struct:doris::ReaderParams +start_keys src/olap/reader.h /^ std::vector start_keys;$/;" m struct:doris::Reader::KeysParam +start_nanos src/runtime/stream_load/stream_load_context.h /^ int64_t start_nanos = 0;$/;" m class:doris::StreamLoadContext +start_plan_fragment_execution src/service/backend_service.cpp /^Status BackendService::start_plan_fragment_execution(const TExecPlanFragmentParams& exec_params) {$/;" f class:doris::BackendService +start_receiver test/runtime/data_stream_test.cpp /^ void start_receiver(TPartitionType::type stream_type, int num_senders, int receiver_num,$/;" f class:doris::DataStreamTest +start_scan src/exec/olap_scan_node.cpp /^Status OlapScanNode::start_scan(RuntimeState* state) {$/;" f class:doris::OlapScanNode +start_scan_thread src/exec/olap_scan_node.cpp /^Status OlapScanNode::start_scan_thread(RuntimeState* state) {$/;" f class:doris::OlapScanNode +start_scanners src/exec/broker_scan_node.cpp /^Status BrokerScanNode::start_scanners() {$/;" f class:doris::BrokerScanNode +start_scanners src/exec/es_http_scan_node.cpp /^Status EsHttpScanNode::start_scanners() {$/;" f class:doris::EsHttpScanNode +start_sender test/runtime/data_stream_test.cpp /^ void start_sender($/;" f class:doris::DataStreamTest +start_task src/runtime/export_task_mgr.cpp /^Status ExportTaskMgr::start_task(const TExportTaskRequest& request) {$/;" f class:doris::ExportTaskMgr +start_thread src/util/thread.cpp /^Status Thread::start_thread(const std::string& category, const std::string& name,$/;" f class:doris::Thread +start_time src/runtime/fragment_mgr.cpp /^ const DateTimeValue& start_time() const {$/;" f class:doris::FragmentExecState +start_trash_sweep src/olap/tablet_manager.cpp /^OLAPStatus TabletManager::start_trash_sweep() {$/;" f class:doris::TabletManager +start_version src/olap/rowset/rowset.h /^ int64_t start_version() const { return rowset_meta()->version().first; }$/;" f class:doris::Rowset +start_version src/olap/rowset/rowset_meta.h /^ int64_t start_version() const {$/;" f class:doris::RowsetMeta +starts_with src/exprs/string_functions.cpp /^BooleanVal StringFunctions::starts_with($/;" f class:doris::StringFunctions +starts_with src/gutil/strings/stringpiece.h /^ bool starts_with(StringPiece x) const {$/;" f class:StringPiece +starts_with src/olap/lru_cache.h /^ bool starts_with(const CacheKey& x) const {$/;" f class:doris::CacheKey +starts_with src/util/slice.h /^ bool starts_with(const Slice& x) const {$/;" f struct:doris::Slice +stat src/gutil/linux_syscall_support.h /^ LSS_INLINE int LSS_NAME(stat)(const char* p, struct kernel_stat* b) {$/;" f +state src/gutil/once.h /^ Atomic32 state;$/;" m struct:GoogleOnceType +state src/http/http_parser.h /^ int state; \/\/ Parse state$/;" m struct:doris::HttpChunkParseCtx +state src/olap/fs/file_block_manager.cpp /^WritableBlock::State FileWritableBlock::state() const {$/;" f class:doris::fs::internal::FileWritableBlock +state src/udf/udf_internal.h /^ RuntimeState* state() { $/;" f class:doris::FunctionContextImpl +state src/util/threadpool.h /^ State state() const { return _state; }$/;" f class:doris::ThreadPoolToken +state test/exprs/timestamp_functions_test.cpp /^ RuntimeState* state;$/;" m class:doris::TimestampFunctionsTest file: +state_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeState* state_;$/;" m class:doris::NewPartitionedAggregationNode +state_ src/exec/new_partitioned_hash_table.h /^ RuntimeState* state_;$/;" m class:doris::NewPartitionedHashTable +state_ src/gutil/once.h /^ Atomic32 state_;$/;" m class:GoogleOnceDynamic +state_ src/runtime/buffered_tuple_stream3.h /^ RuntimeState* const state_;$/;" m class:doris::BufferedTupleStream3 +state_to_string src/util/threadpool.cpp /^const char* ThreadPoolToken::state_to_string(State s) {$/;" f class:doris::ThreadPoolToken +static_cast_assign_vector src/olap/utils.h /^void static_cast_assign_vector(std::vector* v1, const std::vector& v2) {$/;" f namespace:doris +statistic_format src/olap/stream_index_common.h /^ uint32_t statistic_format; \/\/ 统计信息格式,实际上就是OLAP_FIELD_TYPE_XXX$/;" m struct:doris::StreamIndexHeader +stats src/olap/iterators.h /^ OlapReaderStatistics* stats = nullptr;$/;" m class:doris::StorageReadOptions +stats src/olap/reader.h /^ const OlapReaderStatistics& stats() const { return _stats; }$/;" f class:doris::Reader +stats src/olap/rowset/rowset_reader_context.h /^ OlapReaderStatistics* stats = nullptr;$/;" m struct:doris::RowsetReaderContext +stats src/olap/rowset/segment_v2/column_reader.h /^ OlapReaderStatistics* stats = nullptr;$/;" m struct:doris::segment_v2::ColumnIteratorOptions +stats src/olap/rowset/segment_v2/page_io.h /^ OlapReaderStatistics* stats = nullptr;$/;" m struct:doris::segment_v2::PageReadOptions +status src/http/http_response.h /^ HttpStatus status() const {$/;" f class:doris::HttpResponse +status src/plugin/plugin.h /^ void* status;$/;" m struct:doris::Plugin +status src/runtime/buffered_tuple_stream.h /^ Status status() const {$/;" f class:doris::BufferedTupleStream +status src/runtime/export_task_mgr.h /^ Status status;$/;" m struct:doris::ExportTaskCtx +status src/runtime/plan_fragment_executor.h /^ const Status& status() const {$/;" f class:doris::PlanFragmentExecutor +status src/runtime/record_batch_queue.h /^ Status status() {$/;" f class:doris::RecordBatchQueue +status src/runtime/stream_load/stream_load_context.h /^ Status status;$/;" m class:doris::StreamLoadContext +status test/runtime/data_stream_test.cpp /^ Status status;$/;" m struct:doris::DataStreamTest::ReceiverInfo file: +status test/runtime/data_stream_test.cpp /^ Status status;$/;" m struct:doris::DataStreamTest::SenderInfo file: +std src/gutil/port.h /^namespace std {} \/\/ Avoid error if we didn't see std.$/;" n +std src/gutil/strings/stringpiece.cc /^namespace std {$/;" n file: +std src/gutil/strings/stringpiece.h /^namespace std {$/;" n +std src/runtime/datetime_value.h /^namespace std {$/;" n +std src/runtime/decimal_value.h /^namespace std {$/;" n +std src/runtime/decimalv2_value.h /^namespace std {$/;" n +std src/runtime/dpp_sink_internal.h /^namespace std {$/;" n +std src/util/hash_util.hpp /^namespace std {$/;" n +std src/util/uid_util.h /^namespace std {$/;" n +stepping src/gutil/cpu.h /^ int stepping() const { return stepping_; }$/;" f class:base::final +stepping_ src/gutil/cpu.h /^ int stepping_; \/\/ processor revision number$/;" m class:base::final +stop src/http/ev_http_server.cpp /^void EvHttpServer::stop() {$/;" f class:doris::EvHttpServer +stop src/util/runtime_profile.h /^ void stop() {$/;" f class:doris::RuntimeProfile::EventSequence +stop src/util/runtime_profile.h /^ void stop() {$/;" f class:doris::ScopedTimer +stop src/util/stopwatch.hpp /^ void stop() {$/;" f class:doris::MonotonicStopWatch +stop src/util/thrift_server.cpp /^void ThriftServer::stop() {$/;" f class:doris::ThriftServer +stop_backend test/runtime/data_stream_test.cpp /^ void stop_backend() {$/;" f class:doris::DataStreamTest +stop_bucketing_counters_updates src/util/runtime_profile.cpp /^void RuntimeProfile::stop_bucketing_counters_updates($/;" f class:doris::RuntimeProfile +stop_for_testing src/util/thrift_server.cpp /^void ThriftServer::stop_for_testing() {$/;" f class:doris::ThriftServer +stop_rate_counters_updates src/util/runtime_profile.cpp /^void RuntimeProfile::stop_rate_counters_updates(Counter* rate_counter) {$/;" f class:doris::RuntimeProfile +stop_report_thread src/runtime/plan_fragment_executor.cpp /^void PlanFragmentExecutor::stop_report_thread() {$/;" f class:doris::PlanFragmentExecutor +stop_sampling_counters_updates src/util/runtime_profile.cpp /^void RuntimeProfile::stop_sampling_counters_updates(Counter* sampling_counter) {$/;" f class:doris::RuntimeProfile +storage src/common/configbase.h /^ void* storage;$/;" m struct:doris::config::Register::Field +storage_engine src/runtime/exec_env.h /^ StorageEngine* storage_engine() { return _storage_engine; }$/;" f class:doris::ExecEnv +storage_medium src/olap/data_dir.h /^ TStorageMedium::type storage_medium() const { return _storage_medium; }$/;" f class:doris::DataDir +storage_medium src/olap/olap_common.h /^ TStorageMedium::type storage_medium; \/\/ 存储介质类型:SSD|HDD$/;" m struct:doris::DataDirInfo +storage_medium src/olap/options.h /^ TStorageMedium::type storage_medium;$/;" m struct:doris::StorePath +storage_migrate_requests_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::storage_migrate_requests_total;$/;" m class:doris::DorisMetrics file: +storage_migrate_requests_total src/util/doris_metrics.h /^ static IntCounter storage_migrate_requests_total;$/;" m class:doris::DorisMetrics +storage_path src/olap/storage_engine.h /^ const std::string storage_path;$/;" m struct:doris::StorageEngine::CompactionDiskStat +store src/common/atomic.h /^ ALWAYS_INLINE void store(T x) {$/;" f class:doris::AtomicInt +store src/common/atomic.h /^ inline void store(T* val) { _ptr.store(reinterpret_cast(val)); }$/;" f class:doris::AtomicPtr +store_paths src/olap/options.h /^ std::vector store_paths;$/;" m struct:doris::EngineOptions +store_paths src/runtime/exec_env.h /^ const std::vector& store_paths() const { return _store_paths; }$/;" f class:doris::ExecEnv +stored_result src/util/once.h /^ ReturnType stored_result() const {$/;" f class:doris::DorisCallOnce +stores_duplicates src/exec/new_partitioned_hash_table.h /^ int stores_duplicates;$/;" m struct:doris::NewPartitionedHashTableCtx::HashTableReplacedConstants +stores_duplicates src/exec/new_partitioned_hash_table.h /^ bool IR_NO_INLINE stores_duplicates() const { return stores_duplicates_; }$/;" f class:doris::NewPartitionedHashTable +stores_duplicates_ src/exec/new_partitioned_hash_table.h /^ const bool stores_duplicates_;$/;" m class:doris::NewPartitionedHashTable +stores_nulls src/exec/new_partitioned_hash_table.h /^ int stores_nulls;$/;" m struct:doris::NewPartitionedHashTableCtx::HashTableReplacedConstants +stores_nulls src/exec/new_partitioned_hash_table.h /^ bool IR_NO_INLINE stores_nulls() const { return stores_nulls_; }$/;" f class:doris::NewPartitionedHashTableCtx +stores_nulls_ src/exec/new_partitioned_hash_table.h /^ const bool stores_nulls_;$/;" m class:doris::NewPartitionedHashTableCtx +stores_tuples src/exec/new_partitioned_hash_table.h /^ int stores_tuples;$/;" m struct:doris::NewPartitionedHashTableCtx::HashTableReplacedConstants +stores_tuples src/exec/new_partitioned_hash_table.h /^ bool IR_NO_INLINE stores_tuples() const { return stores_tuples_; }$/;" f class:doris::NewPartitionedHashTable +stores_tuples_ src/exec/new_partitioned_hash_table.h /^ const bool stores_tuples_;$/;" m class:doris::NewPartitionedHashTable +str src/olap/utils.cpp /^const char *Errno::str() {$/;" f class:doris::Errno +str src/olap/utils.cpp /^const char *Errno::str(int no) {$/;" f class:doris::Errno +str src/runtime/string_buffer.hpp /^ const StringValue& str() const {$/;" f class:doris::StringBuffer +str_to_date src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::str_to_date($/;" f class:doris::TimestampFunctions +str_to_int64 src/runtime/datetime_value.cpp /^static bool str_to_int64(const char* ptr, const char** endptr, int64_t *ret) {$/;" f namespace:doris +strategy src/olap/rowset/segment_v2/bloom_filter.h /^ HashStrategyPB strategy = HASH_MURMUR3_X64_64;$/;" m struct:doris::segment_v2::BloomFilterOptions +strcasecmp src/gutil/port.h 834;" d +strcaseprefix src/gutil/strings/util.h 117;" d +strcasestr_alnum src/gutil/strings/util.cc /^char *strcasestr_alnum(const char *haystack, const char *needle) {$/;" f +strcasesuffix src/gutil/strings/util.cc /^char* strcasesuffix(char* str, const char* suffix) {$/;" f +strcasesuffix src/gutil/strings/util.h /^inline const char* strcasesuffix(const char* str, const char* suffix) {$/;" f +strchr_sse src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline char* strchr_sse(const char* s, int c) {$/;" f namespace:doris +strchrnth src/gutil/strings/util.cc /^char* strchrnth(const char* str, const char& c, int n) {$/;" f +strcmp_sse src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline int strcmp_sse(const char* s1, const char* s2)$/;" f namespace:doris +strcmpcasenosensitive_internal src/util/minizip/unzip.c /^local int strcmpcasenosensitive_internal (const char* fileName1, const char* fileName2)$/;" f +strcount src/gutil/strings/util.h /^inline ptrdiff_t strcount(const char* buf, char c) {$/;" f +strcount src/gutil/strings/util.h /^inline ptrdiff_t strcount(const char* buf, size_t len, char c) {$/;" f +strcount src/gutil/strings/util.h /^inline ptrdiff_t strcount(const char* buf_begin, const char* buf_end, char c) {$/;" f +strcount src/gutil/strings/util.h /^inline ptrdiff_t strcount(const string& buf, char c) {$/;" f +strdup src/gutil/port.h 842;" d +strdup_nonempty src/gutil/strings/util.h /^inline char* strdup_nonempty(const char* src) {$/;" f +strdup_with_new src/gutil/strings/util.cc /^char* strdup_with_new(const char* the_string) {$/;" f +stream src/util/minizip/unzip.c /^ z_stream stream; \/* zLib stream structure for inflate *\/$/;" m struct:__anon33 file: +stream_factory src/olap/rowset/column_writer.h /^ OutStreamFactory* stream_factory() {$/;" f class:doris::ColumnWriter +stream_initialised src/util/minizip/unzip.c /^ uLong stream_initialised; \/* flag set if stream structure is initialised*\/$/;" m struct:__anon33 file: +stream_length src/olap/file_stream.h /^ uint64_t stream_length() {$/;" f class:doris::ReadOnlyFileStream +stream_length src/olap/in_stream.h /^ uint64_t stream_length() {$/;" f class:doris::InStream +stream_load src/exec/tablet_sink.cpp /^namespace stream_load {$/;" n namespace:doris file: +stream_load src/exec/tablet_sink.h /^namespace stream_load {$/;" n namespace:doris +stream_load test/exec/tablet_sink_test.cpp /^namespace stream_load {$/;" n namespace:doris file: +stream_load_ctx src/http/action/mini_load.cpp /^ StreamLoadContext* stream_load_ctx = nullptr;$/;" m struct:doris::MiniLoadCtx file: +stream_load_executor src/runtime/exec_env.h /^ StreamLoadExecutor* stream_load_executor() { return _stream_load_executor; }$/;" f class:doris::ExecEnv +stream_load_rows_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::stream_load_rows_total;$/;" m class:doris::DorisMetrics file: +stream_load_rows_total src/util/doris_metrics.h /^ static IntCounter stream_load_rows_total;$/;" m class:doris::DorisMetrics +stream_mgr src/runtime/exec_env.h /^ DataStreamMgr* stream_mgr() { return _stream_mgr; }$/;" f class:doris::ExecEnv +stream_receive_bytes_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::stream_receive_bytes_total;$/;" m class:doris::DorisMetrics file: +stream_receive_bytes_total src/util/doris_metrics.h /^ static IntCounter stream_receive_bytes_total;$/;" m class:doris::DorisMetrics +stream_recvr test/runtime/data_stream_test.cpp /^ boost::shared_ptr stream_recvr;$/;" m struct:doris::DataStreamTest::ReceiverInfo file: +stream_type test/runtime/data_stream_test.cpp /^ TPartitionType::type stream_type;$/;" m struct:doris::DataStreamTest::ReceiverInfo file: +streaming_ht_min_reduction src/exec/new_partitioned_aggregation_node.cc /^ double streaming_ht_min_reduction;$/;" m struct:doris::StreamingHtMinReductionEntry file: +streaming_timer_ src/exec/new_partitioned_aggregation_node.h /^ RuntimeProfile::Counter* streaming_timer_;$/;" m class:doris::NewPartitionedAggregationNode +streams src/olap/out_stream.h /^ const std::map& streams() const {$/;" f class:doris::OutStreamFactory +streq src/gutil/strings/util.h /^struct streq : public binary_function {$/;" s +strict_autodigit_greater src/gutil/strings/numbers.h /^struct strict_autodigit_greater$/;" s +strict_autodigit_less src/gutil/strings/numbers.h /^struct strict_autodigit_less$/;" s +string_as_array src/gutil/stl_util.h /^inline char* string_as_array(string* str) {$/;" f +string_at_index src/olap/rowset/segment_v2/binary_plain_page.h /^ Slice string_at_index(size_t idx) const {$/;" f class:doris::segment_v2::BinaryPlainPageDecoder +string_compare src/runtime/string_value.hpp /^static inline int string_compare(const char* s1, int n1, const char* s2, int n2, int len) {$/;" f namespace:doris +string_concat src/exprs/aggregate_functions.cpp /^void AggregateFunctions::string_concat(FunctionContext* ctx, const StringVal& src,$/;" f class:doris::AggregateFunctions +string_concat_finalize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::string_concat_finalize(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +string_concat_merge src/exprs/aggregate_functions.cpp /^void AggregateFunctions::string_concat_merge(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +string_concat_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::string_concat_update(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +string_data src/exprs/expr_value.h /^ std::string string_data;$/;" m struct:doris::ExprValue +string_grouping_exprs_ src/exec/new_partitioned_aggregation_node.h /^ std::vector string_grouping_exprs_;$/;" m class:doris::NewPartitionedAggregationNode +string_list src/exec/olap_scan_node.h /^ typedef boost::variant> string_list;$/;" t class:doris::OlapScanNode +string_printf_empty_block src/gutil/stringprintf.cc /^static const char string_printf_empty_block[256] = { '\\0' };$/;" v file: +string_result src/udf/udf_internal.h /^ std::string& string_result() {$/;" f class:doris::FunctionContextImpl +string_slots src/runtime/descriptors.h /^ const std::vector& string_slots() const {$/;" f class:doris::TupleDescriptor +string_to_bool src/util/string_parser.hpp /^ static inline bool string_to_bool(const char* s, int len, ParseResult* result) {$/;" f class:doris::StringParser +string_to_bool_internal src/util/string_parser.hpp /^inline bool StringParser::string_to_bool_internal(const char* s, int len, ParseResult* result) {$/;" f class:doris::StringParser +string_to_decimal src/util/string_parser.hpp /^inline __int128 StringParser::string_to_decimal(const char* s, int len,$/;" f class:doris::StringParser +string_to_float src/util/string_parser.hpp /^ static inline T string_to_float(const char* s, int len, ParseResult* result) {$/;" f class:doris::StringParser +string_to_float_internal src/util/string_parser.hpp /^inline T StringParser::string_to_float_internal(const char* s, int len, ParseResult* result) {$/;" f class:doris::StringParser +string_to_int src/util/string_parser.hpp /^ static inline T string_to_int(const char* s, int len, ParseResult* result) {$/;" f class:doris::StringParser +string_to_int src/util/string_parser.hpp /^ static inline T string_to_int(const char* s, int len, int base, ParseResult* result) {$/;" f class:doris::StringParser +string_to_int_internal src/util/string_parser.hpp /^inline T StringParser::string_to_int_internal($/;" f class:doris::StringParser +string_to_int_internal src/util/string_parser.hpp /^inline T StringParser::string_to_int_internal(const char* s, int len, ParseResult* result) {$/;" f class:doris::StringParser +string_to_int_no_overflow src/util/string_parser.hpp /^inline T StringParser::string_to_int_no_overflow(const char* s, int len, ParseResult* result) {$/;" f class:doris::StringParser +string_to_unsigned_int src/util/string_parser.hpp /^ static inline T string_to_unsigned_int(const char* s, int len, ParseResult* result) {$/;" f class:doris::StringParser +string_to_unsigned_int_internal src/util/string_parser.hpp /^inline T StringParser::string_to_unsigned_int_internal(const char* s, int len, ParseResult* result) {$/;" f class:doris::StringParser +string_type src/runtime/descriptor_helper.h /^ TSlotDescriptorBuilder& string_type(int len) {$/;" f class:doris::TSlotDescriptorBuilder +string_val src/exprs/expr_value.h /^ StringValue string_val;$/;" m struct:doris::ExprValue +string_val_get_value src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::string_val_get_value($/;" f class:doris::AggregateFunctions +string_val_serialize_or_finalize src/exprs/aggregate_functions.cpp /^StringVal AggregateFunctions::string_val_serialize_or_finalize($/;" f class:doris::AggregateFunctions +strings src/gutil/strings/charset.cc /^namespace strings {$/;" n file: +strings src/gutil/strings/charset.h /^namespace strings {$/;" n +strings src/gutil/strings/escaping.cc /^namespace strings {$/;" n file: +strings src/gutil/strings/escaping.h /^namespace strings {$/;" n +strings src/gutil/strings/fastmem.h /^namespace strings {$/;" n +strings src/gutil/strings/split.cc /^namespace strings {$/;" n file: +strings src/gutil/strings/split.h /^namespace strings {$/;" n +strings src/gutil/strings/split_internal.h /^namespace strings {$/;" n +strings src/gutil/strings/strcat.cc /^namespace strings {$/;" n file: +strings src/gutil/strings/strcat.h /^namespace strings {$/;" n +strings src/gutil/strings/strip.h /^namespace strings {$/;" n +strings src/gutil/strings/substitute.cc /^namespace strings {$/;" n file: +strings src/gutil/strings/substitute.h /^namespace strings {$/;" n +strings src/gutil/strings/util.cc /^namespace strings {$/;" n file: +strings src/gutil/strings/util.h /^namespace strings {$/;" n +strlcpy src/gutil/strings/util.cc /^size_t strings::strlcpy(char* dst, const char* src, size_t dst_size) {$/;" f class:strings +strlen_sse src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline size_t strlen_sse(const char* s) {$/;" f namespace:doris +strliterallen src/gutil/strings/memutil.h 105;" d +strlt src/gutil/strings/util.h /^struct strlt : public binary_function {$/;" s +strncasecmp src/gutil/port.h 835;" d +strncaseprefix src/gutil/strings/util.cc /^const char* strncaseprefix(const char* haystack, int haystack_size,$/;" f +strncasesuffix src/gutil/strings/util.cc /^const char* strncasesuffix(const char* haystack, int haystack_size,$/;" f +strnchr src/gutil/strings/util.h /^inline char* strnchr(const char* buf, char c, int sz) {$/;" f +strncmp_sse src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline int strncmp_sse(const char* s1, const char* s2, size_t n)$/;" f namespace:doris +strndup_with_new src/gutil/strings/util.cc /^char* strndup_with_new(const char* the_string, int max_length) {$/;" f +strnlen src/gutil/port.h /^inline size_t strnlen(const char *s, size_t maxlen) {$/;" f +strnprefix src/gutil/strings/util.cc /^const char* strnprefix(const char* haystack, int haystack_size,$/;" f +strnprefix src/gutil/strings/util.h 134;" d +strnstr src/gutil/strings/util.cc /^char* strnstr(const char* haystack, const char* needle,$/;" f +strnsuffix src/gutil/strings/util.cc /^const char* strnsuffix(const char* haystack, int haystack_size,$/;" f +strprefix src/gutil/strings/util.h 108;" d +strrchr_end_sse src/olap/bhp_lib.h /^inline char* strrchr_end_sse(char const* b, char const* e, char c) {$/;" f namespace:doris +strrchr_sse src/olap/bhp_lib.h /^\/*__attribute__((always_inline))*\/ inline char* strrchr_sse(const char* s, int c) {$/;" f namespace:doris +strrm src/gutil/strings/strip.cc /^int strrm(char* str, char c) {$/;" f +strrmm src/gutil/strings/strip.cc /^int strrmm(char* str, const char* chars) {$/;" f +strrmm src/gutil/strings/strip.cc /^int strrmm(string* str, const string& chars) {$/;" f +strstr_delimited src/gutil/strings/util.cc /^const char* strstr_delimited(const char* haystack,$/;" f +strsuffix src/gutil/strings/util.h /^inline char* strsuffix(char* str, const char* suffix) {$/;" f +strsuffix src/gutil/strings/util.h /^inline const char* strsuffix(const char* str, const char* suffix) {$/;" f +strto32 src/gutil/strtoint.h /^inline int32 strto32(const char *nptr, char **endptr, int base) {$/;" f +strto32_0 src/gutil/strings/split.cc /^static inline long strto32_0(const char * source, char ** end) {$/;" f file: +strto32_10 src/gutil/strings/split.cc /^static inline long strto32_10(const char * source, char ** end) {$/;" f file: +strto32_adapter src/gutil/strtoint.cc /^int32 strto32_adapter(const char *nptr, char **endptr, int base) {$/;" f +strto64 src/gutil/strtoint.h /^inline int64 strto64(const char *nptr, char **endptr, int base) {$/;" f +strto64_0 src/gutil/strings/split.cc /^static inline int64 strto64_0(const char * source, char ** end) {$/;" f file: +strto64_10 src/gutil/strings/split.cc /^static inline int64 strto64_10(const char * source, char ** end) {$/;" f file: +strtointeger src/common/configbase.cpp /^bool Properties::strtointeger(const std::string& valstr, T& retval) {$/;" f class:doris::config::Properties +strtoll src/gutil/port.h 812;" d +strtoq src/gutil/port.h 810;" d +strtoq src/gutil/port.h 85;" d +strtou32 src/gutil/strtoint.h /^inline uint32 strtou32(const char *nptr, char **endptr, int base) {$/;" f +strtou32_0 src/gutil/strings/split.cc /^static inline unsigned long strtou32_0(const char * source, char ** end) {$/;" f file: +strtou32_10 src/gutil/strings/split.cc /^static inline unsigned long strtou32_10(const char * source, char ** end) {$/;" f file: +strtou32_16 src/gutil/strings/split.cc /^static inline uint32 strtou32_16(const char * source, char ** end) {$/;" f file: +strtou32_adapter src/gutil/strtoint.cc /^uint32 strtou32_adapter(const char *nptr, char **endptr, int base) {$/;" f +strtou64 src/gutil/strtoint.h /^inline uint64 strtou64(const char *nptr, char **endptr, int base) {$/;" f +strtou64_0 src/gutil/strings/split.cc /^static inline uint64 strtou64_0(const char * source, char ** end) {$/;" f file: +strtou64_10 src/gutil/strings/split.cc /^static inline uint64 strtou64_10(const char * source, char ** end) {$/;" f file: +strtou64_16 src/gutil/strings/split.cc /^static inline uint64 strtou64_16(const char * source, char ** end) {$/;" f file: +strtoull src/gutil/port.h 813;" d +strtouq src/gutil/port.h 811;" d +strtouq src/gutil/port.h 86;" d +strtox src/common/configbase.cpp /^bool Properties::strtox(const std::string& valstr, bool& retval) {$/;" f class:doris::config::Properties +strtox src/common/configbase.cpp /^bool Properties::strtox(const std::string& valstr, double& retval) {$/;" f class:doris::config::Properties +strtox src/common/configbase.cpp /^bool Properties::strtox(const std::string& valstr, int16_t& retval) {$/;" f class:doris::config::Properties +strtox src/common/configbase.cpp /^bool Properties::strtox(const std::string& valstr, int32_t& retval) {$/;" f class:doris::config::Properties +strtox src/common/configbase.cpp /^bool Properties::strtox(const std::string& valstr, int64_t& retval) {$/;" f class:doris::config::Properties +strtox src/common/configbase.cpp /^bool Properties::strtox(const std::string& valstr, std::string& retval) {$/;" f class:doris::config::Properties +strtox src/common/configbase.cpp /^bool Properties::strtox(const std::string& valstr, std::vector& retval) {$/;" f class:doris::config::Properties +sub src/runtime/decimal_value.cpp /^inline void sub(const int32_t value1, const int32_t value2, int32_t* to, int32_t* carry) {$/;" f namespace:doris +sub2 src/runtime/decimal_value.cpp /^inline void sub2(const int32_t value1, const int32_t value2, int32_t* to, int32_t* carry) {$/;" f namespace:doris +sub_label src/http/action/mini_load.h /^ std::string sub_label;$/;" m struct:doris::LoadHandle +sub_label src/runtime/stream_load/stream_load_context.h /^ std::string sub_label;$/;" m class:doris::StreamLoadContext +sub_string src/exprs/udf_builtins.cpp /^StringVal UdfBuiltins::sub_string($/;" f class:doris::UdfBuiltins +submit src/olap/memtable_flush_executor.cpp /^OLAPStatus FlushToken::submit(const std::shared_ptr& memtable) {$/;" f class:doris::FlushToken +submit src/util/threadpool.cpp /^Status ThreadPool::submit(std::shared_ptr r) {$/;" f class:doris::ThreadPool +submit src/util/threadpool.cpp /^Status ThreadPoolToken::submit(std::shared_ptr r) {$/;" f class:doris::ThreadPoolToken +submit_etl_task src/agent/agent_server.cpp /^void AgentServer::submit_etl_task(TAgentResult& t_agent_result,$/;" f class:doris::AgentServer +submit_etl_task src/service/backend_service.h /^ virtual void submit_etl_task(TAgentResult& result,$/;" f class:doris::BackendService +submit_etl_task test/runtime/data_stream_test.cpp /^ virtual void submit_etl_task($/;" f class:doris::DorisTestBackend +submit_export_task src/service/backend_service.cpp /^void BackendService::submit_export_task(TStatus& t_status, const TExportTaskRequest& request) {$/;" f class:doris::BackendService +submit_func src/util/threadpool.cpp /^Status ThreadPool::submit_func(std::function f) {$/;" f class:doris::ThreadPool +submit_func src/util/threadpool.cpp /^Status ThreadPoolToken::submit_func(std::function f) {$/;" f class:doris::ThreadPoolToken +submit_routine_load_task src/service/backend_service.cpp /^void BackendService::submit_routine_load_task($/;" f class:doris::BackendService +submit_task src/agent/task_worker_pool.cpp /^void TaskWorkerPool::submit_task(const TAgentTaskRequest& task) {$/;" f class:doris::TaskWorkerPool +submit_task src/runtime/routine_load/routine_load_task_executor.cpp /^Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) {$/;" f class:doris::RoutineLoadTaskExecutor +submit_tasks src/agent/agent_server.cpp /^void AgentServer::submit_tasks(TAgentResult& agent_result, const vector& tasks) {$/;" f class:doris::AgentServer +submit_tasks src/service/backend_service.h /^ virtual void submit_tasks($/;" f class:doris::BackendService +submit_tasks test/runtime/data_stream_test.cpp /^ virtual void submit_tasks($/;" f class:doris::DorisTestBackend +submit_time src/util/threadpool.h /^ MonoTime submit_time;$/;" m struct:doris::ThreadPool::Task +submitter test/util/counter_cond_variable_test.cpp /^void submitter() {$/;" f namespace:doris +substr src/gutil/strings/stringpiece.cc /^StringPiece StringPiece::substr(size_type pos, size_type n) const {$/;" f class:StringPiece +substring src/exprs/string_functions.cpp /^StringVal StringFunctions::substring($/;" f class:doris::StringFunctions +substring src/runtime/string_value.hpp /^inline StringValue StringValue::substring(int start_pos) const {$/;" f class:doris::StringValue +substring src/runtime/string_value.hpp /^inline StringValue StringValue::substring(int start_pos, int new_len) const {$/;" f class:doris::StringValue +substring_pattern src/exprs/like_predicate.h /^ StringSearch substring_pattern;$/;" m struct:doris::LikePredicate::LikePredicateState +subtle src/gutil/atomicops-internals-gcc.h /^namespace subtle {$/;" n namespace:base +subtle src/gutil/atomicops-internals-tsan.h /^namespace subtle {$/;" n namespace:base +subtle src/gutil/atomicops-internals-x86.h /^namespace subtle {$/;" n namespace:base +subtle src/gutil/atomicops.h /^namespace subtle {$/;" n namespace:base +subtle src/gutil/gscoped_ptr.h /^namespace subtle {$/;" n namespace:doris +subtle src/gutil/ref_counted.cc /^namespace subtle {$/;" n namespace:doris file: +subtle src/gutil/ref_counted.h /^namespace subtle {$/;" n namespace:doris +sum src/exprs/aggregate_functions.cpp /^ DecimalV2Val sum;$/;" m struct:doris::DecimalV2AvgState file: +sum src/exprs/aggregate_functions.cpp /^ DecimalVal sum;$/;" m struct:doris::DecimalAvgState file: +sum src/exprs/aggregate_functions.cpp /^ double sum = 0;$/;" m struct:doris::AvgState file: +sum src/exprs/aggregate_functions.cpp /^void AggregateFunctions::sum(FunctionContext* ctx, const DecimalV2Val& src, DecimalV2Val* dst) {$/;" f class:doris::AggregateFunctions +sum src/exprs/aggregate_functions.cpp /^void AggregateFunctions::sum(FunctionContext* ctx, const DecimalVal& src, DecimalVal* dst) {$/;" f class:doris::AggregateFunctions +sum src/exprs/aggregate_functions.cpp /^void AggregateFunctions::sum(FunctionContext* ctx, const LargeIntVal& src, LargeIntVal* dst) {$/;" f class:doris::AggregateFunctions +sum src/exprs/aggregate_functions.cpp /^void AggregateFunctions::sum(FunctionContext* ctx, const SRC_VAL& src, DST_VAL* dst) {$/;" f class:doris::AggregateFunctions +sum_distinct_bigint_finalize src/exprs/aggregate_functions.cpp /^BigIntVal AggregateFunctions::sum_distinct_bigint_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +sum_distinct_data_filter src/exprs/agg_fn_evaluator.cpp /^bool AggFnEvaluator::sum_distinct_data_filter(TupleRow* row, Tuple* dst) {$/;" f class:doris::AggFnEvaluator +sum_distinct_decimal_finalize src/exprs/aggregate_functions.cpp /^DecimalVal AggregateFunctions::sum_distinct_decimal_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +sum_distinct_decimalv2_finalize src/exprs/aggregate_functions.cpp /^DecimalV2Val AggregateFunctions::sum_distinct_decimalv2_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +sum_distinct_double_finalize src/exprs/aggregate_functions.cpp /^DoubleVal AggregateFunctions::sum_distinct_double_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +sum_distinct_largeint_finalize src/exprs/aggregate_functions.cpp /^LargeIntVal AggregateFunctions::sum_distinct_largeint_finalize(FunctionContext* ctx, const StringVal& state_sv) {$/;" f class:doris::AggregateFunctions +sum_finalize src/exprs/aggregate_functions.cpp /^ DecimalV2Val sum_finalize() {$/;" f class:doris::MultiDistinctDecimalV2State +sum_finalize src/exprs/aggregate_functions.cpp /^ DecimalVal sum_finalize() {$/;" f class:doris::MultiDistinctDecimalState +sum_finalize_bigint src/exprs/aggregate_functions.cpp /^ BigIntVal sum_finalize_bigint() {$/;" f class:doris::MultiDistinctNumericState +sum_finalize_double src/exprs/aggregate_functions.cpp /^ DoubleVal sum_finalize_double() {$/;" f class:doris::MultiDistinctNumericState +sum_finalize_largeint src/exprs/aggregate_functions.cpp /^ LargeIntVal sum_finalize_largeint() {$/;" f class:doris::MultiDistinctNumericState +sum_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::sum_remove(FunctionContext* ctx, const DecimalV2Val& src,$/;" f class:doris::AggregateFunctions +sum_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::sum_remove(FunctionContext* ctx, const DecimalVal& src,$/;" f class:doris::AggregateFunctions +sum_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::sum_remove(FunctionContext* ctx, const SRC_VAL& src,$/;" f class:doris::AggregateFunctions +supervise src/util/thrift_server.cpp /^void ThriftServer::ThriftServerEventProcessor::supervise() {$/;" f class:doris::ThriftServer::ThriftServerEventProcessor +supervise_thread src/util/thread.cpp /^void* Thread::supervise_thread(void* arg) {$/;" f class:doris::Thread +support_ordinal_seek src/olap/rowset/segment_v2/indexed_column_reader.h /^ bool support_ordinal_seek() const { return _meta.has_ordinal_index_meta(); }$/;" f class:doris::segment_v2::IndexedColumnReader +support_value_seek src/olap/rowset/segment_v2/indexed_column_reader.h /^ bool support_value_seek() const { return _meta.has_value_index_meta(); }$/;" f class:doris::segment_v2::IndexedColumnReader +supports_serialize src/exprs/agg_fn_evaluator.h /^ bool supports_serialize() const {$/;" f class:doris::AggFnEvaluator +suppress src/olap/out_stream.h /^ void suppress() {$/;" f class:doris::OutStream +swap src/common/atomic.h /^ T swap(const T& new_val) {$/;" f class:doris::AtomicInt +swap src/gutil/gscoped_ptr.h /^ void swap(gscoped_array& p2) {$/;" f class:gscoped_array +swap src/gutil/gscoped_ptr.h /^ void swap(gscoped_ptr& p2) {$/;" f class:gscoped_ptr +swap src/gutil/gscoped_ptr.h /^ void swap(gscoped_ptr_impl& p2) {$/;" f class:doris::internal::gscoped_ptr_impl +swap src/gutil/gscoped_ptr.h /^ void swap(gscoped_ptr_malloc & b) {$/;" f class:gscoped_ptr_malloc +swap src/gutil/gscoped_ptr.h /^void swap(gscoped_array& p1, gscoped_array& p2) {$/;" f +swap src/gutil/gscoped_ptr.h /^void swap(gscoped_ptr& p1, gscoped_ptr& p2) {$/;" f +swap src/gutil/gscoped_ptr.h /^void swap(gscoped_ptr_malloc& a, gscoped_ptr_malloc& b) {$/;" f +swap src/gutil/ref_counted.h /^ void swap(T** pp) {$/;" f class:scoped_refptr +swap src/gutil/ref_counted.h /^ void swap(scoped_refptr& r) {$/;" f class:scoped_refptr +swap src/runtime/merge_sorter.cpp /^inline void MergeSorter::TupleSorter::swap(uint8_t* left, uint8_t* right) {$/;" f class:doris::MergeSorter::TupleSorter +swap src/runtime/spill_sorter.cc /^inline void SpillSorter::TupleSorter::swap(uint8_t* left, uint8_t* right) {$/;" f class:doris::SpillSorter::TupleSorter +swap src/util/bitmap_value.h /^ void swap(Roaring64Map& r) { roarings.swap(r.roarings); }$/;" f class:doris::detail::Roaring64Map +swap_big_endian src/olap/key_coder.h /^ static UnsignedCppType swap_big_endian(UnsignedCppType val) {$/;" f class:doris::KeyCoderTraits +switch_to_io_buffers src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::switch_to_io_buffers(bool* got_buffer) {$/;" f class:doris::BufferedTupleStream2 +syms src/util/bfd_parser.cpp /^ bfd_symbol** syms;$/;" m struct:doris::BfdFindCtx file: +sync src/olap/file_helper.h /^ int32_t sync() {$/;" f class:doris::FileHandler +sync src/olap/file_helper.h /^ int32_t sync() {$/;" f class:doris::FileHandlerWithBuf +sync_on_close src/env/env.h /^ bool sync_on_close = false;$/;" m struct:doris::RandomRWFileOptions +sync_on_close src/env/env.h /^ bool sync_on_close = false;$/;" m struct:doris::WritableFileOptions +sync_string_val src/exprs/expr_value.h /^ void sync_string_val() {$/;" f struct:doris::ExprValue +sys_perf_event_open src/util/perf_counters.cpp /^static inline int sys_perf_event_open($/;" f namespace:doris +system_alloc_cost_ns src/runtime/memory/chunk_allocator.cpp /^static IntCounter system_alloc_cost_ns;$/;" m namespace:doris file: +system_alloc_count src/runtime/memory/chunk_allocator.cpp /^static IntCounter system_alloc_count;$/;" m namespace:doris file: +system_allocator_ src/runtime/bufferpool/buffer_allocator.h /^ const boost::scoped_ptr system_allocator_;$/;" m struct:doris::BufferPool::BufferAllocator +system_bytes_limit src/runtime/bufferpool/buffer_allocator.h /^ int64_t system_bytes_limit() const { return system_bytes_limit_; }$/;" f struct:doris::BufferPool::BufferAllocator +system_bytes_limit_ src/runtime/bufferpool/buffer_allocator.h /^ const int64_t system_bytes_limit_;$/;" m struct:doris::BufferPool::BufferAllocator +system_bytes_remaining_ src/runtime/bufferpool/buffer_allocator.h /^ AtomicInt64 system_bytes_remaining_;$/;" m struct:doris::BufferPool::BufferAllocator +system_free_cost_ns src/runtime/memory/chunk_allocator.cpp /^static IntCounter system_free_cost_ns;$/;" m namespace:doris file: +system_free_count src/runtime/memory/chunk_allocator.cpp /^static IntCounter system_free_count;$/;" m namespace:doris file: +system_metrics src/util/doris_metrics.h /^ static SystemMetrics* system_metrics() { return _s_doris_metrics._system_metrics; }$/;" f class:doris::DorisMetrics +system_threads_quota src/runtime/thread_resource_mgr.h /^ int system_threads_quota() const {$/;" f class:doris::ThreadResourceMgr +t src/common/object_pool.h /^ T* t;$/;" m struct:doris::ObjectPool::SpecificElement +t_network_address_comparator src/util/thrift_util.cpp /^bool t_network_address_comparator(const TNetworkAddress& a, const TNetworkAddress& b) {$/;" f namespace:doris +t_network_address_to_string src/util/thrift_util.cpp /^void t_network_address_to_string(const TNetworkAddress& address, std::string* out) {$/;" f namespace:doris +table src/exec/schema_scanner.h /^ const std::string* table;$/;" m struct:doris::SchemaScannerParam +table src/runtime/stream_load/stream_load_context.h /^ std::string table;$/;" m class:doris::StreamLoadContext +table src/util/mysql_load_error_hub.h /^ std::string table;$/;" m struct:doris::MysqlLoadErrorHub::MysqlInfo +table0_ src/util/crc32c.cpp /^static const uint32_t table0_[256] = {$/;" m namespace:doris::crc32c file: +table1_ src/util/crc32c.cpp /^static const uint32_t table1_[256] = {$/;" m namespace:doris::crc32c file: +table2_ src/util/crc32c.cpp /^static const uint32_t table2_[256] = {$/;" m namespace:doris::crc32c file: +table3_ src/util/crc32c.cpp /^static const uint32_t table3_[256] = {$/;" m namespace:doris::crc32c file: +table_ src/exec/new_partitioned_hash_table.h /^ NewPartitionedHashTable* table_;$/;" m class:doris::NewPartitionedHashTable::Iterator +table_arr src/olap/tablet_manager.h /^ std::list table_arr;$/;" m struct:doris::TabletManager::TableInstances +table_desc src/runtime/descriptors.h /^ const TableDescriptor* table_desc() const {$/;" f class:doris::TupleDescriptor +table_id src/exec/tablet_info.h /^ int64_t table_id() const { return _t_param.table_id; }$/;" f class:doris::OlapTableLocationParam +table_id src/exec/tablet_info.h /^ int64_t table_id() const { return _t_param.table_id; }$/;" f class:doris::OlapTablePartitionParam +table_id src/exec/tablet_info.h /^ int64_t table_id() const { return _table_id; }$/;" f class:doris::OlapTableSchemaParam +table_id src/olap/tablet.h /^inline int64_t Tablet::table_id() const {$/;" f class:doris::Tablet +table_id src/olap/tablet_meta.h /^inline int64_t TabletMeta::table_id() const {$/;" f class:doris::TabletMeta +table_name src/runtime/descriptors.h /^ const std::string table_name() const { return table_name_; }$/;" f class:doris::KuduTableDescriptor +table_name_ src/runtime/descriptors.h /^ std::string table_name_;$/;" m class:doris::KuduTableDescriptor +table_num test/exec/schema_scanner/schema_columns_scanner_test.cpp /^int table_num = 0;$/;" m namespace:doris file: +table_num test/exec/schema_scanner/schema_create_table_scanner_test.cpp /^int table_num = 0;$/;" m namespace:doris file: +table_num test/exec/schema_scanner/schema_open_tables_scanner_test.cpp /^int table_num = 0;$/;" m namespace:doris file: +table_num test/exec/schema_scanner/schema_schemata_scanner_test.cpp /^int table_num = 0;$/;" m namespace:doris file: +table_num test/exec/schema_scanner/schema_table_names_scanner_test.cpp /^int table_num = 0;$/;" m namespace:doris file: +table_num test/exec/schema_scanner/schema_tables_scanner_test.cpp /^int table_num = 0;$/;" m namespace:doris file: +tablet src/olap/push_handler.h /^ TabletSharedPtr tablet;$/;" m struct:doris::TabletVars +tablet src/olap/reader.h /^ TabletSharedPtr tablet() { return _tablet; }$/;" f class:doris::Reader +tablet src/olap/reader.h /^ TabletSharedPtr tablet;$/;" m struct:doris::ReaderParams +tablet src/olap/rowset/column_data.h /^ Tablet* tablet() const { return _tablet; }$/;" f class:doris::ColumnData +tablet src/olap/tablet_sync_service.h /^ TabletSharedPtr tablet;$/;" m struct:doris::FetchRowsetMetaTask +tablet src/olap/tablet_sync_service.h /^ TabletSharedPtr tablet;$/;" m struct:doris::FetchTabletMetaTask +tablet test/exec/new_olap_scan_node_test.cpp /^ Tablet* tablet;$/;" m class:doris::TestOlapScanNode file: +tablet test/olap/delete_handler_test.cpp /^ TabletSharedPtr tablet;$/;" m class:doris::TestDeleteConditionHandler file: +tablet test/olap/delete_handler_test.cpp /^ TabletSharedPtr tablet;$/;" m class:doris::TestDeleteConditionHandler2 file: +tablet test/olap/delete_handler_test.cpp /^ TabletSharedPtr tablet;$/;" m class:doris::TestDeleteHandler file: +tablet_base_max_compaction_score src/util/doris_metrics.cpp /^IntGauge DorisMetrics::tablet_base_max_compaction_score;$/;" m class:doris::DorisMetrics file: +tablet_base_max_compaction_score src/util/doris_metrics.h /^ static IntGauge tablet_base_max_compaction_score;$/;" m class:doris::DorisMetrics +tablet_commit_infos src/runtime/runtime_state.h /^ const std::vector& tablet_commit_infos() const {$/;" f class:doris::RuntimeState +tablet_commit_infos src/runtime/runtime_state.h /^ std::vector& tablet_commit_infos() {$/;" f class:doris::RuntimeState +tablet_cumulative_max_compaction_score src/util/doris_metrics.cpp /^IntGauge DorisMetrics::tablet_cumulative_max_compaction_score;$/;" m class:doris::DorisMetrics file: +tablet_cumulative_max_compaction_score src/util/doris_metrics.h /^ static IntGauge tablet_cumulative_max_compaction_score;$/;" m class:doris::DorisMetrics +tablet_footprint src/olap/tablet.h /^inline size_t Tablet::tablet_footprint() {$/;" f class:doris::Tablet +tablet_footprint src/olap/tablet_meta.h /^inline size_t TabletMeta::tablet_footprint() const {$/;" f class:doris::TabletMeta +tablet_id src/olap/delta_writer.h /^ int64_t tablet_id;$/;" m struct:doris::WriteRequest +tablet_id src/olap/memtable.h /^ int64_t tablet_id() const { return _tablet_id; }$/;" f class:doris::MemTable +tablet_id src/olap/olap_common.h /^ TTabletId tablet_id;$/;" m struct:doris::TabletInfo +tablet_id src/olap/rowset/rowset_meta.h /^ int64_t tablet_id() const {$/;" f class:doris::RowsetMeta +tablet_id src/olap/rowset/rowset_writer_context.h /^ int64_t tablet_id;$/;" m struct:doris::RowsetWriterContext +tablet_id src/olap/storage_engine.h /^ int64_t tablet_id;$/;" m struct:doris::StorageEngine::CompactionCandidate +tablet_id src/olap/tablet.h /^inline int64_t Tablet::tablet_id() const {$/;" f class:doris::Tablet +tablet_id src/olap/tablet_meta.h /^inline int64_t TabletMeta::tablet_id() const {$/;" f class:doris::TabletMeta +tablet_id test/olap/txn_manager_test.cpp /^ TTabletId tablet_id = 222;$/;" m class:doris::TxnManagerTest file: +tablet_manager src/olap/storage_engine.h /^ TabletManager* tablet_manager() { return _tablet_manager.get(); }$/;" f class:doris::StorageEngine +tablet_map_t src/olap/tablet_manager.h /^ typedef std::unordered_map tablet_map_t;$/;" t class:doris::TabletManager +tablet_meta src/olap/tablet.h /^inline const TabletMetaSharedPtr Tablet::tablet_meta() {$/;" f class:doris::Tablet +tablet_meta_pb src/olap/tablet_sync_service.h /^ TabletMetaPB tablet_meta_pb;$/;" m struct:doris::PushTabletMetaTask +tablet_path src/olap/tablet.h /^inline string Tablet::tablet_path() const {$/;" f class:doris::Tablet +tablet_schema src/olap/row_block.h /^ const TabletSchema& tablet_schema() const { return *_schema; }$/;" f class:doris::RowBlock +tablet_schema src/olap/rowset/rowset_reader_context.h /^ const TabletSchema* tablet_schema = nullptr;$/;" m struct:doris::RowsetReaderContext +tablet_schema src/olap/rowset/rowset_writer_context.h /^ const TabletSchema* tablet_schema;$/;" m struct:doris::RowsetWriterContext +tablet_schema src/olap/rowset/segment_reader.h /^ inline const TabletSchema& tablet_schema() {$/;" f class:doris::SegmentReader +tablet_schema src/olap/tablet.h /^inline const TabletSchema& Tablet::tablet_schema() const {$/;" f class:doris::Tablet +tablet_schema src/olap/tablet_meta.h /^inline const TabletSchema& TabletMeta::tablet_schema() const {$/;" f class:doris::TabletMeta +tablet_schema_hash src/olap/rowset/rowset_meta.h /^ int32_t tablet_schema_hash() const {$/;" f class:doris::RowsetMeta +tablet_schema_hash src/olap/rowset/rowset_writer_context.h /^ int64_t tablet_schema_hash;$/;" m struct:doris::RowsetWriterContext +tablet_state src/olap/tablet.h /^ TabletState tablet_state() const { return _state; }$/;" f class:doris::Tablet +tablet_state src/olap/tablet_meta.h /^inline TabletState TabletMeta::tablet_state() const {$/;" f class:doris::TabletMeta +tablet_sync_service src/olap/storage_engine.h /^ TabletSyncService* tablet_sync_service() { return nullptr; }$/;" f class:doris::StorageEngine +tablet_uid src/olap/olap_common.h /^ UniqueId tablet_uid;$/;" m struct:doris::TabletInfo +tablet_uid src/olap/rowset/rowset_meta.h /^ TabletUid tablet_uid() const {$/;" f class:doris::RowsetMeta +tablet_uid src/olap/rowset/rowset_writer_context.h /^ TabletUid tablet_uid;$/;" m struct:doris::RowsetWriterContext +tablet_uid src/olap/tablet.h /^inline TabletUid Tablet::tablet_uid() const {$/;" f class:doris::Tablet +tablet_uid src/olap/tablet_meta.h /^inline TabletUid TabletMeta::tablet_uid() const {$/;" f class:doris::TabletMeta +tablet_writer_add_batch src/service/internal_service.cpp /^void PInternalServiceImpl::tablet_writer_add_batch(google::protobuf::RpcController* controller,$/;" f class:doris::PInternalServiceImpl +tablet_writer_cancel src/service/internal_service.cpp /^void PInternalServiceImpl::tablet_writer_cancel(google::protobuf::RpcController* controller,$/;" f class:doris::PInternalServiceImpl +tablet_writer_open src/service/internal_service.cpp /^void PInternalServiceImpl::tablet_writer_open(google::protobuf::RpcController* controller,$/;" f class:doris::PInternalServiceImpl +tail src/olap/rowset/segment_v2/column_writer.h /^ Page* tail = nullptr;$/;" m struct:doris::segment_v2::ColumnWriter::PageHead +tail src/runtime/bufferpool/buffer_pool_internal.h /^ Page* tail() { return list_.tail(); }$/;" f class:doris::BufferPool::PageList +tail src/util/internal_queue.h /^ T* tail() {$/;" f class:doris::InternalQueueBase +tail_ src/util/internal_queue.h /^ Node *head_, *tail_;$/;" m class:doris::InternalQueueBase +targetQuantile src/exprs/aggregate_functions.cpp /^ double targetQuantile = -1.0;$/;" m struct:doris::PercentileApproxState file: +task_remaining src/olap/storage_engine.h /^ uint32_t task_remaining;$/;" m struct:doris::StorageEngine::CompactionDiskStat +task_running src/olap/storage_engine.h /^ uint32_t task_running;$/;" m struct:doris::StorageEngine::CompactionDiskStat +tcmalloc_gc_thread src/common/daemon.cpp /^void* tcmalloc_gc_thread(void* dummy) {$/;" f namespace:doris +tear_down test/olap/delete_handler_test.cpp /^void tear_down() {$/;" f namespace:doris +tear_down test/olap/delta_writer_test.cpp /^void tear_down() {$/;" f namespace:doris +tear_down test/olap/memtable_flush_executor_test.cpp /^void tear_down() {$/;" f namespace:doris +tear_down test/olap/olap_reader_test.cpp /^void tear_down() {$/;" f namespace:doris +tear_down test/olap/options_test.cpp /^void tear_down() { system("rm -rf .\/test_run"); }$/;" f namespace:doris +tear_down test/olap/rowset/alpha_rowset_test.cpp /^void tear_down() {$/;" f namespace:doris +tear_down test/olap/vectorized_olap_reader_test.cpp /^void tear_down() {$/;" f namespace:doris +tear_down_query_states src/runtime/test_env.cc /^void TestEnv::tear_down_query_states() {$/;" f class:doris::TestEnv +tell src/exec/broker_reader.cpp /^Status BrokerReader::tell(int64_t* position) {$/;" f class:doris::BrokerReader +tell src/exec/local_file_reader.cpp /^Status LocalFileReader::tell(int64_t* position) {$/;" f class:doris::LocalFileReader +tell src/olap/file_helper.h /^ off_t tell() const {$/;" f class:doris::FileHandler +tell src/olap/file_helper.h /^ off_t tell() const {$/;" f class:doris::FileHandlerWithBuf +tell_file_func src/util/minizip/ioapi.h /^typedef long (ZCALLBACK *tell_file_func) OF((voidpf opaque, voidpf stream));$/;" t +templated_in src/exprs/new_in_predicate.cpp /^BooleanVal InPredicate::templated_in($/;" f class:doris::InPredicate +tempnam src/gutil/port.h 843;" d +test test/util/utf8_check_test.cpp /^struct test {$/;" s namespace:doris file: +test_all_float_variants test/util/string_parser_test.cpp /^void test_all_float_variants(const std::string& s, StringParser::ParseResult exp_result) {$/;" f namespace:doris +test_base64 test/util/url_coding_test.cpp /^void test_base64(const string& input, const string& expected_encoded) {$/;" f namespace:doris +test_bit src/olap/selection_vector.h /^ bool test_bit(size_t row_idx) {$/;" f class:doris::SelectionVectorView +test_bitmap_intersect test/exprs/bitmap_function_test.cpp /^void test_bitmap_intersect(FunctionContext* ctx, ValType key1, ValType key2) {$/;" f namespace:doris +test_bloom_filter_index_reader_writer_template test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^void test_bloom_filter_index_reader_writer_template(const std::string file_name,$/;" f namespace:doris::segment_v2 +test_bool_value test/util/string_parser_test.cpp /^void test_bool_value(const char* s, bool exp_val, StringParser::ParseResult exp_result) {$/;" f namespace:doris +test_by_small_data_size test/olap/rowset/segment_v2/binary_dict_page_test.cpp /^ void test_by_small_data_size(const std::vector& slices) {$/;" f class:doris::segment_v2::BinaryDictPageTest +test_bytes src/olap/bloom_filter.hpp /^ bool test_bytes(const char* buf, uint32_t len) const {$/;" f class:doris::BloomFilter +test_bytes src/olap/rowset/segment_v2/bloom_filter.h /^ bool test_bytes(char* buf, uint32_t size) const {$/;" f class:doris::segment_v2::BloomFilter +test_char test/olap/storage_types_test.cpp /^void test_char(Slice src_val) {$/;" f namespace:doris +test_convert_from_varchar test/olap/schema_change_test.cpp /^ void test_convert_from_varchar(const std::string& type_name, int type_size, const std::string& value, OLAPStatus expected_st) {$/;" f class:doris::TestColumn +test_convert_to_varchar test/olap/schema_change_test.cpp /^ void test_convert_to_varchar(const std::string& type_name, int type_size, T val, const std::string& expected_val, OLAPStatus expected_st) {$/;" f class:doris::TestColumn +test_encode_and_decode test/olap/rowset/segment_v2/binary_prefix_page_test.cpp /^ void test_encode_and_decode() {$/;" f class:doris::segment_v2::BinaryPrefixPageTest +test_encode_and_decode2 test/olap/rowset/segment_v2/binary_prefix_page_test.cpp /^ void test_encode_and_decode2() {$/;" f class:doris::segment_v2::BinaryPrefixPageTest +test_encode_decode_page_template test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^ void test_encode_decode_page_template(typename TypeTraits::CppType* src,$/;" f class:doris::BitShufflePageTest +test_encode_decode_page_template test/olap/rowset/segment_v2/bloom_filter_page_test.cpp /^ void test_encode_decode_page_template(typename TypeTraits::CppType* src,$/;" f class:doris::segment_v2::BloomFilterPageTest +test_encode_decode_page_template test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp /^ void test_encode_decode_page_template(typename TypeTraits::CppType* src,$/;" f class:doris::FrameOfReferencePageTest +test_encode_decode_page_template test/olap/rowset/segment_v2/plain_page_test.cpp /^ void test_encode_decode_page_template(typename TypeTraits::CppType* src,$/;" f class:doris::segment_v2::PlainPageTest +test_encode_decode_page_template test/olap/rowset/segment_v2/rle_page_test.cpp /^ void test_encode_decode_page_template(typename TypeTraits::CppType* src,$/;" f class:doris::RlePageTest +test_float_value test/util/string_parser_test.cpp /^void test_float_value(const std::string& s, StringParser::ParseResult exp_result) {$/;" f namespace:doris +test_float_value_is_nan test/util/string_parser_test.cpp /^void test_float_value_is_nan(const std::string& s, StringParser::ParseResult exp_result) {$/;" f namespace:doris +test_frame_of_reference_encode_decode test/util/frame_of_reference_coding_test.cpp /^ static void test_frame_of_reference_encode_decode(int32_t element_size) {$/;" f class:doris::TestForCoding +test_hash src/olap/bloom_filter.hpp /^ bool test_hash(uint64_t hash) const {$/;" f class:doris::BloomFilter +test_hash src/olap/rowset/segment_v2/block_split_bloom_filter.cpp /^bool BlockSplitBloomFilter::test_hash(uint64_t hash) const {$/;" f class:doris::segment_v2::BlockSplitBloomFilter +test_int_value test/util/string_parser_test.cpp /^void test_int_value(const char* s, T exp_val, StringParser::ParseResult exp_result) {$/;" f namespace:doris +test_int_value test/util/string_parser_test.cpp /^void test_int_value(const char* s, int base, T exp_val, StringParser::ParseResult exp_result) {$/;" f namespace:doris +test_integer_encode test/olap/key_coder_test.cpp /^void test_integer_encode() {$/;" f namespace:doris +test_max test/olap/aggregate_func_test.cpp /^void test_max() {$/;" f namespace:doris +test_min test/olap/aggregate_func_test.cpp /^void test_min() {$/;" f namespace:doris +test_multi_slices test/util/block_compression_test.cpp /^void test_multi_slices(segment_v2::CompressionTypePB type) {$/;" f namespace:doris +test_normal test/runtime/memory/system_allocator_test.cpp /^void test_normal() {$/;" f namespace:doris +test_nullable_data test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, string test_name) {$/;" f namespace:doris::segment_v2 +test_parse_mem_spec test/util/parse_util_test.cpp /^static void test_parse_mem_spec(const std::string& mem_spec_str, int64_t result) {$/;" f namespace:doris +test_read_default_value test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^void test_read_default_value(string value, void* result) {$/;" f namespace:doris::segment_v2 +test_replace test/olap/aggregate_func_test.cpp /^void test_replace() {$/;" f namespace:doris +test_replace_string test/olap/aggregate_func_test.cpp /^void test_replace_string() {$/;" f namespace:doris +test_seek_at_or_after_value_template test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^ void test_seek_at_or_after_value_template(typename TypeTraits::CppType* src,$/;" f class:doris::BitShufflePageTest +test_seek_at_or_after_value_template test/olap/rowset/segment_v2/plain_page_test.cpp /^ void test_seek_at_or_after_value_template(typename TypeTraits::CppType* src,$/;" f class:doris::segment_v2::PlainPageTest +test_shutdown src/runtime/client_cache.cpp /^void ClientCacheHelper::test_shutdown() {$/;" f class:doris::ClientCacheHelper +test_shutdown src/runtime/client_cache.h /^ void test_shutdown() {$/;" f class:doris::ClientCache +test_single_slice test/util/block_compression_test.cpp /^void test_single_slice(segment_v2::CompressionTypePB type) {$/;" f namespace:doris +test_skip test/util/frame_of_reference_coding_test.cpp /^ static void test_skip(int32_t skip_num) {$/;" f class:doris::TestForCoding +test_str test/util/arrow/arrow_row_batch_test.cpp /^std::string test_str() {$/;" f namespace:doris +test_str test/util/arrow/arrow_row_block_test.cpp /^std::string test_str() {$/;" f namespace:doris +test_stream test/runtime/data_stream_test.cpp /^ void test_stream(TPartitionType::type stream_type, int num_senders,$/;" f class:doris::DataStreamTest +test_string test/olap/rowset/segment_v2/zone_map_index_test.cpp /^ void test_string(std::string testname, Field* field) {$/;" f class:doris::segment_v2::ColumnZoneMapTest +test_sum test/olap/aggregate_func_test.cpp /^void test_sum() {$/;" f namespace:doris +test_unsigned_int_value test/util/string_parser_test.cpp /^void test_unsigned_int_value(const char* s, T exp_val, StringParser::ParseResult exp_result) {$/;" f namespace:doris +test_url test/util/url_coding_test.cpp /^void test_url(const string& input, const string& expected_encoded, bool hive_compat) {$/;" f namespace:doris +test_with_large_data_size test/olap/rowset/segment_v2/binary_dict_page_test.cpp /^ void test_with_large_data_size(const std::vector& contents) {$/;" f class:doris::segment_v2::BinaryDictPageTest +testerror_file_func src/util/minizip/ioapi.h /^typedef int (ZCALLBACK *testerror_file_func) OF((voidpf opaque, voidpf stream));$/;" t +text_ src/gutil/strings/split_internal.h /^ StringPiece text_;$/;" m class:strings::internal::SplitIterator +text_ src/gutil/strings/substitute.h /^ const char* text_;$/;" m class:strings::internal::SubstituteArg +thread_available_cb src/exec/kudu_scan_node.cpp /^void KuduScanNode::thread_available_cb(ThreadResourceMgr::ResourcePool* pool) {$/;" f class:doris::KuduScanNode +thread_available_cb src/runtime/thread_resource_mgr.h /^ typedef boost::function thread_available_cb;$/;" t class:doris::ThreadResourceMgr +thread_handle test/runtime/data_stream_test.cpp /^ thread* thread_handle;$/;" m struct:doris::DataStreamTest::ReceiverInfo file: +thread_handle test/runtime/data_stream_test.cpp /^ thread* thread_handle;$/;" m struct:doris::DataStreamTest::SenderInfo file: +thread_id src/exec/schema_scanner.h /^ int64_t thread_id;$/;" m struct:doris::SchemaScannerParam +thread_id src/util/thread.cpp /^ int64_t thread_id() const { return _thread_id; }$/;" f class:doris::ThreadMgr::ThreadDescriptor +thread_manager src/util/thread.cpp /^static std::shared_ptr thread_manager;$/;" m namespace:doris file: +thread_mgr src/runtime/exec_env.h /^ ThreadResourceMgr* thread_mgr() { return _thread_mgr; }$/;" f class:doris::ExecEnv +thread_pool src/runtime/exec_env.h /^ PriorityThreadPool* thread_pool() { return _thread_pool; }$/;" f class:doris::ExecEnv +thrift_output src/service/doris_main.cpp /^static void thrift_output(const char* x) {$/;" f namespace:doris +thrift_output_function src/util/thrift_util.cpp /^static void thrift_output_function(const char* output) {$/;" f namespace:doris +thrift_to_type src/runtime/primitive_type.cpp /^PrimitiveType thrift_to_type(TPrimitiveType::type ttype) {$/;" f namespace:doris +tid src/util/thread.cpp /^int64_t Thread::tid() const {$/;" f class:doris::Thread +tight_enum_cast src/gutil/casts.h /^inline Enum tight_enum_cast(int e_val) {$/;" f +tight_enum_test src/gutil/casts.h /^inline bool tight_enum_test(int e_val) {$/;" f +tight_enum_test_cast src/gutil/casts.h /^inline bool tight_enum_test_cast(int e_val, Enum* e_var) {$/;" f +time_diff src/exprs/timestamp_functions.cpp /^DoubleVal TimestampFunctions::time_diff($/;" f class:doris::TimestampFunctions +time_part_diff src/runtime/datetime_value.h /^ int64_t time_part_diff(const DateTimeValue& rhs) const {$/;" f class:doris::DateTimeValue +time_str_from_double src/util/date_func.cpp /^std::string time_str_from_double(double time) {$/;" f namespace:doris +time_to_string test/udf/udf_test.cpp /^StringVal time_to_string(FunctionContext* context, const TimestampVal& time) {$/;" f namespace:doris_udf +timeout src/runtime/load_channel.h /^ int64_t timeout() const { return _timeout_s; }$/;" f class:doris::LoadChannel +timeout_second src/runtime/stream_load/stream_load_context.h /^ int32_t timeout_second = -1;$/;" m class:doris::StreamLoadContext +timestamp src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::timestamp($/;" f class:doris::TimestampFunctions +timestamp_avg_finalize src/exprs/aggregate_functions.cpp /^DateTimeVal AggregateFunctions::timestamp_avg_finalize(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +timestamp_avg_get_value src/exprs/aggregate_functions.cpp /^DateTimeVal AggregateFunctions::timestamp_avg_get_value(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +timestamp_avg_remove src/exprs/aggregate_functions.cpp /^void AggregateFunctions::timestamp_avg_remove(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +timestamp_avg_update src/exprs/aggregate_functions.cpp /^void AggregateFunctions::timestamp_avg_update(FunctionContext* ctx,$/;" f class:doris::AggregateFunctions +timestamp_diff src/exprs/timestamp_functions.cpp /^BigIntVal TimestampFunctions::timestamp_diff(FunctionContext* ctx, const DateTimeVal& ts_val2, const DateTimeVal& ts_val1) {$/;" f class:doris::TimestampFunctions +timestamp_from_date src/util/date_func.cpp /^uint24_t timestamp_from_date(const std::string& date_str) {$/;" f namespace:doris +timestamp_from_date test/olap/in_list_predicate_test.cpp /^static uint24_t timestamp_from_date(const char* date_string) {$/;" f namespace:doris::datetime +timestamp_from_datetime src/util/date_func.cpp /^uint64_t timestamp_from_datetime(const std::string& datetime_str) {$/;" f namespace:doris +timestamp_from_datetime test/olap/in_list_predicate_test.cpp /^static uint64_t timestamp_from_datetime(const std::string& value_string) {$/;" f namespace:doris::datetime +timestamp_ms src/runtime/runtime_state.h /^ int64_t timestamp_ms() const {$/;" f class:doris::RuntimeState +timestamp_time_op src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::timestamp_time_op($/;" f class:doris::TimestampFunctions +timezone src/runtime/runtime_state.h /^ const std::string& timezone() const {$/;" f class:doris::RuntimeState +tiny_val test/runtime/dpp_writer_test.cpp /^ int8_t tiny_val;$/;" m struct:doris::TestDataTuple file: +tiny_val test/runtime/mysql_table_writer_test.cpp /^ int8_t tiny_val;$/;" m struct:doris::TestDataTuple file: +tinyint_val src/exprs/expr_value.h /^ int8_t tinyint_val;$/;" m struct:doris::ExprValue +tm_hour src/util/minizip/unzip.h /^ uInt tm_hour; \/* hours since midnight - [0,23] *\/$/;" m struct:tm_unz_s +tm_mday src/util/minizip/unzip.h /^ uInt tm_mday; \/* day of the month - [1,31] *\/$/;" m struct:tm_unz_s +tm_min src/util/minizip/unzip.h /^ uInt tm_min; \/* minutes after the hour - [0,59] *\/$/;" m struct:tm_unz_s +tm_mon src/util/minizip/unzip.h /^ uInt tm_mon; \/* months since January - [0,11] *\/$/;" m struct:tm_unz_s +tm_sec src/util/minizip/unzip.h /^ uInt tm_sec; \/* seconds after the minute - [0,59] *\/$/;" m struct:tm_unz_s +tm_unz src/util/minizip/unzip.h /^} tm_unz;$/;" t typeref:struct:tm_unz_s +tm_unz_s src/util/minizip/unzip.h /^typedef struct tm_unz_s$/;" s +tm_year src/util/minizip/unzip.h /^ uInt tm_year; \/* years - [1980..2044] *\/$/;" m struct:tm_unz_s +tmp_file_mgr src/runtime/exec_env.h /^ TmpFileMgr* tmp_file_mgr() { return _tmp_file_mgr; }$/;" f class:doris::ExecEnv +tmp_file_mgr src/runtime/test_env.h /^ TmpFileMgr* tmp_file_mgr() {$/;" f class:doris::TestEnv +tmp_file_path src/runtime/buffered_block_mgr2.cc /^string BufferedBlockMgr2::Block::tmp_file_path() const {$/;" f class:doris::BufferedBlockMgr2::Block +tmu_date src/util/minizip/unzip.h /^ tm_unz tmu_date;$/;" m struct:unz_file_info64_s +tmu_date src/util/minizip/unzip.h /^ tm_unz tmu_date;$/;" m struct:unz_file_info_s +to src/gutil/spinlock_internal.h /^ int32 to;$/;" m struct:base::internal::SpinLockWaitTransition +to src/olap/rowset/segment_v2/row_ranges.h /^ int64_t to() const {$/;" f class:doris::segment_v2::RowRange +to src/olap/rowset/segment_v2/row_ranges.h /^ int64_t to() {$/;" f class:doris::segment_v2::RowRanges +toString src/util/bitmap_value.h /^ std::string toString() const {$/;" f class:doris::detail::Roaring64Map +toUint64Array src/util/bitmap_value.h /^ void toUint64Array(uint64_t* ans) const {$/;" f class:doris::detail::Roaring64Map +to_abs_value src/runtime/decimal_value.h /^ void to_abs_value() {$/;" f class:doris::DecimalValue +to_abs_value src/runtime/decimalv2_value.h /^ void to_abs_value() {$/;" f class:doris::DecimalV2Value +to_alter_pb src/olap/tablet_meta.cpp /^OLAPStatus AlterTabletTask::to_alter_pb(AlterTabletPB* alter_task) {$/;" f class:doris::AlterTabletTask +to_alter_tablet_pb src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::to_alter_tablet_pb(const SchemaChangeStatusMessage& schema_change_msg,$/;" f class:doris::OlapSnapshotConverter +to_base64 src/exprs/encryption_functions.cpp /^StringVal EncryptionFunctions::to_base64(FunctionContext* ctx, const StringVal &src) {$/;" f class:doris::EncryptionFunctions +to_bitmap src/exprs/bitmap_function.cpp /^StringVal BitmapFunctions::to_bitmap(doris_udf::FunctionContext* ctx, const doris_udf::StringVal& src) {$/;" f class:doris::BitmapFunctions +to_code src/http/http_status.cpp /^std::string to_code(const HttpStatus& status) {$/;" f namespace:doris +to_column_msg src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::to_column_msg(const ColumnPB& column_pb, ColumnMessage* column_msg) {$/;" f class:doris::OlapSnapshotConverter +to_column_pb src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::to_column_pb(const ColumnMessage& column_msg, ColumnPB* column_pb) {$/;" f class:doris::OlapSnapshotConverter +to_date src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::to_date($/;" f class:doris::TimestampFunctions +to_date_int64 src/runtime/datetime_value.cpp /^int64_t DateTimeValue::to_date_int64() const {$/;" f class:doris::DateTimeValue +to_date_string src/runtime/datetime_value.cpp /^char* DateTimeValue::to_date_string(char* to) const {$/;" f class:doris::DateTimeValue +to_date_string test/olap/comparison_predicate_test.cpp /^static std::string to_date_string(uint24_t& date_value) {$/;" f namespace:doris::datetime +to_date_string test/olap/in_list_predicate_test.cpp /^static std::string to_date_string(uint24_t& date_value) {$/;" f namespace:doris::datetime +to_date_timestamp test/olap/comparison_predicate_test.cpp /^static uint24_t to_date_timestamp(const char* date_string) {$/;" f namespace:doris::datetime +to_date_timestamp test/olap/null_predicate_test.cpp /^static uint24_t to_date_timestamp(const char* date_string) {$/;" f namespace:doris::datetime +to_datetime src/runtime/datetime_value.h /^ void to_datetime() {$/;" f class:doris::DateTimeValue +to_datetime_int64 src/runtime/datetime_value.cpp /^int64_t DateTimeValue::to_datetime_int64() const {$/;" f class:doris::DateTimeValue +to_datetime_string src/runtime/datetime_value.cpp /^char* DateTimeValue::to_datetime_string(char* to) const {$/;" f class:doris::DateTimeValue +to_datetime_string test/olap/comparison_predicate_test.cpp /^static std::string to_datetime_string(uint64_t& datetime_value) {$/;" f namespace:doris::datetime +to_datetime_string test/olap/in_list_predicate_test.cpp /^static std::string to_datetime_string(uint64_t& datetime_value) {$/;" f namespace:doris::datetime +to_datetime_timestamp test/olap/comparison_predicate_test.cpp /^static uint64_t to_datetime_timestamp(const std::string& value_string) {$/;" f namespace:doris::datetime +to_datetime_timestamp test/olap/null_predicate_test.cpp /^static uint64_t to_datetime_timestamp(const std::string& value_string) {$/;" f namespace:doris::datetime +to_datetime_val src/runtime/datetime_value.h /^ void to_datetime_val(doris_udf::DateTimeVal* tv) const {$/;" f class:doris::DateTimeValue +to_days src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::to_days($/;" f class:doris::TimestampFunctions +to_decimal_val src/runtime/decimal_value.h /^ void to_decimal_val(doris_udf::DecimalVal* value) const {$/;" f class:doris::DecimalValue +to_decimal_val src/runtime/decimalv2_value.h /^ void to_decimal_val(DecimalV2Val* value) const {$/;" f class:doris::DecimalV2Value +to_ext_literal src/exec/es_scan_node.cpp /^bool EsScanNode::to_ext_literal(ExprContext* context, Expr* expr, TExtLiteral* literal) {$/;" f class:doris::EsScanNode +to_ext_literal src/exec/es_scan_node.cpp /^bool EsScanNode::to_ext_literal(PrimitiveType slot_type, void* value, TExtLiteral* literal) {$/;" f class:doris::EsScanNode +to_format_string src/runtime/datetime_value.cpp /^bool DateTimeValue::to_format_string(const char* format, int len, char* to) const {$/;" f class:doris::DateTimeValue +to_hex src/util/uid_util.h /^inline void to_hex(T val, char* buf) {$/;" f namespace:doris +to_http_method src/http/http_method.cpp /^HttpMethod to_http_method(const char* desc) {$/;" f namespace:doris +to_http_method src/http/http_method.h /^inline HttpMethod to_http_method(evhttp_cmd_type type) {$/;" f namespace:doris +to_http_path src/runtime/etl_job_mgr.cpp /^std::string EtlJobMgr::to_http_path(const std::string& file_name) {$/;" f class:doris::EtlJobMgr +to_http_path src/runtime/fragment_mgr.cpp /^std::string FragmentExecState::to_http_path(const std::string& file_name) {$/;" f class:doris::FragmentExecState +to_in_opcode src/runtime/primitive_type.cpp /^TExprOpcode::type to_in_opcode(PrimitiveType t) {$/;" f namespace:doris +to_index src/olap/field.h /^void Field::to_index(DstCellType* dst, const SrcCellType& src) const {$/;" f class:doris::Field +to_int64 src/runtime/datetime_value.cpp /^int64_t DateTimeValue::to_int64() const {$/;" f class:doris::DateTimeValue +to_int64_date_packed src/runtime/datetime_value.h /^ int64_t to_int64_date_packed() const {$/;" f class:doris::DateTimeValue +to_int64_datetime_packed src/runtime/datetime_value.h /^ int64_t to_int64_datetime_packed() const {$/;" f class:doris::DateTimeValue +to_json src/exec/es/es_query_builder.cpp /^void BooleanQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) {$/;" f class:doris::BooleanQueryBuilder +to_json src/exec/es/es_query_builder.cpp /^void ESQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) {$/;" f class:doris::ESQueryBuilder +to_json src/exec/es/es_query_builder.cpp /^void ExistsQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) {$/;" f class:doris::ExistsQueryBuilder +to_json src/exec/es/es_query_builder.cpp /^void MatchAllQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) {$/;" f class:doris::MatchAllQueryBuilder +to_json src/exec/es/es_query_builder.cpp /^void RangeQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) {$/;" f class:doris::RangeQueryBuilder +to_json src/exec/es/es_query_builder.cpp /^void TermQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) {$/;" f class:doris::TermQueryBuilder +to_json src/exec/es/es_query_builder.cpp /^void TermsInSetQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) {$/;" f class:doris::TermsInSetQueryBuilder +to_json src/exec/es/es_query_builder.cpp /^void WildCardQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) {$/;" f class:doris::WildCardQueryBuilder +to_json src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::to_json(string* json_string, json2pb::Pb2JsonOptions& options) {$/;" f class:doris::TabletMeta +to_json src/runtime/stream_load/stream_load_context.cpp /^std::string StreamLoadContext::to_json() const {$/;" f class:doris::StreamLoadContext +to_json src/util/json_util.cpp /^std::string to_json(const Status& status) {$/;" f namespace:doris +to_json_for_mini_load src/runtime/stream_load/stream_load_context.cpp /^std::string StreamLoadContext::to_json_for_mini_load() const {$/;" f class:doris::StreamLoadContext +to_load_error_http_path src/runtime/etl_job_mgr.cpp /^std::string EtlJobMgr::to_load_error_http_path(const std::string& file_name) {$/;" f class:doris::EtlJobMgr +to_load_error_http_path src/runtime/fragment_mgr.cpp /^std::string to_load_error_http_path(const std::string& file_name) {$/;" f namespace:doris +to_max_decimal src/runtime/decimal_value.cpp /^void DecimalValue::to_max_decimal(int32_t precision, int32_t frac) {$/;" f class:doris::DecimalValue +to_max_decimal src/runtime/decimalv2_value.cpp /^void DecimalV2Value::to_max_decimal(int32_t precision, int32_t scale) {$/;" f class:doris::DecimalV2Value +to_meta_pb src/olap/tablet_meta.cpp /^OLAPStatus TabletMeta::to_meta_pb(TabletMetaPB* tablet_meta_pb) {$/;" f class:doris::TabletMeta +to_method_desc src/http/http_method.cpp /^std::string to_method_desc(const HttpMethod& method) {$/;" f namespace:doris +to_min_decimal src/runtime/decimal_value.h /^ void to_min_decimal(int precision, int frac) {$/;" f class:doris::DecimalValue +to_min_decimal src/runtime/decimalv2_value.h /^ void to_min_decimal(int precision, int frac) {$/;" f class:doris::DecimalV2Value +to_new_snapshot src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::to_new_snapshot(const OLAPHeaderMessage& olap_header, const string& old_data_path_prefix,$/;" f class:doris::OlapSnapshotConverter +to_olap_date src/runtime/datetime_value.h /^ uint64_t to_olap_date() const {$/;" f class:doris::DateTimeValue +to_olap_datetime src/runtime/datetime_value.h /^ uint64_t to_olap_datetime() const {$/;" f class:doris::DateTimeValue +to_olap_filter src/exec/olap_common.h /^ std::string to_olap_filter(std::list &filters) {$/;" f class:doris::ColumnValueRange +to_olap_filter_type src/exec/olap_utils.h /^inline SQLFilterOp to_olap_filter_type(TExprOpcode::type type, bool opposite) {$/;" f namespace:doris +to_olap_header src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::to_olap_header(const TabletMetaPB& tablet_meta_pb, OLAPHeaderMessage* olap_header) {$/;" f class:doris::OlapSnapshotConverter +to_old_snapshot src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::to_old_snapshot(const TabletMetaPB& tablet_meta_pb, string& new_data_path_prefix,$/;" f class:doris::OlapSnapshotConverter +to_pb src/runtime/query_statistics.h /^ void to_pb(PQueryStatistics* statistics) {$/;" f class:doris::QueryStatistics +to_proto src/olap/rowset/segment_v2/page_pointer.h /^ void to_proto(PagePointerPB* to) {$/;" f struct:doris::segment_v2::PagePointer +to_proto src/olap/rowset/segment_v2/zone_map_index.h /^ void to_proto(ZoneMapPB* dst, Field* field) {$/;" f struct:doris::segment_v2::ZoneMap +to_proto src/util/uid_util.h /^ PUniqueId to_proto() const {$/;" f struct:doris::UniqueId +to_protobuf src/common/status.cpp /^void Status::to_protobuf(PStatus* s) const {$/;" f class:doris::Status +to_protobuf src/exec/tablet_info.cpp /^void OlapTableIndexSchema::to_protobuf(POlapTableIndexSchema* pindex) const {$/;" f class:doris::OlapTableIndexSchema +to_protobuf src/exec/tablet_info.cpp /^void OlapTableSchemaParam::to_protobuf(POlapTableSchemaParam* pschema) const {$/;" f class:doris::OlapTableSchemaParam +to_protobuf src/exec/tablet_info.h /^ POlapTableSchemaParam* to_protobuf() const {$/;" f class:doris::OlapTableSchemaParam +to_protobuf src/runtime/descriptors.cpp /^void RowDescriptor::to_protobuf($/;" f class:doris::RowDescriptor +to_protobuf src/runtime/descriptors.cpp /^void SlotDescriptor::to_protobuf(PSlotDescriptor* pslot) const {$/;" f class:doris::SlotDescriptor +to_protobuf src/runtime/descriptors.cpp /^void TupleDescriptor::to_protobuf(PTupleDescriptor* ptuple) const {$/;" f class:doris::TupleDescriptor +to_protobuf src/runtime/types.cpp /^void TypeDescriptor::to_protobuf(PTypeDesc* ptype) const {$/;" f class:doris::TypeDescriptor +to_query src/exec/es/es_query_builder.cpp /^void BooleanQueryBuilder::to_query(const std::vector& predicates, rapidjson::Document* root, rapidjson::Value* query) {$/;" f class:doris::BooleanQueryBuilder +to_rowset_pb src/olap/rowset/rowset.h /^ void to_rowset_pb(RowsetMetaPB* rs_meta) { return rowset_meta()->to_rowset_pb(rs_meta); }$/;" f class:doris::Rowset +to_rowset_pb src/olap/rowset/rowset_meta.h /^ void to_rowset_pb(RowsetMetaPB* rs_meta_pb) const {$/;" f class:doris::RowsetMeta +to_s2loop src/geo/geo_types.cpp /^static GeoParseStatus to_s2loop(const GeoCoordinateList& coords,$/;" f namespace:doris +to_s2point src/geo/geo_types.cpp /^static inline GeoParseStatus to_s2point(const GeoCoordinate& coord, S2Point* point) {$/;" f namespace:doris +to_s2point src/geo/geo_types.cpp /^static inline GeoParseStatus to_s2point(double lng, double lat, S2Point* point) {$/;" f namespace:doris +to_s2polygon src/geo/geo_types.cpp /^static GeoParseStatus to_s2polygon(const GeoCoordinateListList& coords_list,$/;" f namespace:doris +to_s2polyline src/geo/geo_types.cpp /^static GeoParseStatus to_s2polyline(const GeoCoordinateList& coords,$/;" f namespace:doris +to_schema_pb src/olap/tablet_schema.cpp /^OLAPStatus TabletColumn::to_schema_pb(ColumnPB* column) {$/;" f class:doris::TabletColumn +to_schema_pb src/olap/tablet_schema.cpp /^OLAPStatus TabletSchema::to_schema_pb(TabletSchemaPB* tablet_meta_pb) {$/;" f class:doris::TabletSchema +to_status src/util/arrow/utils.cpp /^Status to_status(const arrow::Status& status) {$/;" f namespace:doris +to_string src/common/status.cpp /^std::string Status::to_string() const {$/;" f class:doris::Status +to_string src/exec/es/es_predicate.h /^ const std::string& to_string() const {$/;" f class:doris::ExtLiteral +to_string src/exprs/agg_fn_evaluator.h /^ std::string to_string(TAggregationOp::type index) {$/;" f class:doris::AggFnEvaluator +to_string src/exprs/anyval_util.h /^ static std::string to_string(const StringVal& v) {$/;" f class:doris::AnyValUtil +to_string src/geo/geo_common.cpp /^std::string to_string(GeoParseStatus status) {$/;" f namespace:doris +to_string src/geo/geo_types.cpp /^std::string GeoPoint::to_string() const {$/;" f class:doris::GeoPoint +to_string src/geo/geo_types.h /^ virtual std::string to_string() const { return ""; };$/;" f class:doris::GeoShape +to_string src/http/action/metrics_action.cpp /^ std::string to_string() const { return _ss.str(); }$/;" f class:doris::PrometheusMetricsVisitor +to_string src/http/action/metrics_action.cpp /^ std::string to_string() const { return _ss.str(); }$/;" f class:doris::SimpleCoreMetricsVisitor +to_string src/olap/bloom_filter.hpp /^ std::string to_string() const {$/;" f class:doris::BitSet +to_string src/olap/bloom_filter.hpp /^ std::string to_string() const {$/;" f class:doris::BloomFilter +to_string src/olap/decimal12.h /^ std::string to_string() const {$/;" f struct:doris::decimal12_t +to_string src/olap/field.h /^ inline std::string to_string(const char* src) const {$/;" f class:doris::Field +to_string src/olap/fs/block_id.h /^ std::string to_string() const {$/;" f class:doris::BlockId +to_string src/olap/hll.h /^ std::string to_string() {$/;" f class:doris::HyperLogLog +to_string src/olap/lru_cache.h /^ std::string to_string() const {$/;" f class:doris::CacheKey +to_string src/olap/olap_common.h /^ std::string to_string() const {$/;" f struct:doris::RowsetId +to_string src/olap/olap_common.h /^ std::string to_string() const {$/;" f struct:doris::TabletInfo +to_string src/olap/olap_index.h /^ std::string to_string() const {$/;" f struct:doris::RowBlockPosition +to_string src/olap/reader.h /^ std::string to_string() const {$/;" f struct:doris::Reader::KeysParam +to_string src/olap/reader.h /^ std::string to_string() {$/;" f struct:doris::ReaderParams +to_string src/olap/row_cursor.cpp /^string RowCursor::to_string() const {$/;" f class:doris::RowCursor +to_string src/olap/rowset/segment_v2/row_ranges.h /^ std::string to_string() const {$/;" f class:doris::segment_v2::RowRange +to_string src/olap/rowset/segment_v2/row_ranges.h /^ std::string to_string() {$/;" f class:doris::segment_v2::RowRanges +to_string src/olap/selection_vector.h /^ std::string to_string() const {$/;" f class:doris::SelectionVector +to_string src/olap/types.h /^ static std::string to_string(const void* src) {$/;" f struct:doris::BaseFieldtypeTraits +to_string src/olap/types.h /^ static std::string to_string(const void* src) {$/;" f struct:doris::FieldTypeTraits +to_string src/olap/types.h /^ std::string to_string(const void* src) const { return _to_string(src); }$/;" f class:doris::TypeInfo +to_string src/olap/uint24.h /^ std::string to_string() const {$/;" f struct:doris::uint24_t +to_string src/olap/wrapper_field.h /^ std::string to_string() const {$/;" f class:doris::WrapperField +to_string src/runtime/datetime_value.cpp /^char* DateTimeValue::to_string(char* to) const {$/;" f class:doris::DateTimeValue +to_string src/runtime/decimal_value.cpp /^std::string DecimalValue::to_string() const {$/;" f class:doris::DecimalValue +to_string src/runtime/decimal_value.cpp /^std::string DecimalValue::to_string(int scale) const {$/;" f class:doris::DecimalValue +to_string src/runtime/decimalv2_value.cpp /^std::string DecimalV2Value::to_string() const {$/;" f class:doris::DecimalV2Value +to_string src/runtime/decimalv2_value.cpp /^std::string DecimalV2Value::to_string(int round_scale) const {$/;" f class:doris::DecimalV2Value +to_string src/runtime/large_int_value.h /^ static char* to_string(__int128 value, char* buffer, int* len) {$/;" f class:doris::LargeIntValue +to_string src/runtime/large_int_value.h /^ static std::string to_string(__int128 value) {$/;" f class:doris::LargeIntValue +to_string src/runtime/row_batch.cpp /^std::string RowBatch::to_string() {$/;" f class:doris::RowBatch +to_string src/runtime/string_value.cpp /^std::string StringValue::to_string() const {$/;" f class:doris::StringValue +to_string src/runtime/tablets_channel.cpp /^std::string TabletsChannelKey::to_string() const {$/;" f class:doris::TabletsChannelKey +to_string src/runtime/tuple.cpp /^std::string Tuple::to_string(const Tuple* t, const TupleDescriptor& d) {$/;" f class:doris::Tuple +to_string src/runtime/tuple.cpp /^std::string Tuple::to_string(const TupleDescriptor& d) const {$/;" f class:doris::Tuple +to_string src/runtime/tuple_row.cpp /^std::string TupleRow::to_string(const RowDescriptor& d) {$/;" f class:doris::TupleRow +to_string src/util/bitmap_value.h /^ std::string to_string() const {$/;" f class:doris::BitmapValue +to_string src/util/metrics.h /^ std::string to_string() const {$/;" f struct:doris::MetricLabel +to_string src/util/metrics.h /^ std::string to_string() const {$/;" f struct:doris::MetricLabels +to_string src/util/slice.h /^ static std::string to_string(const std::vector& slices) {$/;" f struct:doris::Slice +to_string src/util/slice.h /^ std::string to_string() const { return std::string(data, size); }$/;" f struct:doris::Slice +to_string src/util/thread.cpp /^std::string Thread::to_string() const {$/;" f class:doris::Thread +to_string src/util/uid_util.h /^ std::string to_string() const {$/;" f struct:doris::UniqueId +to_string test/util/doris_metrics_test.cpp /^ std::string to_string() {$/;" f class:doris::TestMetricsVisitor +to_string test/util/new_metrics_test.cpp /^ std::string to_string() {$/;" f class:doris::TestMetricsVisitor +to_string test/util/system_metrics_test.cpp /^ std::string to_string() {$/;" f class:doris::TestMetricsVisitor +to_string_val src/runtime/string_value.h /^ void to_string_val(doris_udf::StringVal* sv) const {$/;" f struct:doris::StringValue +to_tablet_meta_pb src/olap/olap_snapshot_converter.cpp /^OLAPStatus OlapSnapshotConverter::to_tablet_meta_pb(const OLAPHeaderMessage& olap_header,$/;" f class:doris::OlapSnapshotConverter +to_tcolumn_type_thrift src/runtime/primitive_type.cpp /^TColumnType to_tcolumn_type_thrift(TPrimitiveType::type ttype) {$/;" f namespace:doris +to_thrift src/common/status.cpp /^void Status::to_thrift(TStatus* s) const {$/;" f class:doris::Status +to_thrift src/runtime/descriptors.cpp /^void RowDescriptor::to_thrift(std::vector* row_tuple_ids) {$/;" f class:doris::RowDescriptor +to_thrift src/runtime/primitive_type.cpp /^TPrimitiveType::type to_thrift(PrimitiveType ptype) {$/;" f namespace:doris +to_thrift src/runtime/types.cpp /^void TypeDescriptor::to_thrift(TTypeDesc* thrift_type) const {$/;" f class:doris::TypeDescriptor +to_thrift src/runtime/types.h /^ TTypeDesc to_thrift() const {$/;" f struct:doris::TypeDescriptor +to_thrift src/util/runtime_profile.cpp /^void RuntimeProfile::to_thrift(TRuntimeProfileTree* tree) {$/;" f class:doris::RuntimeProfile +to_thrift src/util/runtime_profile.cpp /^void RuntimeProfile::to_thrift(std::vector* nodes) {$/;" f class:doris::RuntimeProfile +to_thrift src/util/uid_util.h /^ TUniqueId to_thrift() const {$/;" f struct:doris::UniqueId +to_time_int64 src/runtime/datetime_value.cpp /^int64_t DateTimeValue::to_time_int64() const {$/;" f class:doris::DateTimeValue +to_time_string src/runtime/datetime_value.cpp /^char* DateTimeValue::to_time_string(char* to) const {$/;" f class:doris::DateTimeValue +to_tuple src/olap/row_cursor.cpp /^OlapTuple RowCursor::to_tuple() const {$/;" f class:doris::RowCursor +to_unix src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::to_unix($/;" f class:doris::TimestampFunctions +to_unix src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::to_unix(FunctionContext* context) {$/;" f class:doris::TimestampFunctions +to_upper src/olap/options.cpp /^static std::string to_upper(const std::string& str) {$/;" f namespace:doris +token src/runtime/exec_env.cpp /^const std::string& ExecEnv::token() const {$/;" f class:doris::ExecEnv +too_many_disks_are_failed src/olap/storage_engine.cpp /^static bool too_many_disks_are_failed(uint32_t unused_num, uint32_t total_num) {$/;" f namespace:doris +topic src/runtime/stream_load/stream_load_context.h /^ std::string topic;$/;" m class:doris::KafkaLoadInfo +total src/util/progress_updater.h /^ int64_t total() const {$/;" f class:doris::ProgressUpdater +totalSize src/util/tdigest.h /^ size_t totalSize() const { return _processed.size() + _unprocessed.size(); }$/;" f class:doris::TDigest +totalWeight src/util/tdigest.h /^ long totalWeight() const { return static_cast(_processed_weight + _unprocessed_weight); }$/;" f class:doris::TDigest +total_allocated_bytes src/runtime/mem_pool.h /^ int64_t total_allocated_bytes() const { return total_allocated_bytes_; }$/;" f class:doris::MemPool +total_allocated_bytes_ src/runtime/mem_pool.h /^ int64_t total_allocated_bytes_;$/;" m class:doris::MemPool +total_blocks_created src/olap/fs/block_manager_metrics.h /^ IntCounter* total_blocks_created;$/;" m struct:doris::fs::internal::BlockManagerMetrics +total_blocks_deleted src/olap/fs/block_manager_metrics.h /^ IntCounter* total_blocks_deleted;$/;" m struct:doris::fs::internal::BlockManagerMetrics +total_byte_size src/runtime/row_batch.cpp /^int RowBatch::total_byte_size() {$/;" f class:doris::RowBatch +total_byte_size src/runtime/tuple.cpp /^int64_t Tuple::total_byte_size(const TupleDescriptor& desc) const {$/;" f class:doris::Tuple +total_byte_size_ src/runtime/buffered_tuple_stream3.h /^ int64_t total_byte_size_;$/;" m class:doris::BufferedTupleStream3 +total_bytes_read src/olap/fs/block_manager_metrics.h /^ IntCounter* total_bytes_read;$/;" m struct:doris::fs::internal::BlockManagerMetrics +total_bytes_written src/olap/fs/block_manager_metrics.h /^ IntCounter* total_bytes_written;$/;" m struct:doris::fs::internal::BlockManagerMetrics +total_data_page_size_ src/exec/new_partitioned_hash_table.h /^ int64_t total_data_page_size_;$/;" m class:doris::NewPartitionedHashTable +total_disk_size src/olap/rowset/rowset_meta.h /^ size_t total_disk_size() const {$/;" f class:doris::RowsetMeta +total_disk_sync src/olap/fs/block_manager_metrics.h /^ IntCounter* total_disk_sync;$/;" m struct:doris::fs::internal::BlockManagerMetrics +total_get_wait_time src/util/blocking_priority_queue.hpp /^ uint64_t total_get_wait_time() const {$/;" f class:doris::BlockingPriorityQueue +total_get_wait_time src/util/blocking_queue.hpp /^ uint64_t total_get_wait_time() const {$/;" f class:doris::BlockingQueue +total_num_rows src/exec/row_batch_list.h /^ int64_t total_num_rows() {$/;" f class:doris::RowBatchList +total_out_64 src/util/minizip/unzip.c /^ ZPOS64_T total_out_64;$/;" m struct:__anon33 file: +total_pages_num src/olap/olap_common.h /^ int64_t total_pages_num = 0;$/;" m struct:doris::OlapReaderStatistics +total_put_wait_time src/util/blocking_priority_queue.hpp /^ uint64_t total_put_wait_time() const {$/;" f class:doris::BlockingPriorityQueue +total_put_wait_time src/util/blocking_queue.hpp /^ uint64_t total_put_wait_time() const {$/;" f class:doris::BlockingQueue +total_readable_blocks src/olap/fs/block_manager_metrics.h /^ IntCounter* total_readable_blocks;$/;" m struct:doris::fs::internal::BlockManagerMetrics +total_reserved_bytes src/runtime/mem_pool.h /^ int64_t total_reserved_bytes() const { return total_reserved_bytes_; }$/;" f class:doris::MemPool +total_reserved_bytes_ src/runtime/mem_pool.h /^ int64_t total_reserved_bytes_;$/;" m class:doris::MemPool +total_sampled_value src/util/runtime_profile.h /^ int64_t total_sampled_value; \/\/ sum of all sampled values;$/;" m struct:doris::RuntimeProfile::SamplingCounterInfo +total_throughput_counter src/exec/scan_node.h /^ RuntimeProfile::Counter* total_throughput_counter() const {$/;" f class:doris::ScanNode +total_time_counter src/util/runtime_profile.h /^ Counter* total_time_counter() {$/;" f class:doris::RuntimeProfile +total_writable_blocks src/olap/fs/block_manager_metrics.h /^ IntCounter* total_writable_blocks;$/;" m struct:doris::fs::internal::BlockManagerMetrics +track_allocation src/udf/udf.cpp /^void FunctionContext::track_allocation(int64_t bytes) {$/;" f class:doris_udf::FunctionContext +tracker_ src/runtime/bufferpool/buffer_pool.h /^ boost::scoped_ptr tracker_;$/;" m class:doris::BufferPool::SubReservation +trailing_bits src/util/bit_util.h /^ static inline uint64_t trailing_bits(uint64_t v, int num_bits) {$/;" f class:doris::BitUtil +transaction_id src/olap/rowset/segment_group.h /^ inline TTransactionId transaction_id() const { return _txn_id; }$/;" f class:doris::SegmentGroup +transaction_id test/olap/txn_manager_test.cpp /^ TTransactionId transaction_id = 111;$/;" m class:doris::TxnManagerTest file: +transfer_all_resources src/runtime/data_stream_recvr.cc /^void DataStreamRecvr::transfer_all_resources(RowBatch* transfer_batch) {$/;" f class:doris::DataStreamRecvr +transfer_buffer src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::transfer_buffer(Block* dst, Block* src, bool unpin) {$/;" f class:doris::BufferedBlockMgr2 +transfer_resource_ownership src/runtime/row_batch.cpp /^void RowBatch::transfer_resource_ownership(RowBatch* dest) {$/;" f class:doris::RowBatch +transfer_resources src/runtime/spill_sorter.cc /^void SpillSorter::Run::transfer_resources(RowBatch* row_batch) {$/;" f class:doris::SpillSorter::Run +transfer_thread src/exec/olap_scan_node.cpp /^void OlapScanNode::transfer_thread(RuntimeState* state) {$/;" f class:doris::OlapScanNode +transform_is_simple src/util/radix_sort.h /^ static constexpr bool transform_is_simple = false;$/;" m struct:doris::RadixSortFloatTransform +transform_is_simple src/util/radix_sort.h /^ static constexpr bool transform_is_simple = true;$/;" m struct:doris::RadixSortIdentityTransform +transform_is_simple src/util/radix_sort.h /^ static constexpr bool transform_is_simple = true;$/;" m struct:doris::RadixSortSignedTransform +transition src/util/threadpool.cpp /^void ThreadPoolToken::transition(State new_state) {$/;" f class:doris::ThreadPoolToken +transmit_data src/runtime/data_stream_mgr.cpp /^Status DataStreamMgr::transmit_data(const PTransmitDataParams* request, ::google::protobuf::Closure** done) {$/;" f class:doris::DataStreamMgr +transmit_data src/service/backend_service.cpp /^void BackendService::transmit_data(TTransmitDataResult& return_val,$/;" f class:doris::BackendService +transmit_data src/service/internal_service.cpp /^void PInternalServiceImpl::transmit_data(google::protobuf::RpcController* cntl_base,$/;" f class:doris::PInternalServiceImpl +transmit_data test/runtime/data_stream_test.cpp /^ virtual void transmit_data($/;" f class:doris::DorisTestBackend +travel_length_ src/exec/new_partitioned_hash_table.h /^ int64_t travel_length_;$/;" m class:doris::NewPartitionedHashTable +traverse_headers src/olap/tablet_meta_manager.cpp /^OLAPStatus TabletMetaManager::traverse_headers(OlapMeta* meta,$/;" f class:doris::TabletMetaManager +traverse_rowset_metas src/olap/rowset/rowset_meta_manager.cpp /^OLAPStatus RowsetMetaManager::traverse_rowset_metas(OlapMeta* meta,$/;" f class:doris::RowsetMetaManager +trigger_hook src/util/metrics.h /^ void trigger_hook() {$/;" f class:doris::MetricRegistry +trigger_profile_report src/runtime/fragment_mgr.cpp /^Status FragmentMgr::trigger_profile_report(const PTriggerProfileReportRequest* request) {$/;" f class:doris::FragmentMgr +trigger_profile_report src/service/internal_service.cpp /^void PInternalServiceImpl::trigger_profile_report($/;" f class:doris::PInternalServiceImpl +trigger_report src/olap/storage_engine.h /^ void trigger_report() {$/;" f class:doris::StorageEngine +trim src/common/configbase.cpp /^std::string& Properties::trim(std::string& s) {$/;" f class:doris::config::Properties +trim src/exprs/string_functions.cpp /^StringVal StringFunctions::trim(FunctionContext* context, const StringVal& str) {$/;" f class:doris::StringFunctions +trim src/runtime/string_value.hpp /^inline StringValue StringValue::trim() const {$/;" f class:doris::StringValue +true_ src/gutil/template_util.h /^typedef true_type true_;$/;" t namespace:base +true_type src/gutil/template_util.h /^typedef integral_constant true_type;$/;" t namespace:base +truncate src/exprs/math_functions.cpp /^DoubleVal MathFunctions::truncate($/;" f class:doris::MathFunctions +truncate src/util/slice.h /^ void truncate(size_t n) {$/;" f struct:doris::Slice +try_acquire_thread_token src/runtime/thread_resource_mgr.h /^inline bool ThreadResourceMgr::ResourcePool::try_acquire_thread_token() {$/;" f class:doris::ThreadResourceMgr::ResourcePool +try_acquire_tmp_reservation src/runtime/buffered_block_mgr2.cc /^bool BufferedBlockMgr2::try_acquire_tmp_reservation(Client* client, int num_buffers) {$/;" f class:doris::BufferedBlockMgr2 +try_add src/util/runtime_profile.h /^ bool try_add(int64_t delta, int64_t max) {$/;" f class:doris::RuntimeProfile::HighWaterMarkCounter +try_add_block src/runtime/merge_sorter.cpp /^Status MergeSorter::Run::try_add_block(std::vector* block_sequence,$/;" f class:doris::MergeSorter::Run +try_add_block src/runtime/spill_sorter.cc /^Status SpillSorter::Run::try_add_block($/;" f class:doris::SpillSorter::Run +try_add_remaining_results src/exec/analytic_eval_node.cpp /^inline void AnalyticEvalNode::try_add_remaining_results(int64_t partition_idx,$/;" f class:doris::AnalyticEvalNode +try_add_result_tuple_for_curr_row src/exec/analytic_eval_node.cpp /^inline void AnalyticEvalNode::try_add_result_tuple_for_curr_row(int64_t stream_idx,$/;" f class:doris::AnalyticEvalNode +try_add_result_tuple_for_prev_row src/exec/analytic_eval_node.cpp /^inline void AnalyticEvalNode::try_add_result_tuple_for_prev_row(bool next_partition,$/;" f class:doris::AnalyticEvalNode +try_allocate src/runtime/mem_pool.h /^ uint8_t* try_allocate(int64_t size) {$/;" f class:doris::MemPool +try_allocate_aligned src/runtime/mem_pool.h /^ uint8_t* try_allocate_aligned(int64_t size, int alignment) {$/;" f class:doris::MemPool +try_allocate_unaligned src/runtime/mem_pool.h /^ uint8_t* try_allocate_unaligned(int64_t size) {$/;" f class:doris::MemPool +try_consume src/runtime/mem_tracker.h /^ bool try_consume(int64_t bytes) {$/;" f class:doris::MemTracker +try_do_aggregate_serde_improve src/exec/exec_node.cpp /^void ExecNode::try_do_aggregate_serde_improve() {$/;" f class:doris::ExecNode +try_lock src/util/fake_lock.h /^ bool try_lock() { return true; }$/;" f class:doris::FakeLock +try_lock src/util/spinlock.h /^ inline bool try_lock() {$/;" f class:doris::SpinLock +try_remove_rows_before_window src/exec/analytic_eval_node.cpp /^inline void AnalyticEvalNode::try_remove_rows_before_window(int64_t stream_idx) {$/;" f class:doris::AnalyticEvalNode +try_schema_change_lock src/olap/tablet_manager.cpp /^bool TabletManager::try_schema_change_lock(TTabletId tablet_id) {$/;" f class:doris::TabletManager +trylock src/util/mutex.cpp /^OLAPStatus Mutex::trylock() {$/;" f class:doris::Mutex +tryrdlock src/util/mutex.cpp /^OLAPStatus RWMutex::tryrdlock() {$/;" f class:doris::RWMutex +trywrlock src/util/mutex.cpp /^OLAPStatus RWMutex::trywrlock() {$/;" f class:doris::RWMutex +tuple src/exec/new_partitioned_hash_table.h /^ Tuple* tuple;$/;" m union:doris::NewPartitionedHashTable::HtData +tuple src/exec/olap_scan_node.h /^ Tuple* tuple;$/;" m struct:doris::OlapScanNode::__anon40 +tuple src/exec/partitioned_hash_table.h /^ Tuple* tuple;$/;" m union:doris::PartitionedHashTable::HtData +tuple_data_pool src/runtime/row_batch.h /^ MemPool* tuple_data_pool() {$/;" f class:doris::RowBatch +tuple_desc src/exec/kudu_scan_node.h /^ const TupleDescriptor* tuple_desc() const { return _tuple_desc; }$/;" f class:doris::KuduScanNode +tuple_desc src/exec/schema_scanner.h /^ const TupleDescriptor* tuple_desc() const {$/;" f class:doris::SchemaScanner +tuple_desc src/exec/tablet_info.h /^ TupleDescriptor* tuple_desc() const { return _tuple_desc; }$/;" f class:doris::OlapTableSchemaParam +tuple_desc src/olap/delta_writer.h /^ TupleDescriptor* tuple_desc;$/;" m struct:doris::WriteRequest +tuple_descriptors src/runtime/descriptors.h /^ const std::vector& tuple_descriptors() const {$/;" f class:doris::RowDescriptor +tuple_is_nullable src/runtime/descriptors.cpp /^bool RowDescriptor::tuple_is_nullable(int tuple_idx) const {$/;" f class:doris::RowDescriptor +tuple_offset src/runtime/descriptors.h /^ int tuple_offset() const {$/;" f class:doris::SlotDescriptor +tuple_stream_ src/exec/new_partitioned_hash_table.h /^ BufferedTupleStream3* tuple_stream_;$/;" m class:doris::NewPartitionedHashTable +tv_nsec src/gutil/linux_syscall_support.h /^ long tv_nsec;$/;" m struct:kernel_timespec +tv_sec src/gutil/linux_syscall_support.h /^ long tv_sec;$/;" m struct:kernel_timespec +tv_sec src/gutil/linux_syscall_support.h /^ long tv_sec;$/;" m struct:kernel_timeval +tv_usec src/gutil/linux_syscall_support.h /^ long tv_usec;$/;" m struct:kernel_timeval +two_ASCII_digits src/gutil/strings/util.cc /^const char two_ASCII_digits[100][2] = {$/;" v +txn_begin_request_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::txn_begin_request_total;$/;" m class:doris::DorisMetrics file: +txn_begin_request_total src/util/doris_metrics.h /^ static IntCounter txn_begin_request_total;$/;" m class:doris::DorisMetrics +txn_commit_request_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::txn_commit_request_total;$/;" m class:doris::DorisMetrics file: +txn_commit_request_total src/util/doris_metrics.h /^ static IntCounter txn_commit_request_total;$/;" m class:doris::DorisMetrics +txn_exec_plan_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::txn_exec_plan_total;$/;" m class:doris::DorisMetrics file: +txn_exec_plan_total src/util/doris_metrics.h /^ static IntCounter txn_exec_plan_total;$/;" m class:doris::DorisMetrics +txn_id src/olap/delta_writer.h /^ int64_t txn_id;$/;" m struct:doris::WriteRequest +txn_id src/olap/rowset/rowset.h /^ int64_t txn_id() const { return rowset_meta()->txn_id(); }$/;" f class:doris::Rowset +txn_id src/olap/rowset/rowset_meta.h /^ int64_t txn_id() const {$/;" f class:doris::RowsetMeta +txn_id src/olap/rowset/rowset_writer_context.h /^ int64_t txn_id;$/;" m struct:doris::RowsetWriterContext +txn_id src/olap/tablet_sync_service.h /^ int64_t txn_id;$/;" m struct:doris::FetchRowsetMetaTask +txn_id src/runtime/stream_load/stream_load_context.h /^ int64_t txn_id = -1;$/;" m class:doris::StreamLoadContext +txn_manager src/olap/storage_engine.h /^ TxnManager* txn_manager() { return _txn_manager.get(); }$/;" f class:doris::StorageEngine +txn_rollback_request_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::txn_rollback_request_total;$/;" m class:doris::DorisMetrics file: +txn_rollback_request_total src/util/doris_metrics.h /^ static IntCounter txn_rollback_request_total;$/;" m class:doris::DorisMetrics +type output/udf/include/udf.h /^ Type type;$/;" m struct:doris_udf::FunctionContext::TypeDesc +type output/udf/include/udf.h /^ int type;$/;" m struct:doris_udf::DateTimeVal +type src/common/configbase.h /^ const char* type;$/;" m struct:doris::config::Register::Field +type src/exec/es/es_predicate.h /^ TypeDescriptor type;$/;" m struct:doris::ExtColumnDesc +type src/exec/exec_node.h /^ TPlanNodeType::type type() const {$/;" f class:doris::ExecNode +type src/exec/olap_common.h /^ PrimitiveType type() const {$/;" f class:doris::ColumnValueRange +type src/exec/schema_scanner.h /^ PrimitiveType type;$/;" m struct:doris::SchemaScanner::ColumnDesc +type src/exprs/expr.h /^ const TypeDescriptor& type() const {$/;" f class:doris::Expr +type src/exprs/new_in_predicate.h /^ const FunctionContext::TypeDesc* type;$/;" m struct:doris::InPredicate::SetLookupState +type src/gutil/cpu.h /^ int type() const { return type_; }$/;" f class:base::final +type src/gutil/strings/split_internal.h /^ typedef void type;$/;" t struct:strings::internal::IsNotInitializerList +type src/gutil/template_util.h /^ typedef A type;$/;" t struct:base::if_ +type src/gutil/template_util.h /^ typedef B type;$/;" t struct:base::if_ +type src/gutil/template_util.h /^ typedef T type;$/;" t struct:base::identity_ +type src/gutil/template_util.h /^ typedef integral_constant type;$/;" t struct:base::integral_constant +type src/gutil/type_traits.h /^ typedef T type; };$/;" t struct:base::remove_pointer +type src/gutil/type_traits.h /^ typedef typename remove_const::type>::type type;$/;" t struct:base::remove_cv +type src/gutil/type_traits.h /^template struct add_reference { typedef T& type; };$/;" t struct:base::add_reference +type src/gutil/type_traits.h /^template struct add_reference { typedef T& type; };$/;" t struct:base::add_reference +type src/gutil/type_traits.h /^template struct enable_if { typedef T type; };$/;" t struct:base::enable_if +type src/gutil/type_traits.h /^template struct remove_const { typedef T type; };$/;" t struct:base::remove_const +type src/gutil/type_traits.h /^template struct remove_const { typedef T type; };$/;" t struct:base::remove_const +type src/gutil/type_traits.h /^template struct remove_pointer { typedef T type; };$/;" t struct:base::remove_pointer +type src/gutil/type_traits.h /^template struct remove_pointer { typedef T type; };$/;" t struct:base::remove_pointer +type src/gutil/type_traits.h /^template struct remove_pointer { typedef T type; };$/;" t struct:base::remove_pointer +type src/gutil/type_traits.h /^template struct remove_pointer { typedef T type; };$/;" t struct:base::remove_pointer +type src/gutil/type_traits.h /^template struct remove_reference { typedef T type; };$/;" t struct:base::remove_reference +type src/gutil/type_traits.h /^template struct remove_reference { typedef T type; };$/;" t struct:base::remove_reference +type src/gutil/type_traits.h /^template struct remove_volatile { typedef T type; };$/;" t struct:base::remove_volatile +type src/gutil/type_traits.h /^template struct remove_volatile { typedef T type; };$/;" t struct:base::remove_volatile +type src/olap/aggregate_func.h /^ static const FieldType type = fieldType;$/;" m struct:doris::AggregateTraits +type src/olap/field.h /^ FieldType type() const { return _type_info->type(); }$/;" f class:doris::Field +type src/olap/out_stream.h /^ uint8_t type; \/\/ 256种类型, 应该足够以后的扩展了$/;" m struct:doris::StreamHead +type src/olap/rowset/run_length_integer_writer.h /^ type: 2;$/;" m struct:doris::RunLengthIntegerWriter::DeltaHead +type src/olap/rowset/run_length_integer_writer.h /^ type: 2;$/;" m struct:doris::RunLengthIntegerWriter::DirectHead +type src/olap/rowset/run_length_integer_writer.h /^ type: 2;$/;" m struct:doris::RunLengthIntegerWriter::PatchedBaseHead +type src/olap/rowset/run_length_integer_writer.h /^ type: 2;$/;" m struct:doris::RunLengthIntegerWriter::ShortRepeatHead +type src/olap/rowset/segment_v2/encoding_info.cpp /^ static const FieldType type = field_type;$/;" m struct:doris::segment_v2::EncodingTraits file: +type src/olap/rowset/segment_v2/encoding_info.h /^ FieldType type() const { return _type; }$/;" f class:doris::segment_v2::EncodingInfo +type src/olap/tablet_schema.h /^ inline FieldType type() const { return _type; }$/;" f class:doris::TabletColumn +type src/olap/types.h /^ inline FieldType type() const { return _field_type; }$/;" f class:doris::TypeInfo +type src/olap/types.h /^ static const FieldType type = field_type;$/;" m struct:doris::TypeTraits +type src/runtime/descriptor_helper.h /^ TSlotDescriptorBuilder& type(PrimitiveType type) {$/;" f class:doris::TSlotDescriptorBuilder +type src/runtime/descriptors.h /^ const TypeDescriptor& type() const {$/;" f class:doris::SlotDescriptor +type src/runtime/disk_io_mgr.h /^ enum type {$/;" g struct:doris::DiskIoMgr::RequestType +type src/runtime/dpp_sink.cpp /^ HllDataType type;$/;" m struct:doris::HllDppSinkMerge::HllMergeValue file: +type src/runtime/types.h /^ PrimitiveType type;$/;" m struct:doris::TypeDescriptor +type src/udf/udf.h /^ Type type;$/;" m struct:doris_udf::FunctionContext::TypeDesc +type src/udf/udf.h /^ int type;$/;" m struct:doris_udf::DateTimeVal +type src/util/bitmap_value.h /^ enum type {$/;" g struct:doris::BitmapTypeCode +type src/util/metrics.h /^ MetricType type() const { return _type; }$/;" f class:doris::Metric +type src/util/metrics.h /^ MetricType type() const { return _type; }$/;" f class:doris::MetricCollector +type src/util/perf_counters.h /^ TUnit::type type;$/;" m struct:doris::PerfCounters::CounterData +type src/util/runtime_profile.h /^ TUnit::type type() const {$/;" f class:doris::RuntimeProfile::Counter +type src/util/slice.h /^ typedef std::map type;$/;" t struct:doris::SliceMap +type_ src/gutil/cpu.h /^ int type_; \/\/ process type$/;" m class:base::final +type_equals_ src/gutil/template_util.h /^struct type_equals_ : public false_ {$/;" s namespace:base +type_equals_ src/gutil/template_util.h /^struct type_equals_ : public true_ {$/;" s namespace:base +type_info src/olap/column_block.h /^ const TypeInfo* type_info() const { return _block->type_info(); }$/;" f class:doris::ColumnBlockView +type_info src/olap/column_block.h /^ const TypeInfo* type_info() const { return _type_info; }$/;" f class:doris::ColumnBlock +type_info src/olap/field.h /^ const TypeInfo* type_info() const { return _type_info; }$/;" f class:doris::Field +type_info src/olap/rowset/segment_v2/bitmap_index_reader.h /^ const TypeInfo* type_info() {$/;" f class:doris::segment_v2::BitmapIndexReader +type_info src/olap/rowset/segment_v2/bloom_filter_index_reader.h /^ const TypeInfo* type_info() const {$/;" f class:doris::segment_v2::BloomFilterIndexReader +type_info src/olap/rowset/segment_v2/column_reader.h /^ const TypeInfo* type_info() const { return _type_info; }$/;" f class:doris::segment_v2::ColumnReader +type_info src/olap/rowset/segment_v2/indexed_column_reader.h /^ const TypeInfo* type_info() const { return _type_info; }$/;" f class:doris::segment_v2::IndexedColumnReader +type_must_be_complete src/gutil/gscoped_ptr.h /^ enum { type_must_be_complete = sizeof(element_type) };$/;" e enum:gscoped_ptr::__anon24 +type_of_iterator src/util/bitmap_value.h /^ typedef Roaring64MapSetBitForwardIterator type_of_iterator;$/;" t class:doris::detail::final +type_to_odbc_string src/runtime/primitive_type.cpp /^std::string type_to_odbc_string(PrimitiveType t) {$/;" f namespace:doris +type_to_string src/exec/schema_scanner/schema_columns_scanner.cpp /^std::string SchemaColumnsScanner::type_to_string(TColumnDesc &desc) {$/;" f class:doris::SchemaColumnsScanner +type_to_string src/runtime/primitive_type.cpp /^std::string type_to_string(PrimitiveType t) {$/;" f namespace:doris +type_without_cast src/exprs/expr.cpp /^TExprNodeType::type Expr::type_without_cast(const Expr* expr) {$/;" f class:doris::Expr +u64tostr_base36 src/gutil/strings/numbers.cc /^size_t u64tostr_base36(uint64 number, size_t buf_size, char* buffer) {$/;" f +u_int16_t src/gutil/port.h /^typedef uint16_t u_int16_t;$/;" t +u_int16_t src/gutil/port.h /^typedef unsigned short u_int16_t;$/;" t +uchar src/gutil/utf/utfdef.h /^typedef unsigned char uchar;$/;" t +uchar src/gutil/utf/utfdef.h 1;" d +uchar src/util/mysql_global.h /^typedef unsigned char uchar;$/;" t namespace:doris +uid_t src/gutil/port.h /^typedef int uid_t;$/;" t +uint src/gutil/port.h /^typedef unsigned int uint;$/;" t +uint src/gutil/utf/utfdef.h /^typedef unsigned int uint;$/;" t +uint src/gutil/utf/utfdef.h 3;" d +uint128 src/gutil/int128.h /^class uint128 {$/;" c +uint128 src/gutil/int128.h /^inline uint128::uint128(): lo_(0), hi_(0) { }$/;" f class:uint128 +uint128 src/gutil/int128.h /^inline uint128::uint128(const uint128 &v) : lo_(v.lo_), hi_(v.hi_) { }$/;" f class:uint128 +uint128 src/gutil/int128.h /^inline uint128::uint128(const uint128_pod &v) : lo_(v.lo), hi_(v.hi) { }$/;" f class:uint128 +uint128 src/gutil/int128.h /^inline uint128::uint128(int bottom) : lo_(bottom), hi_(0) {$/;" f class:uint128 +uint128 src/gutil/int128.h /^inline uint128::uint128(uint32 bottom) : lo_(bottom), hi_(0) { }$/;" f class:uint128 +uint128 src/gutil/int128.h /^inline uint128::uint128(uint64 bottom) : lo_(bottom), hi_(0) { }$/;" f class:uint128 +uint128 src/gutil/int128.h /^inline uint128::uint128(uint64 top, uint64 bottom) : lo_(bottom), hi_(top) { }$/;" f class:uint128 +uint128_pod src/gutil/int128.h /^struct uint128_pod {$/;" s +uint128_t src/olap/olap_common.h /^typedef unsigned __int128 uint128_t;$/;" t namespace:doris +uint16 src/gutil/integral_types.h /^typedef uint16_t uint16;$/;" t +uint24_t src/olap/uint24.h /^ uint24_t() {$/;" f struct:doris::uint24_t +uint24_t src/olap/uint24.h /^ uint24_t(const uint24_t& value) {$/;" f struct:doris::uint24_t +uint24_t src/olap/uint24.h /^ uint24_t(const uint32_t& value) {$/;" f struct:doris::uint24_t +uint24_t src/olap/uint24.h /^struct uint24_t {$/;" s namespace:doris +uint32 src/gutil/integral_types.h /^typedef uint32_t uint32;$/;" t +uint32_t src/util/murmur_hash3.h /^typedef unsigned int uint32_t;$/;" t +uint64 src/gutil/integral_types.h /^typedef uint64_t uint64;$/;" t +uint64 src/gutil/integral_types.h /^typedef unsigned __int64 uint64;$/;" t +uint64_t src/util/murmur_hash3.h /^typedef unsigned __int64 uint64_t;$/;" t +uint8 src/gutil/integral_types.h /^typedef uint8_t uint8;$/;" t +uint8_t src/util/murmur_hash3.h /^typedef unsigned char uint8_t;$/;" t +ulong src/gutil/port.h /^typedef unsigned long ulong;$/;" t +ulong src/gutil/utf/utfdef.h /^typedef unsigned long ulong;$/;" t +ulong src/gutil/utf/utfdef.h 4;" d +unaggregated_row_stream src/exec/new_partitioned_aggregation_node.h /^ boost::scoped_ptr unaggregated_row_stream;$/;" m struct:doris::NewPartitionedAggregationNode::Partition +unaggregated_row_stream src/exec/partitioned_aggregation_node.h /^ boost::scoped_ptr unaggregated_row_stream;$/;" m struct:doris::PartitionedAggregationNode::Partition +uncompressed_bytes_read src/olap/olap_common.h /^ int64_t uncompressed_bytes_read = 0;$/;" m struct:doris::OlapReaderStatistics +uncompressed_size src/util/minizip/unzip.h /^ ZPOS64_T uncompressed_size; \/* uncompressed size 8 bytes *\/$/;" m struct:unz_file_info64_s +uncompressed_size src/util/minizip/unzip.h /^ uLong uncompressed_size; \/* uncompressed size 4 bytes *\/$/;" m struct:unz_file_info_s +underflow src/olap/in_stream.h /^ virtual int_type underflow() {$/;" f class:doris::InStreamBufferWrapper +unescape_path src/util/cgroup_util.cpp /^static Status unescape_path(const string& escaped, string* unescaped) {$/;" f namespace:doris +unescape_string src/exec/text_converter.cpp /^void TextConverter::unescape_string(StringValue* value, MemPool* pool) {$/;" f class:doris::TextConverter +unescape_string src/exec/text_converter.cpp /^void TextConverter::unescape_string(const char* src, char* dest, size_t* len) {$/;" f class:doris::TextConverter +unexpected_remote_bytes src/runtime/disk_io_mgr.cc /^int64_t DiskIoMgr::unexpected_remote_bytes(RequestContext* reader) const {$/;" f class:doris::DiskIoMgr +unhex src/exprs/math_functions.cpp /^StringVal MathFunctions::unhex(FunctionContext* ctx, const StringVal& s) {$/;" f class:doris::MathFunctions +unique_column_id src/olap/rowset/column_writer.h /^ uint32_t unique_column_id() const {$/;" f class:doris::ColumnWriter +unique_column_id src/olap/stream_name.h /^ uint32_t unique_column_id() const {$/;" f class:doris::StreamName +unique_id src/olap/rowset/rowset.h /^ std::string unique_id() const {$/;" f class:doris::Rowset +unique_id src/olap/tablet_schema.h /^ inline int32_t unique_id() const { return _unique_id; }$/;" f class:doris::TabletColumn +unique_thread_id src/util/thread.cpp /^int64_t Thread::unique_thread_id() {$/;" f class:doris::Thread +uniteBytes src/util/bitmap_value.h /^ static uint64_t uniteBytes(const uint32_t highBytes, const uint32_t lowBytes) {$/;" f class:doris::detail::Roaring64Map +units_per_second src/util/runtime_profile.cpp /^int64_t RuntimeProfile::units_per_second($/;" f class:doris::RuntimeProfile +unix_timestamp src/runtime/datetime_value.cpp /^bool DateTimeValue::unix_timestamp(int64_t* timestamp, const std::string& timezone) const{$/;" f class:doris::DateTimeValue +unlock src/util/fake_lock.h /^ void unlock() {}$/;" f class:doris::FakeLock +unlock src/util/mutex.cpp /^OLAPStatus Mutex::unlock() {$/;" f class:doris::Mutex +unlock src/util/mutex.cpp /^OLAPStatus RWMutex::unlock() {$/;" f class:doris::RWMutex +unlock src/util/mutex.h /^ void unlock() {$/;" f class:doris::MutexLock +unlock src/util/spinlock.h /^ void unlock() {$/;" f class:doris::SpinLock +unpin src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::Block::unpin() {$/;" f class:doris::BufferedBlockMgr2::Block +unpin_all_blocks src/runtime/spill_sorter.cc /^Status SpillSorter::Run::unpin_all_blocks() {$/;" f class:doris::SpillSorter::Run +unpin_block src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::unpin_block(Block* block) {$/;" f class:doris::BufferedBlockMgr2 +unpin_block src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::unpin_block(BufferedBlockMgr2::Block* block) {$/;" f class:doris::BufferedTupleStream2 +unpin_buffer src/runtime/buffered_block_mgr2.cc /^ void unpin_buffer(BufferDescriptor* buffer) {$/;" f class:doris::BufferedBlockMgr2::Client +unpin_stream src/runtime/buffered_tuple_stream2.cc /^Status BufferedTupleStream2::unpin_stream(bool all) {$/;" f class:doris::BufferedTupleStream2 +unprocessed src/util/tdigest.h /^ const std::vector& unprocessed() const { return _unprocessed; }$/;" f class:doris::TDigest +unprocessedWeight src/util/tdigest.h /^ Weight unprocessedWeight() const { return _unprocessed_weight; }$/;" f class:doris::TDigest +unprotected_trigger_hook src/util/metrics.h /^ void unprotected_trigger_hook() {$/;" f class:doris::MetricRegistry +unref src/runtime/stream_load/stream_load_context.h /^ bool unref() { return _refs.fetch_sub(1) == 1; }$/;" f class:doris::StreamLoadContext +unref src/runtime/user_function_cache.cpp /^ bool unref() { return _refs.fetch_sub(1) == 1; }$/;" f struct:doris::UserFunctionCacheEntry +unref src/util/ref_count_closure.h /^ bool unref() { return _refs.fetch_sub(1) == 1; }$/;" f class:doris::RefCountClosure +unregister_context src/runtime/disk_io_mgr.cc /^void DiskIoMgr::unregister_context(RequestContext* reader) {$/;" f class:doris::DiskIoMgr +unregister_from_parent src/runtime/mem_tracker.h /^ void unregister_from_parent() {$/;" f class:doris::MemTracker +unregister_pool src/runtime/thread_resource_mgr.cpp /^void ThreadResourceMgr::unregister_pool(ResourcePool* pool) {$/;" f class:doris::ThreadResourceMgr +unserialize src/exprs/aggregate_functions.cpp /^ void unserialize(StringVal& src) {$/;" f class:doris::MultiDistinctCountDateState +unserialize src/exprs/aggregate_functions.cpp /^ void unserialize(StringVal& src) {$/;" f class:doris::MultiDistinctDecimalState +unserialize src/exprs/aggregate_functions.cpp /^ void unserialize(StringVal& src) {$/;" f class:doris::MultiDistinctDecimalV2State +unserialize src/exprs/aggregate_functions.cpp /^ void unserialize(StringVal& src) {$/;" f class:doris::MultiDistinctNumericState +unserialize src/exprs/aggregate_functions.cpp /^ void unserialize(StringVal& src) {$/;" f class:doris::MultiDistinctStringCountState +unserialize src/olap/file_helper.h /^OLAPStatus FileHeader::unserialize($/;" f class:doris::FileHeader +unserialize src/util/tdigest.h /^ void unserialize(const uint8_t* type_reader) {$/;" f class:doris::TDigest +unstarted_scan_ranges src/runtime/disk_io_mgr_internal.h /^ InternalQueue* unstarted_scan_ranges() { return &_unstarted_scan_ranges; }$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +unstarted_scan_ranges src/runtime/disk_io_mgr_internal.h /^ const InternalQueue* unstarted_scan_ranges() const {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +unstarted_write_ranges src/runtime/disk_io_mgr_internal.h /^ InternalQueue* unstarted_write_ranges() {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +unstarted_write_ranges src/runtime/disk_io_mgr_internal.h /^ const InternalQueue* unstarted_write_ranges() const {$/;" f class:doris::DiskIoMgr::RequestContext::PerDiskState +unused src/util/minizip/unzip.h /^typedef struct TagunzFile__ { int unused; } unzFile__;$/;" m struct:TagunzFile__ +unused_reservation src/runtime/bufferpool/reservation_tracker.h /^ inline int64_t unused_reservation() const {$/;" f class:doris::ReservationTracker +unz64_file_pos src/util/minizip/unzip.h /^} unz64_file_pos;$/;" t typeref:struct:unz64_file_pos_s +unz64_file_pos_s src/util/minizip/unzip.h /^typedef struct unz64_file_pos_s$/;" s +unz64_s src/util/minizip/unzip.c /^} unz64_s;$/;" t typeref:struct:__anon34 file: +unz64local_CheckCurrentFileCoherencyHeader src/util/minizip/unzip.c /^local int unz64local_CheckCurrentFileCoherencyHeader (unz64_s* s, uInt* piSizeVar,$/;" f +unz64local_DosDateToTmuDate src/util/minizip/unzip.c /^local void unz64local_DosDateToTmuDate (ZPOS64_T ulDosDate, tm_unz* ptm)$/;" f +unz64local_GetCurrentFileInfoInternal src/util/minizip/unzip.c /^local int unz64local_GetCurrentFileInfoInternal (unzFile file,$/;" f +unz64local_SearchCentralDir src/util/minizip/unzip.c /^local ZPOS64_T unz64local_SearchCentralDir(const zlib_filefunc64_32_def* pzlib_filefunc_def, voidpf filestream)$/;" f +unz64local_SearchCentralDir64 src/util/minizip/unzip.c /^local ZPOS64_T unz64local_SearchCentralDir64(const zlib_filefunc64_32_def* pzlib_filefunc_def,$/;" f +unz64local_getByte src/util/minizip/unzip.c /^local int unz64local_getByte(const zlib_filefunc64_32_def* pzlib_filefunc_def, voidpf filestream, int *pi)$/;" f +unz64local_getLong src/util/minizip/unzip.c /^local int unz64local_getLong (const zlib_filefunc64_32_def* pzlib_filefunc_def,$/;" f +unz64local_getLong64 src/util/minizip/unzip.c /^local int unz64local_getLong64 (const zlib_filefunc64_32_def* pzlib_filefunc_def,$/;" f +unz64local_getShort src/util/minizip/unzip.c /^local int unz64local_getShort (const zlib_filefunc64_32_def* pzlib_filefunc_def,$/;" f +unzClose src/util/minizip/unzip.c /^extern int ZEXPORT unzClose (unzFile file)$/;" f +unzCloseCurrentFile src/util/minizip/unzip.c /^extern int ZEXPORT unzCloseCurrentFile (unzFile file)$/;" f +unzFile src/util/minizip/unzip.h /^typedef unzFile__ *unzFile;$/;" t +unzFile src/util/minizip/unzip.h /^typedef voidp unzFile;$/;" t +unzFile__ src/util/minizip/unzip.h /^typedef struct TagunzFile__ { int unused; } unzFile__;$/;" t typeref:struct:TagunzFile__ +unzGetCurrentFileInfo src/util/minizip/unzip.c /^extern int ZEXPORT unzGetCurrentFileInfo (unzFile file,$/;" f +unzGetCurrentFileInfo64 src/util/minizip/unzip.c /^extern int ZEXPORT unzGetCurrentFileInfo64 (unzFile file,$/;" f +unzGetCurrentFileZStreamPos64 src/util/minizip/unzip.c /^extern ZPOS64_T ZEXPORT unzGetCurrentFileZStreamPos64( unzFile file)$/;" f +unzGetFilePos src/util/minizip/unzip.c /^extern int ZEXPORT unzGetFilePos($/;" f +unzGetFilePos64 src/util/minizip/unzip.c /^extern int ZEXPORT unzGetFilePos64(unzFile file, unz64_file_pos* file_pos)$/;" f +unzGetGlobalComment src/util/minizip/unzip.c /^extern int ZEXPORT unzGetGlobalComment (unzFile file, char * szComment, uLong uSizeBuf)$/;" f +unzGetGlobalInfo src/util/minizip/unzip.c /^extern int ZEXPORT unzGetGlobalInfo (unzFile file, unz_global_info* pglobal_info32)$/;" f +unzGetGlobalInfo64 src/util/minizip/unzip.c /^extern int ZEXPORT unzGetGlobalInfo64 (unzFile file, unz_global_info64* pglobal_info)$/;" f +unzGetLocalExtrafield src/util/minizip/unzip.c /^extern int ZEXPORT unzGetLocalExtrafield (unzFile file, voidp buf, unsigned len)$/;" f +unzGetOffset src/util/minizip/unzip.c /^extern uLong ZEXPORT unzGetOffset (unzFile file)$/;" f +unzGetOffset64 src/util/minizip/unzip.c /^extern ZPOS64_T ZEXPORT unzGetOffset64(unzFile file)$/;" f +unzGoToFilePos src/util/minizip/unzip.c /^extern int ZEXPORT unzGoToFilePos($/;" f +unzGoToFilePos64 src/util/minizip/unzip.c /^extern int ZEXPORT unzGoToFilePos64(unzFile file, const unz64_file_pos* file_pos)$/;" f +unzGoToFirstFile src/util/minizip/unzip.c /^extern int ZEXPORT unzGoToFirstFile (unzFile file)$/;" f +unzGoToNextFile src/util/minizip/unzip.c /^extern int ZEXPORT unzGoToNextFile (unzFile file)$/;" f +unzLocateFile src/util/minizip/unzip.c /^extern int ZEXPORT unzLocateFile (unzFile file, const char *szFileName, int iCaseSensitivity)$/;" f +unzOpen src/util/minizip/unzip.c /^extern unzFile ZEXPORT unzOpen (const char *path)$/;" f +unzOpen2 src/util/minizip/unzip.c /^extern unzFile ZEXPORT unzOpen2 (const char *path,$/;" f +unzOpen2_64 src/util/minizip/unzip.c /^extern unzFile ZEXPORT unzOpen2_64 (const void *path,$/;" f +unzOpen64 src/util/minizip/unzip.c /^extern unzFile ZEXPORT unzOpen64 (const void *path)$/;" f +unzOpenCurrentFile src/util/minizip/unzip.c /^extern int ZEXPORT unzOpenCurrentFile (unzFile file)$/;" f +unzOpenCurrentFile2 src/util/minizip/unzip.c /^extern int ZEXPORT unzOpenCurrentFile2 (unzFile file, int* method, int* level, int raw)$/;" f +unzOpenCurrentFile3 src/util/minizip/unzip.c /^extern int ZEXPORT unzOpenCurrentFile3 (unzFile file, int* method,$/;" f +unzOpenCurrentFilePassword src/util/minizip/unzip.c /^extern int ZEXPORT unzOpenCurrentFilePassword (unzFile file, const char* password)$/;" f +unzOpenInternal src/util/minizip/unzip.c /^local unzFile unzOpenInternal (const void *path,$/;" f +unzReadCurrentFile src/util/minizip/unzip.c /^extern int ZEXPORT unzReadCurrentFile (unzFile file, voidp buf, unsigned len)$/;" f +unzSetOffset src/util/minizip/unzip.c /^extern int ZEXPORT unzSetOffset (unzFile file, uLong pos)$/;" f +unzSetOffset64 src/util/minizip/unzip.c /^extern int ZEXPORT unzSetOffset64(unzFile file, ZPOS64_T pos)$/;" f +unzStringFileNameCompare src/util/minizip/unzip.c /^extern int ZEXPORT unzStringFileNameCompare (const char* fileName1,$/;" f +unz_copyright src/util/minizip/unzip.c /^const char unz_copyright[] =$/;" v +unz_file_info src/util/minizip/unzip.h /^} unz_file_info;$/;" t typeref:struct:unz_file_info_s +unz_file_info64 src/util/minizip/unzip.h /^} unz_file_info64;$/;" t typeref:struct:unz_file_info64_s +unz_file_info64_internal src/util/minizip/unzip.c /^} unz_file_info64_internal;$/;" t typeref:struct:unz_file_info64_internal_s file: +unz_file_info64_internal_s src/util/minizip/unzip.c /^typedef struct unz_file_info64_internal_s$/;" s file: +unz_file_info64_s src/util/minizip/unzip.h /^typedef struct unz_file_info64_s$/;" s +unz_file_info_s src/util/minizip/unzip.h /^typedef struct unz_file_info_s$/;" s +unz_file_pos src/util/minizip/unzip.h /^} unz_file_pos;$/;" t typeref:struct:unz_file_pos_s +unz_file_pos_s src/util/minizip/unzip.h /^typedef struct unz_file_pos_s$/;" s +unz_global_info src/util/minizip/unzip.h /^} unz_global_info;$/;" t typeref:struct:unz_global_info_s +unz_global_info64 src/util/minizip/unzip.h /^} unz_global_info64;$/;" t typeref:struct:unz_global_info64_s +unz_global_info64_s src/util/minizip/unzip.h /^typedef struct unz_global_info64_s$/;" s +unz_global_info_s src/util/minizip/unzip.h /^typedef struct unz_global_info_s$/;" s +unzeof src/util/minizip/unzip.c /^extern int ZEXPORT unzeof (unzFile file)$/;" f +unztell src/util/minizip/unzip.c /^extern z_off_t ZEXPORT unztell (unzFile file)$/;" f +unztell64 src/util/minizip/unzip.c /^extern ZPOS64_T ZEXPORT unztell64 (unzFile file)$/;" f +update src/exprs/agg_fn_evaluator.cpp /^void AggFnEvaluator::update($/;" f class:doris::AggFnEvaluator +update src/exprs/aggregate_functions.cpp /^ inline void update(StringValue* sv) {$/;" f class:doris::MultiDistinctStringCountState +update src/exprs/aggregate_functions.cpp /^ void update(DateTimeVal& t) {$/;" f class:doris::MultiDistinctCountDateState +update src/exprs/aggregate_functions.cpp /^ void update(DecimalV2Val& t) {$/;" f class:doris::MultiDistinctDecimalV2State +update src/exprs/aggregate_functions.cpp /^ void update(DecimalVal& t) {$/;" f class:doris::MultiDistinctDecimalState +update src/exprs/aggregate_functions.cpp /^ void update(T& t) {$/;" f class:doris::MultiDistinctNumericState +update src/exprs/bitmap_function.cpp /^ void update(const T& key, const BitmapValue& bitmap) {$/;" f struct:doris::BitmapIntersect +update src/olap/aggregate_func.h /^ inline void update(RowCursorCell* dst, const RowCursorCell& src, MemPool* mem_pool) const {$/;" f class:doris::AggregateInfo +update src/olap/aggregate_func.h /^ static void update(RowCursorCell* dst, const RowCursorCell& src, MemPool* mem_pool) {$/;" f struct:doris::AggregateFuncTraits +update src/olap/aggregate_func.h /^ static void update(RowCursorCell* dst, const RowCursorCell& src, MemPool* mem_pool) {$/;" f struct:doris::BaseAggregateFuncs +update src/olap/hll.cpp /^void HyperLogLog::update(uint64_t hash_value) {$/;" f class:doris::HyperLogLog +update src/runtime/heartbeat_flags.h /^ void update(uint64_t flags) {$/;" f class:doris::HeartbeatFlags +update src/util/doris_metrics.cpp /^void DorisMetrics::update() {$/;" f class:doris::DorisMetrics +update src/util/md5.cpp /^void Md5Digest::update(const void* data, size_t length) {$/;" f class:doris::Md5Digest +update src/util/progress_updater.cpp /^void ProgressUpdater::update(int64_t delta) {$/;" f class:doris::ProgressUpdater +update src/util/runtime_profile.cpp /^void RuntimeProfile::update(const TRuntimeProfileTree& thrift_profile) {$/;" f class:doris::RuntimeProfile +update src/util/runtime_profile.cpp /^void RuntimeProfile::update(const std::vector& nodes, int* idx) {$/;" f class:doris::RuntimeProfile +update src/util/runtime_profile.h /^ virtual void update(int64_t delta) {$/;" f class:doris::RuntimeProfile::Counter +update src/util/system_metrics.cpp /^void SystemMetrics::update() {$/;" f class:doris::SystemMetrics +updateCumulative src/util/tdigest.h /^ void updateCumulative() {$/;" f class:doris::TDigest +update_agg_row src/exec/pre_aggregation_node.cpp /^Status PreAggregationNode::update_agg_row(TupleRow* agg_row, TupleRow* probe_row) {$/;" f class:doris::PreAggregationNode +update_and_fetch src/common/atomic.h /^ T update_and_fetch(T delta) {$/;" f class:doris::AtomicInt +update_capacity src/olap/data_dir.cpp /^Status DataDir::update_capacity() {$/;" f class:doris::DataDir +update_counter src/exec/olap_scanner.cpp /^void OlapScanner::update_counter() {$/;" f class:doris::OlapScanner +update_eof src/exec/plain_text_line_reader.cpp /^inline bool PlainTextLineReader::update_eof() {$/;" f class:doris::PlainTextLineReader +update_field_pos_and_find_line_delimiter src/exec/plain_text_line_reader.cpp /^uint8_t* PlainTextLineReader::update_field_pos_and_find_line_delimiter($/;" f class:doris::PlainTextLineReader +update_fn_ src/exprs/agg_fn.h /^ void* update_fn_ = nullptr;$/;" m class:doris::AggFn +update_func src/runtime/dpp_sink.cpp /^ typedef void (*update_func) (SlotRef*, TupleRow*, TupleRow*);$/;" t class:doris::Translator file: +update_hll_set src/runtime/dpp_sink.cpp /^void HllDppSinkMerge::update_hll_set(TupleRow* agg_row, TupleRow* row, $/;" f class:doris::HllDppSinkMerge +update_limits src/runtime/mem_tracker.h /^ static void update_limits(int64_t bytes, std::vector* limits) {$/;" f class:doris::MemTracker +update_local_cgroups src/agent/cgroups_mgr.cpp /^AgentStatus CgroupsMgr::update_local_cgroups(const TFetchResourceResult& new_fetched_resource) {$/;" f class:doris::CgroupsMgr +update_max src/common/atomic.h /^ void update_max(T value) {$/;" f class:doris::AtomicInt +update_max src/runtime/dpp_sink.cpp /^static void update_max(SlotRef* ref, TupleRow* agg_row, TupleRow *row) {$/;" f namespace:doris +update_max_slot src/exec/pre_aggregation_node.cpp /^void update_max_slot(void* slot, void* value) {$/;" f namespace:doris +update_mem_limlits src/exprs/agg_fn_evaluator.cpp /^void AggFnEvaluator::update_mem_limlits(int len) {$/;" f class:doris::AggFnEvaluator +update_mem_trackers src/exprs/agg_fn_evaluator.cpp /^inline void AggFnEvaluator::update_mem_trackers(bool is_filter, bool is_add_buckets, int len) {$/;" f class:doris::AggFnEvaluator +update_min src/common/atomic.h /^ void update_min(T value) {$/;" f class:doris::AtomicInt +update_min src/runtime/dpp_sink.cpp /^static void update_min(SlotRef *ref, TupleRow *agg_row, TupleRow *row) {$/;" f namespace:doris +update_min_slot src/exec/pre_aggregation_node.cpp /^void update_min_slot(void* slot, void* value) {$/;" f namespace:doris +update_node_add_batch_counter src/exec/tablet_sink.h /^ void update_node_add_batch_counter(int64_t be_id, int64_t add_batch_time_ns, int64_t wait_lock_time_ns) {$/;" f class:doris::stream_load::OlapTableSink +update_num_rows_load_filtered src/runtime/runtime_state.h /^ void update_num_rows_load_filtered(int64_t num_rows) {$/;" f class:doris::RuntimeState +update_num_rows_load_total src/runtime/runtime_state.h /^ void update_num_rows_load_total(int64_t num_rows) {$/;" f class:doris::RuntimeState +update_num_rows_load_unselected src/runtime/runtime_state.h /^ void update_num_rows_load_unselected(int64_t num_rows) {$/;" f class:doris::RuntimeState +update_or_merge src/exprs/agg_fn_evaluator.cpp /^void AggFnEvaluator::update_or_merge(FunctionContext* agg_fn_ctx, TupleRow* row,$/;" f class:doris::AggFnEvaluator +update_pool_quotas src/runtime/thread_resource_mgr.cpp /^void ThreadResourceMgr::update_pool_quotas(ResourcePool* new_pool) {$/;" f class:doris::ThreadResourceMgr +update_pool_quotas src/runtime/thread_resource_mgr.h /^ void update_pool_quotas() {$/;" f class:doris::ThreadResourceMgr +update_prepared_size src/olap/rowset/segment_v2/binary_plain_page.h /^ void update_prepared_size(size_t added_size) {$/;" f class:doris::segment_v2::BinaryPlainPageBuilder +update_queue_status src/runtime/result_queue_mgr.cpp /^void ResultQueueMgr::update_queue_status(const TUniqueId& fragment_instance_id, const Status& status) {$/;" f class:doris::ResultQueueMgr +update_root_path_info src/olap/tablet_manager.cpp /^void TabletManager::update_root_path_info(std::map* path_map,$/;" f class:doris::TabletManager +update_row src/runtime/dpp_sink.cpp /^void Translator::update_row(TupleRow* agg_row, TupleRow* row) {$/;" f class:doris::Translator +update_status src/exec/broker_scan_node.h /^ bool update_status(const Status& new_status) {$/;" f class:doris::BrokerScanNode +update_status src/exec/es_http_scan_node.h /^ bool update_status(const Status& new_status) {$/;" f class:doris::EsHttpScanNode +update_status src/runtime/fragment_mgr.cpp /^ Status update_status(Status status) {$/;" f class:doris::FragmentExecState +update_status src/runtime/plan_fragment_executor.cpp /^void PlanFragmentExecutor::update_status(const Status& status) {$/;" f class:doris::PlanFragmentExecutor +update_status src/runtime/record_batch_queue.cpp /^void RecordBatchQueue::update_status(const Status& status) {$/;" f class:doris::RecordBatchQueue +update_sum src/runtime/dpp_sink.cpp /^static void update_sum(SlotRef* ref, TupleRow* agg_row, TupleRow *row) {$/;" f namespace:doris +update_sum src/runtime/dpp_sink.cpp /^void update_sum(SlotRef* ref, TupleRow* agg_row, TupleRow *row) {$/;" f namespace:doris +update_sum_slot src/exec/pre_aggregation_node.cpp /^void update_sum_slot(void* slot, void* value) {$/;" f namespace:doris +update_thread src/util/runtime_profile.h /^ boost::scoped_ptr update_thread;$/;" m struct:doris::RuntimeProfile::PeriodicCounterUpdateState +update_tuple src/exec/aggregation_node.cpp /^void AggregationNode::update_tuple(Tuple* tuple, TupleRow* row) {$/;" f class:doris::AggregationNode +update_tuple src/exec/partitioned_aggregation_node.cc /^void PartitionedAggregationNode::update_tuple(FunctionContext** agg_fn_ctxs,$/;" f class:doris::PartitionedAggregationNode +update_users_resource src/agent/user_resource_listener.cpp /^void UserResourceListener::update_users_resource(int64_t new_version) {$/;" f class:doris::UserResourceListener +updater test/util/core_local_test.cpp /^void updater(CoreLocalValue* value, int64_t* used_ns) {$/;" f namespace:doris +upload src/runtime/snapshot_loader.cpp /^Status SnapshotLoader::upload($/;" f class:doris::SnapshotLoader +upper src/exprs/string_functions.cpp /^StringVal StringFunctions::upper(FunctionContext* context, const StringVal& str) {$/;" f class:doris::StringFunctions +upper_bound src/olap/rowset/segment_v2/segment.h /^ ShortKeyIndexIterator upper_bound(const Slice& key) const {$/;" f class:doris::segment_v2::Segment +upper_bound src/olap/short_key_index.h /^ ShortKeyIndexIterator upper_bound(const Slice& key) const {$/;" f class:doris::ShortKeyIndexDecoder +upper_bound_keys src/olap/rowset/rowset_reader_context.h /^ const std::vector* upper_bound_keys = nullptr;$/;" m struct:doris::RowsetReaderContext +upper_key src/olap/iterators.h /^ const RowCursor* upper_key;$/;" m struct:doris::StorageReadOptions::KeyRange +upper_udf test/udf/udf_test.cpp /^StringVal upper_udf(FunctionContext* context, const StringVal& input) {$/;" f namespace:doris_udf +uri src/http/http_request.h /^ const std::string& uri() const {$/;" f class:doris::HttpRequest +url_decode src/util/url_coding.cpp /^bool url_decode(const std::string& in, std::string* out) {$/;" f namespace:doris +url_encode src/util/url_coding.cpp /^static inline void url_encode(const char* in, int in_len, std::string* out) {$/;" f namespace:doris +url_encode src/util/url_coding.cpp /^void url_encode(const std::string& in, std::string* out) {$/;" f namespace:doris +url_encode src/util/url_coding.cpp /^void url_encode(const std::vector& in, std::string* out) {$/;" f namespace:doris +use_page_cache src/olap/iterators.h /^ bool use_page_cache = false;$/;" m class:doris::StorageReadOptions +use_page_cache src/olap/reader.h /^ bool use_page_cache = false;$/;" m struct:doris::ReaderParams +use_page_cache src/olap/rowset/rowset_reader_context.h /^ bool use_page_cache = false;$/;" m struct:doris::RowsetReaderContext +use_page_cache src/olap/rowset/segment_v2/column_reader.h /^ bool use_page_cache = false;$/;" m struct:doris::segment_v2::ColumnIteratorOptions +use_page_cache src/olap/rowset/segment_v2/page_io.h /^ bool use_page_cache = true;$/;" m struct:doris::segment_v2::PageReadOptions +use_streaming src/runtime/stream_load/stream_load_context.h /^ bool use_streaming = false;$/;" m class:doris::StreamLoadContext +used_bits src/olap/serialize.h /^inline uint32_t used_bits(uint64_t value) {$/;" f namespace:doris::ser +used_reservation_ src/runtime/bufferpool/reservation_tracker.h /^ int64_t used_reservation_;$/;" m class:doris::ReservationTracker +user src/common/utils.h /^ std::string user;$/;" m struct:doris::AuthInfo +user src/exec/mysql_scanner.h /^ std::string user;$/;" m struct:doris::MysqlScannerParam +user src/exec/schema_scanner.h /^ const std::string* user; \/\/ deprecated$/;" m struct:doris::SchemaScannerParam +user src/runtime/descriptors.h /^ const std::string user() const {$/;" f class:doris::MySQLTableDescriptor +user src/runtime/mysql_table_writer.h /^ std::string user;$/;" m struct:doris::MysqlConnInfo +user src/runtime/runtime_state.h /^ const std::string& user() const {$/;" f class:doris::RuntimeState +user src/udf/udf.cpp /^ const std::string user() const {$/;" f class:doris::RuntimeState +user src/udf/udf.cpp /^const char* FunctionContext::user() const {$/;" f class:doris_udf::FunctionContext +user src/util/mysql_load_error_hub.h /^ std::string user;$/;" m struct:doris::MysqlLoadErrorHub::MysqlInfo +user_ip src/common/utils.h /^ std::string user_ip;$/;" m struct:doris::AuthInfo +user_ip src/exec/schema_scanner.h /^ const std::string* user_ip; \/\/ deprecated$/;" m struct:doris::SchemaScannerParam +ushort src/gutil/port.h /^typedef unsigned short ushort;$/;" t +ushort src/gutil/utf/utfdef.h /^typedef unsigned short ushort;$/;" t +ushort src/gutil/utf/utfdef.h 2;" d +using_small_buffers src/runtime/buffered_tuple_stream.h /^ bool using_small_buffers() const {$/;" f class:doris::BufferedTupleStream +using_small_buffers src/runtime/buffered_tuple_stream2.h /^ bool using_small_buffers() const { return _use_small_buffers; }$/;" f class:doris::BufferedTupleStream2 +utc_timestamp src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::utc_timestamp(FunctionContext* context) {$/;" f class:doris::TimestampFunctions +utf8_range src/util/utf8_check.cpp /^bool utf8_range(const char *data, size_t len)$/;" f namespace:doris +util_hash src/gutil/hash/city.cc /^namespace util_hash {$/;" n file: +util_hash src/gutil/hash/city.h /^namespace util_hash {$/;" n +utils test/exprs/bitmap_function_test.cpp /^ FunctionUtils* utils;$/;" m class:doris::BitmapFunctionsTest file: +utils test/exprs/hll_function_test.cpp /^ FunctionUtils* utils;$/;" m class:doris::HllFunctionsTest file: +utils test/exprs/timestamp_functions_test.cpp /^ FunctionUtils* utils;$/;" m class:doris::TimestampFunctionsTest file: +uvlong src/gutil/utf/utfdef.h 6;" d +uword_t src/gutil/integral_types.h /^typedef unsigned long uword_t;$/;" t +va_copy src/gutil/port.h /^inline void va_copy(va_list& a, va_list& b) {$/;" f +val output/udf/include/udf.h /^ __int128 val;$/;" m struct:doris_udf::DecimalV2Val +val output/udf/include/udf.h /^ __int128 val;$/;" m struct:doris_udf::LargeIntVal +val output/udf/include/udf.h /^ bool val;$/;" m struct:doris_udf::BooleanVal +val output/udf/include/udf.h /^ double val;$/;" m struct:doris_udf::DoubleVal +val output/udf/include/udf.h /^ float val;$/;" m struct:doris_udf::FloatVal +val output/udf/include/udf.h /^ int16_t val;$/;" m struct:doris_udf::SmallIntVal +val output/udf/include/udf.h /^ int32_t val;$/;" m struct:doris_udf::IntVal +val output/udf/include/udf.h /^ int64_t val;$/;" m struct:doris_udf::BigIntVal +val output/udf/include/udf.h /^ int8_t val;$/;" m struct:doris_udf::TinyIntVal +val src/udf/udf.h /^ __int128 val;$/;" m struct:doris_udf::DecimalV2Val +val src/udf/udf.h /^ __int128 val;$/;" m struct:doris_udf::LargeIntVal +val src/udf/udf.h /^ bool val;$/;" m struct:doris_udf::BooleanVal +val src/udf/udf.h /^ double val;$/;" m struct:doris_udf::DoubleVal +val src/udf/udf.h /^ float val;$/;" m struct:doris_udf::FloatVal +val src/udf/udf.h /^ int16_t val;$/;" m struct:doris_udf::SmallIntVal +val src/udf/udf.h /^ int32_t val;$/;" m struct:doris_udf::IntVal +val src/udf/udf.h /^ int64_t val;$/;" m struct:doris_udf::BigIntVal +val src/udf/udf.h /^ int8_t val;$/;" m struct:doris_udf::TinyIntVal +val_set src/exprs/new_in_predicate.h /^ std::set val_set;$/;" m struct:doris::InPredicate::SetLookupState +valid src/olap/generic_iterators.cpp /^ bool valid() const { return _valid; }$/;" f class:doris::MergeIteratorContext +valid src/olap/rowset/segment_v2/ordinal_page_index.h /^ bool valid() const { return _cur_idx < _index->_num_pages; }$/;" f class:doris::segment_v2::OrdinalPageIndexIterator +valid src/olap/short_key_index.h /^inline bool ShortKeyIndexIterator::valid() const {$/;" f class:doris::ShortKeyIndexIterator +valid_closing_ src/gutil/strings/split.cc /^ bool valid_closing_[256];$/;" m class:__anon31::ClosingSymbolLookup file: +valid_data_len src/runtime/buffered_block_mgr.h /^ int64_t valid_data_len() const {$/;" f class:doris::BufferedBlockMgr::Block +valid_data_len src/runtime/buffered_block_mgr2.h /^ int64_t valid_data_len() const {$/;" f class:doris::BufferedBlockMgr2::Block +valid_datetime src/olap/utils.cpp /^bool valid_datetime(const string &value_str) {$/;" f namespace:doris +valid_decimal src/olap/utils.cpp /^bool valid_decimal(const string &value_str, const uint32_t precision, const uint32_t frac) {$/;" f namespace:doris +valid_signed_number src/olap/utils.cpp /^bool valid_signed_number(const std::string& value_str) {$/;" f namespace:doris +valid_signed_number src/olap/utils.h /^bool valid_signed_number(const std::string& value_str) {$/;" f namespace:doris +valid_thread_id_ src/gutil/threading/thread_collision_warner.h /^ volatile subtle::Atomic64 valid_thread_id_;$/;" m class:base::ThreadCollisionWarner +valid_unsigned_number src/olap/utils.h /^bool valid_unsigned_number(const std::string& value_str) {$/;" f namespace:doris +validat_udf test/udf/udf_test.cpp /^IntVal validat_udf(FunctionContext* context) {$/;" f namespace:doris_udf +validate src/exec/es/es_query_builder.cpp /^void BooleanQueryBuilder::validate(const std::vector& espredicates, std::vector* result) {$/;" f class:doris::BooleanQueryBuilder +validate src/olap/file_helper.h /^OLAPStatus FileHeader::validate($/;" f class:doris::FileHeader +validate src/olap/rowset/segment_group.cpp /^OLAPStatus SegmentGroup::validate() {$/;" f class:doris::SegmentGroup +validate src/runtime/buffered_block_mgr2.cc /^bool BufferedBlockMgr2::Block::validate() const {$/;" f class:doris::BufferedBlockMgr2::Block +validate src/runtime/buffered_block_mgr2.cc /^bool BufferedBlockMgr2::validate() const {$/;" f class:doris::BufferedBlockMgr2 +validate src/runtime/disk_io_mgr_reader_context.cc /^bool DiskIoMgr::RequestContext::validate() const {$/;" f class:doris::DiskIoMgr::RequestContext +validate src/runtime/disk_io_mgr_scan_range.cc /^bool DiskIoMgr::ScanRange::validate() {$/;" f class:doris::DiskIoMgr::ScanRange +validate src/util/internal_queue.h /^ bool validate() {$/;" f class:doris::InternalQueueBase +validate_all_inactive src/runtime/disk_io_mgr.cc /^ bool validate_all_inactive() {$/;" f class:doris::DiskIoMgr::RequestContextCache +validate_checksum src/olap/push_handler.h /^ bool validate_checksum() {$/;" f class:doris::IBinaryReader +validate_empty_or_correct test/runtime/disk_io_mgr_test.cpp /^ static void validate_empty_or_correct(const char* expected, const char* buffer, int len) {$/;" f class:doris::DiskIoMgrTest +validate_fail test/udf/udf_test.cpp /^IntVal validate_fail(FunctionContext* context) {$/;" f namespace:doris_udf +validate_match test/exec/hash_table_test.cpp /^ void validate_match(TupleRow* probe_row, TupleRow* build_row) {$/;" f class:doris::HashTableTest +validate_mem test/udf/udf_test.cpp /^IntVal validate_mem(FunctionContext* context) {$/;" f namespace:doris_udf +validate_scan_range src/runtime/disk_io_mgr.cc /^Status DiskIoMgr::validate_scan_range(ScanRange* range) {$/;" f class:doris::DiskIoMgr +validate_scan_range test/runtime/disk_io_mgr_test.cpp /^ static void validate_scan_range(DiskIoMgr::ScanRange* range, const char* expected,$/;" f class:doris::DiskIoMgrTest +validate_string test/runtime/string_buffer_test.cpp /^void validate_string(const std::string& std_str, const StringBuffer& str) {$/;" f namespace:doris +validate_sync_read test/runtime/disk_io_mgr_test.cpp /^ static void validate_sync_read(DiskIoMgr* io_mgr, DiskIoMgr::RequestContext* reader,$/;" f class:doris::DiskIoMgrTest +validate_utf8 src/util/utf8_check.cpp /^bool validate_utf8(const char *src, size_t len) {$/;" f namespace:doris +validate_utf8_fast src/util/simdutf8check.h /^static bool validate_utf8_fast(const char *src, size_t len) {$/;" f +validate_utf8_fast_avx src/util/simdutf8check.h /^static bool validate_utf8_fast_avx(const char *src, size_t len) {$/;" f +validate_utf8_fast_avx_asciipath src/util/simdutf8check.h /^static bool validate_utf8_fast_avx_asciipath(const char *src, size_t len) {$/;" f +validate_utf8_naive src/util/utf8_check.cpp /^bool validate_utf8_naive(const char *data, size_t len) {$/;" f namespace:doris +value output/udf/include/udf.h /^ const __int128& value() const { return val; }$/;" f struct:doris_udf::DecimalV2Val +value src/exec/es/es_predicate.h /^ ExtLiteral value;$/;" m struct:doris::ExtBinaryPredicate +value src/exec/es/es_predicate.h /^ ExtLiteral value;$/;" m struct:doris::ExtLikePredicate +value src/exec/schema_scanner/schema_variables_scanner.h /^ const char *value;$/;" m struct:doris::SchemaVariablesScanner::VariableStruct +value src/gutil/gscoped_ptr.h /^ value = !base::is_convertible::value &&$/;" e enum:doris::internal::IsNotRefCounted::__anon23 +value src/gutil/strings/split_internal.h /^ static const bool value = (sizeof(test(0)) == sizeof(base::big_));$/;" m struct:strings::internal::Splitter::is_map +value src/gutil/template_util.h /^ static const T value = v;$/;" m struct:base::integral_constant +value src/gutil/template_util.h /^template const T integral_constant::value;$/;" m class:base::integral_constant +value src/gutil/type_traits.h /^ static const bool value = sizeof(tester(0)) == sizeof(small_);$/;" m struct:base::internal::is_class_or_union +value src/olap/lru_cache.cpp /^void* ShardedLRUCache::value(Handle* handle) {$/;" f class:doris::ShardedLRUCache +value src/olap/lru_cache.h /^ void* value;$/;" m struct:doris::CachePriority::LRUHandle +value src/olap/olap_common.h /^ int64_t value;$/;" m struct:doris::Vertex +value src/runtime/decimalv2_value.h /^ inline const int128_t& value() const { return _value;}$/;" f class:doris::DecimalV2Value +value src/runtime/decimalv2_value.h /^ inline int128_t& value() { return _value; }$/;" f class:doris::DecimalV2Value +value src/udf/udf.h /^ const __int128& value() const { return val; }$/;" f struct:doris_udf::DecimalV2Val +value src/util/metrics.h /^ T value() const {$/;" f class:doris::CoreLocalCounter +value src/util/metrics.h /^ T value() const {$/;" f class:doris::LockSimpleMetric +value src/util/metrics.h /^ std::string value;$/;" m struct:doris::MetricLabel +value src/util/runtime_profile.h /^ virtual int64_t value() const {$/;" f class:doris::RuntimeProfile::Counter +value src/util/runtime_profile.h /^ virtual int64_t value() const {$/;" f class:doris::RuntimeProfile::DerivedCounter +value src/util/types.h /^ __int128 value;$/;" m struct:doris::PackedInt128 +value test/udf/uda_test.cpp /^ uint8_t* value;$/;" m struct:doris_udf::MinState file: +value test/util/internal_queue_test.cpp /^ int value;$/;" m struct:doris::IntNode file: +value_bytes src/olap/rowset/run_length_integer_writer.h /^ value_bytes: 3,$/;" m struct:doris::RunLengthIntegerWriter::ShortRepeatHead +value_ops src/runtime/dpp_sink_internal.h /^ const std::vector& value_ops() const {$/;" f class:doris::RollupSchema +value_slice src/olap/lru_cache.cpp /^Slice ShardedLRUCache::value_slice(Handle* handle) {$/;" f class:doris::ShardedLRUCache +value_to_string src/exec/es/es_predicate.cpp /^std::string ExtLiteral::value_to_string() {$/;" f class:doris::ExtLiteral +value_type src/gutil/strings/stringpiece.h /^ typedef char value_type;$/;" t class:StringPiece +value_type src/gutil/template_util.h /^ typedef T value_type;$/;" t struct:base::integral_constant +value_type src/util/bitmap_value.h /^ typedef uint64_t value_type;$/;" t class:doris::detail::final +values src/exec/es/es_predicate.h /^ const std::vector values;$/;" m struct:doris::ExtFunction +values src/exec/es/es_predicate.h /^ std::vector values;$/;" m struct:doris::ExtInPredicate +values src/olap/tuple.h /^ const std::vector& values() const { return _values; }$/;" f class:doris::OlapTuple +values src/runtime/dpp_sink_internal.h /^ const std::vector& values() const {$/;" f class:doris::RollupSchema +var_result_offset src/exec/new_partitioned_hash_table.h /^ int ALWAYS_INLINE var_result_offset() const { return var_result_offset_; }$/;" f class:doris::NewPartitionedHashTableCtx::ExprValuesCache +var_result_offset_ src/exec/new_partitioned_hash_table.h /^ int var_result_offset_;$/;" m class:doris::NewPartitionedHashTableCtx::ExprValuesCache +var_strcaseprefix src/gutil/strings/util.h /^inline CharStar var_strcaseprefix(CharStar str, const char* prefix) {$/;" f +var_strprefix src/gutil/strings/util.h /^inline CharStar var_strprefix(CharStar str, const char* prefix) {$/;" f +varargs_buffer src/udf/udf_internal.h /^ uint8_t* varargs_buffer() { $/;" f class:doris::FunctionContextImpl +varchar_val test/runtime/dpp_writer_test.cpp /^ std::string varchar_val;$/;" m struct:doris::TestDataTuple file: +varchar_val test/runtime/mysql_table_writer_test.cpp /^ std::string varchar_val;$/;" m struct:doris::TestDataTuple file: +variable src/plugin/plugin.h /^ void* variable;$/;" m struct:doris::Plugin +variable_length src/exprs/expr.cpp /^ bool variable_length;$/;" m struct:doris::MemLayoutData file: +varint_length src/util/coding.h /^inline int varint_length(uint64_t v) {$/;" f namespace:doris +varlen_byte_size src/runtime/tuple.cpp /^int64_t Tuple::varlen_byte_size(const TupleDescriptor& desc) const {$/;" f class:doris::Tuple +vec_cond_ns src/olap/olap_common.h /^ int64_t vec_cond_ns = 0;$/;" m struct:doris::OlapReaderStatistics +vector_as_array src/gutil/stl_util.h /^inline T* vector_as_array(vector* v) {$/;" f +vector_as_array src/gutil/stl_util.h /^inline const T* vector_as_array(const vector* v) {$/;" f +vector_compute_fn src/exprs/slot_ref.h /^inline bool SlotRef::vector_compute_fn(Expr* expr, VectorizedRowBatch* \/* batch *\/) {$/;" f class:doris::SlotRef +vendor_name src/gutil/cpu.h /^ const std::string& vendor_name() const { return cpu_vendor_; }$/;" f class:base::final +verify_checksum src/olap/rowset/segment_v2/column_reader.h /^ bool verify_checksum = true;$/;" m struct:doris::segment_v2::ColumnReaderOptions +verify_checksum src/olap/rowset/segment_v2/page_io.h /^ bool verify_checksum = true;$/;" m struct:doris::segment_v2::PageReadOptions +verify_cpu_requirements src/util/cpu_info.cpp /^void CpuInfo::verify_cpu_requirements() {$/;" f class:doris::CpuInfo +verify_is_directory src/util/filesystem_util.cc /^Status FileSystemUtil::verify_is_directory(const string& directory_path) {$/;" f class:doris::FileSystemUtil +verify_performance_governor src/util/cpu_info.cpp /^void CpuInfo::verify_performance_governor() {$/;" f class:doris::CpuInfo +verify_turbo_disabled src/util/cpu_info.cpp /^void CpuInfo::verify_turbo_disabled() {$/;" f class:doris::CpuInfo +version src/exec/decompressor.h /^ uint16_t version;$/;" m struct:doris::LzopDecompressor::HeaderInfo +version src/exec/tablet_info.h /^ int64_t version() const { return _t_param.version; }$/;" f class:doris::OlapTableLocationParam +version src/exec/tablet_info.h /^ int64_t version() const { return _t_param.version; }$/;" f class:doris::OlapTablePartitionParam +version src/exec/tablet_info.h /^ int64_t version() const { return _version; }$/;" f class:doris::OlapTableSchemaParam +version src/exprs/utility_functions.cpp /^StringVal UtilityFunctions::version(FunctionContext* ctx) {$/;" f class:doris::UtilityFunctions +version src/olap/file_helper.h /^ uint32_t version;$/;" m struct:doris::_FixedFileHeaderV2 +version src/olap/olap_common.h /^ int8_t version = 0;$/;" m struct:doris::RowsetId +version src/olap/reader.cpp /^ int32_t version() const {$/;" f class:doris::CollectIterator::ChildCtx +version src/olap/reader.h /^ Version version;$/;" m struct:doris::ReaderParams +version src/olap/rowset/alpha_rowset_reader.cpp /^Version AlphaRowsetReader::version() {$/;" f class:doris::AlphaRowsetReader +version src/olap/rowset/column_data.h /^ Version version() const {$/;" f class:doris::ColumnData +version src/olap/rowset/rowset.h /^ Version version() const { return rowset_meta()->version(); }$/;" f class:doris::Rowset +version src/olap/rowset/rowset_meta.h /^ Version version() const {$/;" f class:doris::RowsetMeta +version src/olap/rowset/rowset_writer_context.h /^ Version version;$/;" m struct:doris::RowsetWriterContext +version src/olap/rowset/segment_group.h /^ inline Version version() const { return _version; }$/;" f class:doris::SegmentGroup +version src/olap/tablet_sync_service.h /^ Version version;$/;" m struct:doris::FetchRowsetMetaTask +version src/udf/udf.cpp /^FunctionContext::DorisVersion FunctionContext::version() const {$/;" f class:doris_udf::FunctionContext +version src/util/minizip/unzip.h /^ uLong version; \/* version made by 2 bytes *\/$/;" m struct:unz_file_info64_s +version src/util/minizip/unzip.h /^ uLong version; \/* version made by 2 bytes *\/$/;" m struct:unz_file_info_s +version_count src/olap/tablet.h /^inline int Tablet::version_count() const {$/;" f class:doris::Tablet +version_count src/olap/tablet_meta.h /^inline size_t TabletMeta::version_count() const {$/;" f class:doris::TabletMeta +version_for_delete_predicate src/olap/tablet.cpp /^bool Tablet::version_for_delete_predicate(const Version& version) {$/;" f class:doris::Tablet +version_for_delete_predicate src/olap/tablet_meta.cpp /^bool TabletMeta::version_for_delete_predicate(const Version& version) {$/;" f class:doris::TabletMeta +version_for_load_deletion src/olap/tablet.cpp /^bool Tablet::version_for_load_deletion(const Version& version) {$/;" f class:doris::Tablet +version_hash src/olap/rowset/alpha_rowset_reader.cpp /^VersionHash AlphaRowsetReader::version_hash() {$/;" f class:doris::AlphaRowsetReader +version_hash src/olap/rowset/column_data.h /^ VersionHash version_hash() const {$/;" f class:doris::ColumnData +version_hash src/olap/rowset/rowset.h /^ VersionHash version_hash() const { return rowset_meta()->version_hash(); }$/;" f class:doris::Rowset +version_hash src/olap/rowset/rowset_meta.h /^ VersionHash version_hash() const {$/;" f class:doris::RowsetMeta +version_hash src/olap/rowset/rowset_writer_context.h /^ VersionHash version_hash;$/;" m struct:doris::RowsetWriterContext +version_hash src/olap/rowset/segment_group.h /^ inline VersionHash version_hash() const { return _version_hash; }$/;" f class:doris::SegmentGroup +version_needed src/exec/decompressor.h /^ uint16_t version_needed;$/;" m struct:doris::LzopDecompressor::HeaderInfo +version_needed src/util/minizip/unzip.h /^ uLong version_needed; \/* version needed to extract 2 bytes *\/$/;" m struct:unz_file_info64_s +version_needed src/util/minizip/unzip.h /^ uLong version_needed; \/* version needed to extract 2 bytes *\/$/;" m struct:unz_file_info_s +visit src/http/action/metrics_action.cpp /^void PrometheusMetricsVisitor::visit(const std::string& prefix,$/;" f class:doris::PrometheusMetricsVisitor +visit src/http/action/metrics_action.cpp /^void SimpleCoreMetricsVisitor::visit(const std::string& prefix,$/;" f class:doris::SimpleCoreMetricsVisitor +visit test/util/doris_metrics_test.cpp /^ void visit(const std::string& prefix, const std::string& name,$/;" f class:doris::TestMetricsVisitor +visit test/util/new_metrics_test.cpp /^ void visit(const std::string& prefix, const std::string& name,$/;" f class:doris::TestMetricsVisitor +visit test/util/system_metrics_test.cpp /^ void visit(const std::string& prefix, const std::string& name,$/;" f class:doris::TestMetricsVisitor +vlong src/gutil/utf/utfdef.h 5;" d +volatile build/CMakeFiles/3.12.3/CompilerIdC/CMakeCCompilerId.c 11;" d file: +vsnprintf src/gutil/port.h 832;" d +wait src/olap/memtable_flush_executor.cpp /^OLAPStatus FlushToken::wait() {$/;" f class:doris::FlushToken +wait src/util/barrier.h /^ void wait() {$/;" f class:doris::Barrier +wait src/util/condition_variable.cpp /^void ConditionVariable::wait() const {$/;" f class:doris::ConditionVariable +wait src/util/countdown_latch.h /^ void wait() const {$/;" f class:doris::CountDownLatch +wait src/util/semaphore.hpp /^ void wait() {$/;" f class:__anon35::Semaphore +wait src/util/threadpool.cpp /^void ThreadPool::wait() {$/;" f class:doris::ThreadPool +wait src/util/threadpool.cpp /^void ThreadPoolToken::wait() {$/;" f class:doris::ThreadPoolToken +wait test/olap/skiplist_test.cpp /^ void wait(ReaderState s) {$/;" f class:doris::TestState +wait_for src/util/condition_variable.cpp /^bool ConditionVariable::wait_for(const MonoDelta& delta) const {$/;" f class:doris::ConditionVariable +wait_for src/util/countdown_latch.h /^ bool wait_for(const MonoDelta& delta) const {$/;" f class:doris::CountDownLatch +wait_for src/util/threadpool.cpp /^bool ThreadPool::wait_for(const MonoDelta& delta) {$/;" f class:doris::ThreadPool +wait_for src/util/threadpool.cpp /^bool ThreadPoolToken::wait_for(const MonoDelta& delta) {$/;" f class:doris::ThreadPoolToken +wait_for_local_server src/util/thrift_util.cpp /^Status wait_for_local_server($/;" f namespace:doris +wait_for_report_notify src/olap/storage_engine.h /^ void wait_for_report_notify(int64_t timeout_sec, bool from_report_tablet_thread) {$/;" f class:doris::StorageEngine +wait_for_server src/util/thrift_util.cpp /^Status wait_for_server($/;" f namespace:doris +wait_for_tid src/util/thread.cpp /^int64_t Thread::wait_for_tid() const {$/;" f class:doris::Thread +wait_lock_time_ns test/runtime/load_channel_mgr_test.cpp /^int64_t wait_lock_time_ns;$/;" m namespace:doris file: +wait_until src/util/condition_variable.cpp /^bool ConditionVariable::wait_until(const MonoTime& until) const {$/;" f class:doris::ConditionVariable +wait_until src/util/countdown_latch.h /^ bool wait_until(const MonoTime& when) const {$/;" f class:doris::CountDownLatch +wait_until src/util/threadpool.cpp /^bool ThreadPool::wait_until(const MonoTime& until) {$/;" f class:doris::ThreadPool +wait_until src/util/threadpool.cpp /^bool ThreadPoolToken::wait_until(const MonoTime& until) {$/;" f class:doris::ThreadPoolToken +waiter test/util/counter_cond_variable_test.cpp /^void waiter() {$/;" f namespace:doris +waitpid src/gutil/linux_syscall_support.h /^ LSS_INLINE pid_t LSS_NAME(waitpid)(pid_t pid, int *status, int options){$/;" f +walltime_internal src/gutil/walltime.h /^namespace walltime_internal {$/;" n +warn src/gutil/threading/thread_collision_warner.cc /^void DCheckAsserter::warn(int64_t previous_thread_id, int64_t current_thread_id) {$/;" f class:base::DCheckAsserter +warn_after_ms src/util/thread.cpp /^ThreadJoiner& ThreadJoiner::warn_after_ms(int ms) {$/;" f class:doris::ThreadJoiner +warn_every_ms src/util/thread.cpp /^ThreadJoiner& ThreadJoiner::warn_every_ms(int ms) {$/;" f class:doris::ThreadJoiner +warn_with_bitshuffle_error src/olap/rowset/segment_v2/bitshuffle_page.cpp /^void warn_with_bitshuffle_error(int64_t val) {$/;" f namespace:doris::segment_v2 +warner_ src/gutil/threading/thread_collision_warner.h /^ ThreadCollisionWarner* warner_;$/;" m class:base::ThreadCollisionWarner::Check +warner_ src/gutil/threading/thread_collision_warner.h /^ ThreadCollisionWarner* warner_;$/;" m class:base::ThreadCollisionWarner::ScopedCheck +warner_ src/gutil/threading/thread_collision_warner.h /^ ThreadCollisionWarner* warner_;$/;" m class:base::ThreadCollisionWarner::ScopedRecursiveCheck +week src/runtime/datetime_value.cpp /^uint8_t DateTimeValue::week(uint8_t mode) const {$/;" f class:doris::DateTimeValue +week_of_year src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::week_of_year($/;" f class:doris::TimestampFunctions +weekday src/runtime/datetime_value.h /^ inline uint8_t weekday() const {$/;" f class:doris::DateTimeValue +weeks_add src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::weeks_add($/;" f class:doris::TimestampFunctions +weeks_diff src/exprs/timestamp_functions.cpp /^BigIntVal TimestampFunctions::weeks_diff($/;" f class:doris::TimestampFunctions +weeks_sub src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::weeks_sub($/;" f class:doris::TimestampFunctions +weightedAverage src/util/tdigest.h /^ static Value weightedAverage(Value x1, Value w1, Value x2, Value w2) {$/;" f class:doris::TDigest +weightedAverageSorted src/util/tdigest.h /^ static Value weightedAverageSorted(Value x1, Value w1, Value x2, Value w2) {$/;" f class:doris::TDigest +what src/olap/utils.h /^ virtual const char* what() const throw() {$/;" f class:doris::ComparatorException +when_val src/exprs/case_expr.cpp /^ AnyVal* when_val;$/;" m struct:doris::CaseExprState file: +whether_to_filter_value src/olap/rowset/segment_v2/segment_writer.h /^ bool whether_to_filter_value = false;$/;" m struct:doris::segment_v2::SegmentWriterOptions +wild src/exec/schema_scanner.h /^ const std::string* wild;$/;" m struct:doris::SchemaScannerParam +work_available src/runtime/disk_io_mgr_internal.h /^ boost::condition_variable work_available;$/;" m struct:doris::DiskIoMgr::DiskQueue +work_function src/util/priority_thread_pool.hpp /^ WorkFunction work_function;$/;" m struct:doris::PriorityThreadPool::Task +work_loop src/runtime/disk_io_mgr.cc /^void DiskIoMgr::work_loop(DiskQueue* disk_queue) {$/;" f class:doris::DiskIoMgr +work_thread src/util/batch_process_thread_pool.hpp /^ void work_thread(int thread_id) {$/;" f class:doris::BatchProcessThreadPool +work_thread src/util/priority_thread_pool.hpp /^ void work_thread(int thread_id) {$/;" f class:doris::PriorityThreadPool +worker test/util/counter_cond_variable_test.cpp /^void worker() {$/;" f namespace:doris +writable_blocks_total src/util/doris_metrics.cpp /^IntCounter DorisMetrics::writable_blocks_total;$/;" m class:doris::DorisMetrics file: +writable_blocks_total src/util/doris_metrics.h /^ static IntCounter writable_blocks_total;$/;" m class:doris::DorisMetrics +write src/exec/broker_writer.cpp /^Status BrokerWriter::write(const uint8_t* buf, size_t buf_len, size_t* written_len) {$/;" f class:doris::BrokerWriter +write src/exec/local_file_writer.cpp /^Status LocalFileWriter::write(const uint8_t* buf, size_t buf_len, size_t* written_len) {$/;" f class:doris::LocalFileWriter +write src/olap/delta_writer.cpp /^OLAPStatus DeltaWriter::write(Tuple* tuple) {$/;" f class:doris::DeltaWriter +write src/olap/file_helper.cpp /^OLAPStatus FileHandler::write(const void* buf, size_t buf_size) {$/;" f class:doris::FileHandler +write src/olap/file_helper.cpp /^OLAPStatus FileHandlerWithBuf::write(const void* buf, size_t buf_size) {$/;" f class:doris::FileHandlerWithBuf +write src/olap/out_stream.cpp /^OLAPStatus OutStream::write(const char* buffer, uint64_t length) {$/;" f class:doris::OutStream +write src/olap/out_stream.h /^ inline OLAPStatus write(char byte) {$/;" f class:doris::OutStream +write src/olap/rowset/bit_field_writer.cpp /^OLAPStatus BitFieldWriter::write(bool bit_value) {$/;" f class:doris::BitFieldWriter +write src/olap/rowset/column_data_writer.cpp /^OLAPStatus ColumnDataWriter::write(const RowType& row) {$/;" f class:doris::ColumnDataWriter +write src/olap/rowset/column_writer.cpp /^OLAPStatus ColumnWriter::write(RowCursor* row_cursor) {$/;" f class:doris::ColumnWriter +write src/olap/rowset/column_writer.cpp /^OLAPStatus VarStringColumnWriter::write(const char* str, uint32_t len) {$/;" f class:doris::VarStringColumnWriter +write src/olap/rowset/column_writer.h /^ OLAPStatus write(int64_t data) {$/;" f class:doris::IntegerColumnWriter +write src/olap/rowset/run_length_byte_writer.cpp /^OLAPStatus RunLengthByteWriter::write(char value) {$/;" f class:doris::RunLengthByteWriter +write src/olap/rowset/run_length_integer_writer.cpp /^OLAPStatus RunLengthIntegerWriter::write(int64_t value) {$/;" f class:doris::RunLengthIntegerWriter +write src/runtime/disk_io_mgr.cc /^void DiskIoMgr::write(RequestContext* writer_context, WriteRange* write_range) {$/;" f class:doris::DiskIoMgr +write src/runtime/raw_value.cpp /^void RawValue::write(const void* value, Tuple* tuple, const SlotDescriptor* slot_desc,$/;" f class:doris::RawValue +write src/runtime/raw_value.cpp /^void RawValue::write(const void* value, const TypeDescriptor& type, void* dst, uint8_t** buf) {$/;" f class:doris::RawValue +write src/runtime/raw_value.cpp /^void RawValue::write(const void* value, void* dst, const TypeDescriptor& type, MemPool* pool) {$/;" f class:doris::RawValue +write src/util/bitmap_value.h /^ size_t write(char* buf) const {$/;" f class:doris::detail::Roaring64Map +write src/util/bitmap_value.h /^ void write(char* dst) {$/;" f class:doris::BitmapValue +write test/runtime/load_channel_mgr_test.cpp /^OLAPStatus DeltaWriter::write(Tuple* tuple) {$/;" f class:doris::DeltaWriter +write_batch src/olap/rowset/segment_writer.cpp /^OLAPStatus SegmentWriter::write_batch(RowBlock* block, RowCursor* cursor, bool is_finalize) {$/;" f class:doris::SegmentWriter +write_bitmap_index src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::write_bitmap_index() {$/;" f class:doris::segment_v2::ColumnWriter +write_bloom_filter_index src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::write_bloom_filter_index() {$/;" f class:doris::segment_v2::ColumnWriter +write_bloom_filter_index_file test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^void write_bloom_filter_index_file(const std::string& file_name, const void* values,$/;" f namespace:doris::segment_v2 +write_bytes src/olap/push_handler.h /^ int64_t write_bytes() const { return _write_bytes; }$/;" f class:doris::PushHandler +write_complete src/runtime/buffered_block_mgr2.cc /^void BufferedBlockMgr2::write_complete(Block* block, const Status& write_status) {$/;" f class:doris::BufferedBlockMgr2 +write_complete_callback test/runtime/disk_io_mgr_test.cpp /^ void write_complete_callback(int num_writes, const Status& status) {$/;" f class:doris::DiskIoMgrTest +write_data src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::write_data() {$/;" f class:doris::segment_v2::ColumnWriter +write_end_ptr_ src/runtime/buffered_tuple_stream3.h /^ const uint8_t* write_end_ptr_;$/;" m class:doris::BufferedTupleStream3 +write_header src/runtime/dpp_writer.cpp /^Status DppWriter::write_header() {$/;" f class:doris::DppWriter +write_index_by_index src/olap/row_cursor.h /^ void write_index_by_index(size_t index, char* index_ptr) const {$/;" f class:doris::RowCursor +write_index_file test/olap/rowset/segment_v2/bitmap_index_test.cpp /^void write_index_file(std::string& filename, const void* values,$/;" f namespace:doris::segment_v2 +write_ints src/olap/serialize.cpp /^OLAPStatus write_ints(OutStream* output, int64_t* data, uint32_t count, uint32_t bit_width) {$/;" f namespace:doris::ser +write_io_ops src/runtime/bufferpool/buffer_pool_counters.h /^ RuntimeProfile::Counter* write_io_ops;$/;" m struct:doris::BufferPoolClientCounters +write_json_to_file src/agent/utils.cpp /^bool AgentUtils::write_json_to_file(const map& info, const string& path) {$/;" f class:doris::AgentUtils +write_log_info src/olap/utils.cpp /^void write_log_info(char *buf, size_t buf_len, const char *fmt, ...) {$/;" f namespace:doris +write_mysql src/util/mysql_load_error_hub.cpp /^Status MysqlLoadErrorHub::write_mysql() {$/;" f class:doris::MysqlLoadErrorHub +write_ordinal_index src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::write_ordinal_index() {$/;" f class:doris::segment_v2::ColumnWriter +write_ordinal_index src/olap/rowset/segment_v2/indexed_column_writer.h /^ bool write_ordinal_index = false;$/;" m struct:doris::segment_v2::IndexedColumnWriterOptions +write_page src/olap/rowset/segment_v2/page_io.cpp /^Status PageIO::write_page(fs::WritableBlock* wblock,$/;" f class:doris::segment_v2::PageIO +write_page_ src/runtime/buffered_tuple_stream3.h /^ Page* write_page_;$/;" m class:doris::BufferedTupleStream3 +write_page_reservation_ src/runtime/buffered_tuple_stream3.h /^ BufferPool::SubReservation write_page_reservation_;$/;" m class:doris::BufferedTupleStream3 +write_ptr_ src/runtime/buffered_tuple_stream3.h /^ uint8_t* write_ptr_;$/;" m class:doris::BufferedTupleStream3 +write_range_helper src/runtime/disk_io_mgr.cc /^Status DiskIoMgr::write_range_helper(FILE* file_handle, WriteRange* write_range) {$/;" f class:doris::DiskIoMgr +write_rows src/olap/push_handler.h /^ int64_t write_rows() const { return _write_rows; }$/;" f class:doris::PushHandler +write_slot src/exec/text_converter.hpp /^inline bool TextConverter::write_slot(const SlotDescriptor* slot_desc,$/;" f class:doris::TextConverter +write_status_ src/runtime/bufferpool/buffer_pool_internal.h /^ Status write_status_;$/;" m class:doris::BufferPool::Client +write_step test/olap/skiplist_test.cpp /^ void write_step(Random* rnd) {$/;" f class:doris::ConcurrentTest +write_text_slot src/exec/mysql_scan_node.cpp /^Status MysqlScanNode::write_text_slot(char* value, int value_length,$/;" f class:doris::MysqlScanNode +write_time_ms src/util/system_metrics.cpp /^ IntLockCounter write_time_ms;$/;" m struct:doris::DiskMetrics file: +write_to src/exprs/bitmap_function.cpp /^char* write_to(const DateTimeValue& v, char* dest) {$/;" f namespace:doris::detail +write_to src/exprs/bitmap_function.cpp /^char* write_to(const DecimalV2Value& v, char* dest) {$/;" f namespace:doris::detail +write_to src/exprs/bitmap_function.cpp /^char* write_to(const StringValue& v, char* dest) {$/;" f namespace:doris::detail +write_to src/exprs/bitmap_function.cpp /^char* write_to(const T& v, char* dest) {$/;" f namespace:doris::detail +write_to_broker src/util/broker_load_error_hub.cpp /^Status BrokerLoadErrorHub::write_to_broker() {$/;" f class:doris::BrokerLoadErrorHub +write_to_buffer src/olap/bloom_filter_writer.cpp /^OLAPStatus BloomFilterIndexWriter::write_to_buffer(OutStream* out_stream) {$/;" f class:doris::BloomFilterIndexWriter +write_to_buffer src/olap/bloom_filter_writer.cpp /^OLAPStatus BloomFilterIndexWriter::write_to_buffer(char* buffer, size_t buffer_size) {$/;" f class:doris::BloomFilterIndexWriter +write_to_buffer src/olap/stream_index_common.cpp /^OLAPStatus ColumnStatistics::write_to_buffer(char* buffer, size_t size) {$/;" f class:doris::ColumnStatistics +write_to_buffer src/olap/stream_index_writer.cpp /^OLAPStatus StreamIndexWriter::write_to_buffer(char* buffer, size_t buffer_size) {$/;" f class:doris::StreamIndexWriter +write_to_buffer src/olap/stream_index_writer.cpp /^void PositionEntryWriter::write_to_buffer(char* out_buffer) {$/;" f class:doris::PositionEntryWriter +write_to_file src/olap/out_stream.cpp /^OLAPStatus OutStream::write_to_file(FileHandler* file_handle,$/;" f class:doris::OutStream +write_type src/olap/delta_writer.h /^ WriteType write_type;$/;" m struct:doris::WriteRequest +write_unpinned_block src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::write_unpinned_block(Block* block) {$/;" f class:doris::BufferedBlockMgr2 +write_unpinned_blocks src/runtime/buffered_block_mgr2.cc /^Status BufferedBlockMgr2::write_unpinned_blocks() {$/;" f class:doris::BufferedBlockMgr2 +write_validate_callback test/runtime/disk_io_mgr_test.cpp /^ void write_validate_callback(int num_writes, DiskIoMgr::WriteRange** written_range,$/;" f class:doris::DiskIoMgrTest +write_value_index src/olap/rowset/segment_v2/indexed_column_writer.h /^ bool write_value_index = false;$/;" m struct:doris::segment_v2::IndexedColumnWriterOptions +write_var_signed src/olap/serialize.h /^inline OLAPStatus write_var_signed(OutStream* stream, int64_t value) {$/;" f namespace:doris::ser +write_var_unsigned src/olap/serialize.cpp /^OLAPStatus write_var_unsigned(OutStream* stream, int64_t value) {$/;" f namespace:doris::ser +write_wait_time src/runtime/bufferpool/buffer_pool_counters.h /^ RuntimeProfile::Counter* write_wait_time;$/;" m struct:doris::BufferPoolClientCounters +write_zone_map src/olap/rowset/segment_v2/column_writer.cpp /^Status ColumnWriter::write_zone_map() {$/;" f class:doris::segment_v2::ColumnWriter +writes_completed src/util/system_metrics.cpp /^ IntLockCounter writes_completed;$/;" m struct:doris::DiskMetrics file: +writes_issued src/runtime/buffered_block_mgr2.h /^ int writes_issued() const { {$/;" f class:doris::BufferedBlockMgr2 +written_bytes src/olap/rowset/column_data_writer.cpp /^uint64_t ColumnDataWriter::written_bytes() {$/;" f class:doris::ColumnDataWriter +wrlock src/util/mutex.cpp /^OLAPStatus RWMutex::wrlock() {$/;" f class:doris::RWMutex +x src/geo/geo_types.cpp /^double GeoPoint::x() const {$/;" f class:doris::GeoPoint +x src/geo/wkt_parse_type.h /^ double x;$/;" m struct:doris::GeoCoordinate +x86_64 src/gutil/hash/string_hash.h /^enum { x86_64 = true, sixty_four_bit = true };$/;" e enum:hash_internal::__anon21 +xgetbv src/gutil/cpu.cc /^uint64_t xgetbv(uint32_t xcr) {$/;" f namespace:base::__anon16 +y src/geo/geo_types.cpp /^double GeoPoint::y() const {$/;" f class:doris::GeoPoint +y src/geo/wkt_parse_type.h /^ double y;$/;" m struct:doris::GeoCoordinate +year src/exprs/timestamp_functions.cpp /^IntVal TimestampFunctions::year($/;" f class:doris::TimestampFunctions +year src/runtime/datetime_value.h /^ int year() const {$/;" f class:doris::DateTimeValue +year src/runtime/datetime_value.h /^ int32_t year;$/;" m struct:doris::TimeInterval +years_add src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::years_add($/;" f class:doris::TimestampFunctions +years_diff src/exprs/timestamp_functions.cpp /^BigIntVal TimestampFunctions::years_diff($/;" f class:doris::TimestampFunctions +years_sub src/exprs/timestamp_functions.cpp /^DateTimeVal TimestampFunctions::years_sub($/;" f class:doris::TimestampFunctions +yyscan_t src/geo/wkt_parse_ctx.h /^typedef void* yyscan_t;$/;" t +z_filefunc src/util/minizip/unzip.c /^ zlib_filefunc64_32_def z_filefunc;$/;" m struct:__anon33 file: +z_filefunc src/util/minizip/unzip.c /^ zlib_filefunc64_32_def z_filefunc;$/;" m struct:__anon34 file: +zclose_file src/util/minizip/ioapi.h /^ close_file_func zclose_file;$/;" m struct:zlib_filefunc64_def_s +zclose_file src/util/minizip/ioapi.h /^ close_file_func zclose_file;$/;" m struct:zlib_filefunc_def_s +zero_num_rows src/olap/olap_index.h /^ bool zero_num_rows() const {$/;" f class:doris::MemIndex +zero_num_rows src/olap/rowset/column_data.h /^ bool zero_num_rows() const { return _segment_group->zero_num_rows(); }$/;" f class:doris::ColumnData +zero_num_rows src/olap/rowset/rowset.h /^ bool zero_num_rows() const { return rowset_meta()->num_rows() == 0; }$/;" f class:doris::Rowset +zero_num_rows src/olap/rowset/segment_group.h /^ bool zero_num_rows() const {$/;" f class:doris::SegmentGroup +zero_udf test/udf/udf_test.cpp /^DoubleVal zero_udf(FunctionContext* context) {$/;" f namespace:doris_udf +zerror_file src/util/minizip/ioapi.h /^ testerror_file_func zerror_file;$/;" m struct:zlib_filefunc64_def_s +zerror_file src/util/minizip/ioapi.h /^ testerror_file_func zerror_file;$/;" m struct:zlib_filefunc_def_s +zfile_func64 src/util/minizip/ioapi.h /^ zlib_filefunc64_def zfile_func64;$/;" m struct:zlib_filefunc64_32_def_s +zig_zag_decode src/olap/serialize.h /^inline int64_t zig_zag_decode(int64_t value) {$/;" f namespace:doris::ser +zig_zag_encode src/olap/serialize.h /^inline int64_t zig_zag_encode(int64_t value) {$/;" f namespace:doris::ser +zlib_crc32 src/runtime/raw_value.h /^inline uint32_t RawValue::zlib_crc32(const void* v, const TypeDescriptor& type, uint32_t seed) {$/;" f class:doris::RawValue +zlib_crc_hash src/util/hash_util.hpp /^ static uint32_t zlib_crc_hash(const void* data, int32_t bytes, uint32_t hash) {$/;" f class:doris::HashUtil +zlib_filefunc64_32_def src/util/minizip/ioapi.h /^} zlib_filefunc64_32_def;$/;" t typeref:struct:zlib_filefunc64_32_def_s +zlib_filefunc64_32_def_s src/util/minizip/ioapi.h /^typedef struct zlib_filefunc64_32_def_s$/;" s +zlib_filefunc64_def src/util/minizip/ioapi.h /^} zlib_filefunc64_def;$/;" t typeref:struct:zlib_filefunc64_def_s +zlib_filefunc64_def_s src/util/minizip/ioapi.h /^typedef struct zlib_filefunc64_def_s$/;" s +zlib_filefunc_def src/util/minizip/ioapi.h /^} zlib_filefunc_def;$/;" t typeref:struct:zlib_filefunc_def_s +zlib_filefunc_def_s src/util/minizip/ioapi.h /^typedef struct zlib_filefunc_def_s$/;" s +zone_maps src/olap/rowset/rowset_meta.h /^ void zone_maps(std::vector* zone_maps) {$/;" f class:doris::RowsetMeta +zopen32_file src/util/minizip/ioapi.h /^ open_file_func zopen32_file;$/;" m struct:zlib_filefunc64_32_def_s +zopen64_file src/util/minizip/ioapi.h /^ open64_file_func zopen64_file;$/;" m struct:zlib_filefunc64_def_s +zopen_file src/util/minizip/ioapi.h /^ open_file_func zopen_file;$/;" m struct:zlib_filefunc_def_s +zread_file src/util/minizip/ioapi.h /^ read_file_func zread_file;$/;" m struct:zlib_filefunc64_def_s +zread_file src/util/minizip/ioapi.h /^ read_file_func zread_file;$/;" m struct:zlib_filefunc_def_s +zseek32_file src/util/minizip/ioapi.h /^ seek_file_func zseek32_file;$/;" m struct:zlib_filefunc64_32_def_s +zseek64_file src/util/minizip/ioapi.h /^ seek64_file_func zseek64_file;$/;" m struct:zlib_filefunc64_def_s +zseek_file src/util/minizip/ioapi.h /^ seek_file_func zseek_file;$/;" m struct:zlib_filefunc_def_s +ztell32_file src/util/minizip/ioapi.h /^ tell_file_func ztell32_file;$/;" m struct:zlib_filefunc64_32_def_s +ztell64_file src/util/minizip/ioapi.h /^ tell64_file_func ztell64_file;$/;" m struct:zlib_filefunc64_def_s +ztell_file src/util/minizip/ioapi.h /^ tell_file_func ztell_file;$/;" m struct:zlib_filefunc_def_s +zwrite_file src/util/minizip/ioapi.h /^ write_file_func zwrite_file;$/;" m struct:zlib_filefunc64_def_s +zwrite_file src/util/minizip/ioapi.h /^ write_file_func zwrite_file;$/;" m struct:zlib_filefunc_def_s +~AddExpr src/exprs/arithmetic_expr.h /^ virtual ~AddExpr() { }$/;" f class:doris::AddExpr +~AgentServer src/agent/agent_server.cpp /^AgentServer::~AgentServer() { }$/;" f class:doris::AgentServer +~AgentUtils src/agent/utils.h /^ virtual ~AgentUtils() {};$/;" f class:doris::AgentUtils +~AggFnEvaluator src/exprs/agg_fn_evaluator.cpp /^AggFnEvaluator::~AggFnEvaluator() {$/;" f class:doris::AggFnEvaluator +~AggregateFuncResolver src/olap/aggregate_func.cpp /^AggregateFuncResolver::~AggregateFuncResolver() {$/;" f class:doris::AggregateFuncResolver +~AggregateFuncTest test/olap/aggregate_func_test.cpp /^ virtual ~AggregateFuncTest() {$/;" f class:doris::AggregateFuncTest +~AggregationNode src/exec/aggregation_node.cpp /^AggregationNode::~AggregationNode() {$/;" f class:doris::AggregationNode +~AlphaRowsetReader src/olap/rowset/alpha_rowset_reader.cpp /^AlphaRowsetReader::~AlphaRowsetReader() {$/;" f class:doris::AlphaRowsetReader +~AlphaRowsetWriter src/olap/rowset/alpha_rowset_writer.cpp /^AlphaRowsetWriter::~AlphaRowsetWriter() {$/;" f class:doris::AlphaRowsetWriter +~AnalyticEvalNode src/exec/analytic_eval_node.h /^ ~AnalyticEvalNode() {}$/;" f class:doris::AnalyticEvalNode +~ArithmeticExpr src/exprs/arithmetic_expr.h /^ virtual ~ArithmeticExpr() { }$/;" f class:doris::ArithmeticExpr +~ArrowRowBatchTest test/util/arrow/arrow_row_batch_test.cpp /^ virtual ~ArrowRowBatchTest() {$/;" f class:doris::ArrowRowBatchTest +~ArrowRowBlockTest test/util/arrow/arrow_row_block_test.cpp /^ virtual ~ArrowRowBlockTest() {$/;" f class:doris::ArrowRowBlockTest +~ArrowWorkFlowTest test/util/arrow/arrow_work_flow_test.cpp /^ ~ArrowWorkFlowTest(){}$/;" f class:doris::ArrowWorkFlowTest +~AssertNumRowsNode src/exec/assert_num_rows_node.h /^ virtual ~AssertNumRowsNode() {};$/;" f class:doris::AssertNumRowsNode +~AsserterBase src/gutil/threading/thread_collision_warner.h /^ virtual ~AsserterBase() {}$/;" f struct:base::AsserterBase +~BRpcService src/service/brpc_service.cpp /^BRpcService::~BRpcService() {$/;" f class:doris::BRpcService +~BackendService src/service/backend_service.h /^ virtual ~BackendService() {$/;" f class:doris::BackendService +~Barrier src/util/barrier.h /^ ~Barrier() {}$/;" f class:doris::Barrier +~BaseCompaction src/olap/base_compaction.cpp /^BaseCompaction::~BaseCompaction() { }$/;" f class:doris::BaseCompaction +~BaseDeleter src/gutil/stl_util.h /^ virtual ~BaseDeleter() {}$/;" f class:BaseDeleter +~BaseScanner src/exec/base_scanner.h /^ virtual ~BaseScanner() {$/;" f class:doris::BaseScanner +~BatchProcessThreadPool src/util/batch_process_thread_pool.hpp /^ ~BatchProcessThreadPool() {$/;" f class:doris::BatchProcessThreadPool +~BatchedRowSupplier src/runtime/sorted_run_merger.cc /^ ~BatchedRowSupplier() {}$/;" f class:doris::SortedRunMerger::BatchedRowSupplier +~BetaRowset src/olap/rowset/beta_rowset.cpp /^BetaRowset::~BetaRowset() { }$/;" f class:doris::BetaRowset +~BetaRowsetWriter src/olap/rowset/beta_rowset_writer.cpp /^BetaRowsetWriter::~BetaRowsetWriter() {$/;" f class:doris::BetaRowsetWriter +~BfdParser src/util/bfd_parser.cpp /^BfdParser::~BfdParser() {$/;" f class:doris::BfdParser +~BinaryFile src/olap/push_handler.h /^ virtual ~BinaryFile() {$/;" f class:doris::BinaryFile +~BinaryOpTest test/exprs/binary_predicate_test.cpp /^ ~BinaryOpTest() {$/;" f class:doris::BinaryOpTest +~BinaryPlainPageTest test/olap/rowset/segment_v2/binary_plain_page_test.cpp /^ virtual ~BinaryPlainPageTest() {$/;" f class:doris::segment_v2::BinaryPlainPageTest +~BinaryPredicate src/exprs/binary_predicate.h /^ virtual ~BinaryPredicate() { }$/;" f class:doris::BinaryPredicate +~BinaryReader src/olap/push_handler.h /^ virtual ~BinaryReader() {$/;" f class:doris::BinaryReader +~BitAndExpr src/exprs/arithmetic_expr.h /^ virtual ~BitAndExpr() { }$/;" f class:doris::BitAndExpr +~BitFieldReader src/olap/rowset/bit_field_reader.cpp /^BitFieldReader::~BitFieldReader() {$/;" f class:doris::BitFieldReader +~BitFieldWriter src/olap/rowset/bit_field_writer.cpp /^BitFieldWriter::~BitFieldWriter() {$/;" f class:doris::BitFieldWriter +~BitMapTest test/util/bitmap_test.cpp /^ virtual ~BitMapTest() {$/;" f class:doris::BitMapTest +~BitNotExpr src/exprs/arithmetic_expr.h /^ virtual ~BitNotExpr() { }$/;" f class:doris::BitNotExpr +~BitOrExpr src/exprs/arithmetic_expr.h /^ virtual ~BitOrExpr() { }$/;" f class:doris::BitOrExpr +~BitRle test/util/rle_encoding_test.cpp /^ virtual ~BitRle() {$/;" f class:doris::BitRle +~BitSet src/olap/bloom_filter.hpp /^ ~BitSet() {$/;" f class:doris::BitSet +~BitShufflePageTest test/olap/rowset/segment_v2/bitshuffle_page_test.cpp /^ virtual ~BitShufflePageTest() {}$/;" f class:doris::BitShufflePageTest +~BitXorExpr src/exprs/arithmetic_expr.h /^ virtual ~BitXorExpr() { }$/;" f class:doris::BitXorExpr +~BitmapRangeIterator src/olap/rowset/segment_v2/segment_iterator.cpp /^ ~BitmapRangeIterator() {$/;" f class:doris::segment_v2::SegmentIterator::BitmapRangeIterator +~Block src/olap/fs/block_manager.h /^ virtual ~Block() {}$/;" f class:doris::fs::Block +~Block src/runtime/buffered_block_mgr.h /^ ~Block() {}$/;" f class:doris::BufferedBlockMgr::Block +~Block src/runtime/buffered_block_mgr2.h /^ ~Block() {}$/;" f class:doris::BufferedBlockMgr2::Block +~BlockBloomFilterTest test/olap/rowset/segment_v2/block_bloom_filter_test.cpp /^ virtual ~BlockBloomFilterTest() { }$/;" f class:doris::segment_v2::BlockBloomFilterTest +~BlockCompressionCodec src/util/block_compression.h /^ virtual ~BlockCompressionCodec() { }$/;" f class:doris::BlockCompressionCodec +~BlockCompressionTest test/util/block_compression_test.cpp /^ virtual ~BlockCompressionTest() {$/;" f class:doris::BlockCompressionTest +~BlockManager src/olap/fs/block_manager.h /^ virtual ~BlockManager() {}$/;" f class:doris::fs::BlockManager +~BlockingJoinNode src/exec/blocking_join_node.cpp /^BlockingJoinNode::~BlockingJoinNode() {$/;" f class:doris::BlockingJoinNode +~BloomFilter src/olap/bloom_filter.hpp /^ ~BloomFilter() {}$/;" f class:doris::BloomFilter +~BloomFilter src/olap/rowset/segment_v2/bloom_filter.h /^ virtual ~BloomFilter() {$/;" f class:doris::segment_v2::BloomFilter +~BloomFilterIndexReader src/olap/bloom_filter_reader.cpp /^BloomFilterIndexReader::~BloomFilterIndexReader() {$/;" f class:doris::BloomFilterIndexReader +~BloomFilterIndexReaderWriterTest test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp /^ virtual ~BloomFilterIndexReaderWriterTest() {$/;" f class:doris::segment_v2::BloomFilterIndexReaderWriterTest +~BloomFilterIndexWriter src/olap/bloom_filter_writer.cpp /^BloomFilterIndexWriter::~BloomFilterIndexWriter() {$/;" f class:doris::BloomFilterIndexWriter +~BloomFilterPageTest test/olap/rowset/segment_v2/bloom_filter_page_test.cpp /^ virtual ~BloomFilterPageTest() { }$/;" f class:doris::segment_v2::BloomFilterPageTest +~BooleanQueryBuilder src/exec/es/es_query_builder.cpp /^BooleanQueryBuilder::~BooleanQueryBuilder() {$/;" f class:doris::BooleanQueryBuilder +~BooleanQueryBuilderTest test/exec/es_query_builder_test.cpp /^ virtual ~BooleanQueryBuilderTest() { }$/;" f class:doris::BooleanQueryBuilderTest +~BrokerLoadErrorHub src/util/broker_load_error_hub.cpp /^BrokerLoadErrorHub::~BrokerLoadErrorHub() {$/;" f class:doris::BrokerLoadErrorHub +~BrokerMgr src/runtime/broker_mgr.cpp /^BrokerMgr::~BrokerMgr() {$/;" f class:doris::BrokerMgr +~BrokerReader src/exec/broker_reader.cpp /^BrokerReader::~BrokerReader() {$/;" f class:doris::BrokerReader +~BrokerScanNode src/exec/broker_scan_node.cpp /^BrokerScanNode::~BrokerScanNode() {$/;" f class:doris::BrokerScanNode +~BrokerScanner src/exec/broker_scanner.cpp /^BrokerScanner::~BrokerScanner() {$/;" f class:doris::BrokerScanner +~BrokerTableDescriptor src/runtime/descriptors.cpp /^BrokerTableDescriptor::~BrokerTableDescriptor() {$/;" f class:doris::BrokerTableDescriptor +~BrokerWriter src/exec/broker_writer.cpp /^BrokerWriter::~BrokerWriter() {$/;" f class:doris::BrokerWriter +~BrpcStubCache src/util/brpc_stub_cache.h /^ ~BrpcStubCache() {$/;" f class:doris::BrpcStubCache +~BrpcStubCacheTest test/util/brpc_stub_cache_test.cpp /^ virtual ~BrpcStubCacheTest() {$/;" f class:doris::BrpcStubCacheTest +~BufferAllocator src/runtime/bufferpool/buffer_allocator.cc /^BufferPool::BufferAllocator::~BufferAllocator() {$/;" f class:doris::BufferPool::BufferAllocator +~BufferBuilder src/util/buffer_builder.hpp /^ ~BufferBuilder() {$/;" f class:doris::BufferBuilder +~BufferControlBlock src/runtime/buffer_control_block.cpp /^BufferControlBlock::~BufferControlBlock() {$/;" f class:doris::BufferControlBlock +~BufferControlBlockTest test/runtime/buffer_control_block_test.cpp /^ virtual ~BufferControlBlockTest() {$/;" f class:doris::BufferControlBlockTest +~BufferDescriptor src/runtime/disk_io_mgr.h /^ ~BufferDescriptor() {}$/;" f class:doris::DiskIoMgr::BufferDescriptor +~BufferHandle src/runtime/bufferpool/buffer_pool.h /^ ~BufferHandle() { DCHECK(!is_open()); }$/;" f class:doris::BufferPool::BufferHandle +~BufferPool src/runtime/bufferpool/buffer_pool.cc /^BufferPool::~BufferPool() {}$/;" f class:doris::BufferPool +~BufferedBlockMgr src/runtime/buffered_block_mgr.h /^ ~BufferedBlockMgr() {};$/;" f class:doris::BufferedBlockMgr +~BufferedBlockMgr2 src/runtime/buffered_block_mgr2.cc /^BufferedBlockMgr2::~BufferedBlockMgr2() {$/;" f class:doris::BufferedBlockMgr2 +~BufferedTupleStream src/runtime/buffered_tuple_stream.h /^ ~BufferedTupleStream() {}$/;" f class:doris::BufferedTupleStream +~BufferedTupleStream2 src/runtime/buffered_tuple_stream2.h /^ ~BufferedTupleStream2() {}$/;" f class:doris::BufferedTupleStream2 +~BufferedTupleStream3 src/runtime/buffered_tuple_stream3.cc /^BufferedTupleStream3::~BufferedTupleStream3() {$/;" f class:BufferedTupleStream3 +~BufferedTupleStreamTest test/runtime/buffered_tuple_stream_test.cpp /^ virtual ~BufferedTupleStreamTest() {$/;" f class:doris::BufferedTupleStreamTest +~ByteBuffer src/util/byte_buffer.h /^ ~ByteBuffer() { delete[] ptr; }$/;" f struct:doris::ByteBuffer +~ByteBufferTest test/util/byte_buffer_test2.cpp /^ virtual ~ByteBufferTest() { }$/;" f class:doris::ByteBufferTest +~ByteColumnWriter src/olap/rowset/column_writer.cpp /^ByteColumnWriter::~ByteColumnWriter() {$/;" f class:doris::ByteColumnWriter +~Bzip2Decompressor src/exec/decompressor.cpp /^Bzip2Decompressor::~Bzip2Decompressor() {$/;" f class:doris::Bzip2Decompressor +~CGroupUtilTest test/util/cgroup_util_test.cpp /^ virtual ~CGroupUtilTest() {$/;" f class:doris::CGroupUtilTest +~Cache src/olap/lru_cache.cpp /^Cache::~Cache() {$/;" f class:doris::Cache +~CacheKey src/olap/lru_cache.h /^ ~CacheKey() {}$/;" f class:doris::CacheKey +~CacheTest test/olap/lru_cache_test.cpp /^ ~CacheTest() {$/;" f class:doris::CacheTest +~CaseExpr src/exprs/case_expr.cpp /^CaseExpr::~CaseExpr() {$/;" f class:doris::CaseExpr +~CastExpr src/exprs/cast_expr.h /^ virtual ~CastExpr() { }$/;" f class:doris::CastExpr +~CgroupsMgr src/agent/cgroups_mgr.cpp /^CgroupsMgr::~CgroupsMgr() {$/;" f class:doris::CgroupsMgr +~Channel src/runtime/data_stream_sender.cpp /^ virtual ~Channel() {$/;" f class:doris::DataStreamSender::Channel +~Check src/gutil/threading/thread_collision_warner.h /^ ~Check() {}$/;" f class:base::ThreadCollisionWarner::Check +~ChecksumAction src/http/action/checksum_action.h /^ virtual ~ChecksumAction() { }$/;" f class:doris::ChecksumAction +~ChunkArena src/runtime/memory/chunk_allocator.cpp /^ ~ChunkArena() {$/;" f class:doris::ChunkArena +~Client src/runtime/buffered_block_mgr2.cc /^ ~Client() {}$/;" f class:doris::BufferedBlockMgr2::Client +~Client src/runtime/bufferpool/buffer_pool_internal.h /^ ~Client() {$/;" f class:doris::BufferPool::Client +~ClientCacheHelper src/runtime/client_cache.cpp /^ClientCacheHelper::~ClientCacheHelper() {$/;" f class:doris::ClientCacheHelper +~ClientConnection src/runtime/client_cache.h /^ ~ClientConnection() {$/;" f class:doris::ClientConnection +~ClientHandle src/runtime/bufferpool/buffer_pool.h /^ ~ClientHandle() { DCHECK(!is_registered()); }$/;" f class:doris::BufferPool::ClientHandle +~CmdlineAction src/http/action/pprof_actions.cpp /^ virtual ~CmdlineAction() {}$/;" f class:doris::CmdlineAction +~CodingTest test/util/coding_test.cpp /^ virtual ~CodingTest() {$/;" f class:doris::CodingTest +~CollectIterator src/olap/reader.cpp /^CollectIterator::~CollectIterator() {$/;" f class:doris::CollectIterator +~ColumnData src/olap/rowset/column_data.cpp /^ColumnData::~ColumnData() {$/;" f class:doris::ColumnData +~ColumnDataComparator src/olap/rowset/column_data.h /^ ~ColumnDataComparator() {}$/;" f class:doris::ColumnDataComparator +~ColumnDataWriter src/olap/rowset/column_data_writer.cpp /^ColumnDataWriter::~ColumnDataWriter() {$/;" f class:doris::ColumnDataWriter +~ColumnIterator src/olap/rowset/segment_v2/column_reader.h /^ virtual ~ColumnIterator() { }$/;" f class:doris::segment_v2::ColumnIterator +~ColumnMapping src/olap/column_mapping.h /^ virtual ~ColumnMapping() {}$/;" f struct:doris::ColumnMapping +~ColumnPredicate src/olap/column_predicate.h /^ virtual ~ColumnPredicate() {}$/;" f class:doris::ColumnPredicate +~ColumnReader src/olap/rowset/column_reader.cpp /^ColumnReader::~ColumnReader() {$/;" f class:doris::ColumnReader +~ColumnReaderWriterTest test/olap/rowset/segment_v2/column_reader_writer_test.cpp /^ virtual ~ColumnReaderWriterTest() { }$/;" f class:doris::segment_v2::ColumnReaderWriterTest +~ColumnStatistics src/olap/stream_index_common.cpp /^ColumnStatistics::~ColumnStatistics() {$/;" f class:doris::ColumnStatistics +~ColumnVector src/runtime/vectorized_row_batch.h /^ ~ColumnVector() {}$/;" f class:doris::ColumnVector +~ColumnWriter src/olap/rowset/column_writer.cpp /^ColumnWriter::~ColumnWriter() {$/;" f class:doris::ColumnWriter +~ColumnWriter src/olap/rowset/segment_v2/column_writer.cpp /^ColumnWriter::~ColumnWriter() {$/;" f class:doris::segment_v2::ColumnWriter +~Compaction src/olap/compaction.cpp /^Compaction::~Compaction() {}$/;" f class:doris::Compaction +~CompactionAction src/http/action/compaction_action.h /^ virtual ~CompactionAction() {}$/;" f class:doris::CompactionAction +~Cond src/olap/olap_cond.cpp /^Cond::~Cond() {$/;" f class:doris::Cond +~CondColumn src/olap/olap_cond.cpp /^CondColumn::~CondColumn() {$/;" f class:doris::CondColumn +~ConditionVariable src/util/condition_variable.cpp /^ConditionVariable::~ConditionVariable() {$/;" f class:doris::ConditionVariable +~ContentionAction src/http/action/pprof_actions.cpp /^ virtual ~ContentionAction() {}$/;" f class:doris::ContentionAction +~CoreDataAllocator src/util/core_local.h /^ virtual ~CoreDataAllocator() { }$/;" f class:doris::CoreDataAllocator +~CoreDataAllocatorFactory src/util/core_local.cpp /^CoreDataAllocatorFactory::~CoreDataAllocatorFactory() {$/;" f class:doris::CoreDataAllocatorFactory +~CoreDataAllocatorImpl src/util/core_local.cpp /^CoreDataAllocatorImpl::~CoreDataAllocatorImpl() {$/;" f class:doris::CoreDataAllocatorImpl +~CoreLocalCounter src/util/metrics.h /^ virtual ~CoreLocalCounter() { }$/;" f class:doris::CoreLocalCounter +~CoreLocalTest test/util/core_local_test.cpp /^ ~CoreLocalTest() {$/;" f class:doris::CoreLocalTest +~CoreLocalValue src/util/core_local.h /^ ~CoreLocalValue() {$/;" f class:doris::CoreLocalValue +~CoreLocalValueController src/util/core_local.h /^ ~CoreLocalValueController() { }$/;" f class:doris::CoreLocalValueController +~CountDownOnScopeExit src/util/countdown_latch.h /^ ~CountDownOnScopeExit() {$/;" f class:doris::CountDownOnScopeExit +~Counter src/util/runtime_profile.h /^ virtual ~Counter() { }$/;" f class:doris::RuntimeProfile::Counter +~CounterCondVariableTest test/util/counter_cond_variable_test.cpp /^ virtual ~CounterCondVariableTest() { }$/;" f class:doris::CounterCondVariableTest +~CsvScanNode src/exec/csv_scan_node.cpp /^CsvScanNode::~CsvScanNode() {$/;" f class:doris::CsvScanNode +~CsvScanNodeBenchTest test/exec/csv_scan_bench_test.cpp /^ ~CsvScanNodeBenchTest(){}$/;" f class:doris::CsvScanNodeBenchTest +~CsvScanNodeTest test/exec/csv_scan_node_test.cpp /^ ~CsvScanNodeTest(){}$/;" f class:doris::CsvScanNodeTest +~CsvScanner src/exec/csv_scanner.cpp /^ CsvScanner::~CsvScanner() {$/;" f class:doris::CsvScanner +~CumulativeCompaction src/olap/cumulative_compaction.cpp /^CumulativeCompaction::~CumulativeCompaction() { }$/;" f class:doris::CumulativeCompaction +~DCheckAsserter src/gutil/threading/thread_collision_warner.h /^ virtual ~DCheckAsserter() {}$/;" f struct:base::DCheckAsserter +~DataConsumer src/runtime/routine_load/data_consumer.h /^ virtual ~DataConsumer() {$/;" f class:doris::DataConsumer +~DataConsumerGroup src/runtime/routine_load/data_consumer_group.h /^ virtual ~DataConsumerGroup() {$/;" f class:doris::DataConsumerGroup +~DataConsumerPool src/runtime/routine_load/data_consumer_pool.h /^ ~DataConsumerPool() {$/;" f class:doris::DataConsumerPool +~DataDir src/olap/data_dir.cpp /^DataDir::~DataDir() {$/;" f class:doris::DataDir +~DataSink src/exec/data_sink.h /^ virtual ~DataSink() {}$/;" f class:doris::DataSink +~DataSplitTest test/runtime/data_spliter_test.cpp /^ ~DataSplitTest() {$/;" f class:doris::DataSplitTest +~DataSpliter src/runtime/data_spliter.cpp /^DataSpliter::~DataSpliter() {$/;" f class:doris::DataSpliter +~DataStreamRecvr src/runtime/data_stream_recvr.cc /^DataStreamRecvr::~DataStreamRecvr() {$/;" f class:doris::DataStreamRecvr +~DataStreamRecvr src/runtime/data_stream_recvr.hpp /^ ~DataStreamRecvr() {$/;" f class:doris::DataStreamRecvr +~DataStreamSender src/runtime/data_stream_sender.cpp /^DataStreamSender::~DataStreamSender() {$/;" f class:doris::DataStreamSender +~DataStreamTest test/runtime/data_stream_test.cpp /^ ~DataStreamTest() {}$/;" f class:doris::DataStreamTest +~DecimalColumnReader src/olap/rowset/column_reader.cpp /^DecimalColumnReader::~DecimalColumnReader() {$/;" f class:doris::DecimalColumnReader +~DecimalColumnWriter src/olap/rowset/column_writer.cpp /^DecimalColumnWriter::~DecimalColumnWriter() {$/;" f class:doris::DecimalColumnWriter +~Decompressor src/exec/decompressor.cpp /^Decompressor::~Decompressor() {$/;" f class:doris::Decompressor +~DefaultValueReader src/olap/rowset/column_reader.h /^ virtual ~DefaultValueReader() {}$/;" f class:doris::DefaultValueReader +~DeferOp src/util/defer_op.h /^ ~DeferOp() {$/;" f class:doris::DeferOp +~DeleteConditionHandler src/olap/delete_handler.h /^ ~DeleteConditionHandler() {}$/;" f class:doris::DeleteConditionHandler +~DeleteConditions src/olap/delete_handler.h /^ ~DeleteConditions() {}$/;" f struct:doris::DeleteConditions +~DeleteHandler src/olap/delete_handler.h /^ ~DeleteHandler() {}$/;" f class:doris::DeleteHandler +~DeltaWriter src/olap/delta_writer.cpp /^DeltaWriter::~DeltaWriter() {$/;" f class:doris::DeltaWriter +~DeltaWriter test/runtime/load_channel_mgr_test.cpp /^DeltaWriter::~DeltaWriter() {$/;" f class:doris::DeltaWriter +~DescriptorTblBuilder src/testutil/desc_tbl_builder.h /^ ~DescriptorTblBuilder() {}$/;" f class:doris::DescriptorTblBuilder +~DiskIoMgr src/runtime/disk_io_mgr.cc /^DiskIoMgr::~DiskIoMgr() {$/;" f class:doris::DiskIoMgr +~DivExpr src/exprs/arithmetic_expr.h /^ virtual ~DivExpr() { }$/;" f class:doris::DivExpr +~DorisMetrics src/util/doris_metrics.cpp /^DorisMetrics::~DorisMetrics() {$/;" f class:doris::DorisMetrics +~DorisMetricsTest test/util/doris_metrics_test.cpp /^ virtual ~DorisMetricsTest() {$/;" f class:doris::DorisMetricsTest +~DorisTestBackend test/runtime/data_stream_test.cpp /^ virtual ~DorisTestBackend() {}$/;" f class:doris::DorisTestBackend +~DoubleColumnWriterBase src/olap/rowset/column_writer.h /^ virtual ~DoubleColumnWriterBase() {}$/;" f class:doris::DoubleColumnWriterBase +~DownloadAction src/http/download_action.h /^ virtual ~DownloadAction() {}$/;" f class:doris::DownloadAction +~DppSink src/runtime/dpp_sink.h /^ ~DppSink() {$/;" f class:doris::DppSink +~DppSinkTest test/runtime/dpp_sink_test.cpp /^ ~DppSinkTest() {$/;" f class:doris::DppSinkTest +~DppWriter src/runtime/dpp_writer.cpp /^DppWriter::~DppWriter() {$/;" f class:doris::DppWriter +~DummyProfile src/util/dummy_runtime_profile.h /^ virtual ~DummyProfile() {$/;" f class:doris::DummyProfile +~ESScanReader src/exec/es/es_scan_reader.cpp /^ESScanReader::~ESScanReader() {$/;" f class:doris::ESScanReader +~ESScrollQueryBuilder src/exec/es/es_scroll_query.cpp /^ESScrollQueryBuilder::~ESScrollQueryBuilder() {$/;" f class:doris::ESScrollQueryBuilder +~ElementDeleter src/gutil/stl_util.h /^ ~ElementDeleter() {$/;" f class:ElementDeleter +~EncodingInfoResolver src/olap/rowset/segment_v2/encoding_info.cpp /^EncodingInfoResolver::~EncodingInfoResolver() {$/;" f class:doris::segment_v2::EncodingInfoResolver +~EncodingInfoTest test/olap/rowset/segment_v2/encoding_info_test.cpp /^ virtual ~EncodingInfoTest() {$/;" f class:doris::segment_v2::EncodingInfoTest +~EngineAlterTabletTask src/olap/task/engine_alter_tablet_task.h /^ ~EngineAlterTabletTask() {}$/;" f class:doris::EngineAlterTabletTask +~EngineBatchLoadTask src/olap/task/engine_batch_load_task.cpp /^EngineBatchLoadTask::~EngineBatchLoadTask() {$/;" f class:doris::EngineBatchLoadTask +~EngineChecksumTask src/olap/task/engine_checksum_task.h /^ ~EngineChecksumTask() {}$/;" f class:doris::EngineChecksumTask +~EngineCloneTask src/olap/task/engine_clone_task.h /^ ~EngineCloneTask() {}$/;" f class:doris::EngineCloneTask +~EnginePublishVersionTask src/olap/task/engine_publish_version_task.h /^ ~EnginePublishVersionTask() {}$/;" f class:doris::EnginePublishVersionTask +~EngineStorageMigrationTask src/olap/task/engine_storage_migration_task.h /^ ~EngineStorageMigrationTask() {}$/;" f class:doris::EngineStorageMigrationTask +~Env src/env/env.h /^ virtual ~Env() { }$/;" f class:doris::Env +~EnvPosixTest test/env/env_posix_test.cpp /^ virtual ~EnvPosixTest() { }$/;" f class:doris::EnvPosixTest +~EsHttpScanNode src/exec/es_http_scan_node.cpp /^EsHttpScanNode::~EsHttpScanNode() {$/;" f class:doris::EsHttpScanNode +~EsHttpScanner src/exec/es_http_scanner.cpp /^EsHttpScanner::~EsHttpScanner() {$/;" f class:doris::EsHttpScanner +~EsPredicate src/exec/es/es_predicate.cpp /^EsPredicate::~EsPredicate() {$/;" f class:doris::EsPredicate +~EsScanNode src/exec/es_scan_node.cpp /^EsScanNode::~EsScanNode() {$/;" f class:doris::EsScanNode +~EsTableDescriptor src/runtime/descriptors.cpp /^EsTableDescriptor::~EsTableDescriptor() {$/;" f class:doris::EsTableDescriptor +~EtlJobMgr src/runtime/etl_job_mgr.cpp /^EtlJobMgr::~EtlJobMgr() {$/;" f class:doris::EtlJobMgr +~EvHttpServer src/http/ev_http_server.cpp /^EvHttpServer::~EvHttpServer() {$/;" f class:doris::EvHttpServer +~ExchangeNode src/exec/exchange_node.h /^ virtual ~ExchangeNode() {}$/;" f class:doris::ExchangeNode +~ExchangeNode src/exec/pl_task_root.cpp /^ExchangeNode::~ExchangeNode() {$/;" f class:doris::ExchangeNode +~ExecEnv src/runtime/exec_env.cpp /^ExecEnv::~ExecEnv() {$/;" f class:doris::ExecEnv +~ExecNode src/exec/exec_node.cpp /^ExecNode::~ExecNode() {$/;" f class:doris::ExecNode +~ExportSink src/runtime/export_sink.cpp /^ExportSink::~ExportSink() {$/;" f class:doris::ExportSink +~ExportTaskMgr src/runtime/export_task_mgr.cpp /^ExportTaskMgr::~ExportTaskMgr() {$/;" f class:doris::ExportTaskMgr +~Expr src/exprs/expr.cpp /^Expr::~Expr() {$/;" f class:doris::Expr +~ExprContext src/exprs/expr_context.cpp /^ExprContext::~ExprContext() {$/;" f class:doris::ExprContext +~ExtLiteral src/exec/es/es_predicate.cpp /^ExtLiteral::~ExtLiteral(){$/;" f class:doris::ExtLiteral +~ExternalScanContextMgr src/runtime/external_scan_context_mgr.h /^ ~ExternalScanContextMgr() {$/;" f class:doris::ExternalScanContextMgr +~ExternalScanContextMgrTest test/runtime/external_scan_context_mgr_test.cpp /^ virtual ~ExternalScanContextMgrTest() {$/;" f class:doris::ExternalScanContextMgrTest +~File src/runtime/tmp_file_mgr.h /^ ~File(){$/;" f class:doris::TmpFileMgr::File +~FileBlockManager src/olap/fs/file_block_manager.cpp /^FileBlockManager::~FileBlockManager() {$/;" f class:doris::fs::FileBlockManager +~FileCache src/util/file_cache.h /^ ~FileCache() { }$/;" f class:doris::FileCache +~FileCursor src/olap/file_stream.h /^ ~FileCursor() {}$/;" f class:doris::ReadOnlyFileStream::FileCursor +~FileDescriptor src/olap/file_helper.h /^ ~FileDescriptor() {$/;" f struct:doris::FileDescriptor +~FileHandler src/olap/file_helper.cpp /^FileHandler::~FileHandler() {$/;" f class:doris::FileHandler +~FileHandlerWithBuf src/olap/file_helper.cpp /^FileHandlerWithBuf::~FileHandlerWithBuf() {$/;" f class:doris::FileHandlerWithBuf +~FileHeader src/olap/file_helper.h /^ ~FileHeader() {}$/;" f class:doris::FileHeader +~FileManager src/util/file_manager.h /^ ~FileManager() { }$/;" f class:doris::FileManager +~FileReadableBlock src/olap/fs/file_block_manager.cpp /^FileReadableBlock::~FileReadableBlock() {$/;" f class:doris::fs::internal::FileReadableBlock +~FileReader src/exec/file_reader.h /^ virtual ~FileReader() {$/;" f class:doris::FileReader +~FileWritableBlock src/olap/fs/file_block_manager.cpp /^FileWritableBlock::~FileWritableBlock() {$/;" f class:doris::fs::internal::FileWritableBlock +~FileWriter src/exec/file_writer.h /^ virtual ~FileWriter() {$/;" f class:doris::FileWriter +~FixLengthStringColumnReader src/olap/rowset/column_reader.h /^ virtual ~FixLengthStringColumnReader() {$/;" f class:doris::FixLengthStringColumnReader +~FixLengthStringColumnWriter src/olap/rowset/column_writer.cpp /^FixLengthStringColumnWriter::~FixLengthStringColumnWriter() {}$/;" f class:doris::FixLengthStringColumnWriter +~FloatintPointColumnReader src/olap/rowset/column_reader.h /^ virtual ~FloatintPointColumnReader() {}$/;" f class:doris::FloatintPointColumnReader +~FragmentExecState src/runtime/fragment_mgr.cpp /^FragmentExecState::~FragmentExecState() {$/;" f class:doris::FragmentExecState +~FragmentMgr src/runtime/fragment_mgr.cpp /^FragmentMgr::~FragmentMgr() {$/;" f class:doris::FragmentMgr +~FragmentMgr test/runtime/etl_job_mgr_test.cpp /^FragmentMgr::~FragmentMgr() {$/;" f class:doris::FragmentMgr +~FragmentMgr test/runtime/export_task_mgr_test.cpp /^FragmentMgr::~FragmentMgr() {$/;" f class:doris::FragmentMgr +~FreeBufferArena src/runtime/bufferpool/buffer_allocator.cc /^BufferPool::FreeBufferArena::~FreeBufferArena() {$/;" f class:doris::BufferPool::FreeBufferArena +~FreePool src/runtime/free_pool.hpp /^ virtual ~FreePool() {}$/;" f class:doris::FreePool +~FunctionContext src/udf/udf.cpp /^FunctionContext::~FunctionContext() {$/;" f class:doris_udf::FunctionContext +~FunctionContextImpl src/udf/udf_internal.h /^ ~FunctionContextImpl() {$/;" f class:doris::FunctionContextImpl +~FunctionUtils src/testutil/function_utils.cpp /^FunctionUtils::~FunctionUtils() {$/;" f class:doris::FunctionUtils +~GenericElement src/common/object_pool.h /^ virtual ~GenericElement() {}$/;" f struct:doris::ObjectPool::GenericElement +~GenericIteratorsTest test/olap/generic_iterators_test.cpp /^ virtual ~GenericIteratorsTest() {$/;" f class:doris::GenericIteratorsTest +~GeoCircle src/geo/geo_types.h /^ ~GeoCircle() { }$/;" f class:doris::GeoCircle +~GeoCoordinateListList src/geo/wkt_parse_type.h /^ ~GeoCoordinateListList() {$/;" f struct:doris::GeoCoordinateListList +~GeoFunctionsTest test/geo/geo_functions_test.cpp /^ virtual ~GeoFunctionsTest() { }$/;" f class:doris::GeoFunctionsTest +~GeoShape src/geo/geo_types.h /^ virtual ~GeoShape() { }$/;" f class:doris::GeoShape +~GeoTypesTest test/geo/geo_types_test.cpp /^ virtual ~GeoTypesTest() { }$/;" f class:doris::GeoTypesTest +~GrowthAction src/http/action/pprof_actions.cpp /^ virtual ~GrowthAction() {}$/;" f class:doris::GrowthAction +~GzipDecompressor src/exec/decompressor.cpp /^GzipDecompressor::~GzipDecompressor() {$/;" f class:doris::GzipDecompressor +~HandleTable src/olap/lru_cache.h /^ ~HandleTable() {$/;" f class:doris::CachePriority::HandleTable +~HashJoinNode src/exec/hash_join_node.cpp /^HashJoinNode::~HashJoinNode() {$/;" f class:doris::HashJoinNode +~HashTable src/exec/hash_table.cpp /^HashTable::~HashTable() {$/;" f class:doris::HashTable +~HealthAction src/http/action/health_action.h /^ virtual ~HealthAction() {};$/;" f class:doris::HealthAction +~HeapAction src/http/action/pprof_actions.cpp /^ virtual ~HeapAction() {}$/;" f class:doris::HeapAction +~HeartbeatServer src/agent/heartbeat_server.h /^ virtual ~HeartbeatServer() {};$/;" f class:doris::HeartbeatServer +~HllSetResolver src/olap/hll.h /^ ~HllSetResolver() {}$/;" f class:doris::HllSetResolver +~HttpClient src/http/http_client.cpp /^HttpClient::~HttpClient() {$/;" f class:doris::HttpClient +~HttpHandler src/http/http_handler.h /^ virtual ~HttpHandler() { }$/;" f class:doris::HttpHandler +~HttpRequest src/http/http_request.cpp /^HttpRequest::~HttpRequest() {$/;" f class:doris::HttpRequest +~HttpService src/service/http_service.cpp /^HttpService::~HttpService() {$/;" f class:doris::HttpService +~HttpUtilsTest test/http/http_utils_test.cpp /^ virtual ~HttpUtilsTest() {$/;" f class:doris::HttpUtilsTest +~HybirdMap src/exprs/hybird_map.h /^ virtual ~HybirdMap() {$/;" f class:doris::HybirdMap +~HybirdSet src/exprs/hybird_set.h /^ virtual ~HybirdSet() {$/;" f class:doris::HybirdSet +~HybirdSetBase src/exprs/hybird_set.h /^ virtual ~HybirdSetBase() {$/;" f class:doris::HybirdSetBase +~HyperLogLog src/olap/hll.h /^ ~HyperLogLog() {$/;" f class:doris::HyperLogLog +~IBinaryReader src/olap/push_handler.h /^ virtual ~IBinaryReader() {}$/;" f class:doris::IBinaryReader +~InOpTest test/exprs/in_op_test.cpp /^ ~InOpTest() {$/;" f class:doris::InOpTest +~InPredicate src/exprs/in_predicate.cpp /^InPredicate::~InPredicate() {$/;" f class:doris::InPredicate +~InStream src/olap/in_stream.cpp /^InStream::~InStream() {$/;" f class:doris::InStream +~InStreamBufferWrapper src/olap/in_stream.h /^ virtual ~InStreamBufferWrapper() {}$/;" f class:doris::InStreamBufferWrapper +~IndexChannel src/exec/tablet_sink.cpp /^IndexChannel::~IndexChannel() {$/;" f class:doris::stream_load::IndexChannel +~IndexComparator src/olap/olap_index.h /^ ~IndexComparator() {}$/;" f class:doris::IndexComparator +~InfoFunc src/exprs/info_func.h /^ virtual ~InfoFunc() { }$/;" f class:doris::InfoFunc +~IntegerColumnReader src/olap/rowset/column_reader.cpp /^IntegerColumnReader::~IntegerColumnReader() {$/;" f class:doris::IntegerColumnReader +~IntegerColumnReaderWrapper src/olap/rowset/column_reader.h /^ virtual ~IntegerColumnReaderWrapper() {}$/;" f class:doris::IntegerColumnReaderWrapper +~IntegerColumnWriter src/olap/rowset/column_writer.cpp /^IntegerColumnWriter::~IntegerColumnWriter() {$/;" f class:doris::IntegerColumnWriter +~IntegerColumnWriterWrapper src/olap/rowset/column_writer.h /^ virtual ~IntegerColumnWriterWrapper() {}$/;" f class:doris::IntegerColumnWriterWrapper +~Iterator src/exprs/hybird_set.h /^ virtual ~Iterator() {$/;" f class:doris::HybirdSet::Iterator +~Iterator src/exprs/hybird_set.h /^ virtual ~Iterator() {$/;" f class:doris::StringValueSet::Iterator +~IteratorBase src/exprs/hybird_set.h /^ virtual ~IteratorBase() {$/;" f class:doris::HybirdSetBase::IteratorBase +~JsonUtilTest test/util/json_util_test.cpp /^ virtual ~JsonUtilTest() { }$/;" f class:doris::JsonUtilTest +~KafkaConsumerPipe src/runtime/routine_load/kafka_consumer_pipe.h /^ virtual ~KafkaConsumerPipe() {}$/;" f class:doris::KafkaConsumerPipe +~KafkaConsumerPipeTest test/runtime/kafka_consumer_pipe_test.cpp /^ virtual ~ KafkaConsumerPipeTest() { }$/;" f class:doris::KafkaConsumerPipeTest +~KafkaDataConsumer src/runtime/routine_load/data_consumer.h /^ virtual ~KafkaDataConsumer() {$/;" f class:doris::KafkaDataConsumer +~KafkaDataConsumerGroup src/runtime/routine_load/data_consumer_group.cpp /^KafkaDataConsumerGroup::~KafkaDataConsumerGroup() {$/;" f class:doris::KafkaDataConsumerGroup +~KeyCoderResolver src/olap/key_coder.cpp /^ ~KeyCoderResolver() {$/;" f class:doris::KeyCoderResolver +~KeyCoderTest test/olap/key_coder_test.cpp /^ virtual ~KeyCoderTest() {$/;" f class:doris::KeyCoderTest +~KeysParam src/olap/reader.h /^ ~KeysParam() {$/;" f struct:doris::Reader::KeysParam +~KuduScanNode src/exec/kudu_scan_node.cpp /^KuduScanNode::~KuduScanNode() {$/;" f class:doris::KuduScanNode +~LRUCache src/olap/lru_cache.cpp /^LRUCache::~LRUCache() {$/;" f class:doris::LRUCache +~LargeIntColumnReader src/olap/rowset/column_reader.cpp /^LargeIntColumnReader::~LargeIntColumnReader() {$/;" f class:doris::LargeIntColumnReader +~LargeIntColumnWriter src/olap/rowset/column_writer.cpp /^LargeIntColumnWriter::~LargeIntColumnWriter() {$/;" f class:doris::LargeIntColumnWriter +~LineReader src/exec/line_reader.h /^ virtual ~LineReader() {$/;" f class:doris::LineReader +~LinkedSchemaChange src/olap/schema_change.h /^ ~LinkedSchemaChange() {}$/;" f class:doris::LinkedSchemaChange +~Literal src/exprs/literal.cpp /^Literal::~Literal() {$/;" f class:doris::Literal +~LoadChannel src/runtime/load_channel.cpp /^LoadChannel::~LoadChannel() {$/;" f class:doris::LoadChannel +~LoadChannelMgr src/runtime/load_channel_mgr.cpp /^LoadChannelMgr::~LoadChannelMgr() {$/;" f class:doris::LoadChannelMgr +~LoadChannelMgrTest test/runtime/load_channel_mgr_test.cpp /^ virtual ~LoadChannelMgrTest() { }$/;" f class:doris::LoadChannelMgrTest +~LoadErrorHub src/util/load_error_hub.h /^ virtual ~LoadErrorHub() {$/;" f class:doris::LoadErrorHub +~LoadPathMgr src/runtime/load_path_mgr.h /^ ~LoadPathMgr() {$/;" f class:doris::LoadPathMgr +~LoadStreamMgr src/runtime/stream_load/load_stream_mgr.h /^ ~LoadStreamMgr() { }$/;" f class:doris::LoadStreamMgr +~LocalFileReader src/exec/local_file_reader.cpp /^LocalFileReader::~LocalFileReader() {$/;" f class:doris::LocalFileReader +~LocalFileWriter src/exec/local_file_writer.cpp /^LocalFileWriter::~LocalFileWriter() {$/;" f class:doris::LocalFileWriter +~LockCounter src/util/metrics.h /^ virtual ~LockCounter() { }$/;" f class:doris::LockCounter +~LockGauge src/util/metrics.h /^ virtual ~LockGauge() { }$/;" f class:doris::LockGauge +~LockSimpleMetric src/util/metrics.h /^ virtual ~LockSimpleMetric() { }$/;" f class:doris::LockSimpleMetric +~Lz4FrameDecompressor src/exec/decompressor.cpp /^Lz4FrameDecompressor::~Lz4FrameDecompressor() {$/;" f class:doris::Lz4FrameDecompressor +~LzoBinaryReader src/olap/push_handler.h /^ virtual ~LzoBinaryReader() {$/;" f class:doris::LzoBinaryReader +~LzopDecompressor src/exec/lzo_decompressor.cpp /^LzopDecompressor::~LzopDecompressor() {$/;" f class:doris::LzopDecompressor +~MasterServerClient src/agent/utils.h /^ virtual ~MasterServerClient() {};$/;" f class:doris::MasterServerClient +~Md5Test test/util/md5_test.cpp /^ virtual ~Md5Test() { }$/;" f class:doris::Md5Test +~MemIndex src/olap/olap_index.cpp /^MemIndex::~MemIndex() {$/;" f class:doris::MemIndex +~MemPool src/runtime/mem_pool.cpp /^MemPool::~MemPool() {$/;" f class:doris::MemPool +~MemTable src/olap/memtable.cpp /^MemTable::~MemTable() {$/;" f class:doris::MemTable +~MemTableFlushExecutor src/olap/memtable_flush_executor.h /^ ~MemTableFlushExecutor() {}$/;" f class:doris::MemTableFlushExecutor +~MemTracker src/runtime/mem_tracker.cpp /^MemTracker::~MemTracker() {$/;" f class:doris::MemTracker +~MemoryScratchSink src/runtime/memory_scratch_sink.cpp /^MemoryScratchSink::~MemoryScratchSink() {$/;" f class:doris::MemoryScratchSink +~MemoryScratchSinkTest test/runtime/memory_scratch_sink_test.cpp /^ virtual ~MemoryScratchSinkTest() {}$/;" f class:doris::MemoryScratchSinkTest +~MergeJoinNode src/exec/merge_join_node.cpp /^MergeJoinNode::~MergeJoinNode() {$/;" f class:doris::MergeJoinNode +~MergeNode src/exec/merge_node.h /^ virtual ~MergeNode() { }$/;" f class:doris::MergeNode +~MergeSorter src/runtime/merge_sorter.cpp /^MergeSorter::~MergeSorter() {$/;" f class:doris::MergeSorter +~MessageBodyFileSink src/runtime/message_body_sink.cpp /^MessageBodyFileSink::~MessageBodyFileSink() {$/;" f class:doris::MessageBodyFileSink +~MessageBodySink src/runtime/message_body_sink.h /^ virtual ~MessageBodySink() { }$/;" f class:doris::MessageBodySink +~MessageBodySinkTest test/http/message_body_sink_test.cpp /^ virtual ~MessageBodySinkTest() { }$/;" f class:doris::MessageBodySinkTest +~MetaAction src/http/action/meta_action.h /^ virtual ~MetaAction() {}$/;" f class:doris::MetaAction +~Metric src/util/metrics.h /^ virtual ~Metric() { hide(); }$/;" f class:doris::Metric +~MetricRegistry src/util/metrics.cpp /^MetricRegistry::~MetricRegistry() {$/;" f class:doris::MetricRegistry +~MetricsAction src/http/action/metrics_action.h /^ virtual ~MetricsAction() { }$/;" f class:doris::MetricsAction +~MetricsActionTest test/http/metrics_action_test.cpp /^ virtual ~MetricsActionTest() {$/;" f class:doris::MetricsActionTest +~MetricsTest test/util/new_metrics_test.cpp /^ virtual ~MetricsTest() {$/;" f class:doris::MetricsTest +~MetricsVisitor src/util/metrics.h /^ virtual ~MetricsVisitor() { }$/;" f class:doris::MetricsVisitor +~MiniLoadAction src/http/action/mini_load.h /^ virtual ~MiniLoadAction() {$/;" f class:doris::MiniLoadAction +~MiniLoadAsyncCtx src/http/action/mini_load.cpp /^ ~MiniLoadAsyncCtx() {$/;" f struct:doris::MiniLoadAsyncCtx +~ModExpr src/exprs/arithmetic_expr.h /^ virtual ~ModExpr() { }$/;" f class:doris::ModExpr +~MonitorAction src/http/monitor_action.h /^ virtual ~MonitorAction() { }$/;" f class:doris::MonitorAction +~MulExpr src/exprs/arithmetic_expr.h /^ virtual ~MulExpr() { }$/;" f class:doris::MulExpr +~Mutex src/util/mutex.cpp /^Mutex::~Mutex() {$/;" f class:doris::Mutex +~MutexLock src/util/mutex.h /^ ~MutexLock() {$/;" f class:doris::MutexLock +~MysqlLoadErrorHub src/util/mysql_load_error_hub.cpp /^MysqlLoadErrorHub::~MysqlLoadErrorHub() {$/;" f class:doris::MysqlLoadErrorHub +~MysqlRowBuffer src/util/mysql_row_buffer.cpp /^MysqlRowBuffer::~MysqlRowBuffer() {$/;" f class:doris::MysqlRowBuffer +~MysqlScanNode src/exec/mysql_scan_node.cpp /^MysqlScanNode::~MysqlScanNode() {$/;" f class:doris::MysqlScanNode +~MysqlScanner src/exec/mysql_scanner.cpp /^MysqlScanner::~MysqlScanner() {$/;" f class:doris::MysqlScanner +~MysqlTableSink src/runtime/mysql_table_sink.cpp /^MysqlTableSink::~MysqlTableSink() {$/;" f class:doris::MysqlTableSink +~MysqlTableWriter src/runtime/mysql_table_writer.cpp /^MysqlTableWriter::~MysqlTableWriter() {$/;" f class:doris::MysqlTableWriter +~MysqlTableWriterTest test/runtime/mysql_table_writer_test.cpp /^ ~MysqlTableWriterTest() {$/;" f class:doris::MysqlTableWriterTest +~NewAggFnEvaluator src/exprs/new_agg_fn_evaluator.cc /^NewAggFnEvaluator::~NewAggFnEvaluator() {$/;" f class:NewAggFnEvaluator +~Node src/util/internal_queue.h /^ virtual ~Node() {}$/;" f struct:doris::InternalQueueBase::Node +~NodeChannel src/exec/tablet_sink.cpp /^NodeChannel::~NodeChannel() {$/;" f class:doris::stream_load::NodeChannel +~NullLoadErrorHub src/util/null_load_error_hub.cpp /^NullLoadErrorHub::~NullLoadErrorHub() {$/;" f class:doris::NullLoadErrorHub +~NullPredicate src/olap/null_predicate.cpp /^NullPredicate::~NullPredicate() {}$/;" f class:doris::NullPredicate +~ORCScanner src/exec/orc_scanner.cpp /^ORCScanner::~ORCScanner() {$/;" f class:doris::ORCScanner +~ObjectPool src/common/object_pool.h /^ ~ObjectPool() {$/;" f class:doris::ObjectPool +~OlapMeta src/olap/olap_meta.cpp /^OlapMeta::~OlapMeta() {$/;" f class:doris::OlapMeta +~OlapRewriteNode src/exec/olap_rewrite_node.h /^ virtual ~OlapRewriteNode() { }$/;" f class:doris::OlapRewriteNode +~OlapScanNode src/exec/olap_scan_node.cpp /^OlapScanNode::~OlapScanNode() {$/;" f class:doris::OlapScanNode +~OlapScanner src/exec/olap_scanner.cpp /^OlapScanner::~OlapScanner() {$/;" f class:doris::OlapScanner +~OlapTablePartitionParam src/exec/tablet_info.cpp /^OlapTablePartitionParam::~OlapTablePartitionParam() {$/;" f class:doris::OlapTablePartitionParam +~OlapTablePartitionParamTest test/exec/tablet_info_test.cpp /^ virtual ~OlapTablePartitionParamTest() { }$/;" f class:doris::OlapTablePartitionParamTest +~OlapTableSink src/exec/tablet_sink.cpp /^OlapTableSink::~OlapTableSink() {$/;" f class:doris::stream_load::OlapTableSink +~OlapTableSinkTest test/exec/tablet_sink_test.cpp /^ virtual ~OlapTableSinkTest() { }$/;" f class:doris::stream_load::OlapTableSinkTest +~OpenedFileHandle src/util/file_cache.h /^ ~OpenedFileHandle() {$/;" f class:doris::OpenedFileHandle +~OptionsTest test/olap/options_test.cpp /^ virtual ~OptionsTest() {}$/;" f class:doris::OptionsTest +~OutStream src/olap/out_stream.cpp /^OutStream::~OutStream() {$/;" f class:doris::OutStream +~OutStreamFactory src/olap/out_stream.cpp /^OutStreamFactory::~OutStreamFactory() {$/;" f class:doris::OutStreamFactory +~OwnedSlice src/util/slice.h /^ ~OwnedSlice(){$/;" f class:doris::OwnedSlice +~PInternalServiceImpl src/service/internal_service.cpp /^PInternalServiceImpl::~PInternalServiceImpl() {$/;" f class:doris::PInternalServiceImpl +~PageBuilder src/olap/rowset/segment_v2/page_builder.h /^ virtual ~PageBuilder() { }$/;" f class:doris::segment_v2::PageBuilder +~PageCacheHandle src/olap/page_cache.h /^ ~PageCacheHandle() {$/;" f class:doris::PageCacheHandle +~PageDecoder src/olap/rowset/segment_v2/page_decoder.h /^ virtual ~PageDecoder() { }$/;" f class:doris::segment_v2::PageDecoder +~PageHandle src/olap/rowset/segment_v2/page_handle.h /^ ~PageHandle() {$/;" f class:doris::segment_v2::PageHandle +~PageHandle src/runtime/bufferpool/buffer_pool.h /^ ~PageHandle() { DCHECK(!is_open()); }$/;" f class:doris::BufferPool::PageHandle +~PageList src/runtime/bufferpool/buffer_pool_internal.h /^ ~PageList() {$/;" f class:doris::BufferPool::PageList +~ParquetFile src/exec/parquet_reader.cpp /^ParquetFile::~ParquetFile() {$/;" f class:doris::ParquetFile +~ParquetReaderWrap src/exec/parquet_reader.cpp /^ParquetReaderWrap::~ParquetReaderWrap() {$/;" f class:doris::ParquetReaderWrap +~ParquetScanner src/exec/parquet_scanner.cpp /^ParquetScanner::~ParquetScanner() {$/;" f class:doris::ParquetScanner +~ParsedPage src/olap/rowset/segment_v2/parsed_page.h /^ ~ParsedPage() {$/;" f struct:doris::segment_v2::ParsedPage +~PartRange src/runtime/dpp_sink_internal.h /^ ~PartRange() {$/;" f class:doris::PartRange +~PartRangeKey src/runtime/dpp_sink_internal.h /^ ~PartRangeKey() {$/;" f class:doris::PartRangeKey +~Partition src/exec/new_partitioned_aggregation_node.cc /^NewPartitionedAggregationNode::Partition::~Partition() {$/;" f class:doris::NewPartitionedAggregationNode::Partition +~PartitionedAggregationNode src/exec/partitioned_aggregation_node.h /^ virtual ~PartitionedAggregationNode() {}$/;" f class:doris::PartitionedAggregationNode +~PartitionedHashTableTest test/exec/partitioned_hash_table_test.cpp /^ ~PartitionedHashTableTest() {}$/;" f class:doris::PartitionedHashTableTest +~PathTrie src/util/path_trie.hpp /^ ~PathTrie() {$/;" f class:doris::PathTrie +~PercentileApproxState src/exprs/aggregate_functions.cpp /^ ~PercentileApproxState() {$/;" f struct:doris::PercentileApproxState +~PerfCounters src/util/perf_counters.cpp /^PerfCounters::~PerfCounters() {$/;" f class:doris::PerfCounters +~PeriodicCounterUpdateState src/util/runtime_profile.cpp /^RuntimeProfile::PeriodicCounterUpdateState::~PeriodicCounterUpdateState() {$/;" f class:doris::RuntimeProfile::PeriodicCounterUpdateState +~PlainPageTest test/olap/rowset/segment_v2/plain_page_test.cpp /^ virtual ~PlainPageTest() {$/;" f class:doris::segment_v2::PlainPageTest +~PlainTextLineReader src/exec/plain_text_line_reader.cpp /^PlainTextLineReader::~PlainTextLineReader() {$/;" f class:doris::PlainTextLineReader +~PlanFragmentExecutor src/runtime/plan_fragment_executor.cpp /^PlanFragmentExecutor::~PlanFragmentExecutor() {$/;" f class:doris::PlanFragmentExecutor +~PlanFragmentExecutor test/runtime/fragment_mgr_test.cpp /^PlanFragmentExecutor::~PlanFragmentExecutor() {$/;" f class:doris::PlanFragmentExecutor +~PmuProfileAction src/http/action/pprof_actions.cpp /^ virtual ~PmuProfileAction() {}$/;" f class:doris::PmuProfileAction +~PositionEntryReader src/olap/stream_index_reader.h /^ ~PositionEntryReader() {}$/;" f class:doris::PositionEntryReader +~PositionProvider src/olap/stream_index_reader.h /^ ~PositionProvider() {}$/;" f class:doris::PositionProvider +~PosixRandomRWFile src/env/env_posix.cpp /^ ~PosixRandomRWFile() {$/;" f class:doris::PosixRandomRWFile +~PreAggregationNode src/exec/pre_aggregation_node.cpp /^PreAggregationNode::~PreAggregationNode() {$/;" f class:doris::PreAggregationNode +~PriorityThreadPool src/util/priority_thread_pool.hpp /^ ~PriorityThreadPool() {$/;" f class:doris::PriorityThreadPool +~ProfileAction src/http/action/pprof_actions.cpp /^ virtual ~ProfileAction() {}$/;" f class:doris::ProfileAction +~PrometheusMetricsVisitor src/http/action/metrics_action.cpp /^ virtual ~PrometheusMetricsVisitor() {}$/;" f class:doris::PrometheusMetricsVisitor +~PushHandler src/olap/push_handler.h /^ ~PushHandler() {}$/;" f class:doris::PushHandler +~Pusher src/agent/pusher.cpp /^Pusher::~Pusher() {$/;" f class:doris::Pusher +~QSorter src/runtime/qsorter.h /^ virtual ~QSorter() {$/;" f class:doris::QSorter +~QSorterTest test/runtime/qsorter_test.cpp /^ ~QSorterTest() {$/;" f class:doris::QSorterTest +~QueryBuilder src/exec/es/es_query_builder.h /^ virtual ~QueryBuilder() {$/;" f class:doris::QueryBuilder +~QueryStatisticsRecvr src/runtime/query_statistics.cpp /^QueryStatisticsRecvr::~QueryStatisticsRecvr() {$/;" f class:doris::QueryStatisticsRecvr +~RWMutex src/util/mutex.cpp /^RWMutex::~RWMutex() {$/;" f class:doris::RWMutex +~RadixSortTest test/util/radix_sort_test.cpp /^ virtual ~RadixSortTest() {$/;" f class:doris::RadixSortTest +~RandomAccessFile src/env/env.h /^ virtual ~RandomAccessFile() { }$/;" f class:doris::RandomAccessFile +~RandomRWFile src/env/env.h /^ virtual ~RandomRWFile() { }$/;" f class:doris::RandomRWFile +~ReadLock src/util/mutex.h /^ ~ReadLock() { $/;" f class:doris::ReadLock +~ReadOnlyFileStream src/olap/file_stream.h /^ ~ReadOnlyFileStream() {$/;" f class:doris::ReadOnlyFileStream +~ReadableBlock src/olap/fs/block_manager.h /^ virtual ~ReadableBlock() {}$/;" f class:doris::fs::ReadableBlock +~Reader src/olap/reader.cpp /^Reader::~Reader() {$/;" f class:doris::Reader +~ReceiverInfo test/runtime/data_stream_test.cpp /^ ~ReceiverInfo() {$/;" f struct:doris::DataStreamTest::ReceiverInfo +~RefCountClosure src/util/ref_count_closure.h /^ ~RefCountClosure() { }$/;" f class:doris::RefCountClosure +~RefCounted src/gutil/ref_counted.h /^ ~RefCounted() {}$/;" f class:doris::RefCounted +~RefCountedBase src/gutil/ref_counted.cc /^RefCountedBase::~RefCountedBase() {$/;" f class:doris::subtle::RefCountedBase +~RefCountedData src/gutil/ref_counted.h /^ ~RefCountedData() {}$/;" f class:doris::RefCountedData +~RefCountedThreadSafe src/gutil/ref_counted.h /^ ~RefCountedThreadSafe() {}$/;" f class:doris::RefCountedThreadSafe +~RefCountedThreadSafeBase src/gutil/ref_counted.cc /^RefCountedThreadSafeBase::~RefCountedThreadSafeBase() {$/;" f class:doris::subtle::RefCountedThreadSafeBase +~ReloadTabletAction src/http/action/reload_tablet_action.h /^ virtual ~ReloadTabletAction() { }$/;" f class:doris::ReloadTabletAction +~RepeatNode src/exec/repeat_node.cpp /^RepeatNode::~RepeatNode() {$/;" f class:doris::RepeatNode +~RequestContextCache src/runtime/disk_io_mgr.cc /^ ~RequestContextCache() {$/;" f class:doris::DiskIoMgr::RequestContextCache +~ReservationTracker src/runtime/bufferpool/reservation_tracker.cc /^ReservationTracker::~ReservationTracker() {$/;" f class:doris::ReservationTracker +~ResourcePool src/runtime/thread_resource_mgr.h /^ virtual ~ResourcePool() {};$/;" f class:doris::ThreadResourceMgr::ResourcePool +~RestMonitorIface src/http/rest_monitor_iface.h /^ virtual ~RestMonitorIface() { }$/;" f class:doris::RestMonitorIface +~RestoreTabletAction src/http/action/restore_tablet_action.h /^ virtual ~RestoreTabletAction() { }$/;" f class:doris::RestoreTabletAction +~ResultBufferMgr src/runtime/result_buffer_mgr.cpp /^ResultBufferMgr::~ResultBufferMgr() {$/;" f class:doris::ResultBufferMgr +~ResultBufferMgrTest test/runtime/result_buffer_mgr_test.cpp /^ virtual ~ResultBufferMgrTest() {$/;" f class:doris::ResultBufferMgrTest +~ResultQueueMgr src/runtime/result_queue_mgr.cpp /^ResultQueueMgr::~ResultQueueMgr() {$/;" f class:doris::ResultQueueMgr +~ResultQueueMgrTest test/runtime/result_queue_mgr_test.cpp /^ virtual ~ResultQueueMgrTest() {$/;" f class:doris::ResultQueueMgrTest +~ResultSink src/runtime/result_sink.cpp /^ResultSink::~ResultSink() {$/;" f class:doris::ResultSink +~ResultSinkTest test/runtime/result_sink_test.cpp /^ virtual ~ResultSinkTest() {$/;" f class:doris::ResultSinkTest +~ResultWriter src/runtime/result_writer.cpp /^ResultWriter::~ResultWriter() {$/;" f class:doris::ResultWriter +~ResultWriterTest test/runtime/result_writer_test.cpp /^ virtual ~ResultWriterTest() {$/;" f class:doris::ResultWriterTest +~RlePageBuilder src/olap/rowset/segment_v2/rle_page.h /^ ~RlePageBuilder() {$/;" f class:doris::segment_v2::RlePageBuilder +~RlePageTest test/olap/rowset/segment_v2/rle_page_test.cpp /^ virtual ~RlePageTest() { }$/;" f class:doris::RlePageTest +~RollupSchema src/runtime/dpp_sink_internal.cpp /^RollupSchema::~RollupSchema() {$/;" f class:doris::RollupSchema +~RoutineLoadTaskExecutor src/runtime/routine_load/routine_load_task_executor.h /^ ~RoutineLoadTaskExecutor() {$/;" f class:doris::RoutineLoadTaskExecutor +~RoutineLoadTaskExecutorTest test/runtime/routine_load_task_executor_test.cpp /^ virtual ~RoutineLoadTaskExecutorTest() { }$/;" f class:doris::RoutineLoadTaskExecutorTest +~RowBatch src/runtime/row_batch.cpp /^RowBatch::~RowBatch() {$/;" f class:doris::RowBatch +~RowBatchInterface src/runtime/row_batch_interface.hpp /^ virtual ~RowBatchInterface() {}$/;" f class:doris::RowBatchInterface +~RowBatchList src/exec/row_batch_list.h /^ virtual ~RowBatchList() { }$/;" f class:doris::RowBatchList +~RowBatchQueue src/exec/exec_node.cpp /^ExecNode::RowBatchQueue::~RowBatchQueue() {$/;" f class:doris::ExecNode::RowBatchQueue +~RowBlock src/olap/row_block.cpp /^RowBlock::~RowBlock() {$/;" f class:doris::RowBlock +~RowBlockAllocator src/olap/schema_change.cpp /^RowBlockAllocator::~RowBlockAllocator() {$/;" f class:doris::RowBlockAllocator +~RowBlockChanger src/olap/schema_change.cpp /^RowBlockChanger::~RowBlockChanger() {$/;" f class:doris::RowBlockChanger +~RowBlockMerger src/olap/schema_change.cpp /^RowBlockMerger::~RowBlockMerger() {}$/;" f class:doris::RowBlockMerger +~RowBlockSorter src/olap/schema_change.cpp /^RowBlockSorter::~RowBlockSorter() {$/;" f class:doris::RowBlockSorter +~RowBlockV2 src/olap/row_block2.cpp /^RowBlockV2::~RowBlockV2() {$/;" f class:doris::RowBlockV2 +~RowCursor src/olap/row_cursor.cpp /^RowCursor::~RowCursor() {$/;" f class:doris::RowCursor +~RowRangesTest test/olap/rowset/segment_v2/row_ranges_test.cpp /^ virtual ~RowRangesTest() { }$/;" f class:doris::segment_v2::RowRangesTest +~Rowset src/olap/rowset/rowset.h /^ virtual ~Rowset() { }$/;" f class:doris::Rowset +~RowsetIdGenerator src/olap/rowset/rowset_id_generator.h /^ virtual ~RowsetIdGenerator() {}$/;" f class:doris::RowsetIdGenerator +~RowsetMeta src/olap/rowset/rowset_meta.h /^ virtual ~RowsetMeta() { }$/;" f class:doris::RowsetMeta +~RowsetReader src/olap/rowset/rowset_reader.h /^ virtual ~RowsetReader() { }$/;" f class:doris::RowsetReader +~RowwiseIterator src/olap/iterators.h /^ virtual ~RowwiseIterator() { }$/;" f class:doris::RowwiseIterator +~Run src/runtime/merge_sorter.cpp /^ ~Run() {}$/;" f class:doris::MergeSorter::Run +~Run src/runtime/spill_sorter.cc /^ ~Run() {$/;" f class:doris::SpillSorter::Run +~RunLengthByteReader src/olap/rowset/run_length_byte_reader.h /^ ~RunLengthByteReader() {}$/;" f class:doris::RunLengthByteReader +~RunLengthByteWriter src/olap/rowset/run_length_byte_writer.h /^ ~RunLengthByteWriter() {}$/;" f class:doris::RunLengthByteWriter +~RunLengthIntegerReader src/olap/rowset/run_length_integer_reader.h /^ ~RunLengthIntegerReader() {}$/;" f class:doris::RunLengthIntegerReader +~RunLengthIntegerWriter src/olap/rowset/run_length_integer_writer.h /^ ~RunLengthIntegerWriter() {}$/;" f class:doris::RunLengthIntegerWriter +~Runnable src/util/threadpool.h /^ virtual ~Runnable() {}$/;" f class:doris::Runnable +~RuntimeProfile src/util/runtime_profile.cpp /^RuntimeProfile::~RuntimeProfile() {$/;" f class:doris::RuntimeProfile +~RuntimeState src/runtime/runtime_state.cpp /^RuntimeState::~RuntimeState() {$/;" f class:doris::RuntimeState +~STLElementDeleter src/gutil/stl_util.h /^ ~STLElementDeleter() { STLDeleteElements(container_ptr_); }$/;" f class:STLElementDeleter +~STLValueDeleter src/gutil/stl_util.h /^ ~STLValueDeleter() { STLDeleteValues(container_ptr_); }$/;" f class:STLValueDeleter +~ScalarFnCall src/exprs/scalar_fn_call.cpp /^ScalarFnCall::~ScalarFnCall() {$/;" f class:doris::ScalarFnCall +~ScanNode src/exec/scan_node.h /^ virtual ~ScanNode() { }$/;" f class:doris::ScanNode +~ScanRange src/runtime/disk_io_mgr_scan_range.cc /^DiskIoMgr::ScanRange::~ScanRange() {$/;" f class:doris::DiskIoMgr::ScanRange +~Schema src/olap/schema.cpp /^Schema::~Schema() {$/;" f class:doris::Schema +~SchemaChange src/olap/schema_change.h /^ virtual ~SchemaChange() {}$/;" f class:doris::SchemaChange +~SchemaChangeDirectly src/olap/schema_change.cpp /^SchemaChangeDirectly::~SchemaChangeDirectly() {$/;" f class:doris::SchemaChangeDirectly +~SchemaChangeHandler src/olap/schema_change.h /^ virtual ~SchemaChangeHandler() {}$/;" f class:doris::SchemaChangeHandler +~SchemaChangeWithSorting src/olap/schema_change.cpp /^SchemaChangeWithSorting::~SchemaChangeWithSorting() {$/;" f class:doris::SchemaChangeWithSorting +~SchemaCharsetsScanner src/exec/schema_scanner/schema_charsets_scanner.cpp /^SchemaCharsetsScanner::~SchemaCharsetsScanner() {$/;" f class:doris::SchemaCharsetsScanner +~SchemaCollationsScanner src/exec/schema_scanner/schema_collations_scanner.cpp /^SchemaCollationsScanner::~SchemaCollationsScanner() {$/;" f class:doris::SchemaCollationsScanner +~SchemaColumnsScanner src/exec/schema_scanner/schema_columns_scanner.cpp /^SchemaColumnsScanner::~SchemaColumnsScanner() {$/;" f class:doris::SchemaColumnsScanner +~SchemaDummyScanner src/exec/schema_scanner/schema_dummy_scanner.cpp /^SchemaDummyScanner::~SchemaDummyScanner() {$/;" f class:doris::SchemaDummyScanner +~SchemaScanNode src/exec/schema_scan_node.cpp /^SchemaScanNode::~SchemaScanNode() {$/;" f class:doris::SchemaScanNode +~SchemaScanNodeTest test/exec/schema_scan_node_test.cpp /^ virtual ~SchemaScanNodeTest() { }$/;" f class:doris::SchemaScanNodeTest +~SchemaScanner src/exec/schema_scanner.cpp /^SchemaScanner::~SchemaScanner() {$/;" f class:doris::SchemaScanner +~SchemaSchemataScanner src/exec/schema_scanner/schema_schemata_scanner.cpp /^SchemaSchemataScanner::~SchemaSchemataScanner() {$/;" f class:doris::SchemaSchemataScanner +~SchemaTableDescriptor src/runtime/descriptors.cpp /^SchemaTableDescriptor::~SchemaTableDescriptor() {$/;" f class:doris::SchemaTableDescriptor +~SchemaTablesScanner src/exec/schema_scanner/schema_tables_scanner.cpp /^SchemaTablesScanner::~SchemaTablesScanner() {$/;" f class:doris::SchemaTablesScanner +~SchemaVariablesScanner src/exec/schema_scanner/schema_variables_scanner.cpp /^SchemaVariablesScanner::~SchemaVariablesScanner() {$/;" f class:doris::SchemaVariablesScanner +~ScopedCheck src/gutil/threading/thread_collision_warner.h /^ ~ScopedCheck() {$/;" f class:base::ThreadCollisionWarner::ScopedCheck +~ScopedCleanup src/util/scoped_cleanup.h /^ ~ScopedCleanup() {$/;" f class:doris::ScopedCleanup +~ScopedCounter src/util/runtime_profile.h /^ ~ScopedCounter() {$/;" f class:doris::ScopedCounter +~ScopedFdCloser src/env/env_posix.cpp /^ ~ScopedFdCloser() {$/;" f class:doris::ScopedFdCloser +~ScopedRawTimer src/util/runtime_profile.h /^ ~ScopedRawTimer() {$/;" f class:doris::ScopedRawTimer +~ScopedRecursiveCheck src/gutil/threading/thread_collision_warner.h /^ ~ScopedRecursiveCheck() {$/;" f class:base::ThreadCollisionWarner::ScopedRecursiveCheck +~ScopedTimer src/util/runtime_profile.h /^ ~ScopedTimer() {$/;" f class:doris::ScopedTimer +~ScrollParser src/exec/es/es_scroll_parser.cpp /^ScrollParser::~ScrollParser() {$/;" f class:doris::ScrollParser +~SegmentComparator src/olap/olap_index.h /^ ~SegmentComparator() {}$/;" f class:doris::SegmentComparator +~SegmentGroup src/olap/rowset/segment_group.cpp /^SegmentGroup::~SegmentGroup() {$/;" f class:doris::SegmentGroup +~SegmentIterator src/olap/rowset/segment_v2/segment_iterator.cpp /^SegmentIterator::~SegmentIterator() {$/;" f class:doris::segment_v2::SegmentIterator +~SegmentReader src/olap/rowset/segment_reader.cpp /^SegmentReader::~SegmentReader() {$/;" f class:doris::SegmentReader +~SegmentWriter src/olap/rowset/segment_writer.cpp /^SegmentWriter::~SegmentWriter() {$/;" f class:doris::SegmentWriter +~SenderQueue src/runtime/data_stream_recvr.cc /^ ~SenderQueue() {}$/;" f class:doris::DataStreamRecvr::SenderQueue +~SequentialFile src/env/env.h /^ virtual ~SequentialFile() { }$/;" f class:doris::SequentialFile +~SerializeTest test/olap/serialize_test.cpp /^ virtual ~SerializeTest() {$/;" f class:doris::ser::SerializeTest +~ShardedLRUCache src/olap/lru_cache.h /^ virtual ~ShardedLRUCache() {}$/;" f class:doris::CachePriority::ShardedLRUCache +~ShortKeyIndexTest test/olap/short_key_index_test.cpp /^ virtual ~ShortKeyIndexTest() {$/;" f class:doris::ShortKeyIndexTest +~SimpleCoreMetricsVisitor src/http/action/metrics_action.cpp /^ virtual ~SimpleCoreMetricsVisitor() {}$/;" f class:doris::SimpleCoreMetricsVisitor +~SimpleMetric src/util/metrics.h /^ virtual ~SimpleMetric() { }$/;" f class:doris::SimpleMetric +~SimpleTupleStreamTest test/runtime/buffered_tuple_stream2_test.cpp /^ ~SimpleTupleStreamTest() {}$/;" f class:doris::SimpleTupleStreamTest +~SlowDestructorRunnable test/util/threadpool_test.cpp /^ virtual ~SlowDestructorRunnable() {$/;" f class:doris::SlowDestructorRunnable +~SmallFileMgr src/runtime/small_file_mgr.cpp /^SmallFileMgr::~SmallFileMgr() {$/;" f class:doris::SmallFileMgr +~SmallFileMgrTest test/runtime/small_file_mgr_test.cpp /^ virtual ~SmallFileMgrTest() { }$/;" f class:doris::SmallFileMgrTest +~SnapshotAction src/http/action/snapshot_action.h /^ virtual ~SnapshotAction() { }$/;" f class:doris::SnapshotAction +~SnapshotLoader src/runtime/snapshot_loader.cpp /^SnapshotLoader::~SnapshotLoader() {$/;" f class:doris::SnapshotLoader +~SnapshotManager src/olap/snapshot_manager.h /^ ~SnapshotManager() {}$/;" f class:doris::SnapshotManager +~SortNode src/exec/sort_node.cpp /^SortNode::~SortNode() {$/;" f class:doris::SortNode +~SortedRunMerger src/runtime/sorted_run_merger.h /^ ~SortedRunMerger() {}$/;" f class:doris::SortedRunMerger +~Sorter src/runtime/sorter.h /^ virtual ~Sorter() { $/;" f class:doris::Sorter +~SorterTest test/runtime/sorter_test.cpp /^ virtual ~SorterTest() {$/;" f class:doris::SorterTest +~SpecificElement src/common/object_pool.h /^ ~SpecificElement() {$/;" f struct:doris::ObjectPool::SpecificElement +~SpillSortNode src/exec/spill_sort_node.cc /^SpillSortNode::~SpillSortNode() {$/;" f class:doris::SpillSortNode +~SpillSorter src/runtime/spill_sorter.cc /^SpillSorter::~SpillSorter() {$/;" f class:doris::SpillSorter +~StConstructState src/geo/geo_functions.cpp /^ ~StConstructState() { }$/;" f struct:doris::StConstructState +~StContainsState src/geo/geo_functions.cpp /^ ~StContainsState() {$/;" f struct:doris::StContainsState +~StorageEngine src/olap/storage_engine.cpp /^StorageEngine::~StorageEngine() {$/;" f class:doris::StorageEngine +~StoragePageCacheTest test/olap/page_cache_test.cpp /^ virtual ~StoragePageCacheTest() {$/;" f class:doris::StoragePageCacheTest +~StreamIndexReader src/olap/stream_index_reader.cpp /^StreamIndexReader::~StreamIndexReader() {$/;" f class:doris::StreamIndexReader +~StreamIndexWriter src/olap/stream_index_writer.cpp /^StreamIndexWriter::~StreamIndexWriter() {$/;" f class:doris::StreamIndexWriter +~StreamLoadAction src/http/action/stream_load.cpp /^StreamLoadAction::~StreamLoadAction() {$/;" f class:doris::StreamLoadAction +~StreamLoadActionTest test/http/stream_load_test.cpp /^ virtual ~StreamLoadActionTest() { }$/;" f class:doris::StreamLoadActionTest +~StreamLoadContext src/runtime/stream_load/stream_load_context.h /^ ~StreamLoadContext() {$/;" f class:doris::StreamLoadContext +~StreamLoadPipe src/runtime/stream_load/stream_load_pipe.h /^ virtual ~StreamLoadPipe() { }$/;" f class:doris::StreamLoadPipe +~StreamLoadPipeTest test/runtime/stream_load_pipe_test.cpp /^ virtual ~StreamLoadPipeTest() { }$/;" f class:doris::StreamLoadPipeTest +~StringBuffer src/runtime/string_buffer.hpp /^ virtual ~StringBuffer() {}$/;" f class:doris::StringBuffer +~StringColumnDictionaryReader src/olap/rowset/column_reader.cpp /^StringColumnDictionaryReader::~StringColumnDictionaryReader() {$/;" f class:doris::StringColumnDictionaryReader +~StringColumnDirectReader src/olap/rowset/column_reader.cpp /^StringColumnDirectReader::~StringColumnDirectReader() {$/;" f class:doris::StringColumnDirectReader +~StringParserTest test/util/string_parser_test.cpp /^ ~StringParserTest(){}$/;" f class:doris::StringParserTest +~StringRef src/exec/csv_scan_node.cpp /^ ~StringRef() {$/;" f class:doris::StringRef +~StringSearch src/runtime/string_search.hpp /^ virtual ~StringSearch() {}$/;" f class:doris::StringSearch +~StringUtilTest test/util/string_util_test.cpp /^ virtual ~StringUtilTest() {$/;" f class:doris::StringUtilTest +~StringValueSet src/exprs/hybird_set.h /^ virtual ~StringValueSet() {$/;" f class:doris::StringValueSet +~SubExpr src/exprs/arithmetic_expr.h /^ virtual ~SubExpr() { }$/;" f class:doris::SubExpr +~SubReservation src/runtime/bufferpool/buffer_pool.cc /^BufferPool::SubReservation::~SubReservation() {}$/;" f class:doris::BufferPool::SubReservation +~Suballocation src/runtime/bufferpool/suballocator.h /^ ~Suballocation() { DCHECK(!in_use_); }$/;" f class:doris::Suballocation +~Suballocator src/runtime/bufferpool/suballocator.cc /^Suballocator::~Suballocator() {$/;" f class:doris::Suballocator +~SymbolAction src/http/action/pprof_actions.cpp /^ virtual ~SymbolAction() {}$/;" f class:doris::SymbolAction +~SystemMetrics src/util/system_metrics.cpp /^SystemMetrics::~SystemMetrics() {$/;" f class:doris::SystemMetrics +~SystemMetricsTest test/util/system_metrics_test.cpp /^ virtual ~SystemMetricsTest() {$/;" f class:doris::SystemMetricsTest +~TDigestTest test/util/tdigest_test.cpp /^ virtual ~TDigestTest() {$/;" f class:doris::TDigestTest +~TableDescriptor src/runtime/descriptors.h /^ virtual ~TableDescriptor() {}$/;" f class:doris::TableDescriptor +~Tablet src/olap/tablet.cpp /^Tablet::~Tablet() {$/;" f class:doris::Tablet +~TabletManager src/olap/tablet_manager.cpp /^TabletManager::~TabletManager() {$/;" f class:doris::TabletManager +~TabletSyncService src/olap/tablet_sync_service.cpp /^TabletSyncService::~TabletSyncService() {$/;" f class:doris::TabletSyncService +~TabletsChannel src/runtime/tablets_channel.cpp /^TabletsChannel::~TabletsChannel() {$/;" f class:doris::TabletsChannel +~TaskWorkerPool src/agent/task_worker_pool.cpp /^TaskWorkerPool::~TaskWorkerPool() {}$/;" f class:doris::TaskWorkerPool +~TempDisable src/util/cpu_info.h /^ ~TempDisable() {$/;" f struct:doris::CpuInfo::TempDisable +~TemplatedElementDeleter src/gutil/stl_util.h /^ virtual ~TemplatedElementDeleter() {$/;" f class:TemplatedElementDeleter +~TemplatedValueDeleter src/gutil/stl_util.h /^ virtual ~TemplatedValueDeleter() {$/;" f class:TemplatedValueDeleter +~TestBitField test/olap/bit_field_test.cpp /^ virtual ~TestBitField() {$/;" f class:doris::TestBitField +~TestBloomFilter test/olap/bloom_filter_test.cpp /^ virtual ~TestBloomFilter() {}$/;" f class:doris::TestBloomFilter +~TestBloomFilterIndex test/olap/bloom_filter_index_test.cpp /^ virtual ~TestBloomFilterIndex() {}$/;" f class:doris::TestBloomFilterIndex +~TestByteBuffer test/olap/byte_buffer_test.cpp /^ virtual ~TestByteBuffer() {$/;" f class:doris::TestByteBuffer +~TestColumn test/olap/column_reader_test.cpp /^ virtual ~TestColumn() {$/;" f class:doris::TestColumn +~TestColumn test/olap/schema_change_test.cpp /^ virtual ~TestColumn() {$/;" f class:doris::TestColumn +~TestDeltaWriter test/olap/delta_writer_test.cpp /^ ~TestDeltaWriter() { }$/;" f class:doris::TestDeltaWriter +~TestEnv src/runtime/test_env.cc /^TestEnv::~TestEnv() {$/;" f class:doris::TestEnv +~TestHll test/olap/hll_test.cpp /^ virtual ~TestHll() { }$/;" f class:doris::TestHll +~TestInListPredicate test/olap/in_list_predicate_test.cpp /^ ~TestInListPredicate() {$/;" f class:doris::TestInListPredicate +~TestInternalService test/exec/tablet_sink_test.cpp /^ virtual ~TestInternalService() { }$/;" f class:doris::stream_load::TestInternalService +~TestMemTableFlushExecutor test/olap/memtable_flush_executor_test.cpp /^ ~TestMemTableFlushExecutor() { }$/;" f class:doris::TestMemTableFlushExecutor +~TestMetricsVisitor test/util/doris_metrics_test.cpp /^ virtual ~TestMetricsVisitor() { }$/;" f class:doris::TestMetricsVisitor +~TestMetricsVisitor test/util/new_metrics_test.cpp /^ virtual ~TestMetricsVisitor() { }$/;" f class:doris::TestMetricsVisitor +~TestMetricsVisitor test/util/system_metrics_test.cpp /^ virtual ~TestMetricsVisitor() { }$/;" f class:doris::TestMetricsVisitor +~TestNullPredicate test/olap/null_predicate_test.cpp /^ ~TestNullPredicate() {$/;" f class:doris::TestNullPredicate +~TestRunLengthByte test/olap/run_length_byte_test.cpp /^ virtual ~TestRunLengthByte() {$/;" f class:doris::TestRunLengthByte +~TestRunLengthSignInteger test/olap/run_length_integer_test.cpp /^ virtual ~TestRunLengthSignInteger() {$/;" f class:doris::TestRunLengthSignInteger +~TestRunLengthUnsignInteger test/olap/run_length_integer_test.cpp /^ virtual ~TestRunLengthUnsignInteger() {$/;" f class:doris::TestRunLengthUnsignInteger +~TestStreamIndex test/olap/stream_index_test.cpp /^ virtual ~TestStreamIndex() {$/;" f class:doris::TestStreamIndex +~Thread src/util/thread.cpp /^Thread::~Thread() {$/;" f class:doris::Thread +~ThreadCollisionWarner src/gutil/threading/thread_collision_warner.h /^ ~ThreadCollisionWarner() {$/;" f class:base::ThreadCollisionWarner +~ThreadMgr src/util/thread.cpp /^ ~ThreadMgr() {$/;" f class:doris::ThreadMgr +~ThreadPool src/util/threadpool.cpp /^ThreadPool::~ThreadPool() {$/;" f class:doris::ThreadPool +~ThreadPoolToken src/util/threadpool.cpp /^ThreadPoolToken::~ThreadPoolToken() {$/;" f class:doris::ThreadPoolToken +~ThriftClientImpl src/util/thrift_client.h /^ virtual ~ThriftClientImpl() {$/;" f class:doris::ThriftClientImpl +~ThriftServer src/util/thrift_server.h /^ ~ThriftServer() { }$/;" f class:doris::ThriftServer +~ThriftServerEventProcessor src/util/thrift_server.cpp /^ virtual ~ThriftServerEventProcessor() {$/;" f class:doris::ThriftServer::ThriftServerEventProcessor +~TinyColumnReader src/olap/rowset/column_reader.cpp /^TinyColumnReader::~TinyColumnReader() {$/;" f class:doris::TinyColumnReader +~TmpFileMgr src/runtime/tmp_file_mgr.h /^ ~TmpFileMgr(){$/;" f class:doris::TmpFileMgr +~TopNNode src/exec/topn_node.cpp /^TopNNode::~TopNNode() {$/;" f class:doris::TopNNode +~TopicListener src/agent/topic_listener.h /^ virtual ~TopicListener(){}$/;" f class:doris::TopicListener +~TopicSubscriber src/agent/topic_subscriber.cpp /^TopicSubscriber::~TopicSubscriber() {$/;" f class:doris::TopicSubscriber +~Translator src/runtime/dpp_sink.cpp /^Translator::~Translator() {$/;" f class:doris::Translator +~TrieNode src/util/path_trie.hpp /^ ~TrieNode() {$/;" f class:doris::PathTrie::TrieNode +~TupleIterator src/runtime/merge_sorter.cpp /^ ~TupleIterator() {}$/;" f class:doris::MergeSorter::TupleSorter::TupleIterator +~TupleIterator src/runtime/spill_sorter.cc /^ ~TupleIterator() {}$/;" f class:doris::SpillSorter::TupleSorter::TupleIterator +~TupleRowIterator src/exec/row_batch_list.h /^ virtual ~TupleRowIterator() { }$/;" f class:doris::RowBatchList::TupleRowIterator +~TupleSorter src/runtime/merge_sorter.cpp /^ ~TupleSorter() {$/;" f class:doris::MergeSorter::TupleSorter +~TupleSorter src/runtime/spill_sorter.cc /^SpillSorter::TupleSorter::~TupleSorter() {$/;" f class:doris::SpillSorter::TupleSorter +~TypeInfoResolver src/olap/types.cpp /^TypeInfoResolver::~TypeInfoResolver() {}$/;" f class:doris::TypeInfoResolver +~TypesTest test/olap/storage_types_test.cpp /^ virtual ~TypesTest() {$/;" f class:doris::TypesTest +~TypesTest test/util/types_test.cpp /^ virtual ~TypesTest() {$/;" f class:doris::TypesTest +~UdaTestHarness2 output/udf/include/uda_test_harness.h /^ ~UdaTestHarness2() {$/;" f class:doris_udf::UdaTestHarness2 +~UdaTestHarness2 src/udf/uda_test_harness.h /^ ~UdaTestHarness2() {$/;" f class:doris_udf::UdaTestHarness2 +~UdaTestHarness3 output/udf/include/uda_test_harness.h /^ ~UdaTestHarness3() {$/;" f class:doris_udf::UdaTestHarness3 +~UdaTestHarness3 src/udf/uda_test_harness.h /^ ~UdaTestHarness3() {$/;" f class:doris_udf::UdaTestHarness3 +~UdaTestHarness4 output/udf/include/uda_test_harness.h /^ ~UdaTestHarness4() { }$/;" f class:doris_udf::UdaTestHarness4 +~UdaTestHarness4 src/udf/uda_test_harness.h /^ ~UdaTestHarness4() { }$/;" f class:doris_udf::UdaTestHarness4 +~UidUtilTest test/util/uid_util_test.cpp /^ virtual ~UidUtilTest() {$/;" f class:doris::UidUtilTest +~UniqueRowsetIdGenerator src/olap/rowset/unique_rowset_id_generator.h /^ ~UniqueRowsetIdGenerator() {}$/;" f class:doris::UniqueRowsetIdGenerator +~UniqueRowsetIdGeneratorTest test/olap/rowset/unique_rowset_id_generator_test.cpp /^ virtual ~UniqueRowsetIdGeneratorTest() {$/;" f class:doris::UniqueRowsetIdGeneratorTest +~UserFunctionCache src/runtime/user_function_cache.cpp /^UserFunctionCache::~UserFunctionCache() {$/;" f class:doris::UserFunctionCache +~UserFunctionCacheEntry src/runtime/user_function_cache.cpp /^UserFunctionCacheEntry::~UserFunctionCacheEntry() {$/;" f class:doris::UserFunctionCacheEntry +~UserFunctionCacheTest test/runtime/user_function_cache_test.cpp /^ virtual ~UserFunctionCacheTest() { }$/;" f class:doris::UserFunctionCacheTest +~UserResourceListener src/agent/user_resource_listener.cpp /^UserResourceListener::~UserResourceListener() {$/;" f class:doris::UserResourceListener +~Utf8CheckTest test/util/utf8_check_test.cpp /^ virtual ~Utf8CheckTest() { }$/;" f class:doris::Utf8CheckTest +~ValueDeleter src/gutil/stl_util.h /^ ~ValueDeleter() {$/;" f class:ValueDeleter +~VarStringColumnReader src/olap/rowset/column_reader.h /^ virtual ~VarStringColumnReader() {$/;" f class:doris::VarStringColumnReader +~VarStringColumnWriter src/olap/rowset/column_writer.cpp /^VarStringColumnWriter::~VarStringColumnWriter() {$/;" f class:doris::VarStringColumnWriter +~VectorizedRowBatch src/runtime/vectorized_row_batch.h /^ ~VectorizedRowBatch() {$/;" f class:doris::VectorizedRowBatch +~WebPageHandler src/http/web_page_handler.h /^ virtual ~WebPageHandler() {$/;" f class:doris::WebPageHandler +~WktParseTest test/geo/wkt_parse_test.cpp /^ virtual ~WktParseTest() { }$/;" f class:doris::WktParseTest +~WrapperField src/olap/wrapper_field.h /^ virtual ~WrapperField() {$/;" f class:doris::WrapperField +~WritableBlock src/olap/fs/block_manager.h /^ virtual ~WritableBlock() {}$/;" f class:doris::fs::WritableBlock +~WritableFile src/env/env.h /^ virtual ~WritableFile() { }$/;" f class:doris::WritableFile +~WriteLock src/util/mutex.h /^ ~WriteLock() { $/;" f class:doris::WriteLock +~WriteRange src/runtime/disk_io_mgr.h /^ ~WriteRange() {}$/;" f class:doris::DiskIoMgr::WriteRange +~ZipFile src/util/zip_util.h /^ ~ZipFile() {$/;" f class:doris::ZipFile +~ZlibBlockCompression src/util/block_compression.cpp /^ ~ZlibBlockCompression() { }$/;" f class:doris::ZlibBlockCompression +~faststring src/util/faststring.h /^ ~faststring() {$/;" f class:doris::faststring +~gscoped_array src/gutil/gscoped_ptr.h /^ ~gscoped_array() {$/;" f class:gscoped_array +~gscoped_ptr_impl src/gutil/gscoped_ptr.h /^ ~gscoped_ptr_impl() {$/;" f class:doris::internal::gscoped_ptr_impl +~gscoped_ptr_malloc src/gutil/gscoped_ptr.h /^ ~gscoped_ptr_malloc() {$/;" f class:gscoped_ptr_malloc +~scoped_refptr src/gutil/ref_counted.h /^ ~scoped_refptr() {$/;" f class:scoped_refptr diff --git a/be/test/olap/delete_handler_test.cpp.bak b/be/test/olap/delete_handler_test.cpp.bak new file mode 100644 index 000000000000000..eb0d2d7aa2fa7ad --- /dev/null +++ b/be/test/olap/delete_handler_test.cpp.bak @@ -0,0 +1,958 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include + +#include +#include +#include +#include + +#include +#include +#include + +#include "olap/delete_handler.h" +#include "olap/olap_define.h" +#include "olap/storage_engine.h" +#include "olap/push_handler.h" +#include "olap/utils.h" +#include "olap/options.h" +#include "util/logging.h" +#include "util/file_utils.h" + +using namespace std; +using namespace doris; +using namespace boost::assign; +using google::protobuf::RepeatedPtrField; + +namespace doris { + +static const uint32_t MAX_PATH_LEN = 1024; +static StorageEngine* k_engine = nullptr; + +void set_up() { + char buffer[MAX_PATH_LEN]; + getcwd(buffer, MAX_PATH_LEN); + config::storage_root_path = string(buffer) + "/data_test"; + FileUtils::remove_all(config::storage_root_path); + FileUtils::remove_all(string(getenv("DORIS_HOME")) + UNUSED_PREFIX); + FileUtils::create_dir(config::storage_root_path); + std::vector paths; + paths.emplace_back(config::storage_root_path, -1); + config::min_file_descriptor_number = 1000; + config::tablet_map_shard_size = 1; + + doris::EngineOptions options; + options.store_paths = paths; + doris::StorageEngine::open(options, &k_engine); +} + +void tear_down() { + char buffer[MAX_PATH_LEN]; + getcwd(buffer, MAX_PATH_LEN); + config::storage_root_path = string(buffer) + "/data_test"; + FileUtils::remove_all(config::storage_root_path); + FileUtils::remove_all(string(getenv("DORIS_HOME")) + UNUSED_PREFIX); +} + +void set_default_create_tablet_request(TCreateTabletReq* request) { + request->tablet_id = 10003; + request->__set_version(1); + request->__set_version_hash(0); + request->tablet_schema.schema_hash = 270068375; + request->tablet_schema.short_key_column_count = 2; + request->tablet_schema.keys_type = TKeysType::AGG_KEYS; + request->tablet_schema.storage_type = TStorageType::COLUMN; + + TColumn k1; + k1.column_name = "k1"; + k1.__set_is_key(true); + k1.column_type.type = TPrimitiveType::TINYINT; + request->tablet_schema.columns.push_back(k1); + + TColumn k2; + k2.column_name = "k2"; + k2.__set_is_key(true); + k2.column_type.type = TPrimitiveType::SMALLINT; + request->tablet_schema.columns.push_back(k2); + + TColumn k3; + k3.column_name = "k3"; + k3.__set_is_key(true); + k3.column_type.type = TPrimitiveType::INT; + request->tablet_schema.columns.push_back(k3); + + TColumn k4; + k4.column_name = "k4"; + k4.__set_is_key(true); + k4.column_type.type = TPrimitiveType::BIGINT; + request->tablet_schema.columns.push_back(k4); + + TColumn k5; + k5.column_name = "k5"; + k5.__set_is_key(true); + k5.column_type.type = TPrimitiveType::LARGEINT; + request->tablet_schema.columns.push_back(k5); + + TColumn k9; + k9.column_name = "k9"; + k9.__set_is_key(true); + k9.column_type.type = TPrimitiveType::DECIMAL; + k9.column_type.__set_precision(6); + k9.column_type.__set_scale(3); + request->tablet_schema.columns.push_back(k9); + + TColumn k10; + k10.column_name = "k10"; + k10.__set_is_key(true); + k10.column_type.type = TPrimitiveType::DATE; + request->tablet_schema.columns.push_back(k10); + + TColumn k11; + k11.column_name = "k11"; + k11.__set_is_key(true); + k11.column_type.type = TPrimitiveType::DATETIME; + request->tablet_schema.columns.push_back(k11); + + TColumn k12; + k12.column_name = "k12"; + k12.__set_is_key(true); + k12.column_type.__set_len(64); + k12.column_type.type = TPrimitiveType::CHAR; + request->tablet_schema.columns.push_back(k12); + + TColumn k13; + k13.column_name = "k13"; + k13.__set_is_key(true); + k13.column_type.__set_len(64); + k13.column_type.type = TPrimitiveType::VARCHAR; + request->tablet_schema.columns.push_back(k13); + + TColumn v; + v.column_name = "v"; + v.__set_is_key(false); + v.column_type.type = TPrimitiveType::BIGINT; + v.__set_aggregation_type(TAggregationType::SUM); + request->tablet_schema.columns.push_back(v); +} + +void set_default_push_request(TPushReq* request) { + request->tablet_id = 10003; + request->schema_hash = 270068375; + request->timeout = 86400; + request->push_type = TPushType::LOAD; +} + +class TestDeleteConditionHandler : public testing::Test { +protected: + void SetUp() { + // Create local data dir for StorageEngine. + char buffer[MAX_PATH_LEN]; + getcwd(buffer, MAX_PATH_LEN); + config::storage_root_path = string(buffer) + "/data_delete_condition"; + FileUtils::remove_all(config::storage_root_path); + ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok()); + + // 1. Prepare for query split key. + // create base tablet + OLAPStatus res = OLAP_SUCCESS; + set_default_create_tablet_request(&_create_tablet); + res = k_engine->create_tablet(_create_tablet); + ASSERT_EQ(OLAP_SUCCESS, res); + tablet = k_engine->tablet_manager()->get_tablet( + _create_tablet.tablet_id, _create_tablet.tablet_schema.schema_hash); + ASSERT_TRUE(tablet.get() != NULL); + _tablet_path = tablet->tablet_path(); + } + + void TearDown() { + // Remove all dir. + tablet.reset(); + StorageEngine::instance()->tablet_manager()->drop_tablet( + _create_tablet.tablet_id, _create_tablet.tablet_schema.schema_hash); + while (0 == access(_tablet_path.c_str(), F_OK)) { + sleep(1); + } + ASSERT_TRUE(FileUtils::remove_all(config::storage_root_path).ok()); + } + + std::string _tablet_path; + TabletSharedPtr tablet; + TCreateTabletReq _create_tablet; + DeleteConditionHandler _delete_condition_handler; +}; + +TEST_F(TestDeleteConditionHandler, StoreCondSucceed) { + OLAPStatus success_res; + std::vector conditions; + + TCondition condition; + condition.column_name = "k1"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("1"); + conditions.push_back(condition); + + condition.column_name = "k2"; + condition.condition_op = ">"; + condition.condition_values.clear(); + condition.condition_values.push_back("3"); + conditions.push_back(condition); + + condition.column_name = "k2"; + condition.condition_op = "<="; + condition.condition_values.clear(); + condition.condition_values.push_back("5"); + conditions.push_back(condition); + + DeletePredicatePB del_pred; + success_res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred); + ASSERT_EQ(OLAP_SUCCESS, success_res); + + // 验证存储在header中的过滤条件正确 + ASSERT_EQ(size_t(3), del_pred.sub_predicates_size()); + EXPECT_STREQ("k1=1", del_pred.sub_predicates(0).c_str()); + EXPECT_STREQ("k2>>3", del_pred.sub_predicates(1).c_str()); + EXPECT_STREQ("k2<=5", del_pred.sub_predicates(2).c_str()); +} + +// 检测参数不正确的情况,包括:空的过滤条件字符串 +TEST_F(TestDeleteConditionHandler, StoreCondInvalidParameters) { + // 空的过滤条件 + std::vector conditions; + DeletePredicatePB del_pred; + OLAPStatus failed_res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred);; + ASSERT_EQ(OLAP_ERR_DELETE_INVALID_PARAMETERS, failed_res); +} + +// 检测过滤条件中指定的列不存在,或者列不符合要求 +TEST_F(TestDeleteConditionHandler, StoreCondNonexistentColumn) { + // 'k100'是一个不存在的列 + std::vector conditions; + TCondition condition; + condition.column_name = "k100"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("2"); + conditions.push_back(condition); + DeletePredicatePB del_pred; + OLAPStatus failed_res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred);; + ASSERT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, failed_res); + + // 'v'是value列 + conditions.clear(); + condition.column_name = "v"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("5"); + conditions.push_back(condition); + + failed_res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred);; + ASSERT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, failed_res); +} + +// 测试删除条件值不符合类型要求 +class TestDeleteConditionHandler2 : public testing::Test { +protected: + void SetUp() { + // Create local data dir for StorageEngine. + char buffer[MAX_PATH_LEN]; + getcwd(buffer, MAX_PATH_LEN); + config::storage_root_path = string(buffer) + "/data_delete_condition"; + FileUtils::remove_all(config::storage_root_path); + ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok()); + + // 1. Prepare for query split key. + // create base tablet + OLAPStatus res = OLAP_SUCCESS; + set_default_create_tablet_request(&_create_tablet); + res = k_engine->create_tablet(_create_tablet); + ASSERT_EQ(OLAP_SUCCESS, res); + tablet = k_engine->tablet_manager()->get_tablet( + _create_tablet.tablet_id, _create_tablet.tablet_schema.schema_hash); + ASSERT_TRUE(tablet.get() != NULL); + _tablet_path = tablet->tablet_path(); + } + + void TearDown() { + // Remove all dir. + tablet.reset(); + StorageEngine::instance()->tablet_manager()->drop_tablet( + _create_tablet.tablet_id, _create_tablet.tablet_schema.schema_hash); + while (0 == access(_tablet_path.c_str(), F_OK)) { + sleep(1); + } + ASSERT_TRUE(FileUtils::remove_all(config::storage_root_path).ok()); + } + + std::string _tablet_path; + TabletSharedPtr tablet; + TCreateTabletReq _create_tablet; + DeleteConditionHandler _delete_condition_handler; +}; + +TEST_F(TestDeleteConditionHandler2, ValidConditionValue) { + OLAPStatus res; + DeleteConditionHandler cond_handler; + std::vector conditions; + + // 测试数据中, k1,k2,k3,k4类型分别为int8, int16, int32, int64 + TCondition condition; + condition.column_name = "k1"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("-1"); + conditions.push_back(condition); + + condition.column_name = "k2"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("-1"); + conditions.push_back(condition); + + condition.column_name = "k3"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("-1"); + conditions.push_back(condition); + + condition.column_name = "k4"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("-1"); + conditions.push_back(condition); + + DeletePredicatePB del_pred; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred); + ASSERT_EQ(OLAP_SUCCESS, res); + + // k5类型为int128 + conditions.clear(); + condition.column_name = "k5"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("1"); + conditions.push_back(condition); + + DeletePredicatePB del_pred_2; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_2); + ASSERT_EQ(OLAP_SUCCESS, res); + + // k9类型为decimal, precision=6, frac=3 + conditions.clear(); + condition.column_name = "k9"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("2.3"); + conditions.push_back(condition); + + DeletePredicatePB del_pred_3; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_3); + ASSERT_EQ(OLAP_SUCCESS, res); + + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("2"); + DeletePredicatePB del_pred_4; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_4); + ASSERT_EQ(OLAP_SUCCESS, res); + + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("-2"); + DeletePredicatePB del_pred_5; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_5); + ASSERT_EQ(OLAP_SUCCESS, res); + + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("-2.3"); + DeletePredicatePB del_pred_6; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_6); + ASSERT_EQ(OLAP_SUCCESS, res); + + // k10,k11类型分别为date, datetime + conditions.clear(); + condition.column_name = "k10"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("2014-01-01"); + conditions.push_back(condition); + + condition.column_name = "k10"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("2014-01-01 00:00:00"); + conditions.push_back(condition); + + DeletePredicatePB del_pred_7; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_7); + ASSERT_EQ(OLAP_SUCCESS, res); + + // k12,k13类型分别为string(64), varchar(64) + conditions.clear(); + condition.column_name = "k12"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("YWFh"); + conditions.push_back(condition); + + condition.column_name = "k13"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("YWFhYQ=="); + conditions.push_back(condition); + + DeletePredicatePB del_pred_8; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_8); + ASSERT_EQ(OLAP_SUCCESS, res); +} + +TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) { + OLAPStatus res; + DeleteConditionHandler cond_handler; + std::vector conditions; + + // 测试k1的值越上界,k1类型为int8 + TCondition condition; + condition.column_name = "k1"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("1000"); + conditions.push_back(condition); + + DeletePredicatePB del_pred_1; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_1); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k1的值越下界,k1类型为int8 + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("-1000"); + DeletePredicatePB del_pred_2; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_2); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k2的值越上界,k2类型为int16 + conditions[0].condition_values.clear(); + conditions[0].column_name = "k2"; + conditions[0].condition_values.push_back("32768"); + DeletePredicatePB del_pred_3; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_3); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k2的值越下界,k2类型为int16 + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("-32769"); + DeletePredicatePB del_pred_4; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_4); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k3的值越上界,k3类型为int32 + conditions[0].condition_values.clear(); + conditions[0].column_name = "k3"; + conditions[0].condition_values.push_back("2147483648"); + DeletePredicatePB del_pred_5; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_5); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k3的值越下界,k3类型为int32 + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("-2147483649"); + DeletePredicatePB del_pred_6; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_6); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k4的值越上界,k2类型为int64 + conditions[0].condition_values.clear(); + conditions[0].column_name = "k4"; + conditions[0].condition_values.push_back("9223372036854775808"); + DeletePredicatePB del_pred_7; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_7); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k4的值越下界,k1类型为int64 + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("-9223372036854775809"); + DeletePredicatePB del_pred_8; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_8); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k5的值越上界,k5类型为int128 + conditions[0].condition_values.clear(); + conditions[0].column_name = "k5"; + conditions[0].condition_values.push_back("170141183460469231731687303715884105728"); + DeletePredicatePB del_pred_9; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_9); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k5的值越下界,k5类型为int128 + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("-170141183460469231731687303715884105729"); + DeletePredicatePB del_pred_10; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_10); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k9整数部分长度过长,k9类型为decimal, precision=6, frac=3 + conditions[0].condition_values.clear(); + conditions[0].column_name = "k9"; + conditions[0].condition_values.push_back("12347876.5"); + DeletePredicatePB del_pred_11; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_11); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k9小数部分长度过长,k9类型为decimal, precision=6, frac=3 + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("1.2345678"); + DeletePredicatePB del_pred_12; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_12); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k9没有小数部分,但包含小数点 + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("1."); + DeletePredicatePB del_pred_13; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_13); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k10类型的过滤值不符合对应格式,k10为date + conditions[0].condition_values.clear(); + conditions[0].column_name = "k10"; + conditions[0].condition_values.push_back("20130101"); + DeletePredicatePB del_pred_14; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_14); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("2013-64-01"); + DeletePredicatePB del_pred_15; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_15); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("2013-01-40"); + DeletePredicatePB del_pred_16; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_16); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k11类型的过滤值不符合对应格式,k11为datetime + conditions[0].condition_values.clear(); + conditions[0].column_name = "k11"; + conditions[0].condition_values.push_back("20130101 00:00:00"); + DeletePredicatePB del_pred_17; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_17); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("2013-64-01 00:00:00"); + DeletePredicatePB del_pred_18; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_18); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("2013-01-40 00:00:00"); + DeletePredicatePB del_pred_19; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_19); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("2013-01-01 24:00:00"); + DeletePredicatePB del_pred_20; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_20); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("2013-01-01 00:60:00"); + DeletePredicatePB del_pred_21; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_21); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + conditions[0].condition_values.clear(); + conditions[0].condition_values.push_back("2013-01-01 00:00:60"); + DeletePredicatePB del_pred_22; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_22); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + // 测试k12和k13类型的过滤值过长,k12,k13类型分别为string(64), varchar(64) + conditions[0].condition_values.clear(); + conditions[0].column_name = "k12"; + conditions[0].condition_values.push_back("YWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYW" + "FhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYW" + "FhYWFhYWFhYWFhYWFhYWFhYWFhYWE=;k13=YWFhYQ=="); + DeletePredicatePB del_pred_23; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_23); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); + + conditions[0].condition_values.clear(); + conditions[0].column_name = "k13"; + conditions[0].condition_values.push_back("YWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYW" + "FhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYW" + "FhYWFhYWFhYWFhYWFhYWFhYWFhYWE=;k13=YWFhYQ=="); + DeletePredicatePB del_pred_24; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_24); + EXPECT_EQ(OLAP_ERR_DELETE_INVALID_CONDITION, res); +} + +class TestDeleteHandler : public testing::Test { +protected: + void SetUp() { + // Create local data dir for StorageEngine. + char buffer[MAX_PATH_LEN]; + getcwd(buffer, MAX_PATH_LEN); + config::storage_root_path = string(buffer) + "/data_delete_condition"; + FileUtils::remove_all(config::storage_root_path); + ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok()); + + // 1. Prepare for query split key. + // create base tablet + OLAPStatus res = OLAP_SUCCESS; + set_default_create_tablet_request(&_create_tablet); + res = k_engine->create_tablet(_create_tablet); + ASSERT_EQ(OLAP_SUCCESS, res); + tablet = k_engine->tablet_manager()->get_tablet( + _create_tablet.tablet_id, _create_tablet.tablet_schema.schema_hash); + ASSERT_TRUE(tablet != nullptr); + _tablet_path = tablet->tablet_path(); + + _data_row_cursor.init(tablet->tablet_schema()); + _data_row_cursor.allocate_memory_for_string_type(tablet->tablet_schema()); + } + + void TearDown() { + // Remove all dir. + tablet.reset(); + _delete_handler.finalize(); + StorageEngine::instance()->tablet_manager()->drop_tablet( + _create_tablet.tablet_id, _create_tablet.tablet_schema.schema_hash); + while (0 == access(_tablet_path.c_str(), F_OK)) { + sleep(1); + } + ASSERT_TRUE(FileUtils::remove_all(config::storage_root_path).ok()); + } + + std::string _tablet_path; + RowCursor _data_row_cursor; + TabletSharedPtr tablet; + TCreateTabletReq _create_tablet; + DeleteHandler _delete_handler; + DeleteConditionHandler _delete_condition_handler; +}; + +TEST_F(TestDeleteHandler, InitSuccess) { + OLAPStatus res; + std::vector conditions; + DeleteConditionHandler delete_condition_handler; + + // 往头文件中添加过滤条件 + TCondition condition; + condition.column_name = "k1"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("1"); + conditions.push_back(condition); + + condition.column_name = "k2"; + condition.condition_op = ">"; + condition.condition_values.clear(); + condition.condition_values.push_back("3"); + conditions.push_back(condition); + + condition.column_name = "k2"; + condition.condition_op = "<="; + condition.condition_values.clear(); + condition.condition_values.push_back("5"); + conditions.push_back(condition); + + DeletePredicatePB del_pred; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred); + ASSERT_EQ(OLAP_SUCCESS, res); + res = tablet->add_delete_predicate(del_pred, 1); + ASSERT_EQ(OLAP_SUCCESS, res); + + conditions.clear(); + condition.column_name = "k1"; + condition.condition_op = "!="; + condition.condition_values.clear(); + condition.condition_values.push_back("3"); + conditions.push_back(condition); + + DeletePredicatePB del_pred_2; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_2); + ASSERT_EQ(OLAP_SUCCESS, res); + res = tablet->add_delete_predicate(del_pred_2, 2); + ASSERT_EQ(OLAP_SUCCESS, res); + + conditions.clear(); + condition.column_name = "k2"; + condition.condition_op = ">="; + condition.condition_values.clear(); + condition.condition_values.push_back("1"); + conditions.push_back(condition); + + DeletePredicatePB del_pred_3; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_3); + ASSERT_EQ(OLAP_SUCCESS, res); + res = tablet->add_delete_predicate(del_pred_3, 3); + ASSERT_EQ(OLAP_SUCCESS, res); + + conditions.clear(); + condition.column_name = "k2"; + condition.condition_op = "!="; + condition.condition_values.clear(); + condition.condition_values.push_back("3"); + conditions.push_back(condition); + + DeletePredicatePB del_pred_4; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_4); + ASSERT_EQ(OLAP_SUCCESS, res); + res = tablet->add_delete_predicate(del_pred_4, 4); + ASSERT_EQ(OLAP_SUCCESS, res); + + // 从header文件中取出版本号小于等于7的过滤条件 + 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(); + EXPECT_EQ(4, conds_version.size()); + sort(conds_version.begin(), conds_version.end()); + EXPECT_EQ(1, conds_version[0]); + EXPECT_EQ(2, conds_version[1]); + EXPECT_EQ(3, conds_version[2]); + EXPECT_EQ(4, conds_version[3]); + + _delete_handler.finalize(); +} + +// 测试一个过滤条件包含的子条件之间是and关系, +// 即只有满足一条过滤条件包含的所有子条件,这条数据才会被过滤 +TEST_F(TestDeleteHandler, FilterDataSubconditions) { + OLAPStatus res; + DeleteConditionHandler cond_handler; + std::vector conditions; + + // 往Header中添加过滤条件 + // 过滤条件1 + TCondition condition; + condition.column_name = "k1"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("1"); + conditions.push_back(condition); + + condition.column_name = "k2"; + condition.condition_op = "!="; + condition.condition_values.clear(); + condition.condition_values.push_back("4"); + conditions.push_back(condition); + + DeletePredicatePB del_pred; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred); + ASSERT_EQ(OLAP_SUCCESS, res); + res = tablet->add_delete_predicate(del_pred, 1); + + // 指定版本号为10以载入Header中的所有过滤条件(在这个case中,只有过滤条件1) + res = _delete_handler.init(tablet->tablet_schema(), tablet->delete_predicates(), 4); + ASSERT_EQ(OLAP_SUCCESS, res); + ASSERT_EQ(1, _delete_handler.conditions_num()); + + // 构造一行测试数据 + vector data_str; + data_str.push_back("1"); + data_str.push_back("6"); + data_str.push_back("8"); + data_str.push_back("-1"); + data_str.push_back("16"); + data_str.push_back("1.2"); + data_str.push_back("2014-01-01"); + data_str.push_back("2014-01-01 00:00:00"); + data_str.push_back("YWFH"); + data_str.push_back("YWFH=="); + data_str.push_back("1"); + OlapTuple tuple1(data_str); + res = _data_row_cursor.from_tuple(tuple1); + ASSERT_EQ(OLAP_SUCCESS, res); + ASSERT_TRUE(_delete_handler.is_filter_data(1, _data_row_cursor)); + + // 构造一行测试数据 + data_str[1] = "4"; + OlapTuple tuple2(data_str); + res = _data_row_cursor.from_tuple(tuple2); + ASSERT_EQ(OLAP_SUCCESS, res); + // 不满足子条件:k2!=4 + ASSERT_FALSE(_delete_handler.is_filter_data(1, _data_row_cursor)); + + _delete_handler.finalize(); +} + +// 测试多个过滤条件之间是or关系, +// 即如果存在多个过滤条件,会一次检查数据是否符合这些过滤条件;只要有一个过滤条件符合,则过滤数据 +TEST_F(TestDeleteHandler, FilterDataConditions) { + OLAPStatus res; + DeleteConditionHandler cond_handler; + std::vector conditions; + + // 往Header中添加过滤条件 + // 过滤条件1 + TCondition condition; + condition.column_name = "k1"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("1"); + conditions.push_back(condition); + + condition.column_name = "k2"; + condition.condition_op = "!="; + condition.condition_values.clear(); + condition.condition_values.push_back("4"); + conditions.push_back(condition); + + DeletePredicatePB del_pred; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred); + ASSERT_EQ(OLAP_SUCCESS, res); + res = tablet->add_delete_predicate(del_pred, 1); + + // 过滤条件2 + conditions.clear(); + condition.column_name = "k1"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("3"); + conditions.push_back(condition); + + DeletePredicatePB del_pred_2; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_2); + ASSERT_EQ(OLAP_SUCCESS, res); + res = tablet->add_delete_predicate(del_pred_2, 2); + + // 过滤条件3 + conditions.clear(); + condition.column_name = "k2"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("5"); + conditions.push_back(condition); + + DeletePredicatePB del_pred_3; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_3); + ASSERT_EQ(OLAP_SUCCESS, res); + res = tablet->add_delete_predicate(del_pred_3, 3); + + // 指定版本号为4以载入meta中的所有过滤条件(在这个case中,只有过滤条件1) + res = _delete_handler.init(tablet->tablet_schema(), tablet->delete_predicates(), 4); + ASSERT_EQ(OLAP_SUCCESS, res); + ASSERT_EQ(3, _delete_handler.conditions_num()); + + vector data_str; + data_str.push_back("4"); + data_str.push_back("5"); + data_str.push_back("8"); + data_str.push_back("-1"); + data_str.push_back("16"); + data_str.push_back("1.2"); + data_str.push_back("2014-01-01"); + data_str.push_back("2014-01-01 00:00:00"); + data_str.push_back("YWFH"); + data_str.push_back("YWFH=="); + data_str.push_back("1"); + OlapTuple tuple(data_str); + res = _data_row_cursor.from_tuple(tuple); + ASSERT_EQ(OLAP_SUCCESS, res); + // 这行数据会因为过滤条件3而被过滤 + ASSERT_TRUE(_delete_handler.is_filter_data(3, _data_row_cursor)); + + _delete_handler.finalize(); +} + +// 测试在过滤时,版本号小于数据版本的过滤条件将不起作用 +TEST_F(TestDeleteHandler, FilterDataVersion) { + OLAPStatus res; + DeleteConditionHandler cond_handler; + std::vector conditions; + + // 往Header中添加过滤条件 + // 过滤条件1 + TCondition condition; + condition.column_name = "k1"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("1"); + conditions.push_back(condition); + + condition.column_name = "k2"; + condition.condition_op = "!="; + condition.condition_values.clear(); + condition.condition_values.push_back("4"); + conditions.push_back(condition); + + DeletePredicatePB del_pred; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred); + ASSERT_EQ(OLAP_SUCCESS, res); + res = tablet->add_delete_predicate(del_pred, 3); + + // 过滤条件2 + conditions.clear(); + condition.column_name = "k1"; + condition.condition_op = "="; + condition.condition_values.clear(); + condition.condition_values.push_back("3"); + conditions.push_back(condition); + + DeletePredicatePB del_pred_2; + res = _delete_condition_handler.generate_delete_predicate(tablet->tablet_schema(), conditions, &del_pred_2); + ASSERT_EQ(OLAP_SUCCESS, res); + res = tablet->add_delete_predicate(del_pred_2, 4); + + // 指定版本号为4以载入meta中的所有过滤条件(过滤条件1,过滤条件2) + res = _delete_handler.init(tablet->tablet_schema(), tablet->delete_predicates(), 4); + ASSERT_EQ(OLAP_SUCCESS, res); + ASSERT_EQ(2, _delete_handler.conditions_num()); + + // 构造一行测试数据 + vector data_str; + data_str.push_back("1"); + data_str.push_back("6"); + data_str.push_back("8"); + data_str.push_back("-1"); + data_str.push_back("16"); + data_str.push_back("1.2"); + data_str.push_back("2014-01-01"); + data_str.push_back("2014-01-01 00:00:00"); + data_str.push_back("YWFH"); + data_str.push_back("YWFH=="); + data_str.push_back("1"); + OlapTuple tuple(data_str); + res = _data_row_cursor.from_tuple(tuple); + ASSERT_EQ(OLAP_SUCCESS, res); + // 如果数据版本小于3,则过滤条件1生效,这条数据被过滤 + ASSERT_TRUE(_delete_handler.is_filter_data(2, _data_row_cursor)); + // 如果数据版本大于3,则过滤条件1会被跳过 + ASSERT_FALSE(_delete_handler.is_filter_data(4, _data_row_cursor)); + + _delete_handler.finalize(); +} + +} // namespace doris + +int main(int argc, char** argv) { + doris::init_glog("be-test"); + int ret = doris::OLAP_SUCCESS; + testing::InitGoogleTest(&argc, argv); + + doris::set_up(); + ret = RUN_ALL_TESTS(); + doris::tear_down(); + + google::protobuf::ShutdownProtobufLibrary(); + return ret; +} diff --git a/be/test/olap/delta_writer_test.cpp b/be/test/olap/delta_writer_test.cpp index 1a6418e0cb73548..d7014428b228649 100644 --- a/be/test/olap/delta_writer_test.cpp +++ b/be/test/olap/delta_writer_test.cpp @@ -364,7 +364,7 @@ TEST_F(TestDeltaWriter, open) { WriteRequest write_req = {10003, 270068375, WriteType::LOAD, 20001, 30001, load_id, false, tuple_desc}; DeltaWriter* delta_writer = nullptr; - DeltaWriter::open(&write_req, k_mem_tracker, &delta_writer); + DeltaWriter::open(write_req, k_mem_tracker, &delta_writer); ASSERT_NE(delta_writer, nullptr); res = delta_writer->close(); ASSERT_EQ(OLAP_SUCCESS, res); @@ -399,7 +399,7 @@ TEST_F(TestDeltaWriter, write) { 20002, 30002, load_id, false, tuple_desc, &(tuple_desc->slots())}; DeltaWriter* delta_writer = nullptr; - DeltaWriter::open(&write_req, k_mem_tracker, &delta_writer); + DeltaWriter::open(write_req, k_mem_tracker, &delta_writer); ASSERT_NE(delta_writer, nullptr); auto tracker = std::make_shared(); @@ -518,7 +518,7 @@ TEST_F(TestDeltaWriter, sequence_col) { 20003, 30003, load_id, false, tuple_desc, &(tuple_desc->slots())}; DeltaWriter* delta_writer = nullptr; - DeltaWriter::open(&write_req, k_mem_tracker, &delta_writer); + DeltaWriter::open(write_req, k_mem_tracker, &delta_writer); ASSERT_NE(delta_writer, nullptr); MemTracker tracker; diff --git a/be/test/olap/row_block_test.cpp b/be/test/olap/row_block_test.cpp index 6136892be159b51..c1e7dd7f369dafa 100644 --- a/be/test/olap/row_block_test.cpp +++ b/be/test/olap/row_block_test.cpp @@ -89,8 +89,7 @@ TEST_F(TestRowBlock, init) { RowBlockInfo block_info; block_info.row_num = 1024; block_info.null_supported = true; - auto res = block.init(block_info); - ASSERT_EQ(OLAP_SUCCESS, res); + block.init(block_info); ASSERT_EQ(9 + 17 + 17, block._mem_row_bytes); } { @@ -99,8 +98,7 @@ TEST_F(TestRowBlock, init) { RowBlockInfo block_info; block_info.row_num = 1024; block_info.null_supported = false; - auto res = block.init(block_info); - ASSERT_EQ(OLAP_SUCCESS, res); + block.init(block_info); ASSERT_EQ(9 + 17 + 17, block._mem_row_bytes); } { @@ -109,8 +107,7 @@ TEST_F(TestRowBlock, init) { block_info.row_num = 1024; block_info.null_supported = true; block_info.column_ids.push_back(1); - auto res = block.init(block_info); - ASSERT_EQ(OLAP_SUCCESS, res); + block.init(block_info); // null + sizeof(Slice) ASSERT_EQ(17, block._mem_row_bytes); ASSERT_EQ(std::numeric_limits::max(), block._field_offset_in_memory[0]); @@ -126,8 +123,7 @@ TEST_F(TestRowBlock, write_and_read) { RowBlockInfo block_info; block_info.row_num = 1024; block_info.null_supported = true; - auto res = block.init(block_info); - ASSERT_EQ(OLAP_SUCCESS, res); + block.init(block_info); RowCursor row; row.init(tablet_schema); @@ -168,8 +164,7 @@ TEST_F(TestRowBlock, write_and_read_without_nullbyte) { RowBlockInfo block_info; block_info.row_num = 1024; block_info.null_supported = false; - auto res = block.init(block_info); - ASSERT_EQ(OLAP_SUCCESS, res); + block.init(block_info); RowCursor row; row.init(tablet_schema); @@ -210,8 +205,7 @@ TEST_F(TestRowBlock, compress_failed) { RowBlockInfo block_info; block_info.row_num = 1024; block_info.null_supported = true; - auto res = block.init(block_info); - ASSERT_EQ(OLAP_SUCCESS, res); + block.init(block_info); RowCursor row; row.init(tablet_schema); @@ -249,8 +243,7 @@ TEST_F(TestRowBlock, decompress_failed) { RowBlockInfo block_info; block_info.row_num = 1024; block_info.null_supported = true; - auto res = block.init(block_info); - ASSERT_EQ(OLAP_SUCCESS, res); + block.init(block_info); RowCursor row; row.init(tablet_schema); @@ -288,8 +281,7 @@ TEST_F(TestRowBlock, clear) { RowBlockInfo block_info; block_info.row_num = 1024; block_info.null_supported = true; - auto res = block.init(block_info); - ASSERT_EQ(OLAP_SUCCESS, res); + block.init(block_info); block.finalize(5); ASSERT_EQ(5, block.row_num()); @@ -305,8 +297,7 @@ TEST_F(TestRowBlock, pos_limit) { RowBlockInfo block_info; block_info.row_num = 1024; block_info.null_supported = true; - auto res = block.init(block_info); - ASSERT_EQ(OLAP_SUCCESS, res); + block.init(block_info); // assert init value ASSERT_EQ(0, block.pos()); diff --git a/be/test/olap/row_block_v2_test.cpp b/be/test/olap/row_block_v2_test.cpp index 8c377b48f27b11d..89c67c5c1890cbb 100644 --- a/be/test/olap/row_block_v2_test.cpp +++ b/be/test/olap/row_block_v2_test.cpp @@ -90,8 +90,7 @@ TEST_F(TestRowBlockV2, test_convert) { RowBlockInfo block_info; block_info.row_num = 1024; block_info.null_supported = true; - auto res = output_block.init(block_info); - ASSERT_EQ(OLAP_SUCCESS, res); + output_block.init(block_info); auto tracker = std::make_shared(); MemPool pool(tracker.get()); for (int i = 0; i < input_block.capacity(); ++i) { diff --git a/be/test/runtime/external_scan_context_mgr_test.cpp b/be/test/runtime/external_scan_context_mgr_test.cpp index 9d6ed82783b9cc5..339dec7bdbd60ac 100644 --- a/be/test/runtime/external_scan_context_mgr_test.cpp +++ b/be/test/runtime/external_scan_context_mgr_test.cpp @@ -53,21 +53,19 @@ class ExternalScanContextMgrTest : public testing::Test { TEST_F(ExternalScanContextMgrTest, create_normal) { std::shared_ptr context; ExternalScanContextMgr context_mgr(&_exec_env); - Status st = context_mgr.create_scan_context(&context); - ASSERT_TRUE(st.ok()); + context_mgr.create_scan_context(&context); ASSERT_TRUE(context != nullptr); } TEST_F(ExternalScanContextMgrTest, get_normal) { std::shared_ptr context; ExternalScanContextMgr context_mgr(&_exec_env); - Status st = context_mgr.create_scan_context(&context); - ASSERT_TRUE(st.ok()); + context_mgr.create_scan_context(&context); ASSERT_TRUE(context != nullptr); std::string context_id = context->context_id; std::shared_ptr result; - st = context_mgr.get_scan_context(context_id, &result); + Status st = context_mgr.get_scan_context(context_id, &result); ASSERT_TRUE(st.ok()); ASSERT_TRUE(context != nullptr); } @@ -84,16 +82,14 @@ TEST_F(ExternalScanContextMgrTest, get_abnormal) { TEST_F(ExternalScanContextMgrTest, clear_context) { std::shared_ptr context; ExternalScanContextMgr context_mgr(&_exec_env); - Status st = context_mgr.create_scan_context(&context); - ASSERT_TRUE(st.ok()); + context_mgr.create_scan_context(&context); ASSERT_TRUE(context != nullptr); std::string context_id = context->context_id; - st = context_mgr.clear_scan_context(context_id); - ASSERT_TRUE(st.ok()); + context_mgr.clear_scan_context(context_id); std::shared_ptr result; - st = context_mgr.get_scan_context(context_id, &result); + Status st = context_mgr.get_scan_context(context_id, &result); ASSERT_TRUE(!st.ok()); ASSERT_TRUE(result == nullptr); } diff --git a/be/test/runtime/fragment_mgr_test.cpp b/be/test/runtime/fragment_mgr_test.cpp index a8b0ce7519227fa..fd6d3ae8f8520ec 100644 --- a/be/test/runtime/fragment_mgr_test.cpp +++ b/be/test/runtime/fragment_mgr_test.cpp @@ -42,6 +42,7 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request) { } Status PlanFragmentExecutor::open() { + // TODO(yingchun): what does it means? SleepFor(MonoDelta::FromMilliseconds(50)); return s_open_status; } diff --git a/be/test/runtime/load_channel_mgr_test.cpp b/be/test/runtime/load_channel_mgr_test.cpp index 1c7af66ec68479e..f8ab18a60522b7b 100644 --- a/be/test/runtime/load_channel_mgr_test.cpp +++ b/be/test/runtime/load_channel_mgr_test.cpp @@ -45,9 +45,9 @@ OLAPStatus close_status; int64_t wait_lock_time_ns; // mock -DeltaWriter::DeltaWriter(WriteRequest* req, const std::shared_ptr& mem_tracker, +DeltaWriter::DeltaWriter(const WriteRequest& req, const std::shared_ptr& mem_tracker, StorageEngine* storage_engine) : - _req(*req) { + _req(req) { } DeltaWriter::~DeltaWriter() { @@ -57,12 +57,12 @@ OLAPStatus DeltaWriter::init() { return OLAP_SUCCESS; } -OLAPStatus DeltaWriter::open(WriteRequest* req, const std::shared_ptr& mem_tracker, DeltaWriter** writer) { +void DeltaWriter::open(const WriteRequest& req, const std::shared_ptr& mem_tracker, DeltaWriter** writer) { if (open_status != OLAP_SUCCESS) { - return open_status; + return; } + *writer = new DeltaWriter(req, mem_tracker, nullptr); - return open_status; } OLAPStatus DeltaWriter::write(Tuple* tuple) { @@ -288,9 +288,8 @@ TEST_F(LoadChannelMgrTest, cancel) { PTabletWriterCancelRequest request; request.set_allocated_id(&load_id); request.set_index_id(4); - auto st = mgr.cancel(request); + mgr.cancel(request); request.release_id(); - ASSERT_TRUE(st.ok()); } } diff --git a/be/test/runtime/result_queue_mgr_test.cpp b/be/test/runtime/result_queue_mgr_test.cpp index 2b9c52b359d4807..7a9b37e14e6cd37 100644 --- a/be/test/runtime/result_queue_mgr_test.cpp +++ b/be/test/runtime/result_queue_mgr_test.cpp @@ -120,7 +120,7 @@ TEST_F(ResultQueueMgrTest, normal_cancel) { BlockQueueSharedPtr block_queue_t; queue_mgr.create_queue(query_id, &block_queue_t); ASSERT_TRUE(block_queue_t != nullptr); - ASSERT_TRUE(queue_mgr.cancel(query_id).ok()); + queue_mgr.cancel(query_id); } TEST_F(ResultQueueMgrTest, cancel_no_block) { @@ -131,7 +131,7 @@ TEST_F(ResultQueueMgrTest, cancel_no_block) { BlockQueueSharedPtr block_queue_t; queue_mgr.create_queue(query_id, &block_queue_t); ASSERT_TRUE(block_queue_t != nullptr); - ASSERT_TRUE(queue_mgr.cancel(query_id).ok()); + queue_mgr.cancel(query_id); } } diff --git a/build.sh b/build.sh index 20f74c503f51da8..69a21c3c72528b1 100755 --- a/build.sh +++ b/build.sh @@ -41,7 +41,7 @@ if [[ ! -f ${DORIS_THIRDPARTY}/installed/lib/libs2.a ]]; then ${DORIS_THIRDPARTY}/build-thirdparty.sh fi -PARALLEL=$[$(nproc)/4+1] +PARALLEL=32 #$[$(nproc)/4+1] # Check args usage() { diff --git a/run-be-ut.sh b/run-be-ut.sh index dafd484681510a6..99e5d6b037c24cb 100755 --- a/run-be-ut.sh +++ b/run-be-ut.sh @@ -39,7 +39,7 @@ export DORIS_HOME=${ROOT} . ${DORIS_HOME}/env.sh -PARALLEL=$[$(nproc)/4+1] +PARALLEL=32 #$[$(nproc)/4+1] # Check args usage() {