From f565f60bc3e1ee4ad69e6bb127bc32c1bb26eb7e Mon Sep 17 00:00:00 2001 From: ShowCode <108080669+hongkun-Shao@users.noreply.github.com> Date: Tue, 28 Nov 2023 13:02:30 +0800 Subject: [PATCH] [refactor](standard)BE:Initialize pointer variables in the class to nullptr by default (#27587) --- be/src/agent/heartbeat_server.h | 4 +- be/src/agent/task_worker_pool.h | 4 +- be/src/agent/workload_group_listener.h | 2 +- be/src/common/dwarf.h | 14 +- be/src/common/elf.h | 8 +- be/src/common/object_pool.h | 2 +- be/src/common/signal_handler.h | 6 +- be/src/common/symbol_index.h | 10 +- be/src/exec/decompressor.h | 2 +- be/src/exec/exec_node.h | 18 +- be/src/exec/odbc_connector.h | 2 +- be/src/exec/scan_node.h | 8 +- be/src/exec/schema_scanner.h | 22 +- .../schema_scanner/schema_charsets_scanner.h | 6 +- .../schema_collations_scanner.h | 8 +- .../schema_scanner/schema_variables_scanner.h | 4 +- be/src/exec/table_connector.h | 2 +- be/src/exprs/function_filter.h | 2 +- be/src/exprs/hybrid_map.h | 2 +- be/src/exprs/runtime_filter.h | 16 +- be/src/geo/ByteOrderDataInStream.h | 4 +- be/src/geo/geo_tobinary_type.h | 2 +- be/src/gutil/gscoped_ptr.h | 4 +- be/src/gutil/move.h | 2 +- be/src/gutil/port.h | 2 +- be/src/gutil/ref_counted.h | 2 +- be/src/gutil/stl_util.h | 14 +- be/src/gutil/strings/numbers.h | 4 +- be/src/gutil/strings/stringpiece.h | 2 +- be/src/gutil/strings/substitute.h | 2 +- .../gutil/threading/thread_collision_warner.h | 8 +- be/src/http/action/download_action.h | 2 +- be/src/http/action/download_binlog_action.h | 2 +- be/src/io/cache/block/block_file_segment.h | 2 +- be/src/io/cache/block/block_lru_file_cache.h | 2 +- be/src/io/fs/broker_file_writer.h | 2 +- be/src/io/fs/buffered_reader.h | 22 +- be/src/io/fs/hdfs_file_reader.h | 16 +- be/src/io/fs/hdfs_file_system.h | 4 +- be/src/io/fs/hdfs_file_writer.h | 2 +- be/src/io/fs/multi_table_pipe.h | 2 +- be/src/io/fs/s3_file_writer.h | 2 +- be/src/olap/base_tablet.h | 10 +- be/src/olap/block_column_predicate.h | 2 +- be/src/olap/column_mapping.h | 4 +- be/src/olap/data_dir.h | 20 +- be/src/olap/delta_writer_context.h | 2 +- be/src/olap/hll.h | 10 +- be/src/olap/inverted_index_parser.h | 2 +- be/src/olap/iterators.h | 4 +- be/src/olap/like_column_predicate.h | 2 +- be/src/olap/lru_cache.h | 10 +- be/src/olap/memtable.h | 10 +- be/src/olap/memtable_flush_executor.h | 2 +- be/src/olap/page_cache.h | 8 +- be/src/olap/push_handler.h | 6 +- be/src/olap/reader.h | 2 +- be/src/olap/row_cursor.h | 2 +- be/src/olap/row_cursor_cell.h | 2 +- be/src/olap/rowset/beta_rowset_reader.h | 4 +- be/src/olap/rowset/pending_rowset_helper.h | 2 +- be/src/olap/rowset/segcompaction.h | 2 +- be/src/olap/rowset/segment_creator.h | 6 +- .../olap/rowset/segment_v2/binary_dict_page.h | 3 +- .../rowset/segment_v2/bitmap_index_reader.h | 4 +- be/src/olap/rowset/segment_v2/bloom_filter.h | 4 +- .../segment_v2/bloom_filter_index_reader.h | 6 +- .../segment_v2/bloom_filter_index_writer.h | 2 +- be/src/olap/rowset/segment_v2/column_reader.h | 10 +- be/src/olap/rowset/segment_v2/column_writer.h | 2 +- be/src/olap/rowset/segment_v2/encoding_info.h | 2 +- be/src/olap/rowset/segment_v2/index_page.h | 2 +- .../rowset/segment_v2/indexed_column_reader.h | 2 +- .../rowset/segment_v2/indexed_column_writer.h | 8 +- .../rowset/segment_v2/inverted_index_cache.h | 2 +- .../inverted_index_compound_directory.h | 6 +- .../inverted_index_compound_reader.h | 8 +- .../rowset/segment_v2/inverted_index_reader.h | 4 +- .../rowset/segment_v2/ordinal_page_index.h | 2 +- be/src/olap/rowset/segment_v2/parsed_page.h | 2 +- be/src/olap/rowset/segment_v2/rle_page.h | 2 +- .../olap/rowset/segment_v2/segment_iterator.h | 2 +- .../olap/rowset/segment_v2/segment_writer.h | 4 +- .../segment_v2/vertical_segment_writer.h | 4 +- .../olap/rowset/segment_v2/zone_map_index.h | 2 +- be/src/olap/segment_loader.h | 2 +- be/src/olap/selection_vector.h | 2 +- be/src/olap/short_key_index.h | 2 +- be/src/olap/skiplist.h | 6 +- be/src/olap/storage_engine.h | 2 +- be/src/olap/tablet.h | 2 +- be/src/olap/task/engine_clone_task.h | 2 +- .../olap/task/engine_publish_version_task.h | 8 +- .../olap/task/engine_storage_migration_task.h | 2 +- be/src/olap/txn_manager.h | 14 +- be/src/olap/wal_manager.h | 2 +- .../pipeline/exec/aggregation_sink_operator.h | 40 +-- .../exec/aggregation_source_operator.h | 14 +- be/src/pipeline/exec/analytic_sink_operator.h | 6 +- .../pipeline/exec/analytic_source_operator.h | 10 +- be/src/pipeline/exec/datagen_operator.h | 2 +- ...inct_streaming_aggregation_sink_operator.h | 2 +- be/src/pipeline/exec/es_scan_operator.h | 8 +- be/src/pipeline/exec/exchange_sink_buffer.h | 18 +- be/src/pipeline/exec/exchange_sink_operator.h | 6 +- be/src/pipeline/exec/hashjoin_build_sink.h | 36 +-- .../pipeline/exec/hashjoin_probe_operator.h | 16 +- .../pipeline/exec/join_build_sink_operator.h | 6 +- be/src/pipeline/exec/join_probe_operator.h | 8 +- .../pipeline/exec/multi_cast_data_streamer.h | 6 +- .../exec/nested_loop_join_probe_operator.h | 2 +- .../pipeline/exec/olap_table_sink_operator.h | 2 +- .../exec/olap_table_sink_v2_operator.h | 2 +- be/src/pipeline/exec/operator.h | 10 +- .../exec/partition_sort_sink_operator.h | 10 +- be/src/pipeline/exec/repeat_operator.h | 4 +- .../pipeline/exec/result_file_sink_operator.h | 12 +- be/src/pipeline/exec/result_sink_operator.h | 2 +- be/src/pipeline/exec/scan_operator.h | 14 +- be/src/pipeline/exec/schema_scan_operator.h | 6 +- be/src/pipeline/exec/set_sink_operator.h | 2 +- be/src/pipeline/exec/sort_sink_operator.h | 4 +- .../streaming_aggregation_sink_operator.h | 10 +- be/src/pipeline/pipeline_fragment_context.h | 6 +- be/src/pipeline/pipeline_task.h | 60 ++-- be/src/pipeline/pipeline_x/dependency.h | 21 +- be/src/pipeline/pipeline_x/operator.h | 44 +-- be/src/pipeline/pipeline_x/pipeline_x_task.h | 4 +- be/src/runtime/broker_mgr.h | 2 +- be/src/runtime/buffer_control_block.h | 2 +- be/src/runtime/cache/result_cache.h | 4 +- be/src/runtime/cache/result_node.h | 6 +- be/src/runtime/client_cache.h | 6 +- be/src/runtime/collection_value.h | 4 +- be/src/runtime/descriptors.h | 2 +- be/src/runtime/exec_env.h | 2 +- be/src/runtime/external_scan_context_mgr.h | 2 +- be/src/runtime/fragment_mgr.h | 7 +- be/src/runtime/group_commit_mgr.h | 6 +- be/src/runtime/load_channel.h | 2 +- be/src/runtime/load_path_mgr.h | 2 +- be/src/runtime/load_stream.h | 4 +- be/src/runtime/load_stream_mgr.h | 4 +- be/src/runtime/load_stream_writer.h | 2 +- be/src/runtime/map_value.h | 4 +- be/src/runtime/memory/mem_tracker.h | 2 +- be/src/runtime/plan_fragment_executor.h | 12 +- be/src/runtime/query_context.h | 8 +- .../routine_load/routine_load_task_executor.h | 2 +- be/src/runtime/runtime_filter_mgr.h | 10 +- be/src/runtime/runtime_predicate.h | 6 +- be/src/runtime/runtime_state.h | 4 +- be/src/runtime/small_file_mgr.h | 2 +- be/src/runtime/snapshot_loader.h | 2 +- .../runtime/stream_load/stream_load_context.h | 2 +- .../stream_load/stream_load_executor.h | 2 +- .../stream_load/stream_load_recorder.h | 2 +- be/src/runtime/struct_value.h | 2 +- be/src/runtime/tablets_channel.h | 2 +- be/src/runtime/task_group/task_group.h | 4 +- be/src/runtime/thread_context.h | 8 +- .../arrow_flight/auth_server_middleware.h | 2 +- be/src/service/backend_service.h | 2 +- be/src/service/http_service.h | 4 +- be/src/service/internal_service.h | 2 +- be/src/service/point_query_executor.h | 8 +- be/src/udf/udf.h | 2 +- be/src/util/bfd_parser.h | 4 +- be/src/util/bit_stream_utils.h | 4 +- be/src/util/bitmap.h | 2 +- be/src/util/countdown_latch.h | 2 +- be/src/util/doris_metrics.h | 304 +++++++++--------- be/src/util/easy_json.h | 2 +- be/src/util/faststring.h | 2 +- be/src/util/frame_of_reference_coding.h | 2 +- be/src/util/interval_tree-inl.h | 4 +- be/src/util/interval_tree.h | 2 +- be/src/util/jni-util.h | 8 +- be/src/util/jsonb_document.h | 6 +- be/src/util/jsonb_stream.h | 2 +- be/src/util/jsonb_updater.h | 4 +- be/src/util/jsonb_writer.h | 2 +- be/src/util/lru_multi_cache.h | 2 +- be/src/util/mem_range.h | 2 +- be/src/util/mysql_row_buffer.h | 4 +- be/src/util/obj_lru_cache.h | 2 +- be/src/util/runtime_profile.h | 16 +- be/src/util/slice.h | 2 +- be/src/util/sm3.h | 4 +- be/src/util/system_metrics.h | 8 +- be/src/util/thread.h | 2 +- be/src/util/threadpool.h | 2 +- be/src/util/thrift_server.h | 6 +- .../aggregate_function_collect.h | 8 +- .../aggregate_function_percentile_approx.h | 2 +- .../aggregate_function_sort.h | 2 +- be/src/vec/common/arena.h | 10 +- be/src/vec/common/columns_hashing_impl.h | 2 +- be/src/vec/common/cow.h | 2 +- .../vec/common/hash_table/hash_map_context.h | 2 +- be/src/vec/common/hash_table/hash_table.h | 8 +- .../common/hash_table/hash_table_set_build.h | 4 +- .../common/hash_table/hash_table_set_probe.h | 2 +- .../vec/common/hash_table/string_hash_table.h | 4 +- be/src/vec/common/sort/partition_sorter.h | 4 +- be/src/vec/common/sort/sorter.h | 10 +- be/src/vec/core/block_spill_reader.h | 8 +- be/src/vec/core/block_spill_writer.h | 8 +- be/src/vec/core/sort_cursor.h | 2 +- .../vdata_gen_function_inf.h | 2 +- be/src/vec/exec/format/avro/avro_jni_reader.h | 6 +- be/src/vec/exec/format/csv/csv_reader.h | 8 +- .../file_reader/new_plain_text_line_reader.h | 16 +- be/src/vec/exec/format/format_common.h | 2 +- be/src/vec/exec/format/jni_reader.h | 4 +- be/src/vec/exec/format/json/new_json_reader.h | 18 +- be/src/vec/exec/format/orc/vorc_reader.h | 40 +-- be/src/vec/exec/format/parquet/decoder.h | 4 +- .../format/parquet/parquet_column_convert.h | 2 +- .../parquet/vparquet_column_chunk_reader.h | 8 +- .../format/parquet/vparquet_column_reader.h | 12 +- .../format/parquet/vparquet_group_reader.h | 16 +- .../format/parquet/vparquet_page_reader.h | 4 +- .../vec/exec/format/parquet/vparquet_reader.h | 80 ++--- .../format/table/max_compute_jni_reader.h | 10 +- be/src/vec/exec/format/table/paimon_reader.h | 4 +- .../format/table/transactional_hive_reader.h | 14 +- be/src/vec/exec/format/wal/wal_reader.h | 4 +- be/src/vec/exec/jni_connector.h | 12 +- be/src/vec/exec/join/join_op.h | 6 +- .../vec/exec/join/process_hash_table_probe.h | 22 +- be/src/vec/exec/join/vhash_join_node.h | 60 ++-- be/src/vec/exec/join/vjoin_node_base.h | 22 +- be/src/vec/exec/join/vnested_loop_join_node.h | 4 +- be/src/vec/exec/runtime_filter_consumer.h | 4 +- be/src/vec/exec/scan/mysql_scanner.h | 4 +- be/src/vec/exec/scan/new_es_scan_node.h | 10 +- be/src/vec/exec/scan/new_es_scanner.h | 2 +- be/src/vec/exec/scan/new_jdbc_scanner.h | 2 +- be/src/vec/exec/scan/new_odbc_scanner.h | 2 +- be/src/vec/exec/scan/new_olap_scanner.h | 4 +- be/src/vec/exec/scan/scan_task_queue.h | 2 +- be/src/vec/exec/scan/scanner_context.h | 12 +- be/src/vec/exec/scan/scanner_scheduler.h | 6 +- be/src/vec/exec/scan/vfile_scanner.h | 10 +- be/src/vec/exec/scan/vmeta_scanner.h | 2 +- be/src/vec/exec/scan/vscan_node.h | 14 +- be/src/vec/exec/scan/vscanner.h | 8 +- be/src/vec/exec/vaggregation_node.h | 50 +-- be/src/vec/exec/vanalytic_eval_node.h | 2 +- be/src/vec/exec/vdata_gen_scan_node.h | 2 +- be/src/vec/exec/vjdbc_connector.h | 2 +- be/src/vec/exec/vmysql_scan_node.h | 2 +- be/src/vec/exec/vpartition_sort_node.h | 14 +- be/src/vec/exec/vschema_scan_node.h | 2 +- be/src/vec/exec/vset_operation_node.h | 6 +- be/src/vec/exec/vsort_node.h | 4 +- .../exprs/table_function/vexplode_bitmap.h | 2 +- be/src/vec/exprs/vectorized_agg_fn.h | 10 +- be/src/vec/exprs/vslot_ref.h | 2 +- .../functions/array/function_array_remove.h | 4 +- be/src/vec/functions/function_cast.h | 2 +- be/src/vec/functions/regexps.h | 2 +- be/src/vec/io/reader_buffer.h | 4 +- be/src/vec/olap/vcollect_iterator.h | 2 +- be/src/vec/runtime/vcsv_transformer.h | 2 +- be/src/vec/runtime/vdata_stream_recvr.h | 36 +-- be/src/vec/runtime/vfile_format_transformer.h | 2 +- be/src/vec/runtime/vorc_transformer.h | 6 +- be/src/vec/runtime/vparquet_transformer.h | 4 +- be/src/vec/runtime/vsorted_run_merger.h | 4 +- be/src/vec/sink/delta_writer_v2_pool.h | 2 +- be/src/vec/sink/load_stream_stub.h | 2 +- be/src/vec/sink/load_stream_stub_pool.h | 2 +- be/src/vec/sink/varrow_flight_result_writer.h | 4 +- be/src/vec/sink/vdata_stream_sender.h | 18 +- be/src/vec/sink/vmysql_result_writer.h | 2 +- be/src/vec/sink/vresult_file_sink.h | 2 +- be/src/vec/sink/vrow_distribution.h | 24 +- be/src/vec/sink/vtablet_block_convertor.h | 2 +- be/src/vec/sink/vtablet_finder.h | 2 +- be/src/vec/sink/vtablet_sink.h | 2 +- be/src/vec/sink/vtablet_sink_v2.h | 2 +- be/src/vec/sink/writer/vfile_result_writer.h | 2 +- be/src/vec/sink/writer/vmysql_table_writer.h | 2 +- be/src/vec/sink/writer/vtablet_writer.h | 10 +- be/src/vec/sink/writer/vtablet_writer_v2.h | 2 +- be/src/vec/sink/writer/vwal_writer.h | 2 +- be/test/testutil/desc_tbl_builder.h | 2 +- samples/connect/cpp/doris_client.h | 2 +- 290 files changed, 1112 insertions(+), 1109 deletions(-) diff --git a/be/src/agent/heartbeat_server.h b/be/src/agent/heartbeat_server.h index aee053860a..928efb5c62 100644 --- a/be/src/agent/heartbeat_server.h +++ b/be/src/agent/heartbeat_server.h @@ -53,13 +53,13 @@ public: private: Status _heartbeat(const TMasterInfo& master_info); - StorageEngine* _olap_engine; + StorageEngine* _olap_engine = nullptr; int64_t _be_epoch; // mutex to protect master_info and _epoch std::mutex _hb_mtx; // Not owned. Point to the ExecEnv::_master_info - TMasterInfo* _master_info; + TMasterInfo* _master_info = nullptr; int64_t _fe_epoch; DISALLOW_COPY_AND_ASSIGN(HeartbeatServer); diff --git a/be/src/agent/task_worker_pool.h b/be/src/agent/task_worker_pool.h index 50c8842166..696453fa4e 100644 --- a/be/src/agent/task_worker_pool.h +++ b/be/src/agent/task_worker_pool.h @@ -227,7 +227,7 @@ protected: // Reference to the ExecEnv::_master_info const TMasterInfo& _master_info; std::unique_ptr _agent_utils; - ExecEnv* _env; + ExecEnv* _env = nullptr; // Protect task queue std::mutex _worker_thread_lock; @@ -242,7 +242,7 @@ protected: std::atomic _is_doing_work; std::shared_ptr _metric_entity; - UIntGauge* agent_task_queue_size; + UIntGauge* agent_task_queue_size = nullptr; // Always 1 when _thread_model is SINGLE_THREAD uint32_t _worker_count; diff --git a/be/src/agent/workload_group_listener.h b/be/src/agent/workload_group_listener.h index 732f5752e4..f596535908 100644 --- a/be/src/agent/workload_group_listener.h +++ b/be/src/agent/workload_group_listener.h @@ -32,6 +32,6 @@ public: void handle_topic_info(const std::vector& topic_info_list) override; private: - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; }; } // namespace doris \ No newline at end of file diff --git a/be/src/common/dwarf.h b/be/src/common/dwarf.h index 6807721856..9d15e6c15f 100644 --- a/be/src/common/dwarf.h +++ b/be/src/common/dwarf.h @@ -347,13 +347,13 @@ private: std::string_view debugLineStr_; // DWARF 5 /// NOLINT // Header - uint16_t version_; /// NOLINT - uint8_t minLength_; /// NOLINT - bool defaultIsStmt_; /// NOLINT - int8_t lineBase_; /// NOLINT - uint8_t lineRange_; /// NOLINT - uint8_t opcodeBase_; /// NOLINT - const uint8_t* standardOpcodeLengths_; /// NOLINT + uint16_t version_; /// NOLINT + uint8_t minLength_; /// NOLINT + bool defaultIsStmt_; /// NOLINT + int8_t lineBase_; /// NOLINT + uint8_t lineRange_; /// NOLINT + uint8_t opcodeBase_; /// NOLINT + const uint8_t* standardOpcodeLengths_ = nullptr; /// NOLINT // 6.2.4 The Line Number Program Header. struct { diff --git a/be/src/common/elf.h b/be/src/common/elf.h index 5bb5d93cb7..4480edcd0e 100644 --- a/be/src/common/elf.h +++ b/be/src/common/elf.h @@ -83,10 +83,10 @@ private: int _fd = -1; std::filesystem::path _file; size_t elf_size; - char* mapped; - const ElfEhdr* header; - const ElfShdr* section_headers; - const ElfPhdr* program_headers; + char* mapped = nullptr; + const ElfEhdr* header = nullptr; + const ElfShdr* section_headers = nullptr; + const ElfPhdr* program_headers = nullptr; const char* section_names = nullptr; }; diff --git a/be/src/common/object_pool.h b/be/src/common/object_pool.h index b0552f7e43..3c000371a9 100644 --- a/be/src/common/object_pool.h +++ b/be/src/common/object_pool.h @@ -78,7 +78,7 @@ private: /// For each object, a pointer to the object and a function that deletes it. struct Element { - void* obj; + void* obj = nullptr; DeleteFn delete_fn; }; diff --git a/be/src/common/signal_handler.h b/be/src/common/signal_handler.h index 013b0e1c64..dad48c163c 100644 --- a/be/src/common/signal_handler.h +++ b/be/src/common/signal_handler.h @@ -64,7 +64,7 @@ namespace { // The list should be synced with the comment in signalhandler.h. const struct { int number; - const char* name; + const char* name = nullptr; } kFailureSignals[] = { {SIGSEGV, "SIGSEGV"}, {SIGILL, "SIGILL"}, {SIGFPE, "SIGFPE"}, {SIGABRT, "SIGABRT"}, {SIGBUS, "SIGBUS"}, {SIGTERM, "SIGTERM"}, @@ -218,8 +218,8 @@ public: } private: - char* buffer_; - char* cursor_; + char* buffer_ = nullptr; + char* cursor_ = nullptr; const char* const end_; }; diff --git a/be/src/common/symbol_index.h b/be/src/common/symbol_index.h index 0a9187c1f4..b713599c58 100644 --- a/be/src/common/symbol_index.h +++ b/be/src/common/symbol_index.h @@ -46,14 +46,14 @@ public: static void reload(); struct Symbol { - const void* address_begin; - const void* address_end; - const char* name; + const void* address_begin = nullptr; + const void* address_end = nullptr; + const char* name = nullptr; }; struct Object { - const void* address_begin; - const void* address_end; + const void* address_begin = nullptr; + const void* address_end = nullptr; std::string name; std::shared_ptr elf; }; diff --git a/be/src/exec/decompressor.h b/be/src/exec/decompressor.h index 2b07e71139..713c5db2bf 100644 --- a/be/src/exec/decompressor.h +++ b/be/src/exec/decompressor.h @@ -140,7 +140,7 @@ private: size_t get_block_size(const LZ4F_frameInfo_t* info); private: - LZ4F_dctx* _dctx; + LZ4F_dctx* _dctx = nullptr; size_t _expect_dec_buf_size; const static unsigned DORIS_LZ4F_VERSION; }; diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index f5f918731f..eeed37907f 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -252,7 +252,7 @@ protected: int _id; // unique w/in single plan tree TPlanNodeType::type _type; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; std::vector _tuple_ids; vectorized::VExprContextSPtrs _conjuncts; @@ -276,15 +276,15 @@ protected: // which will providea reference for operator memory. std::unique_ptr _mem_tracker; - RuntimeProfile::Counter* _exec_timer; - RuntimeProfile::Counter* _rows_returned_counter; - RuntimeProfile::Counter* _output_bytes_counter; - RuntimeProfile::Counter* _block_count_counter; - RuntimeProfile::Counter* _rows_returned_rate; - RuntimeProfile::Counter* _memory_used_counter; - RuntimeProfile::Counter* _projection_timer; + RuntimeProfile::Counter* _exec_timer = nullptr; + RuntimeProfile::Counter* _rows_returned_counter = nullptr; + RuntimeProfile::Counter* _output_bytes_counter = nullptr; + RuntimeProfile::Counter* _block_count_counter = nullptr; + RuntimeProfile::Counter* _rows_returned_rate = nullptr; + RuntimeProfile::Counter* _memory_used_counter = nullptr; + RuntimeProfile::Counter* _projection_timer = nullptr; // Account for peak memory used by this node - RuntimeProfile::Counter* _peak_memory_usage_counter; + RuntimeProfile::Counter* _peak_memory_usage_counter = nullptr; //NOTICE: now add a faker profile, because sometimes the profile record is useless //so we want remove some counters and timers, eg: in join node, if it's broadcast_join diff --git a/be/src/exec/odbc_connector.h b/be/src/exec/odbc_connector.h index 4edd1e879c..edefcd9fe0 100644 --- a/be/src/exec/odbc_connector.h +++ b/be/src/exec/odbc_connector.h @@ -43,7 +43,7 @@ struct ODBCConnectorParam { // only use in insert std::string table_name; bool use_transaction = false; - const TupleDescriptor* tuple_desc; + const TupleDescriptor* tuple_desc = nullptr; }; // Because the DataBinding have the mem alloc, so diff --git a/be/src/exec/scan_node.h b/be/src/exec/scan_node.h index c127a7c7b1..2347a64b85 100644 --- a/be/src/exec/scan_node.h +++ b/be/src/exec/scan_node.h @@ -99,11 +99,11 @@ public: static const std::string _s_num_disks_accessed_counter; protected: - RuntimeProfile::Counter* _bytes_read_counter; // # bytes read from the scanner - RuntimeProfile::Counter* _rows_read_counter; + RuntimeProfile::Counter* _bytes_read_counter = nullptr; // # bytes read from the scanner + RuntimeProfile::Counter* _rows_read_counter = nullptr; // Wall based aggregate read throughput [bytes/sec] - RuntimeProfile::Counter* _total_throughput_counter; - RuntimeProfile::Counter* _num_disks_accessed_counter; + RuntimeProfile::Counter* _total_throughput_counter = nullptr; + RuntimeProfile::Counter* _num_disks_accessed_counter = nullptr; }; } // namespace doris diff --git a/be/src/exec/schema_scanner.h b/be/src/exec/schema_scanner.h index ec4bc9e0d3..d1d11ba935 100644 --- a/be/src/exec/schema_scanner.h +++ b/be/src/exec/schema_scanner.h @@ -53,16 +53,16 @@ struct SchemaScannerCommonParam { ip(nullptr), port(0), catalog(nullptr) {} - const std::string* db; - const std::string* table; - const std::string* wild; - const std::string* user; // deprecated - const std::string* user_ip; // deprecated - const TUserIdentity* current_user_ident; // to replace the user and user ip - const std::string* ip; // frontend ip - int32_t port; // frontend thrift port + const std::string* db = nullptr; + const std::string* table = nullptr; + const std::string* wild = nullptr; + const std::string* user = nullptr; // deprecated + const std::string* user_ip = nullptr; // deprecated + const TUserIdentity* current_user_ident = nullptr; // to replace the user and user ip + const std::string* ip = nullptr; // frontend ip + int32_t port; // frontend thrift port int64_t thread_id; - const std::string* catalog; + const std::string* catalog = nullptr; }; // scanner parameter from frontend @@ -79,7 +79,7 @@ class SchemaScanner { public: struct ColumnDesc { - const char* name; + const char* name = nullptr; PrimitiveType type; int size; bool is_null; @@ -107,7 +107,7 @@ protected: bool _is_init; // this is used for sub class - SchemaScannerParam* _param; + SchemaScannerParam* _param = nullptr; // schema table's column desc std::vector _columns; diff --git a/be/src/exec/schema_scanner/schema_charsets_scanner.h b/be/src/exec/schema_scanner/schema_charsets_scanner.h index 174c11223b..1f01070875 100644 --- a/be/src/exec/schema_scanner/schema_charsets_scanner.h +++ b/be/src/exec/schema_scanner/schema_charsets_scanner.h @@ -40,9 +40,9 @@ public: private: struct CharsetStruct { - const char* charset; - const char* default_collation; - const char* description; + const char* charset = nullptr; + const char* default_collation = nullptr; + const char* description = nullptr; int64_t maxlen; }; diff --git a/be/src/exec/schema_scanner/schema_collations_scanner.h b/be/src/exec/schema_scanner/schema_collations_scanner.h index 5e9e76d053..f0f60538ca 100644 --- a/be/src/exec/schema_scanner/schema_collations_scanner.h +++ b/be/src/exec/schema_scanner/schema_collations_scanner.h @@ -40,11 +40,11 @@ public: private: struct CollationStruct { - const char* name; - const char* charset; + const char* name = nullptr; + const char* charset = nullptr; int64_t id; - const char* is_default; - const char* is_compile; + const char* is_default = nullptr; + const char* is_compile = nullptr; int64_t sortlen; }; diff --git a/be/src/exec/schema_scanner/schema_variables_scanner.h b/be/src/exec/schema_scanner/schema_variables_scanner.h index b928365dd4..2d207ff8b2 100644 --- a/be/src/exec/schema_scanner/schema_variables_scanner.h +++ b/be/src/exec/schema_scanner/schema_variables_scanner.h @@ -44,8 +44,8 @@ public: private: struct VariableStruct { - const char* name; - const char* value; + const char* name = nullptr; + const char* value = nullptr; }; Status _fill_block_impl(vectorized::Block* block); diff --git a/be/src/exec/table_connector.h b/be/src/exec/table_connector.h index 7a0dd63e06..a1d6410ddb 100644 --- a/be/src/exec/table_connector.h +++ b/be/src/exec/table_connector.h @@ -86,7 +86,7 @@ protected: bool _use_tranaction; bool _is_in_transaction; std::string_view _table_name; - const TupleDescriptor* _tuple_desc; + const TupleDescriptor* _tuple_desc = nullptr; // only use in query std::string _sql_str; // only use in write diff --git a/be/src/exprs/function_filter.h b/be/src/exprs/function_filter.h index 9802f1b408..f48010623a 100644 --- a/be/src/exprs/function_filter.h +++ b/be/src/exprs/function_filter.h @@ -35,7 +35,7 @@ public: bool _opposite; std::string _col_name; // these pointer's life time controlled by scan node - doris::FunctionContext* _fn_ctx; + doris::FunctionContext* _fn_ctx = nullptr; // only one param from conjunct, because now only support like predicate doris::StringRef _string_param; }; diff --git a/be/src/exprs/hybrid_map.h b/be/src/exprs/hybrid_map.h index 0f24744018..d34e83c648 100644 --- a/be/src/exprs/hybrid_map.h +++ b/be/src/exprs/hybrid_map.h @@ -33,7 +33,7 @@ public: virtual ~HybridMap() {} virtual HybridSetBase* find_or_insert_set(uint64_t dst, bool* is_add_buckets) { - HybridSetBase* _set_ptr; + HybridSetBase* _set_ptr = nullptr; typename std::unordered_map::const_iterator it = _map.find(dst); if (it == _map.end()) { diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h index a5fd309ca4..797b217662 100644 --- a/be/src/exprs/runtime_filter.h +++ b/be/src/exprs/runtime_filter.h @@ -149,9 +149,9 @@ struct UpdateRuntimeFilterParams { UpdateRuntimeFilterParams(const PPublishFilterRequest* req, butil::IOBufAsZeroCopyInputStream* data_stream, ObjectPool* obj_pool) : request(req), data(data_stream), pool(obj_pool) {} - const PPublishFilterRequest* request; - butil::IOBufAsZeroCopyInputStream* data; - ObjectPool* pool; + const PPublishFilterRequest* request = nullptr; + butil::IOBufAsZeroCopyInputStream* data = nullptr; + ObjectPool* pool = nullptr; }; struct UpdateRuntimeFilterParamsV2 { @@ -161,15 +161,15 @@ struct UpdateRuntimeFilterParamsV2 { : request(req), data(data_stream), pool(obj_pool) {} const PPublishFilterRequestV2* request; butil::IOBufAsZeroCopyInputStream* data; - ObjectPool* pool; + ObjectPool* pool = nullptr; }; struct MergeRuntimeFilterParams { MergeRuntimeFilterParams(const PMergeFilterRequest* req, butil::IOBufAsZeroCopyInputStream* data_stream) : request(req), data(data_stream) {} - const PMergeFilterRequest* request; - butil::IOBufAsZeroCopyInputStream* data; + const PMergeFilterRequest* request = nullptr; + butil::IOBufAsZeroCopyInputStream* data = nullptr; }; enum RuntimeFilterState { @@ -428,10 +428,10 @@ protected: RuntimeState* _state = nullptr; QueryContext* _query_ctx = nullptr; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; // _wrapper is a runtime filter function wrapper // _wrapper should alloc from _pool - RuntimePredicateWrapper* _wrapper; + RuntimePredicateWrapper* _wrapper = nullptr; // runtime filter id int _filter_id; // Specific types BoardCast or Shuffle diff --git a/be/src/geo/ByteOrderDataInStream.h b/be/src/geo/ByteOrderDataInStream.h index f42d685808..d54caffdaf 100644 --- a/be/src/geo/ByteOrderDataInStream.h +++ b/be/src/geo/ByteOrderDataInStream.h @@ -84,8 +84,8 @@ public: private: int byteOrder; - const unsigned char* buf; - const unsigned char* end; + const unsigned char* buf = nullptr; + const unsigned char* end = nullptr; }; } // namespace doris diff --git a/be/src/geo/geo_tobinary_type.h b/be/src/geo/geo_tobinary_type.h index 294548f837..0f3513be88 100644 --- a/be/src/geo/geo_tobinary_type.h +++ b/be/src/geo/geo_tobinary_type.h @@ -39,5 +39,5 @@ struct ToBinaryContext { unsigned char buf[8]; - std::ostream* outStream; + std::ostream* outStream = nullptr; }; diff --git a/be/src/gutil/gscoped_ptr.h b/be/src/gutil/gscoped_ptr.h index 688a5cfd22..3c8f558bbb 100644 --- a/be/src/gutil/gscoped_ptr.h +++ b/be/src/gutil/gscoped_ptr.h @@ -283,7 +283,7 @@ private: struct Data : public D { explicit Data(T* ptr_in) : ptr(ptr_in) {} Data(T* ptr_in, D other) : D(std::move(other)), ptr(ptr_in) {} - T* ptr; + T* ptr = nullptr; }; Data data_; @@ -775,7 +775,7 @@ public: } private: - C* ptr_; + C* ptr_ = nullptr; // no reason to use these: each gscoped_ptr_malloc should have its own object template diff --git a/be/src/gutil/move.h b/be/src/gutil/move.h index ec19de7095..aa835c1bb9 100644 --- a/be/src/gutil/move.h +++ b/be/src/gutil/move.h @@ -204,7 +204,7 @@ private: \ struct rvalue_type { \ explicit rvalue_type(type* object) : object(object) {} \ - type* object; \ + type* object = nullptr; \ }; \ type(type&); \ void operator=(type&); \ diff --git a/be/src/gutil/port.h b/be/src/gutil/port.h index 3f8a431d65..9e2cf6599f 100644 --- a/be/src/gutil/port.h +++ b/be/src/gutil/port.h @@ -255,7 +255,7 @@ using namespace std; // Just like VC++, we need a using here. #define GOOGLE_OBSCURE_SIGNAL 0 struct stack_t { - void* ss_sp; + void* ss_sp = nullptr; int ss_flags; size_t ss_size; }; diff --git a/be/src/gutil/ref_counted.h b/be/src/gutil/ref_counted.h index 320f757dea..8f23a4daf0 100644 --- a/be/src/gutil/ref_counted.h +++ b/be/src/gutil/ref_counted.h @@ -312,7 +312,7 @@ public: void reset(T* p = NULL) { *this = p; } protected: - T* ptr_; + T* ptr_ = nullptr; private: template diff --git a/be/src/gutil/stl_util.h b/be/src/gutil/stl_util.h index e3aa6dc57a..007e8bc62b 100644 --- a/be/src/gutil/stl_util.h +++ b/be/src/gutil/stl_util.h @@ -244,7 +244,7 @@ inline void STLAssignToVectorChar(vector* vec, const char* ptr, size_t n) // A struct that mirrors the GCC4 implementation of a string. See: // /usr/crosstool/v8/gcc-4.1.0-glibc-2.2.2/i686-unknown-linux-gnu/include/c++/4.1.0/ext/sso_string_base.h struct InternalStringRepGCC4 { - char* _M_data; + char* _M_data = nullptr; size_t _M_string_length; enum { _S_local_capacity = 15 }; @@ -437,7 +437,7 @@ public: virtual ~TemplatedElementDeleter() { STLDeleteElements(container_ptr_); } private: - STLContainer* container_ptr_; + STLContainer* container_ptr_ = nullptr; DISALLOW_EVIL_CONSTRUCTORS(TemplatedElementDeleter); }; @@ -454,7 +454,7 @@ public: ~ElementDeleter() { delete deleter_; } private: - BaseDeleter* deleter_; + BaseDeleter* deleter_ = nullptr; DISALLOW_EVIL_CONSTRUCTORS(ElementDeleter); }; @@ -470,7 +470,7 @@ public: virtual ~TemplatedValueDeleter() { STLDeleteValues(container_ptr_); } private: - STLContainer* container_ptr_; + STLContainer* container_ptr_ = nullptr; DISALLOW_EVIL_CONSTRUCTORS(TemplatedValueDeleter); }; @@ -486,7 +486,7 @@ public: ~ValueDeleter() { delete deleter_; } private: - BaseDeleter* deleter_; + BaseDeleter* deleter_ = nullptr; DISALLOW_EVIL_CONSTRUCTORS(ValueDeleter); }; @@ -505,7 +505,7 @@ public: ~STLElementDeleter() { STLDeleteElements(container_ptr_); } private: - STLContainer* container_ptr_; + STLContainer* container_ptr_ = nullptr; }; template @@ -515,7 +515,7 @@ public: ~STLValueDeleter() { STLDeleteValues(container_ptr_); } private: - STLContainer* container_ptr_; + STLContainer* container_ptr_ = nullptr; }; // STLSet{Difference,SymmetricDifference,Union,Intersection}(A a, B b, C *c) diff --git a/be/src/gutil/strings/numbers.h b/be/src/gutil/strings/numbers.h index e7c6f93826..f47982d75e 100644 --- a/be/src/gutil/strings/numbers.h +++ b/be/src/gutil/strings/numbers.h @@ -516,9 +516,9 @@ string AccurateItoaKMGT(int64 i); // '\0'-terminated, which is more efficient. // ---------------------------------------------------------------------- struct DoubleRangeOptions { - const char* separators; + const char* separators = nullptr; bool require_separator; - const char* acceptable_terminators; + const char* acceptable_terminators = nullptr; bool null_terminator_ok; bool allow_unbounded_markers; uint32 num_required_bounds; diff --git a/be/src/gutil/strings/stringpiece.h b/be/src/gutil/strings/stringpiece.h index db8e4a3140..60aeb5a274 100644 --- a/be/src/gutil/strings/stringpiece.h +++ b/be/src/gutil/strings/stringpiece.h @@ -127,7 +127,7 @@ class StringPiece { private: - const char* ptr_; + const char* ptr_ = nullptr; int length_; public: diff --git a/be/src/gutil/strings/substitute.h b/be/src/gutil/strings/substitute.h index 7a44639bdf..27b6625ade 100644 --- a/be/src/gutil/strings/substitute.h +++ b/be/src/gutil/strings/substitute.h @@ -131,7 +131,7 @@ public: private: inline SubstituteArg() : text_(NULL), size_(-1) {} - const char* text_; + const char* text_ = nullptr; int size_; char scratch_[kFastToBufferSize]; }; diff --git a/be/src/gutil/threading/thread_collision_warner.h b/be/src/gutil/threading/thread_collision_warner.h index 754e7ae5b1..eb962b1079 100644 --- a/be/src/gutil/threading/thread_collision_warner.h +++ b/be/src/gutil/threading/thread_collision_warner.h @@ -160,7 +160,7 @@ public: ~Check() {} private: - ThreadCollisionWarner* warner_; + ThreadCollisionWarner* warner_ = nullptr; DISALLOW_COPY_AND_ASSIGN(Check); }; @@ -174,7 +174,7 @@ public: ~ScopedCheck() { warner_->Leave(); } private: - ThreadCollisionWarner* warner_; + ThreadCollisionWarner* warner_ = nullptr; DISALLOW_COPY_AND_ASSIGN(ScopedCheck); }; @@ -190,7 +190,7 @@ public: ~ScopedRecursiveCheck() { warner_->Leave(); } private: - ThreadCollisionWarner* warner_; + ThreadCollisionWarner* warner_ = nullptr; DISALLOW_COPY_AND_ASSIGN(ScopedRecursiveCheck); }; @@ -218,7 +218,7 @@ private: // Here only for class unit tests purpose, during the test I need to not // DCHECK but notify the collision with something else. - AsserterBase* asserter_; + AsserterBase* asserter_ = nullptr; DISALLOW_COPY_AND_ASSIGN(ThreadCollisionWarner); }; diff --git a/be/src/http/action/download_action.h b/be/src/http/action/download_action.h index 3aab1a0d31..370059ba88 100644 --- a/be/src/http/action/download_action.h +++ b/be/src/http/action/download_action.h @@ -71,7 +71,7 @@ private: int32_t _num_workers; std::unique_ptr _download_workers; - std::shared_ptr _rate_limit_group {nullptr}; + std::shared_ptr _rate_limit_group; }; // end class DownloadAction } // end namespace doris diff --git a/be/src/http/action/download_binlog_action.h b/be/src/http/action/download_binlog_action.h index 77a2ed0878..310f5f6c62 100644 --- a/be/src/http/action/download_binlog_action.h +++ b/be/src/http/action/download_binlog_action.h @@ -44,7 +44,7 @@ private: private: ExecEnv* _exec_env; - std::shared_ptr _rate_limit_group {nullptr}; + std::shared_ptr _rate_limit_group; }; } // namespace doris diff --git a/be/src/io/cache/block/block_file_segment.h b/be/src/io/cache/block/block_file_segment.h index 24a4e6e174..67f4fc17a0 100644 --- a/be/src/io/cache/block/block_file_segment.h +++ b/be/src/io/cache/block/block_file_segment.h @@ -186,7 +186,7 @@ private: mutable std::mutex _download_mutex; Key _file_key; - IFileCache* _cache; + IFileCache* _cache = nullptr; std::atomic _is_downloaded {false}; CacheType _cache_type; diff --git a/be/src/io/cache/block/block_lru_file_cache.h b/be/src/io/cache/block/block_lru_file_cache.h index bfb9ab0040..c7644ecbd8 100644 --- a/be/src/io/cache/block/block_lru_file_cache.h +++ b/be/src/io/cache/block/block_lru_file_cache.h @@ -205,7 +205,7 @@ private: size_t _num_hit_segments = 0; size_t _num_removed_segments = 0; - std::shared_ptr _entity = nullptr; + std::shared_ptr _entity; DoubleGauge* file_cache_hits_ratio = nullptr; UIntGauge* file_cache_removed_elements = nullptr; diff --git a/be/src/io/fs/broker_file_writer.h b/be/src/io/fs/broker_file_writer.h index 95773f0a04..318cfafc3c 100644 --- a/be/src/io/fs/broker_file_writer.h +++ b/be/src/io/fs/broker_file_writer.h @@ -58,7 +58,7 @@ private: Status _write(const uint8_t* buf, size_t buf_len, size_t* written_bytes); private: - ExecEnv* _env; + ExecEnv* _env = nullptr; const TNetworkAddress _address; const std::map& _properties; int64_t _cur_offset; diff --git a/be/src/io/fs/buffered_reader.h b/be/src/io/fs/buffered_reader.h index 01422cacac..e78c1c7925 100644 --- a/be/src/io/fs/buffered_reader.h +++ b/be/src/io/fs/buffered_reader.h @@ -220,13 +220,13 @@ protected: const IOContext* io_ctx) override; private: - RuntimeProfile::Counter* _copy_time; - RuntimeProfile::Counter* _read_time; - RuntimeProfile::Counter* _request_io; - RuntimeProfile::Counter* _merged_io; - RuntimeProfile::Counter* _request_bytes; - RuntimeProfile::Counter* _merged_bytes; - RuntimeProfile::Counter* _apply_bytes; + RuntimeProfile::Counter* _copy_time = nullptr; + RuntimeProfile::Counter* _read_time = nullptr; + RuntimeProfile::Counter* _request_io = nullptr; + RuntimeProfile::Counter* _merged_io = nullptr; + RuntimeProfile::Counter* _request_bytes = nullptr; + RuntimeProfile::Counter* _merged_bytes = nullptr; + RuntimeProfile::Counter* _apply_bytes = nullptr; int _search_read_range(size_t start_offset, size_t end_offset); void _clean_cached_data(RangeCachedData& cached_data); @@ -311,8 +311,8 @@ struct PrefetchBuffer : std::enable_shared_from_this { size_t _size {0}; size_t _len {0}; size_t _whole_buffer_size; - io::FileReader* _reader; - const IOContext* _io_ctx; + io::FileReader* _reader = nullptr; + const IOContext* _io_ctx = nullptr; std::unique_ptr _buf; BufferStatus _buffer_status {BufferStatus::RESET}; std::mutex _lock; @@ -420,7 +420,7 @@ private: io::FileReaderSPtr _reader; PrefetchRange _file_range; const std::vector* _random_access_ranges = nullptr; - const IOContext* _io_ctx; + const IOContext* _io_ctx = nullptr; int64_t s_max_pre_buffer_size = 4 * 1024 * 1024; // 4MB std::vector> _pre_buffers; int64_t _whole_pre_buffer_size; @@ -457,7 +457,7 @@ protected: private: Status _close_internal(); io::FileReaderSPtr _reader; - std::unique_ptr _data = nullptr; + std::unique_ptr _data; size_t _size; bool _closed = false; }; diff --git a/be/src/io/fs/hdfs_file_reader.h b/be/src/io/fs/hdfs_file_reader.h index 850bc997c7..837e6376a8 100644 --- a/be/src/io/fs/hdfs_file_reader.h +++ b/be/src/io/fs/hdfs_file_reader.h @@ -60,14 +60,14 @@ protected: private: #ifdef USE_HADOOP_HDFS struct HDFSProfile { - RuntimeProfile::Counter* total_bytes_read; - RuntimeProfile::Counter* total_local_bytes_read; - RuntimeProfile::Counter* total_short_circuit_bytes_read; - RuntimeProfile::Counter* total_total_zero_copy_bytes_read; + RuntimeProfile::Counter* total_bytes_read = nullptr; + RuntimeProfile::Counter* total_local_bytes_read = nullptr; + RuntimeProfile::Counter* total_short_circuit_bytes_read = nullptr; + RuntimeProfile::Counter* total_total_zero_copy_bytes_read = nullptr; - RuntimeProfile::Counter* total_hedged_read; - RuntimeProfile::Counter* hedged_read_in_cur_thread; - RuntimeProfile::Counter* hedged_read_wins; + RuntimeProfile::Counter* total_hedged_read = nullptr; + RuntimeProfile::Counter* hedged_read_in_cur_thread = nullptr; + RuntimeProfile::Counter* hedged_read_wins = nullptr; }; #endif @@ -76,7 +76,7 @@ private: FileHandleCache::Accessor _accessor; CachedHdfsFileHandle* _handle = nullptr; // owned by _cached_file_handle std::atomic _closed = false; - RuntimeProfile* _profile; + RuntimeProfile* _profile = nullptr; #ifdef USE_HADOOP_HDFS HDFSProfile _hdfs_profile; #endif diff --git a/be/src/io/fs/hdfs_file_system.h b/be/src/io/fs/hdfs_file_system.h index b00b86022f..f31d67a78e 100644 --- a/be/src/io/fs/hdfs_file_system.h +++ b/be/src/io/fs/hdfs_file_system.h @@ -156,8 +156,8 @@ private: std::string _fs_name; // do not use std::shared_ptr or std::unique_ptr // _fs_handle is managed by HdfsFileSystemCache - HdfsFileSystemHandle* _fs_handle; - RuntimeProfile* _profile; + HdfsFileSystemHandle* _fs_handle = nullptr; + RuntimeProfile* _profile = nullptr; }; } // namespace io } // namespace doris diff --git a/be/src/io/fs/hdfs_file_writer.h b/be/src/io/fs/hdfs_file_writer.h index 371cae6e0c..03e89cd594 100644 --- a/be/src/io/fs/hdfs_file_writer.h +++ b/be/src/io/fs/hdfs_file_writer.h @@ -50,7 +50,7 @@ private: private: hdfsFile _hdfs_file = nullptr; // A convenient pointer to _fs - HdfsFileSystem* _hdfs_fs; + HdfsFileSystem* _hdfs_fs = nullptr; }; } // namespace io diff --git a/be/src/io/fs/multi_table_pipe.h b/be/src/io/fs/multi_table_pipe.h index 9f89672689..3ded0fc608 100644 --- a/be/src/io/fs/multi_table_pipe.h +++ b/be/src/io/fs/multi_table_pipe.h @@ -80,7 +80,7 @@ private: // note: Use raw pointer here to avoid cycle reference with StreamLoadContext. // Life cycle of MultiTablePipe is under control of StreamLoadContext, which means StreamLoadContext is created // before NultiTablePipe and released after it. It is safe to use raw pointer here. - StreamLoadContext* _ctx; + StreamLoadContext* _ctx = nullptr; Status _status; // save the first error status of all executing plan fragment #ifndef BE_TEST std::mutex _tablet_commit_infos_lock; diff --git a/be/src/io/fs/s3_file_writer.h b/be/src/io/fs/s3_file_writer.h index 119f07199b..959caa0b8e 100644 --- a/be/src/io/fs/s3_file_writer.h +++ b/be/src/io/fs/s3_file_writer.h @@ -76,7 +76,7 @@ private: std::vector> _completed_parts; IFileCache::Key _cache_key; - IFileCache* _cache; + IFileCache* _cache = nullptr; // **Attention** call add_count() before submitting buf to async thread pool bthread::CountdownEvent _countdown_event {0}; diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index d7ed2ca83f..b3fc9f8b9b 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -94,11 +94,11 @@ protected: std::shared_ptr _metric_entity; public: - IntCounter* query_scan_bytes; - IntCounter* query_scan_rows; - IntCounter* query_scan_count; - IntCounter* flush_bytes; - IntCounter* flush_finish_count; + IntCounter* query_scan_bytes = nullptr; + IntCounter* query_scan_rows = nullptr; + IntCounter* query_scan_count = nullptr; + IntCounter* flush_bytes = nullptr; + IntCounter* flush_finish_count = nullptr; std::atomic published_count = 0; }; diff --git a/be/src/olap/block_column_predicate.h b/be/src/olap/block_column_predicate.h index c91dc0c367..f70da43f56 100644 --- a/be/src/olap/block_column_predicate.h +++ b/be/src/olap/block_column_predicate.h @@ -126,7 +126,7 @@ public: } private: - const ColumnPredicate* _predicate; + const ColumnPredicate* _predicate = nullptr; }; class MutilColumnBlockPredicate : public BlockColumnPredicate { diff --git a/be/src/olap/column_mapping.h b/be/src/olap/column_mapping.h index da77a9bfb8..ba6e0a6857 100644 --- a/be/src/olap/column_mapping.h +++ b/be/src/olap/column_mapping.h @@ -34,9 +34,9 @@ struct ColumnMapping { // >=0: use origin column int32_t ref_column; // normally for default value. stores values for filters - WrapperField* default_value; + WrapperField* default_value = nullptr; std::shared_ptr expr; - const TabletColumn* new_column; + const TabletColumn* new_column = nullptr; }; using SchemaMapping = std::vector; diff --git a/be/src/olap/data_dir.h b/be/src/olap/data_dir.h index 0cdb15b241..8cdb54b5cc 100644 --- a/be/src/olap/data_dir.h +++ b/be/src/olap/data_dir.h @@ -175,8 +175,8 @@ private: TStorageMedium::type _storage_medium; bool _is_used; - TabletManager* _tablet_manager; - TxnManager* _txn_manager; + TabletManager* _tablet_manager = nullptr; + TxnManager* _txn_manager = nullptr; int32_t _cluster_id; bool _cluster_id_incomplete = false; // This flag will be set true if this store was not in root path when reloading @@ -193,14 +193,14 @@ private: RowsetIdGenerator* _id_generator = nullptr; std::shared_ptr _data_dir_metric_entity; - IntGauge* disks_total_capacity; - IntGauge* disks_avail_capacity; - IntGauge* disks_local_used_capacity; - IntGauge* disks_remote_used_capacity; - IntGauge* disks_trash_used_capacity; - IntGauge* disks_state; - IntGauge* disks_compaction_score; - IntGauge* disks_compaction_num; + IntGauge* disks_total_capacity = nullptr; + IntGauge* disks_avail_capacity = nullptr; + IntGauge* disks_local_used_capacity = nullptr; + IntGauge* disks_remote_used_capacity = nullptr; + IntGauge* disks_trash_used_capacity = nullptr; + IntGauge* disks_state = nullptr; + IntGauge* disks_compaction_score = nullptr; + IntGauge* disks_compaction_num = nullptr; }; } // namespace doris diff --git a/be/src/olap/delta_writer_context.h b/be/src/olap/delta_writer_context.h index 8656f4e9e1..aa8ee81f87 100644 --- a/be/src/olap/delta_writer_context.h +++ b/be/src/olap/delta_writer_context.h @@ -38,7 +38,7 @@ struct WriteRequest { PUniqueId load_id; TupleDescriptor* tuple_desc = nullptr; // slots are in order of tablet's schema - const std::vector* slots; + const std::vector* slots = nullptr; OlapTableSchemaParam* table_schema_param = nullptr; bool is_high_priority = false; bool write_file_cache = false; diff --git a/be/src/olap/hll.h b/be/src/olap/hll.h index 7c88abfbb5..1f97f30ae8 100644 --- a/be/src/olap/hll.h +++ b/be/src/olap/hll.h @@ -359,11 +359,11 @@ public: void parse(); private: - char* _buf_ref; // set - int _buf_len; // set len - HllDataType _set_type; //set type - char* _full_value_position; - uint64_t* _explicit_value; + char* _buf_ref = nullptr; // set + int _buf_len; // set len + HllDataType _set_type; //set type + char* _full_value_position = nullptr; + uint64_t* _explicit_value = nullptr; ExplicitLengthValueType _explicit_num; std::map _sparse_map; SparseLengthValueType* _sparse_count; diff --git a/be/src/olap/inverted_index_parser.h b/be/src/olap/inverted_index_parser.h index 54455bddef..bf931a3ce4 100644 --- a/be/src/olap/inverted_index_parser.h +++ b/be/src/olap/inverted_index_parser.h @@ -44,7 +44,7 @@ struct InvertedIndexCtx { InvertedIndexParserType parser_type; std::string parser_mode; CharFilterMap char_filter_map; - lucene::analysis::Analyzer* analyzer; + lucene::analysis::Analyzer* analyzer = nullptr; }; using InvertedIndexCtxSPtr = std::shared_ptr; diff --git a/be/src/olap/iterators.h b/be/src/olap/iterators.h index f1b195f8f9..e2d1e67957 100644 --- a/be/src/olap/iterators.h +++ b/be/src/olap/iterators.h @@ -56,11 +56,11 @@ public: include_upper(include_upper_) {} // the lower bound of the range, nullptr if not existed - const RowCursor* lower_key; + const RowCursor* lower_key = nullptr; // whether `lower_key` is included in the range bool include_lower; // the upper bound of the range, nullptr if not existed - const RowCursor* upper_key; + const RowCursor* upper_key = nullptr; // whether `upper_key` is included in the range bool include_upper; }; diff --git a/be/src/olap/like_column_predicate.h b/be/src/olap/like_column_predicate.h index 1449815686..7034efddd9 100644 --- a/be/src/olap/like_column_predicate.h +++ b/be/src/olap/like_column_predicate.h @@ -163,7 +163,7 @@ private: using StateType = vectorized::LikeState; StringRef pattern; - StateType* _state; + StateType* _state = nullptr; // A separate scratch region is required for every concurrent caller of the // Hyperscan API. So here _like_state is separate for each instance of diff --git a/be/src/olap/lru_cache.h b/be/src/olap/lru_cache.h index a682f422e7..b07f3c31aa 100644 --- a/be/src/olap/lru_cache.h +++ b/be/src/olap/lru_cache.h @@ -143,7 +143,7 @@ private: return result; } - const char* _data; + const char* _data = nullptr; size_t _size; }; @@ -239,7 +239,7 @@ private: // An entry is a variable length heap-allocated structure. Entries // are kept in a circular doubly linked list ordered by access time. struct LRUHandle { - void* value; + void* value = nullptr; void (*deleter)(const CacheKey&, void* value); struct LRUHandle* next_hash = nullptr; // next entry in hash table struct LRUHandle* next = nullptr; // next entry in lru list @@ -307,7 +307,7 @@ private: // a linked list of cache entries that hash into the bucket. uint32_t _length; uint32_t _elems; - LRUHandle** _list; + LRUHandle** _list = nullptr; // Return a pointer to slot that points to a cache entry that // matches key/hash. If there is no such cache entry, return a @@ -430,12 +430,12 @@ private: std::string _name; const int _num_shard_bits; const uint32_t _num_shards; - LRUCache** _shards; + LRUCache** _shards = nullptr; std::atomic _last_id; size_t _total_capacity; std::unique_ptr _mem_tracker; - std::shared_ptr _entity = nullptr; + std::shared_ptr _entity; IntGauge* cache_capacity = nullptr; IntGauge* cache_usage = nullptr; DoubleGauge* cache_usage_ratio = nullptr; diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h index a5289dbaf6..04ad022c82 100644 --- a/be/src/olap/memtable.h +++ b/be/src/olap/memtable.h @@ -49,8 +49,8 @@ enum KeysType : int; // row pos in _input_mutable_block struct RowInBlock { size_t _row_pos; - char* _agg_mem; - size_t* _agg_state_offset; + char* _agg_mem = nullptr; + size_t* _agg_state_offset = nullptr; bool _has_init_agg; RowInBlock(size_t row) : _row_pos(row), _has_init_agg(false) {} @@ -136,8 +136,8 @@ public: int operator()(const RowInBlock* left, const RowInBlock* right) const; private: - const TabletSchema* _tablet_schema; - vectorized::MutableBlock* _pblock; // corresponds to Memtable::_input_mutable_block + const TabletSchema* _tablet_schema = nullptr; + vectorized::MutableBlock* _pblock = nullptr; // corresponds to Memtable::_input_mutable_block }; class MemTableStat { @@ -207,7 +207,7 @@ private: bool _enable_unique_key_mow = false; bool _is_partial_update = false; const KeysType _keys_type; - const TabletSchema* _tablet_schema; + const TabletSchema* _tablet_schema = nullptr; std::shared_ptr _vec_row_comparator; diff --git a/be/src/olap/memtable_flush_executor.h b/be/src/olap/memtable_flush_executor.h index d2039ce812..a025bf43cf 100644 --- a/be/src/olap/memtable_flush_executor.h +++ b/be/src/olap/memtable_flush_executor.h @@ -91,7 +91,7 @@ private: FlushStatistic _stats; - RowsetWriter* _rowset_writer; + RowsetWriter* _rowset_writer = nullptr; MemTableStat _memtable_stat; }; diff --git a/be/src/olap/page_cache.h b/be/src/olap/page_cache.h index c5c4e99099..2d93574455 100644 --- a/be/src/olap/page_cache.h +++ b/be/src/olap/page_cache.h @@ -65,7 +65,7 @@ public: } private: - char* _data; + char* _data = nullptr; // Effective size, smaller than capacity, such as data page remove checksum suffix. size_t _size; size_t _capacity = 0; @@ -167,12 +167,12 @@ private: StoragePageCache(); int32_t _index_cache_percentage = 0; - std::unique_ptr _data_page_cache = nullptr; - std::unique_ptr _index_page_cache = nullptr; + std::unique_ptr _data_page_cache; + std::unique_ptr _index_page_cache; // Cache data for primary key index data page, seperated from data // page cache to make it for flexible. we need this cache When construct // delete bitmap in unique key with mow - std::unique_ptr _pk_index_page_cache = nullptr; + std::unique_ptr _pk_index_page_cache; Cache* _get_page_cache(segment_v2::PageTypePB page_type) { switch (page_type) { diff --git a/be/src/olap/push_handler.h b/be/src/olap/push_handler.h index 46e8d47b10..f7e16a7094 100644 --- a/be/src/olap/push_handler.h +++ b/be/src/olap/push_handler.h @@ -113,7 +113,7 @@ private: bool _ready; bool _eof; int _next_range; - vectorized::Block* _src_block_ptr; + vectorized::Block* _src_block_ptr = nullptr; vectorized::Block _src_block; const TDescriptorTable& _t_desc_tbl; std::unordered_map _name_to_col_type; @@ -126,10 +126,10 @@ private: std::vector _src_slot_descs; std::unique_ptr _row_desc; - const TupleDescriptor* _dest_tuple_desc; + const TupleDescriptor* _dest_tuple_desc = nullptr; std::unique_ptr _runtime_state; - RuntimeProfile* _runtime_profile; + RuntimeProfile* _runtime_profile = nullptr; std::unique_ptr _cur_reader; bool _cur_reader_eof; const TBrokerScanRangeParams& _params; diff --git a/be/src/olap/reader.h b/be/src/olap/reader.h index 044eff7808..a6cbd58460 100644 --- a/be/src/olap/reader.h +++ b/be/src/olap/reader.h @@ -138,7 +138,7 @@ public: std::vector rs_splits; // For unique key table with merge-on-write - DeleteBitmap* delete_bitmap {nullptr}; + DeleteBitmap* delete_bitmap = nullptr; // return_columns is init from query schema std::vector return_columns; diff --git a/be/src/olap/row_cursor.h b/be/src/olap/row_cursor.h index e9c58f1eb6..9f39b5d4f7 100644 --- a/be/src/olap/row_cursor.h +++ b/be/src/olap/row_cursor.h @@ -137,7 +137,7 @@ private: char* _variable_buf = nullptr; size_t _variable_len; size_t _string_field_count; - char** _long_text_buf; + char** _long_text_buf = nullptr; DISALLOW_COPY_AND_ASSIGN(RowCursor); }; diff --git a/be/src/olap/row_cursor_cell.h b/be/src/olap/row_cursor_cell.h index 10ef938fce..f874dba14a 100644 --- a/be/src/olap/row_cursor_cell.h +++ b/be/src/olap/row_cursor_cell.h @@ -30,7 +30,7 @@ struct RowCursorCell { void* mutable_cell_ptr() const { return (char*)_ptr + 1; } private: - void* _ptr; + void* _ptr = nullptr; }; } // namespace doris \ No newline at end of file diff --git a/be/src/olap/rowset/beta_rowset_reader.h b/be/src/olap/rowset/beta_rowset_reader.h index 7f5abb42a2..45aef03595 100644 --- a/be/src/olap/rowset/beta_rowset_reader.h +++ b/be/src/olap/rowset/beta_rowset_reader.h @@ -98,11 +98,11 @@ private: std::pair _segment_offsets; SchemaSPtr _input_schema; - RowsetReaderContext* _read_context; + RowsetReaderContext* _read_context = nullptr; BetaRowsetSharedPtr _rowset; OlapReaderStatistics _owned_stats; - OlapReaderStatistics* _stats; + OlapReaderStatistics* _stats = nullptr; std::unique_ptr _iterator; diff --git a/be/src/olap/rowset/pending_rowset_helper.h b/be/src/olap/rowset/pending_rowset_helper.h index 4852de5ee7..53d1f4f16c 100644 --- a/be/src/olap/rowset/pending_rowset_helper.h +++ b/be/src/olap/rowset/pending_rowset_helper.h @@ -56,7 +56,7 @@ private: explicit PendingRowsetGuard(const RowsetId& rowset_id, PendingRowsetSet* set); RowsetId _rowset_id; - PendingRowsetSet* _pending_rowset_set {nullptr}; + PendingRowsetSet* _pending_rowset_set = nullptr; }; // Pending rowsets refer to those rowsets that are under construction, and have not been added to diff --git a/be/src/olap/rowset/segcompaction.h b/be/src/olap/rowset/segcompaction.h index 1c314ba2a9..a0f81e59c7 100644 --- a/be/src/olap/rowset/segcompaction.h +++ b/be/src/olap/rowset/segcompaction.h @@ -75,7 +75,7 @@ private: private: //TODO(zhengyu): current impl depends heavily on the access to feilds of BetaRowsetWriter - BetaRowsetWriter* _writer; + BetaRowsetWriter* _writer = nullptr; io::FileWriterPtr _file_writer; std::atomic _cancelled = false; }; diff --git a/be/src/olap/rowset/segment_creator.h b/be/src/olap/rowset/segment_creator.h index cf5456a28a..054a416e07 100644 --- a/be/src/olap/rowset/segment_creator.h +++ b/be/src/olap/rowset/segment_creator.h @@ -58,7 +58,7 @@ public: } private: - T* _t; + T* _t = nullptr; }; class SegmentCollector { @@ -78,7 +78,7 @@ public: } private: - T* _t; + T* _t = nullptr; }; class SegmentFlusher { @@ -119,7 +119,7 @@ public: private: Writer(SegmentFlusher* flusher, std::unique_ptr& segment_writer); - SegmentFlusher* _flusher; + SegmentFlusher* _flusher = nullptr; std::unique_ptr _writer; }; diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.h b/be/src/olap/rowset/segment_v2/binary_dict_page.h index 0ab50ce634..1592a927e5 100644 --- a/be/src/olap/rowset/segment_v2/binary_dict_page.h +++ b/be/src/olap/rowset/segment_v2/binary_dict_page.h @@ -85,7 +85,8 @@ private: std::unique_ptr _data_page_builder; - std::unique_ptr> _dict_builder; + std::unique_ptr> _dict_builder = + nullptr; EncodingTypePB _encoding_type; struct HashOfSlice { diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_reader.h b/be/src/olap/rowset/segment_v2/bitmap_index_reader.h index 0277003fe1..9753972583 100644 --- a/be/src/olap/rowset/segment_v2/bitmap_index_reader.h +++ b/be/src/olap/rowset/segment_v2/bitmap_index_reader.h @@ -65,7 +65,7 @@ private: friend class BitmapIndexIterator; io::FileReaderSPtr _file_reader; - const TypeInfo* _type_info; + const TypeInfo* _type_info = nullptr; bool _has_null = false; DorisCallOnce _load_once; std::unique_ptr _dict_column_reader; @@ -112,7 +112,7 @@ public: rowid_t current_ordinal() const { return _current_rowid; } private: - BitmapIndexReader* _reader; + BitmapIndexReader* _reader = nullptr; IndexedColumnIterator _dict_column_iter; IndexedColumnIterator _bitmap_column_iter; rowid_t _current_rowid; diff --git a/be/src/olap/rowset/segment_v2/bloom_filter.h b/be/src/olap/rowset/segment_v2/bloom_filter.h index 969d7b8aa9..cd57181cdb 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter.h +++ b/be/src/olap/rowset/segment_v2/bloom_filter.h @@ -172,7 +172,7 @@ public: protected: // bloom filter data // specially add one byte for null flag - char* _data; + char* _data = nullptr; // optimal bloom filter num bytes // it is calculated by optimal_bit_num() / 8 uint32_t _num_bytes; @@ -180,7 +180,7 @@ protected: // last byte is for has_null flag uint32_t _size; // last byte's pointer in data for null flag - bool* _has_null; + bool* _has_null = nullptr; private: std::function _hash_func; diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h index 5e4c848dcb..c2617ef4e4 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h @@ -64,8 +64,8 @@ private: io::FileReaderSPtr _file_reader; DorisCallOnce _load_once; - const TypeInfo* _type_info; - std::unique_ptr _bloom_filter_index_meta; + const TypeInfo* _type_info = nullptr; + std::unique_ptr _bloom_filter_index_meta = nullptr; std::unique_ptr _bloom_filter_reader; }; @@ -80,7 +80,7 @@ public: size_t current_bloom_filter_index() const { return _bloom_filter_iter.get_current_ordinal(); } private: - BloomFilterIndexReader* _reader; + BloomFilterIndexReader* _reader = nullptr; IndexedColumnIterator _bloom_filter_iter; }; diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h index f9fba7e860..dc4707e6e0 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h @@ -90,7 +90,7 @@ public: private: BloomFilterOptions _bf_options; - const TypeInfo* _type_info; + const TypeInfo* _type_info = nullptr; vectorized::Arena _arena; bool _has_null; uint64_t _bf_buffer_size; diff --git a/be/src/olap/rowset/segment_v2/column_reader.h b/be/src/olap/rowset/segment_v2/column_reader.h index 3a792eb762..fbd72dbd33 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.h +++ b/be/src/olap/rowset/segment_v2/column_reader.h @@ -363,10 +363,10 @@ private: Status _read_data_page(const OrdinalPageIndexIterator& iter); Status _read_dict_data(); - ColumnReader* _reader; + ColumnReader* _reader = nullptr; // iterator owned compress codec, should NOT be shared by threads, initialized in init() - BlockCompressionCodec* _compress_codec; + BlockCompressionCodec* _compress_codec = nullptr; // 1. The _page represents current page. // 2. We define an operation is one seek and following read, @@ -464,7 +464,7 @@ public: } private: - ColumnReader* _map_reader; + ColumnReader* _map_reader = nullptr; std::unique_ptr _null_iterator; std::unique_ptr _offsets_iterator; //OffsetFileIterator std::unique_ptr _key_iterator; @@ -502,7 +502,7 @@ public: } private: - ColumnReader* _struct_reader; + ColumnReader* _struct_reader = nullptr; std::unique_ptr _null_iterator; std::vector> _sub_column_iterators; }; @@ -537,7 +537,7 @@ public: } private: - ColumnReader* _array_reader; + ColumnReader* _array_reader = nullptr; std::unique_ptr _offset_iterator; std::unique_ptr _null_iterator; std::unique_ptr _item_iterator; diff --git a/be/src/olap/rowset/segment_v2/column_writer.h b/be/src/olap/rowset/segment_v2/column_writer.h index 67cefc3c9c..77a08607df 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.h +++ b/be/src/olap/rowset/segment_v2/column_writer.h @@ -51,7 +51,7 @@ struct ColumnWriterOptions { // input and output parameter: // - input: column_id/unique_id/type/length/encoding/compression/is_nullable members // - output: encoding/indexes/dict_page members - ColumnMetaPB* meta; + ColumnMetaPB* meta = nullptr; size_t data_page_size = 64 * 1024; // store compressed page only when space saving is above the threshold. // space saving = 1 - compressed_size / uncompressed_size diff --git a/be/src/olap/rowset/segment_v2/encoding_info.h b/be/src/olap/rowset/segment_v2/encoding_info.h index a706683ff5..c6f065a28e 100644 --- a/be/src/olap/rowset/segment_v2/encoding_info.h +++ b/be/src/olap/rowset/segment_v2/encoding_info.h @@ -84,7 +84,7 @@ private: FieldType _type; EncodingTypePB _encoding; - std::unique_ptr _data_page_pre_decoder = nullptr; + std::unique_ptr _data_page_pre_decoder; }; } // namespace segment_v2 diff --git a/be/src/olap/rowset/segment_v2/index_page.h b/be/src/olap/rowset/segment_v2/index_page.h index c889d0fb16..7b15ef6639 100644 --- a/be/src/olap/rowset/segment_v2/index_page.h +++ b/be/src/olap/rowset/segment_v2/index_page.h @@ -147,7 +147,7 @@ public: const PagePointer& current_page_pointer() const { return _reader->get_value(_pos); } private: - const IndexPageReader* _reader; + const IndexPageReader* _reader = nullptr; size_t _pos; }; diff --git a/be/src/olap/rowset/segment_v2/indexed_column_reader.h b/be/src/olap/rowset/segment_v2/indexed_column_reader.h index 6399176e15..ef6e275460 100644 --- a/be/src/olap/rowset/segment_v2/indexed_column_reader.h +++ b/be/src/olap/rowset/segment_v2/indexed_column_reader.h @@ -134,7 +134,7 @@ public: private: Status _read_data_page(const PagePointer& pp); - const IndexedColumnReader* _reader; + const IndexedColumnReader* _reader = nullptr; // iterator for ordinal index page IndexPageIterator _ordinal_iter; // iterator for value index page diff --git a/be/src/olap/rowset/segment_v2/indexed_column_writer.h b/be/src/olap/rowset/segment_v2/indexed_column_writer.h index ecb26782ad..f85fc81b0c 100644 --- a/be/src/olap/rowset/segment_v2/indexed_column_writer.h +++ b/be/src/olap/rowset/segment_v2/indexed_column_writer.h @@ -94,8 +94,8 @@ private: Status _flush_index(IndexPageBuilder* index_builder, BTreeMetaPB* meta); IndexedColumnWriterOptions _options; - const TypeInfo* _type_info; - io::FileWriter* _file_writer; + const TypeInfo* _type_info = nullptr; + io::FileWriter* _file_writer = nullptr; // only used for `_first_value` vectorized::Arena _arena; @@ -115,8 +115,8 @@ private: // builder for index pages of value index, null if write_value_index == false std::unique_ptr _value_index_builder; // encoder for value index's key - const KeyCoder* _value_key_coder; - BlockCompressionCodec* _compress_codec; + const KeyCoder* _value_key_coder = nullptr; + BlockCompressionCodec* _compress_codec = nullptr; DISALLOW_COPY_AND_ASSIGN(IndexedColumnWriter); }; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_cache.h b/be/src/olap/rowset/segment_v2/inverted_index_cache.h index 79439ac462..48a30ecebc 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_cache.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_cache.h @@ -128,7 +128,7 @@ private: Cache::Handle* _insert(const InvertedIndexSearcherCache::CacheKey& key, CacheValue* value); private: - std::unique_ptr _mem_tracker = nullptr; + std::unique_ptr _mem_tracker; }; using IndexCacheValuePtr = std::unique_ptr; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.h b/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.h index f0d068198a..a0ebb05973 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.h @@ -53,7 +53,7 @@ public: int64_t bufferLength); private: - CL_NS(store)::Directory* directory; + CL_NS(store)::Directory* directory = nullptr; }; class CLUCENE_EXPORT DorisCompoundDirectory : public lucene::store::Directory { @@ -127,13 +127,13 @@ class DorisCompoundDirectory::FSIndexInput : public lucene::store::BufferedIndex io::FileReaderSPtr _reader; uint64_t _length; int64_t _fpos; - std::mutex* _shared_lock; + std::mutex* _shared_lock = nullptr; char path[4096]; SharedHandle(const char* path); ~SharedHandle() override; }; - SharedHandle* _handle; + SharedHandle* _handle = nullptr; int64_t _pos; FSIndexInput(SharedHandle* handle, int32_t buffer_size) : BufferedIndexInput(buffer_size) { diff --git a/be/src/olap/rowset/segment_v2/inverted_index_compound_reader.h b/be/src/olap/rowset/segment_v2/inverted_index_compound_reader.h index 75bf1ab633..1a98b44e3f 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_compound_reader.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_compound_reader.h @@ -52,18 +52,18 @@ private: int32_t readBufferSize; // base info - lucene::store::Directory* dir; - lucene::store::RAMDirectory* ram_dir; + lucene::store::Directory* dir = nullptr; + lucene::store::RAMDirectory* ram_dir = nullptr; std::string directory; std::string file_name; - CL_NS(store)::IndexInput* stream; + CL_NS(store)::IndexInput* stream = nullptr; using EntriesType = lucene::util::CLHashMap>; - EntriesType* entries; + EntriesType* entries = nullptr; std::mutex _this_lock; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.h b/be/src/olap/rowset/segment_v2/inverted_index_reader.h index 20c5c731f9..9e94061d29 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.h @@ -180,10 +180,10 @@ public: class InvertedIndexVisitor : public lucene::util::bkd::bkd_reader::intersect_visitor { private: - roaring::Roaring* _hits; + roaring::Roaring* _hits = nullptr; uint32_t _num_hits; bool _only_count; - lucene::util::bkd::bkd_reader* _reader; + lucene::util::bkd::bkd_reader* _reader = nullptr; InvertedIndexQueryType _query_type; public: diff --git a/be/src/olap/rowset/segment_v2/ordinal_page_index.h b/be/src/olap/rowset/segment_v2/ordinal_page_index.h index 648ed2c6b3..77f0b85181 100644 --- a/be/src/olap/rowset/segment_v2/ordinal_page_index.h +++ b/be/src/olap/rowset/segment_v2/ordinal_page_index.h @@ -131,7 +131,7 @@ public: ordinal_t last_ordinal() const { return _index->get_last_ordinal(_cur_idx); } private: - OrdinalIndexReader* _index; + OrdinalIndexReader* _index = nullptr; int32_t _cur_idx; }; diff --git a/be/src/olap/rowset/segment_v2/parsed_page.h b/be/src/olap/rowset/segment_v2/parsed_page.h index df63c543bb..399ec7b360 100644 --- a/be/src/olap/rowset/segment_v2/parsed_page.h +++ b/be/src/olap/rowset/segment_v2/parsed_page.h @@ -83,7 +83,7 @@ struct ParsedPage { bool has_null; Slice null_bitmap; RleDecoder null_decoder; - std::unique_ptr data_decoder = nullptr; + std::unique_ptr data_decoder; // ordinal of the first value in this page ordinal_t first_ordinal = 0; diff --git a/be/src/olap/rowset/segment_v2/rle_page.h b/be/src/olap/rowset/segment_v2/rle_page.h index 102b6d297e..e199f37bc5 100644 --- a/be/src/olap/rowset/segment_v2/rle_page.h +++ b/be/src/olap/rowset/segment_v2/rle_page.h @@ -139,7 +139,7 @@ private: size_t _count; bool _finished; int _bit_width; - RleEncoder* _rle_encoder; + RleEncoder* _rle_encoder = nullptr; faststring _buf; CppType _first_value; CppType _last_value; diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h index 32d1336915..219ca61ee1 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -401,7 +401,7 @@ private: _column_pred_in_remaining_vconjunct; std::set _not_apply_index_pred; - std::shared_ptr _runtime_predicate {nullptr}; + std::shared_ptr _runtime_predicate; // row schema of the key to seek // only used in `_get_row_ranges_by_keys` diff --git a/be/src/olap/rowset/segment_v2/segment_writer.h b/be/src/olap/rowset/segment_v2/segment_writer.h index 37011fdf0f..682aa55d01 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.h +++ b/be/src/olap/rowset/segment_v2/segment_writer.h @@ -177,12 +177,12 @@ private: uint32_t _segment_id; TabletSchemaSPtr _tablet_schema; BaseTabletSPtr _tablet; - DataDir* _data_dir; + DataDir* _data_dir = nullptr; uint32_t _max_row_per_segment; SegmentWriterOptions _opts; // Not owned. owned by RowsetWriter - io::FileWriter* _file_writer; + io::FileWriter* _file_writer = nullptr; SegmentFooterPB _footer; size_t _num_key_columns; diff --git a/be/src/olap/rowset/segment_v2/vertical_segment_writer.h b/be/src/olap/rowset/segment_v2/vertical_segment_writer.h index 773751934b..44c30018f0 100644 --- a/be/src/olap/rowset/segment_v2/vertical_segment_writer.h +++ b/be/src/olap/rowset/segment_v2/vertical_segment_writer.h @@ -149,11 +149,11 @@ private: uint32_t _segment_id; TabletSchemaSPtr _tablet_schema; BaseTabletSPtr _tablet; - DataDir* _data_dir; + DataDir* _data_dir = nullptr; VerticalSegmentWriterOptions _opts; // Not owned. owned by RowsetWriter - io::FileWriter* _file_writer; + io::FileWriter* _file_writer = nullptr; SegmentFooterPB _footer; size_t _num_key_columns; diff --git a/be/src/olap/rowset/segment_v2/zone_map_index.h b/be/src/olap/rowset/segment_v2/zone_map_index.h index d7dcc7d0c7..eeb87eb7da 100644 --- a/be/src/olap/rowset/segment_v2/zone_map_index.h +++ b/be/src/olap/rowset/segment_v2/zone_map_index.h @@ -130,7 +130,7 @@ private: zone_map->pass_all = false; } - Field* _field; + Field* _field = nullptr; // memory will be managed by Arena ZoneMap _page_zone_map; ZoneMap _segment_zone_map; diff --git a/be/src/olap/segment_loader.h b/be/src/olap/segment_loader.h index d6b1d07940..a53ff4bfc9 100644 --- a/be/src/olap/segment_loader.h +++ b/be/src/olap/segment_loader.h @@ -118,7 +118,7 @@ public: private: SegmentLoader(); - std::unique_ptr _segment_cache = nullptr; + std::unique_ptr _segment_cache; }; // A handle for a single rowset from segment lru cache. diff --git a/be/src/olap/selection_vector.h b/be/src/olap/selection_vector.h index 6896c37a0d..1c5f52999a 100644 --- a/be/src/olap/selection_vector.h +++ b/be/src/olap/selection_vector.h @@ -158,7 +158,7 @@ public: } private: - SelectionVector* _sel_vec; + SelectionVector* _sel_vec = nullptr; size_t _row_offset; }; diff --git a/be/src/olap/short_key_index.h b/be/src/olap/short_key_index.h index a5ed018f72..79303a3edb 100644 --- a/be/src/olap/short_key_index.h +++ b/be/src/olap/short_key_index.h @@ -123,7 +123,7 @@ public: ssize_t ordinal() const { return _ordinal; } private: - const ShortKeyIndexDecoder* _decoder; + const ShortKeyIndexDecoder* _decoder = nullptr; ssize_t _ordinal; }; diff --git a/be/src/olap/skiplist.h b/be/src/olap/skiplist.h index 477a3e1e12..d8e3335e50 100644 --- a/be/src/olap/skiplist.h +++ b/be/src/olap/skiplist.h @@ -59,7 +59,7 @@ public: // Note: The user should guarantee that there must not be any other insertion // between calling Find() and InsertWithHint(). struct Hint { - Node* curr; + Node* curr = nullptr; Node* prev[kMaxHeight]; }; @@ -114,8 +114,8 @@ public: void SeekToLast(); private: - const SkipList* list_; - Node* node_; + const SkipList* list_ = nullptr; + Node* node_ = nullptr; // Intentionally copyable }; diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index 95b11e0c66..d71a823f55 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -435,7 +435,7 @@ private: // Type of new loaded data RowsetTypePB _default_rowset_type; - HeartbeatFlags* _heartbeat_flags; + HeartbeatFlags* _heartbeat_flags = nullptr; std::unique_ptr _base_compaction_thread_pool; std::unique_ptr _cumu_compaction_thread_pool; diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index 222435ab99..4174a0ec26 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -612,7 +612,7 @@ public: static const int64_t K_INVALID_CUMULATIVE_POINT = -1; private: - DataDir* _data_dir; + DataDir* _data_dir = nullptr; TimestampedVersionTracker _timestamped_version_tracker; DorisCallOnce _init_once; diff --git a/be/src/olap/task/engine_clone_task.h b/be/src/olap/task/engine_clone_task.h index f98561148e..9c79e5cadf 100644 --- a/be/src/olap/task/engine_clone_task.h +++ b/be/src/olap/task/engine_clone_task.h @@ -88,7 +88,7 @@ private: private: const TCloneReq& _clone_req; - vector* _tablet_infos; + vector* _tablet_infos = nullptr; int64_t _signature; const TMasterInfo& _master_info; int64_t _copy_size; diff --git a/be/src/olap/task/engine_publish_version_task.h b/be/src/olap/task/engine_publish_version_task.h index 8f3790574a..ba738e0699 100644 --- a/be/src/olap/task/engine_publish_version_task.h +++ b/be/src/olap/task/engine_publish_version_task.h @@ -71,7 +71,7 @@ public: void handle(); private: - EnginePublishVersionTask* _engine_publish_version_task; + EnginePublishVersionTask* _engine_publish_version_task = nullptr; TabletSharedPtr _tablet; RowsetSharedPtr _rowset; @@ -103,10 +103,10 @@ private: const TPublishVersionRequest& _publish_version_req; std::mutex _tablet_ids_mutex; - std::set* _error_tablet_ids; + std::set* _error_tablet_ids = nullptr; std::map* _succ_tablets; - std::vector>* _discontinuous_version_tablets; - std::map* _table_id_to_num_delta_rows; + std::vector>* _discontinuous_version_tablets = nullptr; + std::map* _table_id_to_num_delta_rows = nullptr; }; class AsyncTabletPublishTask { diff --git a/be/src/olap/task/engine_storage_migration_task.h b/be/src/olap/task/engine_storage_migration_task.h index ce80fbe4e8..2831fc11df 100644 --- a/be/src/olap/task/engine_storage_migration_task.h +++ b/be/src/olap/task/engine_storage_migration_task.h @@ -78,7 +78,7 @@ private: // tablet to do migrated TabletSharedPtr _tablet; // destination data dir - DataDir* _dest_store; + DataDir* _dest_store = nullptr; int64_t _task_start_time; std::vector _pending_rs_guards; }; // EngineTask diff --git a/be/src/olap/txn_manager.h b/be/src/olap/txn_manager.h index d994596ecb..73ef0cf6e8 100644 --- a/be/src/olap/txn_manager.h +++ b/be/src/olap/txn_manager.h @@ -227,20 +227,20 @@ private: const int32_t _txn_shard_size; // _txn_map_locks[i] protect _txn_tablet_maps[i], i=0,1,2...,and i < _txn_map_shard_size - txn_tablet_map_t* _txn_tablet_maps; + txn_tablet_map_t* _txn_tablet_maps = nullptr; // transaction_id -> corresponding partition ids // This is mainly for the clear txn task received from FE, which may only has transaction id, // so we need this map to find out which partitions are corresponding to a transaction id. // The _txn_partition_maps[i] should be constructed/deconstructed/modified alongside with '_txn_tablet_maps[i]' - txn_partition_map_t* _txn_partition_maps; + txn_partition_map_t* _txn_partition_maps = nullptr; - std::shared_mutex* _txn_map_locks; + std::shared_mutex* _txn_map_locks = nullptr; - std::shared_mutex* _txn_mutex; + std::shared_mutex* _txn_mutex = nullptr; - txn_tablet_delta_writer_map_t* _txn_tablet_delta_writer_map; - ShardedLRUCache* _tablet_version_cache; - std::shared_mutex* _txn_tablet_delta_writer_map_locks; + txn_tablet_delta_writer_map_t* _txn_tablet_delta_writer_map = nullptr; + ShardedLRUCache* _tablet_version_cache = nullptr; + std::shared_mutex* _txn_tablet_delta_writer_map_locks = nullptr; DISALLOW_COPY_AND_ASSIGN(TxnManager); }; // TxnManager diff --git a/be/src/olap/wal_manager.h b/be/src/olap/wal_manager.h index 2cfaaa4ec5..4634916c60 100644 --- a/be/src/olap/wal_manager.h +++ b/be/src/olap/wal_manager.h @@ -72,7 +72,7 @@ public: Status get_wal_column_index(int64_t wal_id, std::vector& column_index); private: - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; std::shared_mutex _lock; scoped_refptr _replay_thread; CountDownLatch _stop_background_threads_latch; diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h index ec4bbe3bc7..7c5483e1e1 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.h +++ b/be/src/pipeline/exec/aggregation_sink_operator.h @@ -298,21 +298,21 @@ protected: int _get_slot_column_id(const vectorized::AggFnEvaluator* evaluator); size_t _memory_usage() const; - RuntimeProfile::Counter* _hash_table_compute_timer; - RuntimeProfile::Counter* _hash_table_emplace_timer; - RuntimeProfile::Counter* _hash_table_input_counter; - RuntimeProfile::Counter* _build_timer; - RuntimeProfile::Counter* _expr_timer; - RuntimeProfile::Counter* _exec_timer; - RuntimeProfile::Counter* _build_table_convert_timer; - RuntimeProfile::Counter* _serialize_key_timer; - RuntimeProfile::Counter* _merge_timer; - RuntimeProfile::Counter* _serialize_data_timer; - RuntimeProfile::Counter* _deserialize_data_timer; - RuntimeProfile::Counter* _max_row_size_counter; - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::Counter* _hash_table_memory_usage; - RuntimeProfile::HighWaterMarkCounter* _serialize_key_arena_memory_usage; + RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; + RuntimeProfile::Counter* _hash_table_emplace_timer = nullptr; + RuntimeProfile::Counter* _hash_table_input_counter = nullptr; + RuntimeProfile::Counter* _build_timer = nullptr; + RuntimeProfile::Counter* _expr_timer = nullptr; + RuntimeProfile::Counter* _exec_timer = nullptr; + RuntimeProfile::Counter* _build_table_convert_timer = nullptr; + RuntimeProfile::Counter* _serialize_key_timer = nullptr; + RuntimeProfile::Counter* _merge_timer = nullptr; + RuntimeProfile::Counter* _serialize_data_timer = nullptr; + RuntimeProfile::Counter* _deserialize_data_timer = nullptr; + RuntimeProfile::Counter* _max_row_size_counter = nullptr; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; + RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; + RuntimeProfile::HighWaterMarkCounter* _serialize_key_arena_memory_usage = nullptr; bool _should_limit_output = false; bool _reach_limit = false; @@ -322,8 +322,8 @@ protected: vectorized::Block _preagg_block = vectorized::Block(); - vectorized::AggregatedDataVariants* _agg_data; - vectorized::Arena* _agg_arena_pool; + vectorized::AggregatedDataVariants* _agg_data = nullptr; + vectorized::Arena* _agg_arena_pool = nullptr; using vectorized_execute = std::function; using vectorized_update_memusage = std::function; @@ -381,10 +381,10 @@ protected: // may be we don't have to know the tuple id TupleId _intermediate_tuple_id; - TupleDescriptor* _intermediate_tuple_desc; + TupleDescriptor* _intermediate_tuple_desc = nullptr; TupleId _output_tuple_id; - TupleDescriptor* _output_tuple_desc; + TupleDescriptor* _output_tuple_desc = nullptr; bool _needs_finalize; bool _is_merge; @@ -399,7 +399,7 @@ protected: size_t _external_agg_bytes_threshold; // group by k1,k2 vectorized::VExprContextSPtrs _probe_expr_ctxs; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; std::vector _make_nullable_keys; size_t _spill_partition_count_bits; int64_t _limit; // -1: no limit diff --git a/be/src/pipeline/exec/aggregation_source_operator.h b/be/src/pipeline/exec/aggregation_source_operator.h index 9418301f15..c30277cbec 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.h +++ b/be/src/pipeline/exec/aggregation_source_operator.h @@ -121,12 +121,12 @@ protected: } } - RuntimeProfile::Counter* _get_results_timer; - RuntimeProfile::Counter* _serialize_result_timer; - RuntimeProfile::Counter* _hash_table_iterate_timer; - RuntimeProfile::Counter* _insert_keys_to_column_timer; - RuntimeProfile::Counter* _serialize_data_timer; - RuntimeProfile::Counter* _hash_table_size_counter; + RuntimeProfile::Counter* _get_results_timer = nullptr; + RuntimeProfile::Counter* _serialize_result_timer = nullptr; + RuntimeProfile::Counter* _hash_table_iterate_timer = nullptr; + RuntimeProfile::Counter* _insert_keys_to_column_timer = nullptr; + RuntimeProfile::Counter* _serialize_data_timer = nullptr; + RuntimeProfile::Counter* _hash_table_size_counter = nullptr; using vectorized_get_result = std::function; @@ -137,7 +137,7 @@ protected: executor _executor; - vectorized::AggregatedDataVariants* _agg_data; + vectorized::AggregatedDataVariants* _agg_data = nullptr; }; class AggSourceOperatorX : public OperatorX { diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index 1e8152a28f..19acfc4d54 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -84,9 +84,9 @@ private: bool need_check_first = false); bool _whether_need_next_partition(vectorized::BlockRowPos& found_partition_end); - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::Counter* _evaluation_timer; - RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; + RuntimeProfile::Counter* _evaluation_timer = nullptr; + RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage = nullptr; std::vector _agg_expr_ctxs; }; diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h index 8b76fbfe26..b9674b908c 100644 --- a/be/src/pipeline/exec/analytic_source_operator.h +++ b/be/src/pipeline/exec/analytic_source_operator.h @@ -120,9 +120,9 @@ private: std::unique_ptr _agg_arena_pool; std::vector _agg_functions; - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::Counter* _evaluation_timer; - RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; + RuntimeProfile::Counter* _evaluation_timer = nullptr; + RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage = nullptr; using vectorized_execute = std::function; @@ -169,8 +169,8 @@ private: vectorized::AnalyticFnScope _fn_scope; - TupleDescriptor* _intermediate_tuple_desc; - TupleDescriptor* _output_tuple_desc; + TupleDescriptor* _intermediate_tuple_desc = nullptr; + TupleDescriptor* _output_tuple_desc = nullptr; /// The offset of the n-th functions. std::vector _offsets_of_aggregate_states; diff --git a/be/src/pipeline/exec/datagen_operator.h b/be/src/pipeline/exec/datagen_operator.h index 613955ae81..1b8ef2977f 100644 --- a/be/src/pipeline/exec/datagen_operator.h +++ b/be/src/pipeline/exec/datagen_operator.h @@ -84,7 +84,7 @@ private: TupleId _tuple_id; // Descriptor of tuples generated - const TupleDescriptor* _tuple_desc; + const TupleDescriptor* _tuple_desc = nullptr; std::vector _runtime_filter_descs; }; diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h index c156a83e1d..b30a829872 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h @@ -96,7 +96,7 @@ private: const size_t num_rows); std::unique_ptr _output_block = vectorized::Block::create_unique(); - std::shared_ptr dummy_mapped_data = nullptr; + std::shared_ptr dummy_mapped_data; vectorized::IColumn::Selector _distinct_row; vectorized::Arena _arena; int64_t _output_distinct_rows = 0; diff --git a/be/src/pipeline/exec/es_scan_operator.h b/be/src/pipeline/exec/es_scan_operator.h index 6f02008383..dbdbbe198b 100644 --- a/be/src/pipeline/exec/es_scan_operator.h +++ b/be/src/pipeline/exec/es_scan_operator.h @@ -61,12 +61,12 @@ private: #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wshadow-field" #endif - RuntimeProfile::Counter* _rows_read_counter; + RuntimeProfile::Counter* _rows_read_counter = nullptr; #ifdef __clang__ #pragma clang diagnostic pop #endif - RuntimeProfile::Counter* _read_timer; - RuntimeProfile::Counter* _materialize_timer; + RuntimeProfile::Counter* _read_timer = nullptr; + RuntimeProfile::Counter* _materialize_timer = nullptr; }; class EsScanOperatorX final : public ScanOperatorX { @@ -81,7 +81,7 @@ private: friend class EsScanLocalState; TupleId _tuple_id; - TupleDescriptor* _tuple_desc; + TupleDescriptor* _tuple_desc = nullptr; std::map _properties; std::map _fields_context; diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index d59872e2a1..c60beb9bcd 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -88,14 +88,14 @@ public: private: AtomicWrapper _ref_count; PBlock pblock; - pipeline::BroadcastDependency* _dep; + pipeline::BroadcastDependency* _dep = nullptr; }; } // namespace vectorized namespace pipeline { template struct TransmitInfo { - vectorized::PipChannel* channel; + vectorized::PipChannel* channel = nullptr; std::unique_ptr block; bool eos; Status exec_status; @@ -103,8 +103,8 @@ struct TransmitInfo { template struct BroadcastTransmitInfo { - vectorized::PipChannel* channel; - vectorized::BroadcastPBlockHolder* block_holder; + vectorized::PipChannel* channel = nullptr; + vectorized::BroadcastPBlockHolder* block_holder = nullptr; bool eos; }; @@ -164,11 +164,11 @@ private: std::function _suc_fn; InstanceLoId _id; bool _eos; - vectorized::BroadcastPBlockHolder* _data; + vectorized::BroadcastPBlockHolder* _data = nullptr; }; struct ExchangeRpcContext { - std::shared_ptr> _send_callback = nullptr; + std::shared_ptr> _send_callback; bool is_cancelled = false; }; @@ -235,7 +235,7 @@ private: int _sender_id; int _be_number; std::atomic _rpc_count = 0; - QueryContext* _context; + QueryContext* _context = nullptr; Status _send_rpc(InstanceLoId); // must hold the _instance_to_package_queue_mutex[id] mutex to opera @@ -249,8 +249,8 @@ private: std::atomic _total_queue_size = 0; static constexpr int QUEUE_CAPACITY_FACTOR = 64; - std::shared_ptr _queue_dependency = nullptr; - std::shared_ptr _finish_dependency = nullptr; + std::shared_ptr _queue_dependency; + std::shared_ptr _finish_dependency; QueryStatistics* _statistics = nullptr; std::atomic _should_stop {false}; }; diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index 751c2768e8..3450292888 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -231,9 +231,9 @@ private: vectorized::BlockSerializer _serializer; - std::shared_ptr _queue_dependency = nullptr; - std::shared_ptr _exchange_sink_dependency = nullptr; - std::shared_ptr _broadcast_dependency = nullptr; + std::shared_ptr _queue_dependency; + std::shared_ptr _exchange_sink_dependency; + std::shared_ptr _broadcast_dependency; std::vector> _local_channels_dependency; std::unique_ptr _partitioner; int _partition_count; diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index b0618d5992..34c3147b7f 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -112,33 +112,33 @@ protected: int64_t _build_side_mem_used = 0; int64_t _build_side_last_mem_used = 0; vectorized::MutableBlock _build_side_mutable_block; - std::shared_ptr _runtime_filter_slots = nullptr; + std::shared_ptr _runtime_filter_slots; bool _has_set_need_null_map_for_build = false; bool _build_side_ignore_null = false; size_t _build_rf_cardinality = 0; std::unordered_map> _inserted_rows; std::shared_ptr _shared_hash_table_dependency; - RuntimeProfile::Counter* _build_table_timer; - RuntimeProfile::Counter* _build_expr_call_timer; - RuntimeProfile::Counter* _build_table_insert_timer; - RuntimeProfile::Counter* _build_table_expanse_timer; - RuntimeProfile::Counter* _build_table_convert_timer; - RuntimeProfile::Counter* _build_buckets_counter; - RuntimeProfile::Counter* _build_buckets_fill_counter; + RuntimeProfile::Counter* _build_table_timer = nullptr; + RuntimeProfile::Counter* _build_expr_call_timer = nullptr; + RuntimeProfile::Counter* _build_table_insert_timer = nullptr; + RuntimeProfile::Counter* _build_table_expanse_timer = nullptr; + RuntimeProfile::Counter* _build_table_convert_timer = nullptr; + RuntimeProfile::Counter* _build_buckets_counter = nullptr; + RuntimeProfile::Counter* _build_buckets_fill_counter = nullptr; - RuntimeProfile::Counter* _build_side_compute_hash_timer; - RuntimeProfile::Counter* _build_side_merge_block_timer; - RuntimeProfile::Counter* _build_runtime_filter_timer; + RuntimeProfile::Counter* _build_side_compute_hash_timer = nullptr; + RuntimeProfile::Counter* _build_side_merge_block_timer = nullptr; + RuntimeProfile::Counter* _build_runtime_filter_timer = nullptr; - RuntimeProfile::Counter* _build_collisions_counter; + RuntimeProfile::Counter* _build_collisions_counter = nullptr; - RuntimeProfile::Counter* _allocate_resource_timer; + RuntimeProfile::Counter* _allocate_resource_timer = nullptr; - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::Counter* _build_blocks_memory_usage; - RuntimeProfile::Counter* _hash_table_memory_usage; - RuntimeProfile::HighWaterMarkCounter* _build_arena_memory_usage; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; + RuntimeProfile::Counter* _build_blocks_memory_usage = nullptr; + RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; + RuntimeProfile::HighWaterMarkCounter* _build_arena_memory_usage = nullptr; }; class HashJoinBuildSinkOperatorX final @@ -177,7 +177,7 @@ private: std::vector _is_null_safe_eq_join; bool _is_broadcast_join = false; - std::shared_ptr _shared_hashtable_controller = nullptr; + std::shared_ptr _shared_hashtable_controller; vectorized::SharedHashTableContextPtr _shared_hash_table_context = nullptr; std::vector _runtime_filter_descs; diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h index 583bba1b00..59a1057b3a 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.h +++ b/be/src/pipeline/exec/hashjoin_probe_operator.h @@ -139,14 +139,14 @@ private: vectorized::HashTableIteratorVariants _outer_join_pull_visited_iter; vectorized::HashTableIteratorVariants _probe_row_match_iter; - RuntimeProfile::Counter* _probe_expr_call_timer; - RuntimeProfile::Counter* _probe_next_timer; - RuntimeProfile::Counter* _probe_side_output_timer; - RuntimeProfile::Counter* _probe_process_hashtable_timer; - RuntimeProfile::HighWaterMarkCounter* _probe_arena_memory_usage; - RuntimeProfile::Counter* _search_hashtable_timer; - RuntimeProfile::Counter* _build_side_output_timer; - RuntimeProfile::Counter* _process_other_join_conjunct_timer; + RuntimeProfile::Counter* _probe_expr_call_timer = nullptr; + RuntimeProfile::Counter* _probe_next_timer = nullptr; + RuntimeProfile::Counter* _probe_side_output_timer = nullptr; + RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; + RuntimeProfile::HighWaterMarkCounter* _probe_arena_memory_usage = nullptr; + RuntimeProfile::Counter* _search_hashtable_timer = nullptr; + RuntimeProfile::Counter* _build_side_output_timer = nullptr; + RuntimeProfile::Counter* _process_other_join_conjunct_timer = nullptr; }; class HashJoinProbeOperatorX final : public JoinProbeOperatorX { diff --git a/be/src/pipeline/exec/join_build_sink_operator.h b/be/src/pipeline/exec/join_build_sink_operator.h index 9034057658..d339c2a977 100644 --- a/be/src/pipeline/exec/join_build_sink_operator.h +++ b/be/src/pipeline/exec/join_build_sink_operator.h @@ -40,9 +40,9 @@ protected: template friend class JoinBuildSinkOperatorX; - RuntimeProfile::Counter* _build_rows_counter; - RuntimeProfile::Counter* _push_down_timer; - RuntimeProfile::Counter* _push_compute_timer; + RuntimeProfile::Counter* _build_rows_counter = nullptr; + RuntimeProfile::Counter* _push_down_timer = nullptr; + RuntimeProfile::Counter* _push_compute_timer = nullptr; }; template diff --git a/be/src/pipeline/exec/join_probe_operator.h b/be/src/pipeline/exec/join_probe_operator.h index 9c874b0b67..67537e65ca 100644 --- a/be/src/pipeline/exec/join_probe_operator.h +++ b/be/src/pipeline/exec/join_probe_operator.h @@ -53,10 +53,10 @@ protected: vectorized::MutableColumnPtr _tuple_is_null_left_flag_column; vectorized::MutableColumnPtr _tuple_is_null_right_flag_column; - RuntimeProfile::Counter* _probe_timer; - RuntimeProfile::Counter* _probe_rows_counter; - RuntimeProfile::Counter* _join_filter_timer; - RuntimeProfile::Counter* _build_output_block_timer; + RuntimeProfile::Counter* _probe_timer = nullptr; + RuntimeProfile::Counter* _probe_rows_counter = nullptr; + RuntimeProfile::Counter* _join_filter_timer = nullptr; + RuntimeProfile::Counter* _build_output_block_timer = nullptr; std::unique_ptr _child_block; SourceState _child_source_state; diff --git a/be/src/pipeline/exec/multi_cast_data_streamer.h b/be/src/pipeline/exec/multi_cast_data_streamer.h index 973b364410..5e4179e0ca 100644 --- a/be/src/pipeline/exec/multi_cast_data_streamer.h +++ b/be/src/pipeline/exec/multi_cast_data_streamer.h @@ -85,7 +85,7 @@ private: void _block_reading(int sender_idx); const RowDescriptor& _row_desc; - RuntimeProfile* _profile; + RuntimeProfile* _profile = nullptr; std::list _multi_cast_blocks; std::vector::iterator> _sender_pos_to_read; std::mutex _mutex; @@ -94,8 +94,8 @@ private: int _closed_sender_count = 0; int64_t _cumulative_mem_size = 0; - RuntimeProfile::Counter* _process_rows; - RuntimeProfile::Counter* _peak_mem_usage; + RuntimeProfile::Counter* _process_rows = nullptr; + RuntimeProfile::Counter* _peak_mem_usage = nullptr; std::vector _dependencies; }; diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.h b/be/src/pipeline/exec/nested_loop_join_probe_operator.h index d7a9b54e89..86905c1418 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.h @@ -207,7 +207,7 @@ private: uint64_t _output_null_idx_build_side = 0; vectorized::VExprContextSPtrs _join_conjuncts; - RuntimeProfile::Counter* _loop_join_timer; + RuntimeProfile::Counter* _loop_join_timer = nullptr; }; class NestedLoopJoinProbeOperatorX final diff --git a/be/src/pipeline/exec/olap_table_sink_operator.h b/be/src/pipeline/exec/olap_table_sink_operator.h index ad35f79728..9330f9520e 100644 --- a/be/src/pipeline/exec/olap_table_sink_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_operator.h @@ -108,7 +108,7 @@ private: vectorized::VExprContextSPtrs _output_vexpr_ctxs; const std::vector& _t_output_expr; const bool _group_commit; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; }; } // namespace pipeline diff --git a/be/src/pipeline/exec/olap_table_sink_v2_operator.h b/be/src/pipeline/exec/olap_table_sink_v2_operator.h index 5fb8f64dd3..d8f7c0b792 100644 --- a/be/src/pipeline/exec/olap_table_sink_v2_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_v2_operator.h @@ -110,7 +110,7 @@ private: vectorized::VExprContextSPtrs _output_vexpr_ctxs; const std::vector& _t_output_expr; const bool _group_commit; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; }; } // namespace pipeline diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index d733064875..a006cda943 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -142,7 +142,7 @@ public: NodeType* exec_node() const { return _node; } protected: - NodeType* _node; + NodeType* _node = nullptr; }; template @@ -160,7 +160,7 @@ public: SinkType* exec_node() const { return _sink; } protected: - SinkType* _sink; + SinkType* _sink = nullptr; }; class OperatorBase { @@ -265,7 +265,7 @@ public: [[nodiscard]] virtual RuntimeProfile* get_runtime_profile() const = 0; protected: - OperatorBuilderBase* _operator_builder; + OperatorBuilderBase* _operator_builder = nullptr; OperatorPtr _child; // Used on pipeline X @@ -323,7 +323,7 @@ public: } protected: - NodeType* _sink; + NodeType* _sink = nullptr; }; /** @@ -400,7 +400,7 @@ public: } protected: - NodeType* _node; + NodeType* _node = nullptr; bool _use_projection; }; diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.h b/be/src/pipeline/exec/partition_sort_sink_operator.h index 62bf7c16b4..aa82cbd233 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.h +++ b/be/src/pipeline/exec/partition_sort_sink_operator.h @@ -82,11 +82,11 @@ private: std::unique_ptr _agg_arena_pool; int _partition_exprs_num = 0; - RuntimeProfile::Counter* _build_timer; - RuntimeProfile::Counter* _emplace_key_timer; - RuntimeProfile::Counter* _selector_block_timer; + RuntimeProfile::Counter* _build_timer = nullptr; + RuntimeProfile::Counter* _emplace_key_timer = nullptr; + RuntimeProfile::Counter* _selector_block_timer = nullptr; - RuntimeProfile::Counter* _hash_table_size_counter; + RuntimeProfile::Counter* _hash_table_size_counter = nullptr; void _init_hash_method(); }; @@ -108,7 +108,7 @@ public: private: friend class PartitionSortSinkLocalState; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; const RowDescriptor _row_descriptor; int64_t _limit = -1; int _partition_exprs_num = 0; diff --git a/be/src/pipeline/exec/repeat_operator.h b/be/src/pipeline/exec/repeat_operator.h index 18d373b77d..03196a22bd 100644 --- a/be/src/pipeline/exec/repeat_operator.h +++ b/be/src/pipeline/exec/repeat_operator.h @@ -67,7 +67,7 @@ private: SourceState _child_source_state; bool _child_eos; int _repeat_id_idx; - std::unique_ptr _intermediate_block {}; + std::unique_ptr _intermediate_block; vectorized::VExprContextSPtrs _expr_ctxs; }; @@ -98,7 +98,7 @@ private: std::vector _repeat_id_list; std::vector> _grouping_list; TupleId _output_tuple_id; - const TupleDescriptor* _output_tuple_desc; + const TupleDescriptor* _output_tuple_desc = nullptr; std::vector _output_slots; diff --git a/be/src/pipeline/exec/result_file_sink_operator.h b/be/src/pipeline/exec/result_file_sink_operator.h index 63217aad04..9c0ada27a1 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.h +++ b/be/src/pipeline/exec/result_file_sink_operator.h @@ -71,18 +71,18 @@ private: template void _handle_eof_channel(RuntimeState* state, ChannelPtrType channel, Status st); - std::unique_ptr _output_block = nullptr; + std::unique_ptr _output_block; std::shared_ptr _sender; std::vector*> _channels; bool _only_local_exchange = false; vectorized::BlockSerializer _serializer; std::unique_ptr _block_holder; - RuntimeProfile::Counter* _brpc_wait_timer; - RuntimeProfile::Counter* _local_send_timer; - RuntimeProfile::Counter* _brpc_send_timer; - RuntimeProfile::Counter* _merge_block_timer; - RuntimeProfile::Counter* _split_block_distribute_by_channel_timer; + RuntimeProfile::Counter* _brpc_wait_timer = nullptr; + RuntimeProfile::Counter* _local_send_timer = nullptr; + RuntimeProfile::Counter* _brpc_send_timer = nullptr; + RuntimeProfile::Counter* _merge_block_timer = nullptr; + RuntimeProfile::Counter* _split_block_distribute_by_channel_timer = nullptr; int _sender_id; }; diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index eedd2d4c05..e3d56f176e 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -93,7 +93,7 @@ private: Status _second_phase_fetch_data(RuntimeState* state, vectorized::Block* final_block); TResultSinkType::type _sink_type; // set file options when sink type is FILE - std::unique_ptr _file_opts; + std::unique_ptr _file_opts = nullptr; // Owned by the RuntimeState. const RowDescriptor& _row_desc; diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h index cdbcdddcae..8ad88b3375 100644 --- a/be/src/pipeline/exec/scan_operator.h +++ b/be/src/pipeline/exec/scan_operator.h @@ -91,7 +91,7 @@ public: void set_scanner_ctx(vectorized::ScannerContext* scanner_ctx) { _scanner_ctx = scanner_ctx; } private: - vectorized::ScannerContext* _scanner_ctx; + vectorized::ScannerContext* _scanner_ctx = nullptr; std::atomic _scanner_done {false}; }; @@ -154,15 +154,15 @@ protected: RuntimeProfile::Counter* _convert_block_timer = nullptr; // time of filter output block from scanner RuntimeProfile::Counter* _filter_timer = nullptr; - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::HighWaterMarkCounter* _queued_blocks_memory_usage; - RuntimeProfile::HighWaterMarkCounter* _free_blocks_memory_usage; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; + RuntimeProfile::HighWaterMarkCounter* _queued_blocks_memory_usage = nullptr; + RuntimeProfile::HighWaterMarkCounter* _free_blocks_memory_usage = nullptr; // rows read from the scanner (including those discarded by (pre)filters) - RuntimeProfile::Counter* _rows_read_counter; + RuntimeProfile::Counter* _rows_read_counter = nullptr; // Wall based aggregate read throughput [rows/sec] - RuntimeProfile::Counter* _total_throughput_counter; - RuntimeProfile::Counter* _num_scanners; + RuntimeProfile::Counter* _total_throughput_counter = nullptr; + RuntimeProfile::Counter* _num_scanners = nullptr; RuntimeProfile::Counter* _wait_for_data_timer = nullptr; RuntimeProfile::Counter* _wait_for_scanner_done_timer = nullptr; diff --git a/be/src/pipeline/exec/schema_scan_operator.h b/be/src/pipeline/exec/schema_scan_operator.h index e7479a75e9..fdd3cb5931 100644 --- a/be/src/pipeline/exec/schema_scan_operator.h +++ b/be/src/pipeline/exec/schema_scan_operator.h @@ -66,7 +66,7 @@ private: friend class SchemaScanOperatorX; SchemaScannerParam _scanner_param; - std::unique_ptr _schema_scanner = nullptr; + std::unique_ptr _schema_scanner; }; class SchemaScanOperatorX final : public OperatorX { @@ -94,13 +94,13 @@ private: TupleId _tuple_id; // Descriptor of dest tuples - const TupleDescriptor* _dest_tuple_desc; + const TupleDescriptor* _dest_tuple_desc = nullptr; // Tuple index in tuple row. int _tuple_idx; // slot num need to fill in and return int _slot_num; - std::unique_ptr _schema_scanner = nullptr; + std::unique_ptr _schema_scanner; }; } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/set_sink_operator.h b/be/src/pipeline/exec/set_sink_operator.h index b8ca789b78..7c4cc7d2cb 100644 --- a/be/src/pipeline/exec/set_sink_operator.h +++ b/be/src/pipeline/exec/set_sink_operator.h @@ -57,7 +57,7 @@ public: bool can_write() override { return true; } private: - vectorized::VSetOperationNode* _set_node; + vectorized::VSetOperationNode* _set_node = nullptr; }; class SetSinkDependency final : public Dependency { diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index 8730780b54..b82871f943 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -72,7 +72,7 @@ private: // Expressions and parameters used for build _sort_description vectorized::VSortExecExprs _vsort_exec_exprs; - RuntimeProfile::Counter* _memory_usage_counter; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; // topn top value vectorized::Field old_top {vectorized::Field::Types::Null}; @@ -99,7 +99,7 @@ private: // Number of rows to skip. const int64_t _offset; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; // Expressions and parameters used for build _sort_description vectorized::VSortExecExprs _vsort_exec_exprs; diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h index 54898864bb..182946188e 100644 --- a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h @@ -66,8 +66,8 @@ public: private: vectorized::Block _preagg_block = vectorized::Block(); - RuntimeProfile::Counter* _queue_byte_size_counter; - RuntimeProfile::Counter* _queue_size_counter; + RuntimeProfile::Counter* _queue_byte_size_counter = nullptr; + RuntimeProfile::Counter* _queue_size_counter = nullptr; std::shared_ptr _data_queue; }; @@ -104,9 +104,9 @@ private: } } - RuntimeProfile::Counter* _queue_byte_size_counter; - RuntimeProfile::Counter* _queue_size_counter; - RuntimeProfile::Counter* _streaming_agg_timer; + RuntimeProfile::Counter* _queue_byte_size_counter = nullptr; + RuntimeProfile::Counter* _queue_size_counter = nullptr; + RuntimeProfile::Counter* _streaming_agg_timer = nullptr; bool _should_expand_hash_table = true; int64_t _num_rows_returned = 0; diff --git a/be/src/pipeline/pipeline_fragment_context.h b/be/src/pipeline/pipeline_fragment_context.h index 39e3dcbe16..00045ee802 100644 --- a/be/src/pipeline/pipeline_fragment_context.h +++ b/be/src/pipeline/pipeline_fragment_context.h @@ -201,8 +201,8 @@ protected: std::shared_ptr _merge_controller_handler; MonotonicStopWatch _fragment_watcher; - RuntimeProfile::Counter* _start_timer; - RuntimeProfile::Counter* _prepare_timer; + RuntimeProfile::Counter* _start_timer = nullptr; + RuntimeProfile::Counter* _prepare_timer = nullptr; std::function _call_back; bool _is_fragment_instance_closed = false; @@ -218,7 +218,7 @@ protected: // profile reporting-related report_status_callback _report_status_cb; - DescriptorTbl* _desc_tbl; + DescriptorTbl* _desc_tbl = nullptr; private: static bool _has_inverted_index_or_partial_update(TOlapTableSink sink); diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h index 026c11430c..67329fd0e3 100644 --- a/be/src/pipeline/pipeline_task.h +++ b/be/src/pipeline/pipeline_task.h @@ -282,13 +282,13 @@ protected: bool _prepared; bool _opened; - RuntimeState* _state; + RuntimeState* _state = nullptr; int _previous_schedule_id = -1; uint32_t _schedule_time = 0; PipelineTaskState _cur_state; SourceState _data_state; std::unique_ptr _block; - PipelineFragmentContext* _fragment_context; + PipelineFragmentContext* _fragment_context = nullptr; TaskQueue* _task_queue = nullptr; // used for priority queue @@ -304,34 +304,34 @@ protected: bool _try_close_flag = false; - RuntimeProfile* _parent_profile; + RuntimeProfile* _parent_profile = nullptr; std::unique_ptr _task_profile; - RuntimeProfile::Counter* _task_cpu_timer; - RuntimeProfile::Counter* _prepare_timer; - RuntimeProfile::Counter* _open_timer; - RuntimeProfile::Counter* _exec_timer; - RuntimeProfile::Counter* _get_block_timer; - RuntimeProfile::Counter* _get_block_counter; - RuntimeProfile::Counter* _sink_timer; - RuntimeProfile::Counter* _close_timer; - RuntimeProfile::Counter* _block_counts; - RuntimeProfile::Counter* _block_by_source_counts; - RuntimeProfile::Counter* _block_by_sink_counts; - RuntimeProfile::Counter* _schedule_counts; + RuntimeProfile::Counter* _task_cpu_timer = nullptr; + RuntimeProfile::Counter* _prepare_timer = nullptr; + RuntimeProfile::Counter* _open_timer = nullptr; + RuntimeProfile::Counter* _exec_timer = nullptr; + RuntimeProfile::Counter* _get_block_timer = nullptr; + RuntimeProfile::Counter* _get_block_counter = nullptr; + RuntimeProfile::Counter* _sink_timer = nullptr; + RuntimeProfile::Counter* _close_timer = nullptr; + RuntimeProfile::Counter* _block_counts = nullptr; + RuntimeProfile::Counter* _block_by_source_counts = nullptr; + RuntimeProfile::Counter* _block_by_sink_counts = nullptr; + RuntimeProfile::Counter* _schedule_counts = nullptr; MonotonicStopWatch _wait_source_watcher; - RuntimeProfile::Counter* _wait_source_timer; + RuntimeProfile::Counter* _wait_source_timer = nullptr; MonotonicStopWatch _wait_bf_watcher; - RuntimeProfile::Counter* _wait_bf_timer; - RuntimeProfile::Counter* _wait_bf_counts; + RuntimeProfile::Counter* _wait_bf_timer = nullptr; + RuntimeProfile::Counter* _wait_bf_counts = nullptr; MonotonicStopWatch _wait_sink_watcher; - RuntimeProfile::Counter* _wait_sink_timer; + RuntimeProfile::Counter* _wait_sink_timer = nullptr; MonotonicStopWatch _wait_worker_watcher; - RuntimeProfile::Counter* _wait_worker_timer; - RuntimeProfile::Counter* _wait_dependency_counts; - RuntimeProfile::Counter* _pending_finish_counts; + RuntimeProfile::Counter* _wait_worker_timer = nullptr; + RuntimeProfile::Counter* _wait_dependency_counts = nullptr; + RuntimeProfile::Counter* _pending_finish_counts = nullptr; // TODO we should calculate the time between when really runnable and runnable - RuntimeProfile::Counter* _yield_counts; - RuntimeProfile::Counter* _core_change_times; + RuntimeProfile::Counter* _yield_counts = nullptr; + RuntimeProfile::Counter* _core_change_times = nullptr; // The monotonic time of the entire lifecycle of the pipelinetask, almost synchronized with the pipfragmentctx // There are several important time points: @@ -343,26 +343,26 @@ protected: MonotonicStopWatch _pipeline_task_watcher; // time 1 bool _is_first_time_to_execute = false; - RuntimeProfile::Counter* _begin_execute_timer; + RuntimeProfile::Counter* _begin_execute_timer = nullptr; int64_t _begin_execute_time = 0; // time 2 bool _is_eos = false; - RuntimeProfile::Counter* _eos_timer; + RuntimeProfile::Counter* _eos_timer = nullptr; int64_t _eos_time = 0; //time 3 bool _is_src_pending_finish_over = false; - RuntimeProfile::Counter* _src_pending_finish_over_timer; + RuntimeProfile::Counter* _src_pending_finish_over_timer = nullptr; int64_t _src_pending_finish_over_time = 0; // time 4 bool _is_dst_pending_finish_over = false; - RuntimeProfile::Counter* _dst_pending_finish_over_timer; + RuntimeProfile::Counter* _dst_pending_finish_over_timer = nullptr; int64_t _dst_pending_finish_over_time = 0; // time 5 bool _is_close_pipeline = false; - RuntimeProfile::Counter* _close_pipeline_timer; + RuntimeProfile::Counter* _close_pipeline_timer = nullptr; int64_t _close_pipeline_time = 0; - RuntimeProfile::Counter* _pip_task_total_timer; + RuntimeProfile::Counter* _pip_task_total_timer = nullptr; std::shared_ptr _query_statistics; Status _collect_query_statistics(); bool _collect_query_statistics_with_every_batch = false; diff --git a/be/src/pipeline/pipeline_x/dependency.h b/be/src/pipeline/pipeline_x/dependency.h index cd5fef95c5..a26b7de781 100644 --- a/be/src/pipeline/pipeline_x/dependency.h +++ b/be/src/pipeline/pipeline_x/dependency.h @@ -55,8 +55,8 @@ static constexpr auto TIME_UNIT_DEPENDENCY_LOG = 30 * 1000L * 1000L * 1000L; static_assert(TIME_UNIT_DEPENDENCY_LOG < SLOW_DEPENDENCY_THRESHOLD); struct BasicSharedState { - Dependency* source_dep; - Dependency* sink_dep; + Dependency* source_dep = nullptr; + Dependency* sink_dep = nullptr; std::atomic ref_count = 0; @@ -126,9 +126,9 @@ protected: const std::string _name; const bool _is_write_dependency; std::atomic _ready; - const QueryContext* _query_ctx; + const QueryContext* _query_ctx = nullptr; - std::shared_ptr _shared_state {nullptr}; + std::shared_ptr _shared_state; MonotonicStopWatch _watcher; std::list> _children; @@ -187,7 +187,7 @@ private: std::mutex _lock; const int64_t _registration_time; const int32_t _wait_time_ms; - IRuntimeFilter* _runtime_filter; + IRuntimeFilter* _runtime_filter = nullptr; }; class RuntimeFilterDependency final : public Dependency { @@ -262,7 +262,7 @@ public: return Status::OK(); } - vectorized::AggregatedDataVariantsUPtr agg_data; + vectorized::AggregatedDataVariantsUPtr agg_data = nullptr; std::unique_ptr aggregate_data_container; vectorized::AggSpillContext spill_context; vectorized::ArenaUPtr agg_arena_pool; @@ -273,7 +273,7 @@ public: size_t input_num_rows = 0; std::vector values; std::unique_ptr agg_profile_arena; - std::unique_ptr data_queue = nullptr; + std::unique_ptr data_queue; /// The total size of the row from the aggregate functions. size_t total_size_of_aggregate_states = 0; size_t align_aggregate_states = 1; @@ -398,7 +398,7 @@ struct HashJoinSharedState : public JoinSharedState { std::make_shared(); const std::vector build_side_child_desc; size_t build_exprs_size = 0; - std::shared_ptr> build_blocks = nullptr; + std::shared_ptr> build_blocks; bool probe_ignore_null = false; }; @@ -416,7 +416,7 @@ public: std::queue blocks_buffer; std::mutex buffer_mutex; std::vector> partition_sorts; - std::unique_ptr previous_row = nullptr; + std::unique_ptr previous_row; }; class AsyncWriterDependency final : public Dependency { @@ -445,7 +445,8 @@ public: //// shared static states (shared, decided in prepare/open...) /// init in setup_local_state - std::unique_ptr hash_table_variants; // the real data HERE. + std::unique_ptr hash_table_variants = + nullptr; // the real data HERE. std::vector build_not_ignore_null; /// init in both upstream side. diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h index 2dc71dec96..ed2dbfc3d5 100644 --- a/be/src/pipeline/pipeline_x/operator.h +++ b/be/src/pipeline/pipeline_x/operator.h @@ -33,7 +33,7 @@ namespace doris::pipeline { // This struct is used only for initializing local state. struct LocalStateInfo { - RuntimeProfile* parent_profile; + RuntimeProfile* parent_profile = nullptr; const std::vector scan_ranges; std::vector& upstream_dependencies; std::shared_ptr local_exchange_state; @@ -44,7 +44,7 @@ struct LocalStateInfo { // This struct is used only for initializing local sink state. struct LocalSinkStateInfo { - RuntimeProfile* parent_profile; + RuntimeProfile* parent_profile = nullptr; const int sender_id; std::vector& dependencys; std::shared_ptr local_exchange_state; @@ -108,7 +108,7 @@ public: protected: friend class OperatorXBase; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; int64_t _num_rows_returned {0}; std::unique_ptr _runtime_profile; @@ -117,20 +117,20 @@ protected: // which will providea reference for operator memory. std::unique_ptr _mem_tracker; - RuntimeProfile::Counter* _rows_returned_counter; - RuntimeProfile::Counter* _blocks_returned_counter; - RuntimeProfile::Counter* _wait_for_dependency_timer; - RuntimeProfile::Counter* _memory_used_counter; - RuntimeProfile::Counter* _wait_for_finish_dependency_timer; - RuntimeProfile::Counter* _projection_timer; - RuntimeProfile::Counter* _exec_timer; + RuntimeProfile::Counter* _rows_returned_counter = nullptr; + RuntimeProfile::Counter* _blocks_returned_counter = nullptr; + RuntimeProfile::Counter* _wait_for_dependency_timer = nullptr; + RuntimeProfile::Counter* _memory_used_counter = nullptr; + RuntimeProfile::Counter* _wait_for_finish_dependency_timer = nullptr; + RuntimeProfile::Counter* _projection_timer = nullptr; + RuntimeProfile::Counter* _exec_timer = nullptr; // Account for peak memory used by this node - RuntimeProfile::Counter* _peak_memory_usage_counter; + RuntimeProfile::Counter* _peak_memory_usage_counter = nullptr; RuntimeProfile::Counter* _open_timer = nullptr; RuntimeProfile::Counter* _close_timer = nullptr; - OperatorXBase* _parent; - RuntimeState* _state; + OperatorXBase* _parent = nullptr; + RuntimeState* _state = nullptr; vectorized::VExprContextSPtrs _conjuncts; vectorized::VExprContextSPtrs _projections; bool _closed = false; @@ -272,7 +272,7 @@ protected: const int _operator_id; const int _node_id; // unique w/in single plan tree TPlanNodeType::type _type; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; std::vector _tuple_ids; vectorized::VExprContextSPtrs _conjuncts; @@ -326,7 +326,7 @@ public: Dependency* dependency() override { return _dependency; } protected: - DependencyType* _dependency; + DependencyType* _dependency = nullptr; typename DependencyType::SharedState* _shared_state = nullptr; }; @@ -380,9 +380,9 @@ public: Dependency* finishdependency() { return _finish_dependency.get(); } protected: - DataSinkOperatorXBase* _parent; - RuntimeState* _state; - RuntimeProfile* _profile; + DataSinkOperatorXBase* _parent = nullptr; + RuntimeState* _state = nullptr; + RuntimeProfile* _profile = nullptr; std::unique_ptr _mem_tracker; // Maybe this will be transferred to BufferControlBlock. std::shared_ptr _query_statistics; @@ -396,12 +396,12 @@ protected: std::unique_ptr _faker_runtime_profile = std::make_unique("faker profile"); - RuntimeProfile::Counter* _rows_input_counter; + RuntimeProfile::Counter* _rows_input_counter = nullptr; RuntimeProfile::Counter* _open_timer = nullptr; RuntimeProfile::Counter* _close_timer = nullptr; - RuntimeProfile::Counter* _wait_for_dependency_timer; - RuntimeProfile::Counter* _wait_for_finish_dependency_timer; - RuntimeProfile::Counter* _exec_timer; + RuntimeProfile::Counter* _wait_for_dependency_timer = nullptr; + RuntimeProfile::Counter* _wait_for_finish_dependency_timer = nullptr; + RuntimeProfile::Counter* _exec_timer = nullptr; std::shared_ptr _finish_dependency; }; diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.h b/be/src/pipeline/pipeline_x/pipeline_x_task.h index dc86feec88..e4ee71914b 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.h @@ -213,9 +213,9 @@ private: int _task_idx; bool _dry_run = false; - Dependency* _blocked_dep {nullptr}; + Dependency* _blocked_dep = nullptr; - Dependency* _execution_dep {nullptr}; + Dependency* _execution_dep = nullptr; std::atomic _use_blocking_queue {true}; std::atomic _finished {false}; diff --git a/be/src/runtime/broker_mgr.h b/be/src/runtime/broker_mgr.h index d9238aed2c..deabc623de 100644 --- a/be/src/runtime/broker_mgr.h +++ b/be/src/runtime/broker_mgr.h @@ -44,7 +44,7 @@ private: void ping(const TNetworkAddress& addr); void ping_worker(); - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; std::string _client_id; std::mutex _mutex; std::unordered_set _broker_set; diff --git a/be/src/runtime/buffer_control_block.h b/be/src/runtime/buffer_control_block.h index c388cf7d50..f75008f101 100644 --- a/be/src/runtime/buffer_control_block.h +++ b/be/src/runtime/buffer_control_block.h @@ -175,7 +175,7 @@ private: void _update_batch_queue_empty() override; std::atomic_bool _batch_queue_empty {false}; - std::shared_ptr _result_sink_dependency {nullptr}; + std::shared_ptr _result_sink_dependency; }; } // namespace doris diff --git a/be/src/runtime/cache/result_cache.h b/be/src/runtime/cache/result_cache.h index a629e7ed7d..7473a42d91 100644 --- a/be/src/runtime/cache/result_cache.h +++ b/be/src/runtime/cache/result_cache.h @@ -58,8 +58,8 @@ public: size_t get_node_count() const { return _node_count; } private: - ResultNode* _head; - ResultNode* _tail; + ResultNode* _head = nullptr; + ResultNode* _tail = nullptr; size_t _node_count; }; diff --git a/be/src/runtime/cache/result_node.h b/be/src/runtime/cache/result_node.h index 31ab193219..377c958109 100644 --- a/be/src/runtime/cache/result_node.h +++ b/be/src/runtime/cache/result_node.h @@ -93,7 +93,7 @@ private: private: int64 _partition_key; - PCacheValue* _cache_value; + PCacheValue* _cache_value = nullptr; size_t _data_size; CacheStat _cache_stat; }; @@ -174,8 +174,8 @@ public: private: mutable std::shared_mutex _node_mtx; UniqueId _sql_key; - ResultNode* _prev; - ResultNode* _next; + ResultNode* _prev = nullptr; + ResultNode* _next = nullptr; size_t _data_size; PartitionRowBatchList _partition_list; PartitionRowBatchMap _partition_map; diff --git a/be/src/runtime/client_cache.h b/be/src/runtime/client_cache.h index ff45055ca6..d29db82786 100644 --- a/be/src/runtime/client_cache.h +++ b/be/src/runtime/client_cache.h @@ -124,10 +124,10 @@ private: std::shared_ptr _thrift_client_metric_entity; // Number of clients 'checked-out' from the cache - IntGauge* thrift_used_clients; + IntGauge* thrift_used_clients = nullptr; // Total clients in the cache, including those in use - IntGauge* thrift_opened_clients; + IntGauge* thrift_opened_clients = nullptr; // Create a new client for specific host/port in 'client' and put it in _client_map Status _create_client(const TNetworkAddress& hostport, ClientFactory& factory_method, @@ -192,7 +192,7 @@ public: T* operator->() const { return _client; } private: - ClientCache* _client_cache; + ClientCache* _client_cache = nullptr; T* _client; }; diff --git a/be/src/runtime/collection_value.h b/be/src/runtime/collection_value.h index 82dfc78fe5..2a4e903fc4 100644 --- a/be/src/runtime/collection_value.h +++ b/be/src/runtime/collection_value.h @@ -69,12 +69,12 @@ public: private: // child column data - void* _data; + void* _data = nullptr; uint64_t _length; // item has no null value if has_null is false. // item ```may``` has null value if has_null is true. bool _has_null; // null bitmap - bool* _null_signs; + bool* _null_signs = nullptr; }; } // namespace doris diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h index fbd233a18d..d1cee02169 100644 --- a/be/src/runtime/descriptors.h +++ b/be/src/runtime/descriptors.h @@ -399,7 +399,7 @@ private: friend class TabletSchema; const TupleId _id; - TableDescriptor* _table_desc; + TableDescriptor* _table_desc = nullptr; int64_t _byte_size; int _num_null_slots; int _num_null_bytes; diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 5fd43afce5..2bcf9b1b91 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -303,7 +303,7 @@ private: // Ideally, all threads are expected to attach to the specified tracker, so that "all memory has its own ownership", // and the consumption of the orphan mem tracker is close to 0, but greater than 0. std::shared_ptr _orphan_mem_tracker; - MemTrackerLimiter* _orphan_mem_tracker_raw; + MemTrackerLimiter* _orphan_mem_tracker_raw = nullptr; std::shared_ptr _experimental_mem_tracker; // page size not in cache, data page/index page/etc. std::shared_ptr _page_no_cache_mem_tracker; diff --git a/be/src/runtime/external_scan_context_mgr.h b/be/src/runtime/external_scan_context_mgr.h index 4925821f3a..b112ba34d4 100644 --- a/be/src/runtime/external_scan_context_mgr.h +++ b/be/src/runtime/external_scan_context_mgr.h @@ -63,7 +63,7 @@ public: Status clear_scan_context(const std::string& context_id); private: - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; std::map> _active_contexts; void gc_expired_context(); diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h index 08f95bd335..a20da9387a 100644 --- a/be/src/runtime/fragment_mgr.h +++ b/be/src/runtime/fragment_mgr.h @@ -171,7 +171,7 @@ private: std::shared_ptr& query_ctx); // This is input params - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; // The lock should only be used to protect the structures in fragment manager. Has to be // used in a very small scope because it may dead lock. For example, if the _lock is used @@ -196,11 +196,12 @@ private: // every job is a pool std::unique_ptr _thread_pool; - std::shared_ptr _entity = nullptr; + std::shared_ptr _entity; UIntGauge* timeout_canceled_fragment_count = nullptr; RuntimeFilterMergeController _runtimefilter_controller; - std::unique_ptr _async_report_thread_pool; // used for pipeliine context report + std::unique_ptr _async_report_thread_pool = + nullptr; // used for pipeliine context report }; } // namespace doris diff --git a/be/src/runtime/group_commit_mgr.h b/be/src/runtime/group_commit_mgr.h index ecd227f6fb..53ab6f6117 100644 --- a/be/src/runtime/group_commit_mgr.h +++ b/be/src/runtime/group_commit_mgr.h @@ -108,8 +108,8 @@ private: int64_t txn_id, const TUniqueId& instance_id, Status& status, bool prepare_failed, RuntimeState* state); - ExecEnv* _exec_env; - ThreadPool* _thread_pool; + ExecEnv* _exec_env = nullptr; + ThreadPool* _thread_pool = nullptr; int64_t _db_id; int64_t _table_id; std::mutex _lock; @@ -136,7 +136,7 @@ public: std::shared_ptr& load_block_queue); private: - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; std::mutex _lock; // TODO remove table when unused diff --git a/be/src/runtime/load_channel.h b/be/src/runtime/load_channel.h index 0e8fa4c135..03c1f64a9c 100644 --- a/be/src/runtime/load_channel.h +++ b/be/src/runtime/load_channel.h @@ -122,7 +122,7 @@ private: SpinLock _profile_serialize_lock; std::unique_ptr _profile; - RuntimeProfile* _self_profile; + RuntimeProfile* _self_profile = nullptr; RuntimeProfile::Counter* _add_batch_number_counter = nullptr; RuntimeProfile::Counter* _peak_memory_usage_counter = nullptr; RuntimeProfile::Counter* _add_batch_timer = nullptr; diff --git a/be/src/runtime/load_path_mgr.h b/be/src/runtime/load_path_mgr.h index 6c81b940a2..e079717095 100644 --- a/be/src/runtime/load_path_mgr.h +++ b/be/src/runtime/load_path_mgr.h @@ -61,7 +61,7 @@ private: void clean(); void process_path(time_t now, const std::string& path, int64_t reserve_hours); - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; std::mutex _lock; std::vector _path_vec; int _idx; diff --git a/be/src/runtime/load_stream.h b/be/src/runtime/load_stream.h index 2674e77bc3..cc06f2a2f7 100644 --- a/be/src/runtime/load_stream.h +++ b/be/src/runtime/load_stream.h @@ -61,7 +61,7 @@ private: std::shared_ptr _failed_st; PUniqueId _load_id; int64_t _txn_id; - RuntimeProfile* _profile; + RuntimeProfile* _profile = nullptr; RuntimeProfile::Counter* _append_data_timer = nullptr; RuntimeProfile::Counter* _add_segment_timer = nullptr; RuntimeProfile::Counter* _close_wait_timer = nullptr; @@ -92,7 +92,7 @@ private: int64_t _txn_id; std::shared_ptr _schema; std::unordered_map _tablet_partitions; - RuntimeProfile* _profile; + RuntimeProfile* _profile = nullptr; RuntimeProfile::Counter* _append_data_timer = nullptr; RuntimeProfile::Counter* _close_wait_timer = nullptr; LoadStreamMgr* _load_stream_mgr = nullptr; diff --git a/be/src/runtime/load_stream_mgr.h b/be/src/runtime/load_stream_mgr.h index debdd2f2fb..1228061bb4 100644 --- a/be/src/runtime/load_stream_mgr.h +++ b/be/src/runtime/load_stream_mgr.h @@ -56,8 +56,8 @@ private: std::unordered_map _load_streams_map; std::unique_ptr _file_writer_thread_pool; - FifoThreadPool* _heavy_work_pool; - FifoThreadPool* _light_work_pool; + FifoThreadPool* _heavy_work_pool = nullptr; + FifoThreadPool* _light_work_pool = nullptr; }; } // namespace doris diff --git a/be/src/runtime/load_stream_writer.h b/be/src/runtime/load_stream_writer.h index dc952d2c18..afdbdecdcc 100644 --- a/be/src/runtime/load_stream_writer.h +++ b/be/src/runtime/load_stream_writer.h @@ -83,7 +83,7 @@ private: bool _is_canceled = false; WriteRequest _req; RowsetBuilder _rowset_builder; - std::shared_ptr _rowset_writer = nullptr; + std::shared_ptr _rowset_writer; std::mutex _lock; std::unordered_map _segment_stat_map; diff --git a/be/src/runtime/map_value.h b/be/src/runtime/map_value.h index 26c46692bc..6167b40124 100644 --- a/be/src/runtime/map_value.h +++ b/be/src/runtime/map_value.h @@ -50,8 +50,8 @@ public: private: // child column data pointer - void* _key_data; - void* _value_data; + void* _key_data = nullptr; + void* _value_data = nullptr; // length for map size int32_t _length; diff --git a/be/src/runtime/memory/mem_tracker.h b/be/src/runtime/memory/mem_tracker.h index 3257ddd0eb..2e6e959541 100644 --- a/be/src/runtime/memory/mem_tracker.h +++ b/be/src/runtime/memory/mem_tracker.h @@ -169,7 +169,7 @@ protected: // label used in the make snapshot, not guaranteed unique. std::string _label; - std::shared_ptr _consumption; + std::shared_ptr _consumption = nullptr; // Tracker is located in group num in mem_tracker_pool int64_t _parent_group_num = 0; diff --git a/be/src/runtime/plan_fragment_executor.h b/be/src/runtime/plan_fragment_executor.h index a51f362f5f..68701755e5 100644 --- a/be/src/runtime/plan_fragment_executor.h +++ b/be/src/runtime/plan_fragment_executor.h @@ -153,8 +153,8 @@ public: Status update_status(Status status); private: - ExecEnv* _exec_env; // not owned - ExecNode* _plan; // lives in _runtime_state->obj_pool() + ExecEnv* _exec_env = nullptr; // not owned + ExecNode* _plan = nullptr; // lives in _runtime_state->obj_pool() std::shared_ptr _query_ctx; // Id of this instance TUniqueId _fragment_instance_id; @@ -211,12 +211,12 @@ private: std::unique_ptr _sink; // Number of rows returned by this fragment - RuntimeProfile::Counter* _rows_produced_counter; + RuntimeProfile::Counter* _rows_produced_counter = nullptr; // Number of blocks returned by this fragment - RuntimeProfile::Counter* _blocks_produced_counter; + RuntimeProfile::Counter* _blocks_produced_counter = nullptr; - RuntimeProfile::Counter* _fragment_cpu_timer; + RuntimeProfile::Counter* _fragment_cpu_timer = nullptr; std::shared_ptr _merge_controller_handler; @@ -240,7 +240,7 @@ private: bool _group_commit = false; - DescriptorTbl* _desc_tbl; + DescriptorTbl* _desc_tbl = nullptr; ObjectPool* obj_pool() { return _runtime_state->obj_pool(); } diff --git a/be/src/runtime/query_context.h b/be/src/runtime/query_context.h index 3a45afef27..426ea79804 100644 --- a/be/src/runtime/query_context.h +++ b/be/src/runtime/query_context.h @@ -48,8 +48,8 @@ struct ReportStatusRequest { bool is_pipeline_x; const Status status; std::vector runtime_states; - RuntimeProfile* profile; - RuntimeProfile* load_channel_profile; + RuntimeProfile* profile = nullptr; + RuntimeProfile* load_channel_profile = nullptr; bool done; TNetworkAddress coord_addr; TUniqueId query_id; @@ -198,7 +198,7 @@ public: pipeline::Dependency* get_execution_dependency() { return _execution_dependency.get(); } public: - DescriptorTbl* desc_tbl; + DescriptorTbl* desc_tbl = nullptr; bool set_rsc_info = false; std::string user; std::string group; @@ -228,7 +228,7 @@ public: private: TUniqueId _query_id; - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; VecDateTimeValue _start_time; // A token used to submit olap scanner to the "_limited_scan_thread_pool", diff --git a/be/src/runtime/routine_load/routine_load_task_executor.h b/be/src/runtime/routine_load/routine_load_task_executor.h index 6714ce6902..e4ad8be592 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.h +++ b/be/src/runtime/routine_load/routine_load_task_executor.h @@ -80,7 +80,7 @@ private: std::shared_ptr ctx); private: - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; PriorityThreadPool _thread_pool; DataConsumerPool _data_consumer_pool; diff --git a/be/src/runtime/runtime_filter_mgr.h b/be/src/runtime/runtime_filter_mgr.h index 2048229cd2..ff089a7a1f 100644 --- a/be/src/runtime/runtime_filter_mgr.h +++ b/be/src/runtime/runtime_filter_mgr.h @@ -97,7 +97,7 @@ public: private: struct ConsumerFilterHolder { int node_id; - IRuntimeFilter* filter; + IRuntimeFilter* filter = nullptr; }; // RuntimeFilterMgr is owned by RuntimeState, so we only // use filter_id as key @@ -106,8 +106,8 @@ private: std::map> _consumer_map; std::map _producer_map; - RuntimeState* _state; - QueryContext* _query_ctx; + RuntimeState* _state = nullptr; + QueryContext* _query_ctx = nullptr; std::unique_ptr _tracker; ObjectPool _pool; @@ -145,7 +145,7 @@ public: TRuntimeFilterDesc runtime_filter_desc; std::vector target_info; std::vector targetv2_info; - IRuntimeFilter* filter; + IRuntimeFilter* filter = nullptr; std::unordered_set arrive_id; // fragment_instance_id ? std::shared_ptr pool; }; @@ -172,7 +172,7 @@ private: using CntlValwithLock = std::pair, std::unique_ptr>; std::map _filter_map; - RuntimeState* _state; + RuntimeState* _state = nullptr; bool _opt_remote_rf = true; }; diff --git a/be/src/runtime/runtime_predicate.h b/be/src/runtime/runtime_predicate.h index 9607bf37ef..9e3bce17d6 100644 --- a/be/src/runtime/runtime_predicate.h +++ b/be/src/runtime/runtime_predicate.h @@ -65,9 +65,9 @@ public: private: mutable std::shared_mutex _rwlock; Field _orderby_extrem {Field::Types::Null}; - std::shared_ptr _predictate {nullptr}; - TabletSchemaSPtr _tablet_schema {nullptr}; - std::unique_ptr _predicate_arena {nullptr}; + std::shared_ptr _predictate; + TabletSchemaSPtr _tablet_schema = nullptr; + std::unique_ptr _predicate_arena; std::function _get_value_fn; bool _nulls_first = true; bool _inited = false; diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 8c7b3bed9f..51778a54bc 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -507,14 +507,14 @@ private: static const int DEFAULT_BATCH_SIZE = 2048; - std::shared_ptr _query_mem_tracker = nullptr; + std::shared_ptr _query_mem_tracker; // put runtime state before _obj_pool, so that it will be deconstructed after // _obj_pool. Because some of object in _obj_pool will use profile when deconstructing. RuntimeProfile _profile; RuntimeProfile _load_channel_profile; - const DescriptorTbl* _desc_tbl; + const DescriptorTbl* _desc_tbl = nullptr; std::shared_ptr _obj_pool; // runtime filter diff --git a/be/src/runtime/small_file_mgr.h b/be/src/runtime/small_file_mgr.h index 74665a7655..c3ed3f4a39 100644 --- a/be/src/runtime/small_file_mgr.h +++ b/be/src/runtime/small_file_mgr.h @@ -63,7 +63,7 @@ private: private: std::mutex _lock; - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; std::string _local_path; // file id -> small file std::unordered_map _file_cache; diff --git a/be/src/runtime/snapshot_loader.h b/be/src/runtime/snapshot_loader.h index c0d1f0f708..9008399d4a 100644 --- a/be/src/runtime/snapshot_loader.h +++ b/be/src/runtime/snapshot_loader.h @@ -107,7 +107,7 @@ private: Status _list_with_checksum(const std::string& dir, std::map* md5_files); private: - ExecEnv* _env; + ExecEnv* _env = nullptr; int64_t _job_id; int64_t _task_id; const TNetworkAddress _broker_addr; diff --git a/be/src/runtime/stream_load/stream_load_context.h b/be/src/runtime/stream_load/stream_load_context.h index 2b8d271157..b530242743 100644 --- a/be/src/runtime/stream_load/stream_load_context.h +++ b/be/src/runtime/stream_load/stream_load_context.h @@ -235,7 +235,7 @@ public: ExecEnv* exec_env() { return _exec_env; } private: - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; }; } // namespace doris diff --git a/be/src/runtime/stream_load/stream_load_executor.h b/be/src/runtime/stream_load/stream_load_executor.h index e6967304a4..c2169f85a1 100644 --- a/be/src/runtime/stream_load/stream_load_executor.h +++ b/be/src/runtime/stream_load/stream_load_executor.h @@ -54,7 +54,7 @@ private: // return true if stat is set, otherwise, return false bool collect_load_stat(StreamLoadContext* ctx, TTxnCommitAttachment* attachment); - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; }; } // namespace doris diff --git a/be/src/runtime/stream_load/stream_load_recorder.h b/be/src/runtime/stream_load/stream_load_recorder.h index 33a0d196e6..835f844e04 100644 --- a/be/src/runtime/stream_load/stream_load_recorder.h +++ b/be/src/runtime/stream_load/stream_load_recorder.h @@ -52,7 +52,7 @@ public: private: std::string _root_path; - rocksdb::DBWithTTL* _db; + rocksdb::DBWithTTL* _db = nullptr; std::vector _handles; std::atomic _last_compaction_time; diff --git a/be/src/runtime/struct_value.h b/be/src/runtime/struct_value.h index cb29d74b4c..dc9ff1426c 100644 --- a/be/src/runtime/struct_value.h +++ b/be/src/runtime/struct_value.h @@ -53,7 +53,7 @@ public: private: // pointer to the start of the vector of children pointers. These pointers are // point to children values where a null pointer means that this child is NULL. - void** _values; + void** _values = nullptr; // the number of values in this struct value. uint32_t _size; // child has no null value if has_null is false. diff --git a/be/src/runtime/tablets_channel.h b/be/src/runtime/tablets_channel.h index 4dca905033..b8e3de0584 100644 --- a/be/src/runtime/tablets_channel.h +++ b/be/src/runtime/tablets_channel.h @@ -185,7 +185,7 @@ private: bool _write_single_replica = false; - RuntimeProfile* _profile; + RuntimeProfile* _profile = nullptr; RuntimeProfile::Counter* _add_batch_number_counter = nullptr; RuntimeProfile::HighWaterMarkCounter* _memory_usage_counter = nullptr; RuntimeProfile::HighWaterMarkCounter* _write_memory_usage_counter = nullptr; diff --git a/be/src/runtime/task_group/task_group.h b/be/src/runtime/task_group/task_group.h index 41a5e9a162..95a329757a 100644 --- a/be/src/runtime/task_group/task_group.h +++ b/be/src/runtime/task_group/task_group.h @@ -70,10 +70,10 @@ public: void check_and_update_cpu_share(const TaskGroupInfo& tg_info); private: - QueueType* _task_queue; + QueueType* _task_queue = nullptr; uint64_t _vruntime_ns = 0; - taskgroup::TaskGroup* _tg; + taskgroup::TaskGroup* _tg = nullptr; std::string _type; diff --git a/be/src/runtime/thread_context.h b/be/src/runtime/thread_context.h index 1999549049..cbe1a19cae 100644 --- a/be/src/runtime/thread_context.h +++ b/be/src/runtime/thread_context.h @@ -115,7 +115,7 @@ extern bthread_key_t btls_key; // Is true after ThreadContext construction. inline thread_local bool pthread_context_ptr_init = false; -inline thread_local constinit ThreadContext* thread_context_ptr; +inline thread_local constinit ThreadContext* thread_context_ptr = nullptr; // use mem hook to consume thread mem tracker. inline thread_local bool use_mem_hook = false; @@ -291,7 +291,7 @@ public: } private: - int64_t* _scope_mem; + int64_t* _scope_mem = nullptr; }; class AttachTask { @@ -334,7 +334,7 @@ public: ~AddThreadMemTrackerConsumer(); private: - std::shared_ptr _mem_tracker = nullptr; // Avoid mem_tracker being released midway. + std::shared_ptr _mem_tracker; // Avoid mem_tracker being released midway. bool _need_pop = false; }; @@ -344,7 +344,7 @@ public: ~AddThreadMemTrackerConsumerByHook(); private: - std::shared_ptr _mem_tracker = nullptr; + std::shared_ptr _mem_tracker; }; // Basic macros for mem tracker, usually do not need to be modified and used. diff --git a/be/src/service/arrow_flight/auth_server_middleware.h b/be/src/service/arrow_flight/auth_server_middleware.h index e5f40cf626..1bb4947c7b 100644 --- a/be/src/service/arrow_flight/auth_server_middleware.h +++ b/be/src/service/arrow_flight/auth_server_middleware.h @@ -65,7 +65,7 @@ public: private: arrow::flight::CallHeaders _incoming_headers; - bool* _is_valid; + bool* _is_valid = nullptr; }; // Factory for base64 header authentication. diff --git a/be/src/service/backend_service.h b/be/src/service/backend_service.h index 09a79a68bf..f419063a49 100644 --- a/be/src/service/backend_service.h +++ b/be/src/service/backend_service.h @@ -143,7 +143,7 @@ public: private: Status start_plan_fragment_execution(const TExecPlanFragmentParams& exec_params); - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; std::unique_ptr _agent_server; std::unique_ptr _ingest_binlog_workers; }; diff --git a/be/src/service/http_service.h b/be/src/service/http_service.h index eeaf951659..d9283c5254 100644 --- a/be/src/service/http_service.h +++ b/be/src/service/http_service.h @@ -43,13 +43,13 @@ public: int get_real_port() const; private: - ExecEnv* _env; + ExecEnv* _env = nullptr; ObjectPool _pool; std::unique_ptr _ev_http_server; std::unique_ptr _web_page_handler; - std::shared_ptr _rate_limit_group {nullptr}; + std::shared_ptr _rate_limit_group; bool stopped = false; }; diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h index c324986f9b..5713faabab 100644 --- a/be/src/service/internal_service.h +++ b/be/src/service/internal_service.h @@ -252,7 +252,7 @@ private: google::protobuf::Closure* done); private: - ExecEnv* _exec_env; + ExecEnv* _exec_env = nullptr; // every brpc service request should put into thread pool // the reason see issue #16634 diff --git a/be/src/service/point_query_executor.h b/be/src/service/point_query_executor.h index 1d4da1b7f1..c951cc6746 100644 --- a/be/src/service/point_query_executor.h +++ b/be/src/service/point_query_executor.h @@ -96,7 +96,7 @@ public: private: // caching TupleDescriptor, output_expr, etc... std::unique_ptr _runtime_state; - DescriptorTbl* _desc_tbl; + DescriptorTbl* _desc_tbl = nullptr; std::mutex _block_mutex; // prevent from allocte too many tmp blocks std::vector> _block_pool; @@ -187,7 +187,7 @@ public: private: static constexpr uint32_t kDefaultNumShards = 128; RowCache(int64_t capacity, int num_shards = kDefaultNumShards); - std::unique_ptr _cache = nullptr; + std::unique_ptr _cache; }; // A cache used for prepare stmt. @@ -243,7 +243,7 @@ private: } struct CacheValue : public LRUCacheValueBase { - std::shared_ptr item = nullptr; + std::shared_ptr item; }; }; @@ -302,7 +302,7 @@ private: std::unique_ptr _rowset_ptr; }; - PTabletKeyLookupResponse* _response; + PTabletKeyLookupResponse* _response = nullptr; TabletSharedPtr _tablet; std::vector _row_read_ctxs; std::shared_ptr _reusable; diff --git a/be/src/udf/udf.h b/be/src/udf/udf.h index e383904c5b..39af2ad1c2 100644 --- a/be/src/udf/udf.h +++ b/be/src/udf/udf.h @@ -156,7 +156,7 @@ private: // We use the query's runtime state to report errors and warnings. nullptr for test // contexts. - RuntimeState* _state; + RuntimeState* _state = nullptr; // Empty if there's no error std::string _error_msg; diff --git a/be/src/util/bfd_parser.h b/be/src/util/bfd_parser.h index 6029c043d1..34eaabf226 100644 --- a/be/src/util/bfd_parser.h +++ b/be/src/util/bfd_parser.h @@ -66,8 +66,8 @@ private: std::string _file_name; std::mutex _mutex; - bfd* _abfd; - bfd_symbol** _syms; + bfd* _abfd = nullptr; + bfd_symbol** _syms = nullptr; long _num_symbols; unsigned int _symbol_size; }; diff --git a/be/src/util/bit_stream_utils.h b/be/src/util/bit_stream_utils.h index d5abbc4d94..550919440a 100644 --- a/be/src/util/bit_stream_utils.h +++ b/be/src/util/bit_stream_utils.h @@ -82,7 +82,7 @@ private: // buffer_. This is faster than writing values byte by byte directly to buffer_. uint64_t buffered_values_; - faststring* buffer_; + faststring* buffer_ = nullptr; int byte_offset_; // Offset in buffer_ int bit_offset_; // Offset in buffered_values_ }; @@ -150,7 +150,7 @@ private: // the next word into buffer_. void BufferValues(); - const uint8_t* buffer_; + const uint8_t* buffer_ = nullptr; int max_bytes_; // Bytes are memcpy'd from buffer_ and values are read from this variable. This is diff --git a/be/src/util/bitmap.h b/be/src/util/bitmap.h index b523427bee..c40b082d0d 100644 --- a/be/src/util/bitmap.h +++ b/be/src/util/bitmap.h @@ -187,7 +187,7 @@ private: private: size_t offset_; size_t num_bits_; - const uint8_t* map_; + const uint8_t* map_ = nullptr; }; /// Bitmap vector utility class. diff --git a/be/src/util/countdown_latch.h b/be/src/util/countdown_latch.h index 43db2bf189..a41a417d20 100644 --- a/be/src/util/countdown_latch.h +++ b/be/src/util/countdown_latch.h @@ -113,7 +113,7 @@ public: ~CountDownOnScopeExit() { _latch->count_down(); } private: - CountDownLatch* _latch; + CountDownLatch* _latch = nullptr; CountDownOnScopeExit(const CountDownOnScopeExit&) = delete; void operator=(const CountDownOnScopeExit&) = delete; diff --git a/be/src/util/doris_metrics.h b/be/src/util/doris_metrics.h index 409e6dae2b..c540b16c85 100644 --- a/be/src/util/doris_metrics.h +++ b/be/src/util/doris_metrics.h @@ -44,201 +44,201 @@ namespace doris { class DorisMetrics { public: - IntCounter* fragment_requests_total; - IntCounter* fragment_request_duration_us; - IntCounter* query_scan_bytes; - IntCounter* query_scan_rows; + IntCounter* fragment_requests_total = nullptr; + IntCounter* fragment_request_duration_us = nullptr; + IntCounter* query_scan_bytes = nullptr; + IntCounter* query_scan_rows = nullptr; - IntCounter* push_requests_success_total; - IntCounter* push_requests_fail_total; - IntCounter* push_request_duration_us; - IntCounter* push_request_write_bytes; - IntCounter* push_request_write_rows; - IntCounter* create_tablet_requests_total; - IntCounter* create_tablet_requests_failed; - IntCounter* drop_tablet_requests_total; + IntCounter* push_requests_success_total = nullptr; + IntCounter* push_requests_fail_total = nullptr; + IntCounter* push_request_duration_us = nullptr; + IntCounter* push_request_write_bytes = nullptr; + IntCounter* push_request_write_rows = nullptr; + IntCounter* create_tablet_requests_total = nullptr; + IntCounter* create_tablet_requests_failed = nullptr; + IntCounter* drop_tablet_requests_total = nullptr; - IntCounter* report_all_tablets_requests_total; - IntCounter* report_all_tablets_requests_failed; - IntCounter* report_tablet_requests_total; - IntCounter* report_tablet_requests_failed; - IntCounter* report_all_tablets_requests_skip; - IntCounter* report_disk_requests_total; - IntCounter* report_disk_requests_failed; - IntCounter* report_task_requests_total; - IntCounter* report_task_requests_failed; + IntCounter* report_all_tablets_requests_total = nullptr; + IntCounter* report_all_tablets_requests_failed = nullptr; + IntCounter* report_tablet_requests_total = nullptr; + IntCounter* report_tablet_requests_failed = nullptr; + IntCounter* report_all_tablets_requests_skip = nullptr; + IntCounter* report_disk_requests_total = nullptr; + IntCounter* report_disk_requests_failed = nullptr; + IntCounter* report_task_requests_total = nullptr; + IntCounter* report_task_requests_failed = nullptr; - IntCounter* schema_change_requests_total; - IntCounter* schema_change_requests_failed; - IntCounter* create_rollup_requests_total; - IntCounter* create_rollup_requests_failed; - IntCounter* storage_migrate_requests_total; - IntCounter* storage_migrate_v2_requests_total; - IntCounter* storage_migrate_v2_requests_failed; - IntCounter* delete_requests_total; - IntCounter* delete_requests_failed; - IntCounter* clone_requests_total; - IntCounter* clone_requests_failed; - IntCounter* alter_inverted_index_requests_total; - IntCounter* alter_inverted_index_requests_failed; + IntCounter* schema_change_requests_total = nullptr; + IntCounter* schema_change_requests_failed = nullptr; + IntCounter* create_rollup_requests_total = nullptr; + IntCounter* create_rollup_requests_failed = nullptr; + IntCounter* storage_migrate_requests_total = nullptr; + IntCounter* storage_migrate_v2_requests_total = nullptr; + IntCounter* storage_migrate_v2_requests_failed = nullptr; + IntCounter* delete_requests_total = nullptr; + IntCounter* delete_requests_failed = nullptr; + IntCounter* clone_requests_total = nullptr; + IntCounter* clone_requests_failed = nullptr; + IntCounter* alter_inverted_index_requests_total = nullptr; + IntCounter* alter_inverted_index_requests_failed = nullptr; - IntCounter* finish_task_requests_total; - IntCounter* finish_task_requests_failed; + IntCounter* finish_task_requests_total = nullptr; + IntCounter* finish_task_requests_failed = nullptr; - IntCounter* base_compaction_request_total; - IntCounter* base_compaction_request_failed; - IntCounter* cumulative_compaction_request_total; - IntCounter* cumulative_compaction_request_failed; + IntCounter* base_compaction_request_total = nullptr; + IntCounter* base_compaction_request_failed = nullptr; + IntCounter* cumulative_compaction_request_total = nullptr; + IntCounter* cumulative_compaction_request_failed = nullptr; - IntCounter* base_compaction_deltas_total; - IntCounter* base_compaction_bytes_total; - IntCounter* cumulative_compaction_deltas_total; - IntCounter* cumulative_compaction_bytes_total; + IntCounter* base_compaction_deltas_total = nullptr; + IntCounter* base_compaction_bytes_total = nullptr; + IntCounter* cumulative_compaction_deltas_total = nullptr; + IntCounter* cumulative_compaction_bytes_total = nullptr; - IntCounter* publish_task_request_total; - IntCounter* publish_task_failed_total; + IntCounter* publish_task_request_total = nullptr; + IntCounter* publish_task_failed_total = nullptr; - IntCounter* meta_write_request_total; - IntCounter* meta_write_request_duration_us; - IntCounter* meta_read_request_total; - IntCounter* meta_read_request_duration_us; + IntCounter* meta_write_request_total = nullptr; + IntCounter* meta_write_request_duration_us = nullptr; + IntCounter* meta_read_request_total = nullptr; + IntCounter* meta_read_request_duration_us = nullptr; // Counters for segment_v2 // ----------------------- // total number of segments read - IntCounter* segment_read_total; + IntCounter* segment_read_total = nullptr; // total number of rows in queried segments (before index pruning) - IntCounter* segment_row_total; + IntCounter* segment_row_total = nullptr; - IntCounter* stream_load_txn_begin_request_total; - IntCounter* stream_load_txn_commit_request_total; - IntCounter* stream_load_txn_rollback_request_total; - IntCounter* stream_receive_bytes_total; - IntCounter* stream_load_rows_total; - IntCounter* load_rows; - IntCounter* load_bytes; + IntCounter* stream_load_txn_begin_request_total = nullptr; + IntCounter* stream_load_txn_commit_request_total = nullptr; + IntCounter* stream_load_txn_rollback_request_total = nullptr; + IntCounter* stream_receive_bytes_total = nullptr; + IntCounter* stream_load_rows_total = nullptr; + IntCounter* load_rows = nullptr; + IntCounter* load_bytes = nullptr; - IntCounter* memtable_flush_total; - IntCounter* memtable_flush_duration_us; + IntCounter* memtable_flush_total = nullptr; + IntCounter* memtable_flush_duration_us = nullptr; - IntGauge* memory_pool_bytes_total; - IntGauge* process_thread_num; - IntGauge* process_fd_num_used; - IntGauge* process_fd_num_limit_soft; - IntGauge* process_fd_num_limit_hard; + IntGauge* memory_pool_bytes_total = nullptr; + IntGauge* process_thread_num = nullptr; + IntGauge* process_fd_num_used = nullptr; + IntGauge* process_fd_num_limit_soft = nullptr; + IntGauge* process_fd_num_limit_hard = nullptr; // the max compaction score of all tablets. // Record base and cumulative scores separately, because // we need to get the larger of the two. - IntGauge* tablet_cumulative_max_compaction_score; - IntGauge* tablet_base_max_compaction_score; + IntGauge* tablet_cumulative_max_compaction_score = nullptr; + IntGauge* tablet_base_max_compaction_score = nullptr; - IntGauge* all_rowsets_num; - IntGauge* all_segments_num; + IntGauge* all_rowsets_num = nullptr; + IntGauge* all_segments_num = nullptr; // permits have been used for all compaction tasks - IntGauge* compaction_used_permits; + IntGauge* compaction_used_permits = nullptr; // permits required by the compaction task which is waiting for permits - IntGauge* compaction_waitting_permits; + IntGauge* compaction_waitting_permits = nullptr; - HistogramMetric* tablet_version_num_distribution; + HistogramMetric* tablet_version_num_distribution = nullptr; // The following metrics will be calculated // by metric calculator - IntGauge* query_scan_bytes_per_second; + IntGauge* query_scan_bytes_per_second = nullptr; // Metrics related with file reader/writer - IntCounter* local_file_reader_total; - IntCounter* s3_file_reader_total; - IntCounter* hdfs_file_reader_total; - IntCounter* broker_file_reader_total; - IntCounter* local_file_writer_total; - IntCounter* s3_file_writer_total; - IntCounter* file_created_total; - IntCounter* s3_file_created_total; - IntCounter* local_bytes_read_total; - IntCounter* s3_bytes_read_total; - IntCounter* local_bytes_written_total; - IntCounter* s3_bytes_written_total; - IntGauge* local_file_open_reading; - IntGauge* s3_file_open_reading; - IntGauge* hdfs_file_open_reading; - IntGauge* broker_file_open_reading; - IntGauge* local_file_open_writing; - IntGauge* s3_file_open_writing; + IntCounter* local_file_reader_total = nullptr; + IntCounter* s3_file_reader_total = nullptr; + IntCounter* hdfs_file_reader_total = nullptr; + IntCounter* broker_file_reader_total = nullptr; + IntCounter* local_file_writer_total = nullptr; + IntCounter* s3_file_writer_total = nullptr; + IntCounter* file_created_total = nullptr; + IntCounter* s3_file_created_total = nullptr; + IntCounter* local_bytes_read_total = nullptr; + IntCounter* s3_bytes_read_total = nullptr; + IntCounter* local_bytes_written_total = nullptr; + IntCounter* s3_bytes_written_total = nullptr; + IntGauge* local_file_open_reading = nullptr; + IntGauge* s3_file_open_reading = nullptr; + IntGauge* hdfs_file_open_reading = nullptr; + IntGauge* broker_file_open_reading = nullptr; + IntGauge* local_file_open_writing = nullptr; + IntGauge* s3_file_open_writing = nullptr; // Size of some global containers - UIntGauge* rowset_count_generated_and_in_use; - UIntGauge* unused_rowsets_count; - UIntGauge* broker_count; - UIntGauge* data_stream_receiver_count; - UIntGauge* fragment_endpoint_count; - UIntGauge* active_scan_context_count; - UIntGauge* fragment_instance_count; - UIntGauge* load_channel_count; - UIntGauge* result_buffer_block_count; - UIntGauge* result_block_queue_count; - UIntGauge* routine_load_task_count; - UIntGauge* small_file_cache_count; - UIntGauge* stream_load_pipe_count; - UIntGauge* new_stream_load_pipe_count; - UIntGauge* brpc_endpoint_stub_count; - UIntGauge* brpc_function_endpoint_stub_count; - UIntGauge* tablet_writer_count; + UIntGauge* rowset_count_generated_and_in_use = nullptr; + UIntGauge* unused_rowsets_count = nullptr; + UIntGauge* broker_count = nullptr; + UIntGauge* data_stream_receiver_count = nullptr; + UIntGauge* fragment_endpoint_count = nullptr; + UIntGauge* active_scan_context_count = nullptr; + UIntGauge* fragment_instance_count = nullptr; + UIntGauge* load_channel_count = nullptr; + UIntGauge* result_buffer_block_count = nullptr; + UIntGauge* result_block_queue_count = nullptr; + UIntGauge* routine_load_task_count = nullptr; + UIntGauge* small_file_cache_count = nullptr; + UIntGauge* stream_load_pipe_count = nullptr; + UIntGauge* new_stream_load_pipe_count = nullptr; + UIntGauge* brpc_endpoint_stub_count = nullptr; + UIntGauge* brpc_function_endpoint_stub_count = nullptr; + UIntGauge* tablet_writer_count = nullptr; - UIntGauge* segcompaction_mem_consumption; - UIntGauge* compaction_mem_consumption; - UIntGauge* load_mem_consumption; - UIntGauge* load_channel_mem_consumption; - UIntGauge* memtable_memory_limiter_mem_consumption; - UIntGauge* query_mem_consumption; - UIntGauge* schema_change_mem_consumption; - UIntGauge* storage_migration_mem_consumption; - UIntGauge* tablet_meta_mem_consumption; + UIntGauge* segcompaction_mem_consumption = nullptr; + UIntGauge* compaction_mem_consumption = nullptr; + UIntGauge* load_mem_consumption = nullptr; + UIntGauge* load_channel_mem_consumption = nullptr; + UIntGauge* memtable_memory_limiter_mem_consumption = nullptr; + UIntGauge* query_mem_consumption = nullptr; + UIntGauge* schema_change_mem_consumption = nullptr; + UIntGauge* storage_migration_mem_consumption = nullptr; + UIntGauge* tablet_meta_mem_consumption = nullptr; // Cache metrics - UIntGauge* query_cache_memory_total_byte; - UIntGauge* query_cache_sql_total_count; - UIntGauge* query_cache_partition_total_count; + UIntGauge* query_cache_memory_total_byte = nullptr; + UIntGauge* query_cache_sql_total_count = nullptr; + UIntGauge* query_cache_partition_total_count = nullptr; - IntCounter* tablet_schema_cache_count; - UIntGauge* tablet_schema_cache_memory_bytes; - IntGauge* lru_cache_memory_bytes; + IntCounter* tablet_schema_cache_count = nullptr; + UIntGauge* tablet_schema_cache_memory_bytes = nullptr; + IntGauge* lru_cache_memory_bytes = nullptr; - UIntGauge* scanner_thread_pool_queue_size; - UIntGauge* add_batch_task_queue_size; - UIntGauge* send_batch_thread_pool_thread_num; - UIntGauge* send_batch_thread_pool_queue_size; - UIntGauge* download_cache_thread_pool_thread_num; - UIntGauge* download_cache_thread_pool_queue_size; - UIntGauge* fragment_thread_pool_queue_size; + UIntGauge* scanner_thread_pool_queue_size = nullptr; + UIntGauge* add_batch_task_queue_size = nullptr; + UIntGauge* send_batch_thread_pool_thread_num = nullptr; + UIntGauge* send_batch_thread_pool_queue_size = nullptr; + UIntGauge* download_cache_thread_pool_thread_num = nullptr; + UIntGauge* download_cache_thread_pool_queue_size = nullptr; + UIntGauge* fragment_thread_pool_queue_size = nullptr; // Upload metrics - UIntGauge* upload_total_byte; - IntCounter* upload_rowset_count; - IntCounter* upload_fail_count; + UIntGauge* upload_total_byte = nullptr; + IntCounter* upload_rowset_count = nullptr; + IntCounter* upload_fail_count = nullptr; - UIntGauge* light_work_pool_queue_size; - UIntGauge* heavy_work_pool_queue_size; - UIntGauge* heavy_work_active_threads; - UIntGauge* light_work_active_threads; + UIntGauge* light_work_pool_queue_size = nullptr; + UIntGauge* heavy_work_pool_queue_size = nullptr; + UIntGauge* heavy_work_active_threads = nullptr; + UIntGauge* light_work_active_threads = nullptr; - UIntGauge* heavy_work_pool_max_queue_size; - UIntGauge* light_work_pool_max_queue_size; - UIntGauge* heavy_work_max_threads; - UIntGauge* light_work_max_threads; + UIntGauge* heavy_work_pool_max_queue_size = nullptr; + UIntGauge* light_work_pool_max_queue_size = nullptr; + UIntGauge* heavy_work_max_threads = nullptr; + UIntGauge* light_work_max_threads = nullptr; - UIntGauge* flush_thread_pool_queue_size; - UIntGauge* flush_thread_pool_thread_num; + UIntGauge* flush_thread_pool_queue_size = nullptr; + UIntGauge* flush_thread_pool_thread_num = nullptr; - UIntGauge* local_scan_thread_pool_queue_size; - UIntGauge* local_scan_thread_pool_thread_num; - UIntGauge* remote_scan_thread_pool_queue_size; - UIntGauge* remote_scan_thread_pool_thread_num; - UIntGauge* limited_scan_thread_pool_queue_size; - UIntGauge* limited_scan_thread_pool_thread_num; - UIntGauge* group_local_scan_thread_pool_queue_size; - UIntGauge* group_local_scan_thread_pool_thread_num; + UIntGauge* local_scan_thread_pool_queue_size = nullptr; + UIntGauge* local_scan_thread_pool_thread_num = nullptr; + UIntGauge* remote_scan_thread_pool_queue_size = nullptr; + UIntGauge* remote_scan_thread_pool_thread_num = nullptr; + UIntGauge* limited_scan_thread_pool_queue_size = nullptr; + UIntGauge* limited_scan_thread_pool_thread_num = nullptr; + UIntGauge* group_local_scan_thread_pool_queue_size = nullptr; + UIntGauge* group_local_scan_thread_pool_thread_num = nullptr; static DorisMetrics* instance() { static DorisMetrics instance; diff --git a/be/src/util/easy_json.h b/be/src/util/easy_json.h index 5e80956d57..191c0a900d 100644 --- a/be/src/util/easy_json.h +++ b/be/src/util/easy_json.h @@ -182,7 +182,7 @@ private: // A pointer to the underlying Value in the object // tree owned by alloc_. - rapidjson::Value* value_; + rapidjson::Value* value_ = nullptr; }; } // namespace doris diff --git a/be/src/util/faststring.h b/be/src/util/faststring.h index 09f325d111..72a362156d 100644 --- a/be/src/util/faststring.h +++ b/be/src/util/faststring.h @@ -224,7 +224,7 @@ private: void ShrinkToFitInternal(); - uint8_t* data_; + uint8_t* data_ = nullptr; uint8_t initial_data_[kInitialCapacity]; size_t len_; // NOTE: we will make a initial buffer as part of the object, so the smallest diff --git a/be/src/util/frame_of_reference_coding.h b/be/src/util/frame_of_reference_coding.h index 06fd2a25f0..bc0812d53c 100644 --- a/be/src/util/frame_of_reference_coding.h +++ b/be/src/util/frame_of_reference_coding.h @@ -117,7 +117,7 @@ private: static const uint8_t FRAME_VALUE_NUM = 128; T _buffered_values[FRAME_VALUE_NUM]; - faststring* _buffer; + faststring* _buffer = nullptr; std::vector _storage_formats; std::vector _bit_widths; }; diff --git a/be/src/util/interval_tree-inl.h b/be/src/util/interval_tree-inl.h index f93dd788a6..71fb14313f 100644 --- a/be/src/util/interval_tree-inl.h +++ b/be/src/util/interval_tree-inl.h @@ -194,10 +194,10 @@ private: IntervalVector overlapping_by_desc_right_; // Tree node for intervals fully left of split_point_, or NULL. - ITNode* left_; + ITNode* left_ = nullptr; // Tree node for intervals fully right of split_point_, or NULL. - ITNode* right_; + ITNode* right_ = nullptr; DISALLOW_COPY_AND_ASSIGN(ITNode); }; diff --git a/be/src/util/interval_tree.h b/be/src/util/interval_tree.h index dd978e8354..29445de5cd 100644 --- a/be/src/util/interval_tree.h +++ b/be/src/util/interval_tree.h @@ -151,7 +151,7 @@ private: // Create a node containing the given intervals, recursively splitting down the tree. static node_type* CreateNode(const IntervalVector& intervals); - node_type* root_; + node_type* root_ = nullptr; DISALLOW_COPY_AND_ASSIGN(IntervalTree); }; diff --git a/be/src/util/jni-util.h b/be/src/util/jni-util.h index faa79c9133..317ba48d3d 100644 --- a/be/src/util/jni-util.h +++ b/be/src/util/jni-util.h @@ -119,9 +119,9 @@ public: const char* get() { return utf_chars; } private: - JNIEnv* env; + JNIEnv* env = nullptr; jstring jstr; - const char* utf_chars; + const char* utf_chars = nullptr; DISALLOW_COPY_AND_ASSIGN(JniUtfCharGuard); }; @@ -143,7 +143,7 @@ public: private: DISALLOW_COPY_AND_ASSIGN(JniLocalFrame); - JNIEnv* env_; + JNIEnv* env_ = nullptr; }; template @@ -151,7 +151,7 @@ Status SerializeThriftMsg(JNIEnv* env, T* msg, jbyteArray* serialized_msg) { int buffer_size = 100 * 1024; // start out with 100KB ThriftSerializer serializer(false, buffer_size); - uint8_t* buffer = NULL; + uint8_t* buffer = nullptr; uint32_t size = 0; RETURN_IF_ERROR(serializer.serialize(msg, &size, &buffer)); diff --git a/be/src/util/jsonb_document.h b/be/src/util/jsonb_document.h index 05e161e356..c02a034d53 100644 --- a/be/src/util/jsonb_document.h +++ b/be/src/util/jsonb_document.h @@ -318,13 +318,13 @@ public: private: /// The current position in the stream. - const char* m_position; + const char* m_position = nullptr; /// The end of the stream. const char* const m_end; ///path leg ptr - char* leg_ptr; + char* leg_ptr = nullptr; ///path leg len unsigned int leg_len; @@ -335,7 +335,7 @@ private: struct leg_info { ///path leg ptr - char* leg_ptr; + char* leg_ptr = nullptr; ///path leg len unsigned int leg_len; diff --git a/be/src/util/jsonb_stream.h b/be/src/util/jsonb_stream.h index b88143e793..4567ab8384 100644 --- a/be/src/util/jsonb_stream.h +++ b/be/src/util/jsonb_stream.h @@ -185,7 +185,7 @@ private: } private: - char* head_; + char* head_ = nullptr; uint32_t size_; uint32_t capacity_; bool alloc_; diff --git a/be/src/util/jsonb_updater.h b/be/src/util/jsonb_updater.h index 27716d28c9..1300eb1e90 100644 --- a/be/src/util/jsonb_updater.h +++ b/be/src/util/jsonb_updater.h @@ -366,8 +366,8 @@ private: } private: - JsonbDocument* document_; - JsonbValue* root_; + JsonbDocument* document_ = nullptr; + JsonbValue* root_ = nullptr; uint buffer_size_; // This stack store all the key path in the document. It's deserve // noticing that the root node is always in this stack. diff --git a/be/src/util/jsonb_writer.h b/be/src/util/jsonb_writer.h index e937747512..bca956d2aa 100644 --- a/be/src/util/jsonb_writer.h +++ b/be/src/util/jsonb_writer.h @@ -543,7 +543,7 @@ private: }; private: - OS_TYPE* os_; + OS_TYPE* os_ = nullptr; bool alloc_; bool hasHdr_; WriteState kvState_; // key or value state diff --git a/be/src/util/lru_multi_cache.h b/be/src/util/lru_multi_cache.h index f1c73912ec..e1ff32fbe5 100644 --- a/be/src/util/lru_multi_cache.h +++ b/be/src/util/lru_multi_cache.h @@ -237,7 +237,7 @@ public: void destroy(); private: - ValueType_internal* _p_value_internal; + ValueType_internal* _p_value_internal = nullptr; }; }; diff --git a/be/src/util/mem_range.h b/be/src/util/mem_range.h index 544595e0f2..fd01faed79 100644 --- a/be/src/util/mem_range.h +++ b/be/src/util/mem_range.h @@ -41,7 +41,7 @@ public: static MemRange null() { return MemRange(nullptr, 0); } private: - uint8_t* data_; + uint8_t* data_ = nullptr; int64_t len_; }; } // namespace doris diff --git a/be/src/util/mysql_row_buffer.h b/be/src/util/mysql_row_buffer.h index 394d4d0bdd..69f84782df 100644 --- a/be/src/util/mysql_row_buffer.h +++ b/be/src/util/mysql_row_buffer.h @@ -139,8 +139,8 @@ private: // the first few bytes is length, followed by data int append_var_string(const char* data, int64_t len); - char* _pos; - char* _buf; + char* _pos = nullptr; + char* _buf = nullptr; int64_t _buf_size; char _default_buf[4096]; diff --git a/be/src/util/obj_lru_cache.h b/be/src/util/obj_lru_cache.h index 89f126ae6f..db6e937e18 100644 --- a/be/src/util/obj_lru_cache.h +++ b/be/src/util/obj_lru_cache.h @@ -97,7 +97,7 @@ public: private: static constexpr uint32_t kDefaultNumShards = 16; - std::unique_ptr _cache = nullptr; + std::unique_ptr _cache; bool _enabled; }; diff --git a/be/src/util/runtime_profile.h b/be/src/util/runtime_profile.h index fed0debbbd..fbddb9dc4e 100644 --- a/be/src/util/runtime_profile.h +++ b/be/src/util/runtime_profile.h @@ -500,21 +500,21 @@ private: }; struct RateCounterInfo { - Counter* src_counter; + Counter* src_counter = nullptr; SampleFn sample_fn; int64_t elapsed_ms; }; struct SamplingCounterInfo { - Counter* src_counter; // the counter to be sampled + Counter* src_counter = nullptr; // the counter to be sampled SampleFn sample_fn; int64_t total_sampled_value; // sum of all sampled values; int64_t num_sampled; // number of samples taken }; struct BucketCountersInfo { - Counter* src_counter; // the counter to be sampled - int64_t num_sampled; // number of samples taken + Counter* src_counter = nullptr; // the counter to be sampled + int64_t num_sampled; // number of samples taken // TODO: customize bucketing }; @@ -563,7 +563,7 @@ public: private: int64_t _val; - RuntimeProfile::Counter* _counter; + RuntimeProfile::Counter* _counter = nullptr; }; // Utility class to update time elapsed when the object goes out of scope. @@ -608,8 +608,8 @@ public: private: T _sw; - RuntimeProfile::Counter* _counter; - const Bool* _is_cancelled; + RuntimeProfile::Counter* _counter = nullptr; + const Bool* _is_cancelled = nullptr; }; // Utility class to update time elapsed when the object goes out of scope. @@ -628,7 +628,7 @@ public: private: T _sw; - C* _counter; + C* _counter = nullptr; }; } // namespace doris diff --git a/be/src/util/slice.h b/be/src/util/slice.h index 0c5c16d714..b9d05194d2 100644 --- a/be/src/util/slice.h +++ b/be/src/util/slice.h @@ -47,7 +47,7 @@ class faststring; /// external synchronization. struct Slice { public: - char* data; + char* data = nullptr; size_t size; // Intentionally copyable diff --git a/be/src/util/sm3.h b/be/src/util/sm3.h index 7f0ab12a72..66d5bbe6a4 100644 --- a/be/src/util/sm3.h +++ b/be/src/util/sm3.h @@ -36,8 +36,8 @@ public: const std::string& hex() const { return _hex; } private: - EVP_MD_CTX* _ctx; - const EVP_MD* _md; + EVP_MD_CTX* _ctx = nullptr; + const EVP_MD* _md = nullptr; std::string _hex; }; diff --git a/be/src/util/system_metrics.h b/be/src/util/system_metrics.h index 830b11e04c..28db964296 100644 --- a/be/src/util/system_metrics.h +++ b/be/src/util/system_metrics.h @@ -110,11 +110,11 @@ private: char* _line_ptr = nullptr; size_t _line_buf_size = 0; MetricRegistry* _registry = nullptr; - std::shared_ptr _server_entity = nullptr; + std::shared_ptr _server_entity; - IntGauge* max_disk_io_util_percent; - IntGauge* max_network_send_bytes_rate; - IntGauge* max_network_receive_bytes_rate; + IntGauge* max_disk_io_util_percent = nullptr; + IntGauge* max_network_send_bytes_rate = nullptr; + IntGauge* max_network_receive_bytes_rate = nullptr; }; } // namespace doris diff --git a/be/src/util/thread.h b/be/src/util/thread.h index ea9d3161d0..886ec6cc42 100644 --- a/be/src/util/thread.h +++ b/be/src/util/thread.h @@ -279,7 +279,7 @@ private: kDefaultGiveUpAfterMs = -1 // forever }; - Thread* _thread; + Thread* _thread = nullptr; int _warn_after_ms; int _warn_every_ms; diff --git a/be/src/util/threadpool.h b/be/src/util/threadpool.h index c842870bce..526836cb09 100644 --- a/be/src/util/threadpool.h +++ b/be/src/util/threadpool.h @@ -487,7 +487,7 @@ private: ThreadPool::ExecutionMode _mode; // Pointer to the token's thread pool. - ThreadPool* _pool; + ThreadPool* _pool = nullptr; // Token state machine. State _state; diff --git a/be/src/util/thrift_server.h b/be/src/util/thrift_server.h index 08c09a6de5..aa9cbe7080 100644 --- a/be/src/util/thrift_server.h +++ b/be/src/util/thrift_server.h @@ -132,7 +132,7 @@ private: std::shared_ptr _processor; // If not nullptr, called when session events happen. Not owned by us. - SessionHandlerIf* _session_handler; + SessionHandlerIf* _session_handler = nullptr; // Protects _session_keys std::mutex _session_keys_lock; @@ -150,9 +150,9 @@ private: std::shared_ptr _thrift_server_metric_entity; // Number of currently active connections - IntGauge* thrift_current_connections; + IntGauge* thrift_current_connections = nullptr; // Total connections made over the lifetime of this server - IntCounter* thrift_connections_total; + IntCounter* thrift_connections_total = nullptr; }; } // namespace doris diff --git a/be/src/vec/aggregate_functions/aggregate_function_collect.h b/be/src/vec/aggregate_functions/aggregate_function_collect.h index 3a4b0bad20..2188fe9b24 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_collect.h +++ b/be/src/vec/aggregate_functions/aggregate_function_collect.h @@ -301,8 +301,8 @@ struct AggregateFunctionArrayAggData { using ElementType = T; using ColVecType = ColumnVectorOrDecimal; MutableColumnPtr column_data; - ColVecType* nested_column; - NullMap* null_map; + ColVecType* nested_column = nullptr; + NullMap* null_map = nullptr; AggregateFunctionArrayAggData(const DataTypes& argument_types) { if constexpr (IsDecimalNumber) { @@ -369,8 +369,8 @@ struct AggregateFunctionArrayAggData { using ElementType = StringRef; using ColVecType = ColumnString; MutableColumnPtr column_data; - ColVecType* nested_column; - NullMap* null_map; + ColVecType* nested_column = nullptr; + NullMap* null_map = nullptr; AggregateFunctionArrayAggData() { column_data = ColumnNullable::create(ColVecType::create(), ColumnUInt8::create()); diff --git a/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h b/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h index 0fffbb994a..1a1285a9dc 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h +++ b/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h @@ -142,7 +142,7 @@ struct PercentileApproxState { } bool init_flag = false; - std::unique_ptr digest = nullptr; + std::unique_ptr digest; double target_quantile = INIT_QUANTILE; double compressions = 10000; }; diff --git a/be/src/vec/aggregate_functions/aggregate_function_sort.h b/be/src/vec/aggregate_functions/aggregate_function_sort.h index 9026060c82..1528690f6b 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_sort.h +++ b/be/src/vec/aggregate_functions/aggregate_function_sort.h @@ -115,7 +115,7 @@ private: DataTypes _arguments; const SortDescription& _sort_desc; Block _block; - const RuntimeState* _state; + const RuntimeState* _state = nullptr; AggregateDataPtr get_nested_place(AggregateDataPtr __restrict place) const noexcept { return place + prefix_size; diff --git a/be/src/vec/common/arena.h b/be/src/vec/common/arena.h index 90ebe2788b..4ab3ee4c60 100644 --- a/be/src/vec/common/arena.h +++ b/be/src/vec/common/arena.h @@ -52,11 +52,11 @@ private: /// Contiguous chunk of memory and pointer to free space inside it. Member of single-linked list. struct alignas(16) Chunk : private Allocator /// empty base optimization { - char* begin; - char* pos; - char* end; /// does not include padding. + char* begin = nullptr; + char* pos = nullptr; + char* end = nullptr; /// does not include padding. - Chunk* prev; + Chunk* prev = nullptr; Chunk(size_t size_, Chunk* prev_) { begin = reinterpret_cast(Allocator::alloc(size_)); @@ -88,7 +88,7 @@ private: size_t linear_growth_threshold; /// Last contiguous chunk of memory. - Chunk* head; + Chunk* head = nullptr; size_t size_in_bytes; // The memory used by all chunks, excluding head. size_t _used_size_no_head; diff --git a/be/src/vec/common/columns_hashing_impl.h b/be/src/vec/common/columns_hashing_impl.h index a87f719d84..d0ef8a4b01 100644 --- a/be/src/vec/common/columns_hashing_impl.h +++ b/be/src/vec/common/columns_hashing_impl.h @@ -54,7 +54,7 @@ struct LastElementCache { template class FindResultImpl { - Mapped* value; + Mapped* value = nullptr; bool found; public: diff --git a/be/src/vec/common/cow.h b/be/src/vec/common/cow.h index 5cd701e6ba..3cc2f8609a 100644 --- a/be/src/vec/common/cow.h +++ b/be/src/vec/common/cow.h @@ -202,7 +202,7 @@ protected: operator T*() const { return t; } private: - T* t; + T* t = nullptr; }; protected: diff --git a/be/src/vec/common/hash_table/hash_map_context.h b/be/src/vec/common/hash_table/hash_map_context.h index 35df772b16..32c0d0a31b 100644 --- a/be/src/vec/common/hash_table/hash_map_context.h +++ b/be/src/vec/common/hash_table/hash_map_context.h @@ -51,7 +51,7 @@ struct MethodBase { std::shared_ptr hash_table; Iterator iterator; bool inited_iterator = false; - Key* keys; + Key* keys = nullptr; Arena arena; std::vector hash_values; diff --git a/be/src/vec/common/hash_table/hash_table.h b/be/src/vec/common/hash_table/hash_table.h index 39ee5ec9e0..20c8c8e457 100644 --- a/be/src/vec/common/hash_table/hash_table.h +++ b/be/src/vec/common/hash_table/hash_table.h @@ -438,7 +438,7 @@ protected: using cell_type = Cell; size_t m_size = 0; /// Amount of elements - Cell* buf {nullptr}; /// A piece of memory for all elements except the element with zero key. + Cell* buf = nullptr; /// A piece of memory for all elements except the element with zero key. Grower grower; int64_t _resize_timer_ns; @@ -542,8 +542,8 @@ protected: using Container = std::conditional_t; using cell_type = std::conditional_t; - Container* container; - cell_type* ptr; + Container* container = nullptr; + cell_type* ptr = nullptr; friend class HashTable; @@ -877,7 +877,7 @@ public: private: Constructor(Cell* cell) : _cell(cell) {} - Cell* _cell; + Cell* _cell = nullptr; }; /** Insert the key. diff --git a/be/src/vec/common/hash_table/hash_table_set_build.h b/be/src/vec/common/hash_table/hash_table_set_build.h index e3c1ed27b1..34fb691f9e 100644 --- a/be/src/vec/common/hash_table/hash_table_set_build.h +++ b/be/src/vec/common/hash_table/hash_table_set_build.h @@ -62,11 +62,11 @@ struct HashTableBuild { } private: - int64_t* _mem_used; + int64_t* _mem_used = nullptr; const int _rows; const uint8_t _offset; ColumnRawPtrs& _build_raw_ptrs; - RuntimeState* _state; + RuntimeState* _state = nullptr; }; } // namespace doris::vectorized diff --git a/be/src/vec/common/hash_table/hash_table_set_probe.h b/be/src/vec/common/hash_table/hash_table_set_probe.h index eb00cca856..331e519445 100644 --- a/be/src/vec/common/hash_table/hash_table_set_probe.h +++ b/be/src/vec/common/hash_table/hash_table_set_probe.h @@ -58,7 +58,7 @@ struct HashTableProbe { } private: - int64_t* _valid_element_in_hash_tbl; + int64_t* _valid_element_in_hash_tbl = nullptr; const size_t _probe_rows; ColumnRawPtrs& _probe_raw_ptrs; std::vector _probe_keys; diff --git a/be/src/vec/common/hash_table/string_hash_table.h b/be/src/vec/common/hash_table/string_hash_table.h index c2916c58ba..09e326761c 100644 --- a/be/src/vec/common/hash_table/string_hash_table.h +++ b/be/src/vec/common/hash_table/string_hash_table.h @@ -157,7 +157,7 @@ public: private: Constructor(Cell* cell) : _cell(cell) {} - Cell* _cell; + Cell* _cell = nullptr; }; template @@ -254,7 +254,7 @@ protected: class iterator_base { using Container = std::conditional_t; - Container* container; + Container* container = nullptr; int sub_table_index; typename T1::iterator iterator1; typename T2::iterator iterator2; diff --git a/be/src/vec/common/sort/partition_sorter.h b/be/src/vec/common/sort/partition_sorter.h index ca0cd5c493..afe95e6208 100644 --- a/be/src/vec/common/sort/partition_sorter.h +++ b/be/src/vec/common/sort/partition_sorter.h @@ -67,7 +67,7 @@ public: return true; } int row = 0; - MergeSortCursorImpl* impl; + MergeSortCursorImpl* impl = nullptr; }; class PartitionSorter final : public Sorter { @@ -103,7 +103,7 @@ private: bool _has_global_limit = false; int _partition_inner_limit = 0; TopNAlgorithm::type _top_n_algorithm = TopNAlgorithm::type::ROW_NUMBER; - SortCursorCmp* _previous_row; + SortCursorCmp* _previous_row = nullptr; }; } // namespace doris::vectorized diff --git a/be/src/vec/common/sort/sorter.h b/be/src/vec/common/sort/sorter.h index 383b6ff75e..e372e6367d 100644 --- a/be/src/vec/common/sort/sorter.h +++ b/be/src/vec/common/sort/sorter.h @@ -126,10 +126,10 @@ private: Block merge_sorted_block_; std::unique_ptr merger_; - RuntimeProfile* profile_; - RuntimeProfile* block_spill_profile_; - RuntimeProfile::Counter* spilled_block_count_; - RuntimeProfile::Counter* spilled_original_block_size_; + RuntimeProfile* profile_ = nullptr; + RuntimeProfile* block_spill_profile_ = nullptr; + RuntimeProfile::Counter* spilled_block_count_ = nullptr; + RuntimeProfile::Counter* spilled_original_block_size_ = nullptr; }; class Sorter { @@ -172,7 +172,7 @@ protected: VSortExecExprs& _vsort_exec_exprs; int _limit; int64_t _offset; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; std::vector& _is_asc_order; std::vector& _nulls_first; diff --git a/be/src/vec/core/block_spill_reader.h b/be/src/vec/core/block_spill_reader.h index 498a041160..d982d586a1 100644 --- a/be/src/vec/core/block_spill_reader.h +++ b/be/src/vec/core/block_spill_reader.h @@ -75,10 +75,10 @@ private: std::vector block_start_offsets_; RuntimeProfile* profile_ = nullptr; - RuntimeProfile::Counter* read_time_; - RuntimeProfile::Counter* deserialize_time_; - RuntimeProfile::Counter* read_bytes_; - RuntimeProfile::Counter* read_block_num_; + RuntimeProfile::Counter* read_time_ = nullptr; + RuntimeProfile::Counter* deserialize_time_ = nullptr; + RuntimeProfile::Counter* read_bytes_ = nullptr; + RuntimeProfile::Counter* read_block_num_ = nullptr; }; using BlockSpillReaderUPtr = std::unique_ptr; diff --git a/be/src/vec/core/block_spill_writer.h b/be/src/vec/core/block_spill_writer.h index f4d8a6bda9..4785258c15 100644 --- a/be/src/vec/core/block_spill_writer.h +++ b/be/src/vec/core/block_spill_writer.h @@ -80,10 +80,10 @@ private: Block tmp_block_; RuntimeProfile* profile_ = nullptr; - RuntimeProfile::Counter* write_bytes_counter_; - RuntimeProfile::Counter* serialize_timer_; - RuntimeProfile::Counter* write_timer_; - RuntimeProfile::Counter* write_blocks_num_; + RuntimeProfile::Counter* write_bytes_counter_ = nullptr; + RuntimeProfile::Counter* serialize_timer_ = nullptr; + RuntimeProfile::Counter* write_timer_ = nullptr; + RuntimeProfile::Counter* write_blocks_num_ = nullptr; }; using BlockSpillWriterUPtr = std::unique_ptr; diff --git a/be/src/vec/core/sort_cursor.h b/be/src/vec/core/sort_cursor.h index e1efe7cd01..71db342854 100644 --- a/be/src/vec/core/sort_cursor.h +++ b/be/src/vec/core/sort_cursor.h @@ -317,7 +317,7 @@ struct MergeSortCursor { /// For easy copying. struct MergeSortBlockCursor { - MergeSortCursorImpl* impl; + MergeSortCursorImpl* impl = nullptr; MergeSortBlockCursor(MergeSortCursorImpl* impl_) : impl(impl_) {} MergeSortCursorImpl* operator->() const { return impl; } diff --git a/be/src/vec/exec/data_gen_functions/vdata_gen_function_inf.h b/be/src/vec/exec/data_gen_functions/vdata_gen_function_inf.h index c8fc78180e..515be45ad1 100644 --- a/be/src/vec/exec/data_gen_functions/vdata_gen_function_inf.h +++ b/be/src/vec/exec/data_gen_functions/vdata_gen_function_inf.h @@ -48,7 +48,7 @@ public: protected: TupleId _tuple_id; // Descriptor of tuples generated - const TupleDescriptor* _tuple_desc; + const TupleDescriptor* _tuple_desc = nullptr; }; } // namespace vectorized diff --git a/be/src/vec/exec/format/avro/avro_jni_reader.h b/be/src/vec/exec/format/avro/avro_jni_reader.h index 8f8d9f6683..64dac0aba4 100644 --- a/be/src/vec/exec/format/avro/avro_jni_reader.h +++ b/be/src/vec/exec/format/avro/avro_jni_reader.h @@ -87,11 +87,11 @@ public: private: const std::vector& _file_slot_descs; - RuntimeState* _state; - RuntimeProfile* _profile; + RuntimeState* _state = nullptr; + RuntimeProfile* _profile = nullptr; const TFileScanRangeParams _params; const TFileRangeDesc _range; - std::unordered_map* _colname_to_value_range; + std::unordered_map* _colname_to_value_range = nullptr; std::unique_ptr _jni_connector; }; diff --git a/be/src/vec/exec/format/csv/csv_reader.h b/be/src/vec/exec/format/csv/csv_reader.h index 5eb3572e72..28d82183d0 100644 --- a/be/src/vec/exec/format/csv/csv_reader.h +++ b/be/src/vec/exec/format/csv/csv_reader.h @@ -230,9 +230,9 @@ private: // and the line is skipped as unqualified row, and the process should continue. Status _validate_line(const Slice& line, bool* success); - RuntimeState* _state; - RuntimeProfile* _profile; - ScannerCounter* _counter; + RuntimeState* _state = nullptr; + RuntimeProfile* _profile = nullptr; + ScannerCounter* _counter = nullptr; const TFileScanRangeParams& _params; const TFileRangeDesc& _range; io::FileSystemProperties _system_properties; @@ -283,7 +283,7 @@ private: // `should_not_trim` is to manage the case that: user do not expect to trim double quotes but enclose is double quotes bool _not_trim_enclose = true; - io::IOContext* _io_ctx; + io::IOContext* _io_ctx = nullptr; // save source text which have been splitted. std::vector _split_values; diff --git a/be/src/vec/exec/format/file_reader/new_plain_text_line_reader.h b/be/src/vec/exec/format/file_reader/new_plain_text_line_reader.h index 1b35c4c6eb..5588890f98 100644 --- a/be/src/vec/exec/format/file_reader/new_plain_text_line_reader.h +++ b/be/src/vec/exec/format/file_reader/new_plain_text_line_reader.h @@ -210,9 +210,9 @@ private: void extend_input_buf(); void extend_output_buf(); - RuntimeProfile* _profile; + RuntimeProfile* _profile = nullptr; io::FileReaderSPtr _file_reader; - Decompressor* _decompressor; + Decompressor* _decompressor = nullptr; // the min length that should be read. // -1 means endless(for stream load) // and only valid if the content is uncompressed @@ -222,13 +222,13 @@ private: TextLineReaderCtxPtr _line_reader_ctx; // save the data read from file reader - uint8_t* _input_buf; + uint8_t* _input_buf = nullptr; size_t _input_buf_size; size_t _input_buf_pos; size_t _input_buf_limit; // save the data decompressed from decompressor. - uint8_t* _output_buf; + uint8_t* _output_buf = nullptr; size_t _output_buf_size; size_t _output_buf_pos; size_t _output_buf_limit; @@ -241,9 +241,9 @@ private: size_t _current_offset; // Profile counters - RuntimeProfile::Counter* _bytes_read_counter; - RuntimeProfile::Counter* _read_timer; - RuntimeProfile::Counter* _bytes_decompress_counter; - RuntimeProfile::Counter* _decompress_timer; + RuntimeProfile::Counter* _bytes_read_counter = nullptr; + RuntimeProfile::Counter* _read_timer = nullptr; + RuntimeProfile::Counter* _bytes_decompress_counter = nullptr; + RuntimeProfile::Counter* _decompress_timer = nullptr; }; } // namespace doris diff --git a/be/src/vec/exec/format/format_common.h b/be/src/vec/exec/format/format_common.h index bea3322eba..8d1b651c7b 100644 --- a/be/src/vec/exec/format/format_common.h +++ b/be/src/vec/exec/format/format_common.h @@ -133,7 +133,7 @@ private: } uint32_t _num_shards; - KVCache** _shards; + KVCache** _shards = nullptr; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/jni_reader.h b/be/src/vec/exec/format/jni_reader.h index 500301bd4b..5c342383fc 100644 --- a/be/src/vec/exec/format/jni_reader.h +++ b/be/src/vec/exec/format/jni_reader.h @@ -65,8 +65,8 @@ public: private: const std::vector& _file_slot_descs; - RuntimeState* _state; - RuntimeProfile* _profile; + RuntimeState* _state = nullptr; + RuntimeProfile* _profile = nullptr; std::unordered_map* _colname_to_value_range; std::unique_ptr _jni_connector; }; diff --git a/be/src/vec/exec/format/json/new_json_reader.h b/be/src/vec/exec/format/json/new_json_reader.h index 06f245840b..708bceef93 100644 --- a/be/src/vec/exec/format/json/new_json_reader.h +++ b/be/src/vec/exec/format/json/new_json_reader.h @@ -182,9 +182,9 @@ private: Status _fill_missing_column(SlotDescriptor* slot_desc, vectorized::IColumn* column_ptr, bool* valid); - RuntimeState* _state; - RuntimeProfile* _profile; - ScannerCounter* _counter; + RuntimeState* _state = nullptr; + RuntimeProfile* _profile = nullptr; + ScannerCounter* _counter = nullptr; const TFileScanRangeParams& _params; const TFileRangeDesc& _range; io::FileSystemProperties _system_properties; @@ -226,15 +226,15 @@ private: rapidjson::Value* _json_doc; // _json_doc equals _final_json_doc iff not set `json_root` std::unordered_map _name_map; - bool* _scanner_eof; + bool* _scanner_eof = nullptr; size_t _current_offset; - io::IOContext* _io_ctx; + io::IOContext* _io_ctx = nullptr; - RuntimeProfile::Counter* _bytes_read_counter; - RuntimeProfile::Counter* _read_timer; - RuntimeProfile::Counter* _file_read_timer; + RuntimeProfile::Counter* _bytes_read_counter = nullptr; + RuntimeProfile::Counter* _read_timer = nullptr; + RuntimeProfile::Counter* _file_read_timer = nullptr; // ======SIMD JSON====== // name mapping @@ -258,7 +258,7 @@ private: // array_iter pointed to _array simdjson::ondemand::array_iterator _array_iter; simdjson::ondemand::array _array; - std::unique_ptr _ondemand_json_parser = nullptr; + std::unique_ptr _ondemand_json_parser; // column to default value string map std::unordered_map _col_default_value_map; }; diff --git a/be/src/vec/exec/format/orc/vorc_reader.h b/be/src/vec/exec/format/orc/vorc_reader.h index eaf8d34be2..0384b0b979 100644 --- a/be/src/vec/exec/format/orc/vorc_reader.h +++ b/be/src/vec/exec/format/orc/vorc_reader.h @@ -191,16 +191,16 @@ public: private: struct OrcProfile { - RuntimeProfile::Counter* read_time; - RuntimeProfile::Counter* read_calls; - RuntimeProfile::Counter* read_bytes; + RuntimeProfile::Counter* read_time = nullptr; + RuntimeProfile::Counter* read_calls = nullptr; + RuntimeProfile::Counter* read_bytes = nullptr; RuntimeProfile::Counter* column_read_time; - RuntimeProfile::Counter* get_batch_time; - RuntimeProfile::Counter* create_reader_time; - RuntimeProfile::Counter* init_column_time; - RuntimeProfile::Counter* set_fill_column_time; - RuntimeProfile::Counter* decode_value_time; - RuntimeProfile::Counter* decode_null_map_time; + RuntimeProfile::Counter* get_batch_time = nullptr; + RuntimeProfile::Counter* create_reader_time = nullptr; + RuntimeProfile::Counter* init_column_time = nullptr; + RuntimeProfile::Counter* set_fill_column_time = nullptr; + RuntimeProfile::Counter* decode_value_time = nullptr; + RuntimeProfile::Counter* decode_null_map_time = nullptr; }; class ORCFilterImpl : public orc::ORCFilter { @@ -213,7 +213,7 @@ private: } private: - OrcReader* orcReader; + OrcReader* orcReader = nullptr; }; class StringDictFilterImpl : public orc::StringDictFilter { @@ -235,7 +235,7 @@ private: } private: - OrcReader* _orc_reader; + OrcReader* _orc_reader = nullptr; }; // Create inner orc file, @@ -527,7 +527,7 @@ private: std::shared_ptr _file_system; - io::IOContext* _io_ctx; + io::IOContext* _io_ctx = nullptr; bool _enable_lazy_mat = true; std::vector _decimal_scale_params; @@ -535,15 +535,15 @@ private: std::unordered_map* _colname_to_value_range; bool _is_acid = false; - std::unique_ptr _filter = nullptr; + std::unique_ptr _filter; LazyReadContext _lazy_read_ctx; const TransactionalHiveReader::AcidRowIDSet* _delete_rows = nullptr; - std::unique_ptr _delete_rows_filter_ptr = nullptr; + std::unique_ptr _delete_rows_filter_ptr; - const TupleDescriptor* _tuple_descriptor; - const RowDescriptor* _row_descriptor; + const TupleDescriptor* _tuple_descriptor = nullptr; + const RowDescriptor* _row_descriptor = nullptr; VExprContextSPtrs _not_single_slot_filter_conjuncts; - const std::unordered_map* _slot_id_to_filter_conjuncts; + const std::unordered_map* _slot_id_to_filter_conjuncts = nullptr; VExprContextSPtrs _dict_filter_conjuncts; VExprContextSPtrs _non_dict_filter_conjuncts; VExprContextSPtrs _filter_conjuncts; @@ -585,9 +585,9 @@ private: io::FileReaderSPtr _inner_reader; io::FileReaderSPtr _file_reader; // Owned by OrcReader - OrcReader::Statistics* _statistics; - const io::IOContext* _io_ctx; - RuntimeProfile* _profile; + OrcReader::Statistics* _statistics = nullptr; + const io::IOContext* _io_ctx = nullptr; + RuntimeProfile* _profile = nullptr; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/parquet/decoder.h b/be/src/vec/exec/format/parquet/decoder.h index 4e56dea3d1..efb7d3a594 100644 --- a/be/src/vec/exec/format/parquet/decoder.h +++ b/be/src/vec/exec/format/parquet/decoder.h @@ -161,8 +161,8 @@ protected: } // For dictionary encoding - std::unique_ptr _dict = nullptr; - std::unique_ptr> _index_batch_decoder = nullptr; + std::unique_ptr _dict; + std::unique_ptr> _index_batch_decoder; std::vector _indexes; }; diff --git a/be/src/vec/exec/format/parquet/parquet_column_convert.h b/be/src/vec/exec/format/parquet/parquet_column_convert.h index 8054f9b88e..9273d7a1aa 100644 --- a/be/src/vec/exec/format/parquet/parquet_column_convert.h +++ b/be/src/vec/exec/format/parquet/parquet_column_convert.h @@ -221,7 +221,7 @@ struct ColumnConvert { } public: - ConvertParams* _convert_params; + ConvertParams* _convert_params = nullptr; }; template diff --git a/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h b/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h index daf8512b3b..21ee808e48 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h @@ -186,17 +186,17 @@ private: void _get_uncompressed_levels(const tparquet::DataPageHeaderV2& page_v2, Slice& page_data); ColumnChunkReaderState _state = NOT_INIT; - FieldSchema* _field_schema; + FieldSchema* _field_schema = nullptr; level_t _max_rep_level; level_t _max_def_level; tparquet::LogicalType _parquet_logical_type; - io::BufferedStreamReader* _stream_reader; + io::BufferedStreamReader* _stream_reader = nullptr; tparquet::ColumnMetaData _metadata; // cctz::time_zone* _ctz; - io::IOContext* _io_ctx; + io::IOContext* _io_ctx = nullptr; - std::unique_ptr _page_reader = nullptr; + std::unique_ptr _page_reader; BlockCompressionCodec* _block_compress_codec = nullptr; LevelDecoder _rep_level_decoder; diff --git a/be/src/vec/exec/format/parquet/vparquet_column_reader.h b/be/src/vec/exec/format/parquet/vparquet_column_reader.h index f8061d0485..994908711b 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_column_reader.h @@ -142,12 +142,12 @@ protected: void _generate_read_ranges(int64_t start_index, int64_t end_index, std::list& read_ranges); - FieldSchema* _field_schema; + FieldSchema* _field_schema = nullptr; // When scalar column is the child of nested column, we should turn off the filtering by page index and lazy read. bool _nested_column = false; const std::vector& _row_ranges; - cctz::time_zone* _ctz; - io::IOContext* _io_ctx; + cctz::time_zone* _ctz = nullptr; + io::IOContext* _io_ctx = nullptr; int64_t _current_row_index = 0; int _row_range_index = 0; int64_t _decode_null_map_time = 0; @@ -214,7 +214,7 @@ public: void close() override {} private: - std::unique_ptr _element_reader = nullptr; + std::unique_ptr _element_reader; }; class MapColumnReader : public ParquetColumnReader { @@ -248,8 +248,8 @@ public: void close() override {} private: - std::unique_ptr _key_reader = nullptr; - std::unique_ptr _value_reader = nullptr; + std::unique_ptr _key_reader; + std::unique_ptr _value_reader; }; class StructColumnReader : public ParquetColumnReader { diff --git a/be/src/vec/exec/format/parquet/vparquet_group_reader.h b/be/src/vec/exec/format/parquet/vparquet_group_reader.h index 2a7e163ed0..ad918bd97f 100644 --- a/be/src/vec/exec/format/parquet/vparquet_group_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_group_reader.h @@ -196,8 +196,8 @@ private: const int32_t _row_group_id; const tparquet::RowGroup& _row_group_meta; int64_t _remaining_rows; - cctz::time_zone* _ctz; - io::IOContext* _io_ctx; + cctz::time_zone* _ctz = nullptr; + io::IOContext* _io_ctx = nullptr; PositionDeleteContext _position_delete_ctx; // merge the row ranges generated from page index and position delete. std::vector _read_ranges; @@ -206,18 +206,18 @@ private: int64_t _lazy_read_filtered_rows = 0; // If continuous batches are skipped, we can cache them to skip a whole page size_t _cached_filtered_rows = 0; - std::unique_ptr _pos_delete_filter_ptr = nullptr; + std::unique_ptr _pos_delete_filter_ptr; int64_t _total_read_rows = 0; - const TupleDescriptor* _tuple_descriptor; - const RowDescriptor* _row_descriptor; - const std::unordered_map* _col_name_to_slot_id; + const TupleDescriptor* _tuple_descriptor = nullptr; + const RowDescriptor* _row_descriptor = nullptr; + const std::unordered_map* _col_name_to_slot_id = nullptr; VExprContextSPtrs _not_single_slot_filter_conjuncts; - const std::unordered_map* _slot_id_to_filter_conjuncts; + const std::unordered_map* _slot_id_to_filter_conjuncts = nullptr; VExprContextSPtrs _dict_filter_conjuncts; VExprContextSPtrs _filter_conjuncts; // std::pair std::vector> _dict_filter_cols; - RuntimeState* _state; + RuntimeState* _state = nullptr; std::shared_ptr _obj_pool; bool _is_row_group_filtered = false; }; diff --git a/be/src/vec/exec/format/parquet/vparquet_page_reader.h b/be/src/vec/exec/format/parquet/vparquet_page_reader.h index 8f93d2d87a..730b9a3001 100644 --- a/be/src/vec/exec/format/parquet/vparquet_page_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_page_reader.h @@ -66,8 +66,8 @@ public: private: enum PageReaderState { INITIALIZED, HEADER_PARSED }; - io::BufferedStreamReader* _reader; - io::IOContext* _io_ctx; + io::BufferedStreamReader* _reader = nullptr; + io::IOContext* _io_ctx = nullptr; tparquet::PageHeader _cur_page_header; Statistics _statistics; PageReaderState _state = INITIALIZED; diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.h b/be/src/vec/exec/format/parquet/vparquet_reader.h index aa4f0014b2..25b00c5bfe 100644 --- a/be/src/vec/exec/format/parquet/vparquet_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_reader.h @@ -153,33 +153,33 @@ public: private: struct ParquetProfile { - RuntimeProfile::Counter* filtered_row_groups; - RuntimeProfile::Counter* to_read_row_groups; - RuntimeProfile::Counter* filtered_group_rows; - RuntimeProfile::Counter* filtered_page_rows; - RuntimeProfile::Counter* lazy_read_filtered_rows; - RuntimeProfile::Counter* filtered_bytes; - RuntimeProfile::Counter* raw_rows_read; - RuntimeProfile::Counter* to_read_bytes; - RuntimeProfile::Counter* column_read_time; - RuntimeProfile::Counter* parse_meta_time; - RuntimeProfile::Counter* parse_footer_time; - RuntimeProfile::Counter* open_file_time; - RuntimeProfile::Counter* open_file_num; - RuntimeProfile::Counter* row_group_filter_time; - RuntimeProfile::Counter* page_index_filter_time; + RuntimeProfile::Counter* filtered_row_groups = nullptr; + RuntimeProfile::Counter* to_read_row_groups = nullptr; + RuntimeProfile::Counter* filtered_group_rows = nullptr; + RuntimeProfile::Counter* filtered_page_rows = nullptr; + RuntimeProfile::Counter* lazy_read_filtered_rows = nullptr; + RuntimeProfile::Counter* filtered_bytes = nullptr; + RuntimeProfile::Counter* raw_rows_read = nullptr; + RuntimeProfile::Counter* to_read_bytes = nullptr; + RuntimeProfile::Counter* column_read_time = nullptr; + RuntimeProfile::Counter* parse_meta_time = nullptr; + RuntimeProfile::Counter* parse_footer_time = nullptr; + RuntimeProfile::Counter* open_file_time = nullptr; + RuntimeProfile::Counter* open_file_num = nullptr; + RuntimeProfile::Counter* row_group_filter_time = nullptr; + RuntimeProfile::Counter* page_index_filter_time = nullptr; - RuntimeProfile::Counter* file_read_time; - RuntimeProfile::Counter* file_read_calls; - RuntimeProfile::Counter* file_meta_read_calls; - RuntimeProfile::Counter* file_read_bytes; - RuntimeProfile::Counter* decompress_time; - RuntimeProfile::Counter* decompress_cnt; - RuntimeProfile::Counter* decode_header_time; - RuntimeProfile::Counter* decode_value_time; - RuntimeProfile::Counter* decode_dict_time; - RuntimeProfile::Counter* decode_level_time; - RuntimeProfile::Counter* decode_null_map_time; + RuntimeProfile::Counter* file_read_time = nullptr; + RuntimeProfile::Counter* file_read_calls = nullptr; + RuntimeProfile::Counter* file_meta_read_calls = nullptr; + RuntimeProfile::Counter* file_read_bytes = nullptr; + RuntimeProfile::Counter* decompress_time = nullptr; + RuntimeProfile::Counter* decompress_cnt = nullptr; + RuntimeProfile::Counter* decode_header_time = nullptr; + RuntimeProfile::Counter* decode_value_time = nullptr; + RuntimeProfile::Counter* decode_dict_time = nullptr; + RuntimeProfile::Counter* decode_level_time = nullptr; + RuntimeProfile::Counter* decode_null_map_time = nullptr; }; Status _open_file(); @@ -211,7 +211,7 @@ private: std::vector _generate_random_access_ranges( const RowGroupReader::RowGroupIndex& group, size_t* avg_io_size); - RuntimeProfile* _profile; + RuntimeProfile* _profile = nullptr; const TFileScanRangeParams& _scan_params; const TFileRangeDesc& _scan_range; io::FileSystemProperties _system_properties; @@ -227,17 +227,17 @@ private: ObjLRUCache::CacheHandle _meta_cache_handle; std::unique_ptr _file_metadata_ptr; FileMetaData* _file_metadata = nullptr; - const tparquet::FileMetaData* _t_metadata; + const tparquet::FileMetaData* _t_metadata = nullptr; - std::shared_ptr _file_system = nullptr; + std::shared_ptr _file_system; io::FileReaderSPtr _file_reader = nullptr; - std::unique_ptr _current_group_reader = nullptr; + std::unique_ptr _current_group_reader; // read to the end of current reader bool _row_group_eof = true; int32_t _total_groups; // num of groups(stripes) of a parquet(orc) file // table column name to file column name map. For iceberg schema evolution. std::unordered_map _table_col_to_file_col; - std::unordered_map* _colname_to_value_range; + std::unordered_map* _colname_to_value_range = nullptr; std::vector _read_columns; RowRange _whole_range = RowRange(0, 0); const std::vector* _delete_rows = nullptr; @@ -251,26 +251,26 @@ private: size_t _batch_size; int64_t _range_start_offset; int64_t _range_size; - cctz::time_zone* _ctz; + cctz::time_zone* _ctz = nullptr; std::unordered_map _col_offsets; - const std::vector* _column_names; + const std::vector* _column_names = nullptr; std::vector _missing_cols; Statistics _statistics; ParquetColumnReader::Statistics _column_statistics; ParquetProfile _parquet_profile; bool _closed = false; - io::IOContext* _io_ctx; - RuntimeState* _state; + io::IOContext* _io_ctx = nullptr; + RuntimeState* _state = nullptr; // Cache to save some common part such as file footer. // Maybe null if not used FileMetaCache* _meta_cache = nullptr; bool _enable_lazy_mat = true; - const TupleDescriptor* _tuple_descriptor; - const RowDescriptor* _row_descriptor; - const std::unordered_map* _colname_to_slot_id; - const VExprContextSPtrs* _not_single_slot_filter_conjuncts; - const std::unordered_map* _slot_id_to_filter_conjuncts; + const TupleDescriptor* _tuple_descriptor = nullptr; + const RowDescriptor* _row_descriptor = nullptr; + const std::unordered_map* _colname_to_slot_id = nullptr; + const VExprContextSPtrs* _not_single_slot_filter_conjuncts = nullptr; + const std::unordered_map* _slot_id_to_filter_conjuncts = nullptr; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/table/max_compute_jni_reader.h b/be/src/vec/exec/format/table/max_compute_jni_reader.h index 0b3c809c50..94ac23d5dc 100644 --- a/be/src/vec/exec/format/table/max_compute_jni_reader.h +++ b/be/src/vec/exec/format/table/max_compute_jni_reader.h @@ -68,13 +68,13 @@ public: std::unordered_map* colname_to_value_range); private: - const MaxComputeTableDescriptor* _table_desc; + const MaxComputeTableDescriptor* _table_desc = nullptr; const std::vector& _file_slot_descs; const TFileRangeDesc& _range; - RuntimeState* _state; - RuntimeProfile* _profile; - std::unordered_map* _colname_to_value_range; - std::unique_ptr _jni_connector; + RuntimeState* _state = nullptr; + RuntimeProfile* _profile = nullptr; + std::unordered_map* _colname_to_value_range = nullptr; + std::unique_ptr _jni_connector = nullptr; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/table/paimon_reader.h b/be/src/vec/exec/format/table/paimon_reader.h index 33d67a4693..9a0ae8d564 100644 --- a/be/src/vec/exec/format/table/paimon_reader.h +++ b/be/src/vec/exec/format/table/paimon_reader.h @@ -68,8 +68,8 @@ public: private: const std::vector& _file_slot_descs; - RuntimeState* _state; - RuntimeProfile* _profile; + RuntimeState* _state = nullptr; + RuntimeProfile* _profile = nullptr; std::unordered_map* _colname_to_value_range; std::unique_ptr _jni_connector; }; diff --git a/be/src/vec/exec/format/table/transactional_hive_reader.h b/be/src/vec/exec/format/table/transactional_hive_reader.h index d6ecd069de..41d611a468 100644 --- a/be/src/vec/exec/format/table/transactional_hive_reader.h +++ b/be/src/vec/exec/format/table/transactional_hive_reader.h @@ -113,21 +113,21 @@ public: private: struct TransactionalHiveProfile { - RuntimeProfile::Counter* num_delete_files; - RuntimeProfile::Counter* num_delete_rows; - RuntimeProfile::Counter* delete_files_read_time; + RuntimeProfile::Counter* num_delete_files = nullptr; + RuntimeProfile::Counter* num_delete_rows = nullptr; + RuntimeProfile::Counter* delete_files_read_time = nullptr; }; - RuntimeProfile* _profile; - RuntimeState* _state; + RuntimeProfile* _profile = nullptr; + RuntimeState* _state = nullptr; const TFileScanRangeParams& _params; const TFileRangeDesc& _range; TransactionalHiveProfile _transactional_orc_profile; AcidRowIDSet _delete_rows; - std::unique_ptr _delete_rows_filter_ptr = nullptr; + std::unique_ptr _delete_rows_filter_ptr; std::vector _col_names; - io::IOContext* _io_ctx; + io::IOContext* _io_ctx = nullptr; }; inline bool operator<(const TransactionalHiveReader::AcidRowID& lhs, diff --git a/be/src/vec/exec/format/wal/wal_reader.h b/be/src/vec/exec/format/wal/wal_reader.h index bee48fe146..ed55d6e166 100644 --- a/be/src/vec/exec/format/wal/wal_reader.h +++ b/be/src/vec/exec/format/wal/wal_reader.h @@ -34,11 +34,11 @@ public: std::vector get_index() { return _column_index; } private: - RuntimeState* _state; + RuntimeState* _state = nullptr; std::string _wal_path; std::string _path_split = "/"; int64_t _wal_id; - std::shared_ptr _wal_reader = nullptr; + std::shared_ptr _wal_reader; uint32_t _version = 0; std::string _col_ids; std::vector _column_index; diff --git a/be/src/vec/exec/jni_connector.h b/be/src/vec/exec/jni_connector.h index ec678d23fb..a7084ca5eb 100644 --- a/be/src/vec/exec/jni_connector.h +++ b/be/src/vec/exec/jni_connector.h @@ -282,11 +282,11 @@ private: std::vector _column_names; bool _is_table_schema = false; - RuntimeState* _state; - RuntimeProfile* _profile; - RuntimeProfile::Counter* _open_scanner_time; - RuntimeProfile::Counter* _java_scan_time; - RuntimeProfile::Counter* _fill_block_time; + RuntimeState* _state = nullptr; + RuntimeProfile* _profile = nullptr; + RuntimeProfile::Counter* _open_scanner_time = nullptr; + RuntimeProfile::Counter* _java_scan_time = nullptr; + RuntimeProfile::Counter* _fill_block_time = nullptr; std::map _scanner_profile; size_t _has_read = 0; @@ -306,7 +306,7 @@ private: TableMetaAddress _table_meta; int _predicates_length = 0; - std::unique_ptr _predicates = nullptr; + std::unique_ptr _predicates; /** * Set the address of meta information, which is returned by org.apache.doris.common.jni.JniScanner#getNextBatchMeta diff --git a/be/src/vec/exec/join/join_op.h b/be/src/vec/exec/join/join_op.h index 1b8b8f2c69..8b8efe7389 100644 --- a/be/src/vec/exec/join/join_op.h +++ b/be/src/vec/exec/join/join_op.h @@ -66,7 +66,7 @@ struct Batch { static constexpr uint32_t MAX_SIZE = 7; /// Adequate values are 3, 7, 15, 31. uint8_t size = 0; /// It's smaller than size_t but keeps align in Arena. - Batch* next; + Batch* next = nullptr; RowRefType row_refs[MAX_SIZE]; Batch(Batch* parent) : next(parent) {} @@ -121,9 +121,9 @@ public: bool ok() const { return first || batch; } private: - RowRefListType* root; + RowRefListType* root = nullptr; bool first; - Batch* batch; + Batch* batch = nullptr; size_t position; }; diff --git a/be/src/vec/exec/join/process_hash_table_probe.h b/be/src/vec/exec/join/process_hash_table_probe.h index 435cea8418..df75952820 100644 --- a/be/src/vec/exec/join/process_hash_table_probe.h +++ b/be/src/vec/exec/join/process_hash_table_probe.h @@ -92,7 +92,7 @@ struct ProcessHashTableProbe { Status process_data_in_hashtable(HashTableType& hash_table_ctx, MutableBlock& mutable_block, Block* output_block, bool* eos); - Parent* _parent; + Parent* _parent = nullptr; const int _batch_size; std::shared_ptr> _build_blocks; std::unique_ptr _arena; @@ -103,12 +103,12 @@ struct ProcessHashTableProbe { PaddedPODArray _build_block_rows; std::vector> _build_blocks_locs; // only need set the tuple is null in RIGHT_OUTER_JOIN and FULL_OUTER_JOIN - ColumnUInt8::Container* _tuple_is_null_left_flags; + ColumnUInt8::Container* _tuple_is_null_left_flags = nullptr; // only need set the tuple is null in LEFT_OUTER_JOIN and FULL_OUTER_JOIN - ColumnUInt8::Container* _tuple_is_null_right_flags; + ColumnUInt8::Container* _tuple_is_null_right_flags = nullptr; size_t _serialized_key_buffer_size {0}; - uint8_t* _serialized_key_buffer; + uint8_t* _serialized_key_buffer = nullptr; std::unique_ptr _serialize_key_arena; std::vector _probe_side_find_result; @@ -121,15 +121,15 @@ struct ProcessHashTableProbe { bool _have_other_join_conjunct; bool _is_right_semi_anti; - std::vector* _left_output_slot_flags; - std::vector* _right_output_slot_flags; + std::vector* _left_output_slot_flags = nullptr; + std::vector* _right_output_slot_flags = nullptr; bool* _has_null_in_build_side; - RuntimeProfile::Counter* _rows_returned_counter; - RuntimeProfile::Counter* _search_hashtable_timer; - RuntimeProfile::Counter* _build_side_output_timer; - RuntimeProfile::Counter* _probe_side_output_timer; - RuntimeProfile::Counter* _probe_process_hashtable_timer; + RuntimeProfile::Counter* _rows_returned_counter = nullptr; + RuntimeProfile::Counter* _search_hashtable_timer = nullptr; + RuntimeProfile::Counter* _build_side_output_timer = nullptr; + RuntimeProfile::Counter* _probe_side_output_timer = nullptr; + RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; static constexpr int PROBE_SIDE_EXPLODE_RATE = 1; }; diff --git a/be/src/vec/exec/join/vhash_join_node.h b/be/src/vec/exec/join/vhash_join_node.h index 06c28de46c..53cb247d0a 100644 --- a/be/src/vec/exec/join/vhash_join_node.h +++ b/be/src/vec/exec/join/vhash_join_node.h @@ -237,12 +237,12 @@ private: int _skip_rows; Block& _acquired_block; ColumnRawPtrs& _build_raw_ptrs; - Parent* _parent; + Parent* _parent = nullptr; int _batch_size; uint8_t _offset; - RuntimeState* _state; + RuntimeState* _state = nullptr; - ProfileCounter* _build_side_compute_hash_timer; + ProfileCounter* _build_side_compute_hash_timer = nullptr; }; template @@ -427,34 +427,34 @@ private: DataTypes _left_table_data_types; std::vector _right_table_column_names; - RuntimeProfile::Counter* _build_table_timer; - RuntimeProfile::Counter* _build_expr_call_timer; - RuntimeProfile::Counter* _build_table_insert_timer; - RuntimeProfile::Counter* _build_table_expanse_timer; - RuntimeProfile::Counter* _build_table_convert_timer; - RuntimeProfile::Counter* _probe_expr_call_timer; - RuntimeProfile::Counter* _probe_next_timer; - RuntimeProfile::Counter* _build_buckets_counter; - RuntimeProfile::Counter* _build_buckets_fill_counter; - RuntimeProfile::Counter* _search_hashtable_timer; - RuntimeProfile::Counter* _build_side_output_timer; - RuntimeProfile::Counter* _probe_side_output_timer; - RuntimeProfile::Counter* _probe_process_hashtable_timer; - RuntimeProfile::Counter* _build_side_compute_hash_timer; - RuntimeProfile::Counter* _build_side_merge_block_timer; - RuntimeProfile::Counter* _build_runtime_filter_timer; + RuntimeProfile::Counter* _build_table_timer = nullptr; + RuntimeProfile::Counter* _build_expr_call_timer = nullptr; + RuntimeProfile::Counter* _build_table_insert_timer = nullptr; + RuntimeProfile::Counter* _build_table_expanse_timer = nullptr; + RuntimeProfile::Counter* _build_table_convert_timer = nullptr; + RuntimeProfile::Counter* _probe_expr_call_timer = nullptr; + RuntimeProfile::Counter* _probe_next_timer = nullptr; + RuntimeProfile::Counter* _build_buckets_counter = nullptr; + RuntimeProfile::Counter* _build_buckets_fill_counter = nullptr; + RuntimeProfile::Counter* _search_hashtable_timer = nullptr; + RuntimeProfile::Counter* _build_side_output_timer = nullptr; + RuntimeProfile::Counter* _probe_side_output_timer = nullptr; + RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; + RuntimeProfile::Counter* _build_side_compute_hash_timer = nullptr; + RuntimeProfile::Counter* _build_side_merge_block_timer = nullptr; + RuntimeProfile::Counter* _build_runtime_filter_timer = nullptr; - RuntimeProfile::Counter* _build_collisions_counter; + RuntimeProfile::Counter* _build_collisions_counter = nullptr; - RuntimeProfile::Counter* _open_timer; - RuntimeProfile::Counter* _allocate_resource_timer; - RuntimeProfile::Counter* _process_other_join_conjunct_timer; + RuntimeProfile::Counter* _open_timer = nullptr; + RuntimeProfile::Counter* _allocate_resource_timer = nullptr; + RuntimeProfile::Counter* _process_other_join_conjunct_timer = nullptr; - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::Counter* _build_blocks_memory_usage; - RuntimeProfile::Counter* _hash_table_memory_usage; - RuntimeProfile::HighWaterMarkCounter* _build_arena_memory_usage; - RuntimeProfile::HighWaterMarkCounter* _probe_arena_memory_usage; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; + RuntimeProfile::Counter* _build_blocks_memory_usage = nullptr; + RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; + RuntimeProfile::HighWaterMarkCounter* _build_arena_memory_usage = nullptr; + RuntimeProfile::HighWaterMarkCounter* _probe_arena_memory_usage = nullptr; std::shared_ptr _arena; @@ -483,8 +483,8 @@ private: bool _is_broadcast_join = false; bool _should_build_hash_table = true; - std::shared_ptr _shared_hashtable_controller = nullptr; - std::shared_ptr _runtime_filter_slots = nullptr; + std::shared_ptr _shared_hashtable_controller; + std::shared_ptr _runtime_filter_slots; std::vector _hash_output_slot_ids; std::vector _left_output_slot_flags; diff --git a/be/src/vec/exec/join/vjoin_node_base.h b/be/src/vec/exec/join/vjoin_node_base.h index ad68faa5c0..8d26db22ea 100644 --- a/be/src/vec/exec/join/vjoin_node_base.h +++ b/be/src/vec/exec/join/vjoin_node_base.h @@ -134,18 +134,18 @@ protected: MutableColumnPtr _tuple_is_null_left_flag_column; MutableColumnPtr _tuple_is_null_right_flag_column; - RuntimeProfile* _build_phase_profile; - RuntimeProfile::Counter* _build_timer; - RuntimeProfile::Counter* _build_get_next_timer; - RuntimeProfile::Counter* _build_rows_counter; + RuntimeProfile* _build_phase_profile = nullptr; + RuntimeProfile::Counter* _build_timer = nullptr; + RuntimeProfile::Counter* _build_get_next_timer = nullptr; + RuntimeProfile::Counter* _build_rows_counter = nullptr; - RuntimeProfile* _probe_phase_profile; - RuntimeProfile::Counter* _probe_timer; - RuntimeProfile::Counter* _probe_rows_counter; - RuntimeProfile::Counter* _push_down_timer; - RuntimeProfile::Counter* _push_compute_timer; - RuntimeProfile::Counter* _join_filter_timer; - RuntimeProfile::Counter* _build_output_block_timer; + RuntimeProfile* _probe_phase_profile = nullptr; + RuntimeProfile::Counter* _probe_timer = nullptr; + RuntimeProfile::Counter* _probe_rows_counter = nullptr; + RuntimeProfile::Counter* _push_down_timer = nullptr; + RuntimeProfile::Counter* _push_compute_timer = nullptr; + RuntimeProfile::Counter* _join_filter_timer = nullptr; + RuntimeProfile::Counter* _build_output_block_timer = nullptr; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/join/vnested_loop_join_node.h b/be/src/vec/exec/join/vnested_loop_join_node.h index bc5944eefa..7dd63b498f 100644 --- a/be/src/vec/exec/join/vnested_loop_join_node.h +++ b/be/src/vec/exec/join/vnested_loop_join_node.h @@ -53,7 +53,7 @@ struct RuntimeFilterBuild { Status operator()(RuntimeState* state); private: - Parent* _parent; + Parent* _parent = nullptr; }; // Node for nested loop joins. @@ -274,7 +274,7 @@ private: std::stack _build_offset_stack; std::stack _probe_offset_stack; VExprContextSPtrs _join_conjuncts; - RuntimeProfile::Counter* _loop_join_timer; + RuntimeProfile::Counter* _loop_join_timer = nullptr; template friend struct RuntimeFilterBuild; }; diff --git a/be/src/vec/exec/runtime_filter_consumer.h b/be/src/vec/exec/runtime_filter_consumer.h index b3449197b0..3fa822b4b7 100644 --- a/be/src/vec/exec/runtime_filter_consumer.h +++ b/be/src/vec/exec/runtime_filter_consumer.h @@ -57,7 +57,7 @@ protected: RuntimeFilterContext(IRuntimeFilter* rf) : apply_mark(false), runtime_filter(rf) {} // set to true if this runtime filter is already applied to vconjunct_ctx_ptr bool apply_mark; - IRuntimeFilter* runtime_filter; + IRuntimeFilter* runtime_filter = nullptr; }; std::vector _runtime_filter_ctxs; @@ -65,7 +65,7 @@ protected: std::vector _runtime_filter_ready_flag; std::mutex _rf_locks; phmap::flat_hash_set _rf_vexpr_set; - RuntimeState* _state; + RuntimeState* _state = nullptr; private: int32_t _filter_id; diff --git a/be/src/vec/exec/scan/mysql_scanner.h b/be/src/vec/exec/scan/mysql_scanner.h index 8aa324f1e1..63c688460b 100644 --- a/be/src/vec/exec/scan/mysql_scanner.h +++ b/be/src/vec/exec/scan/mysql_scanner.h @@ -65,8 +65,8 @@ private: Status _error_status(const std::string& prefix); const MysqlScannerParam& _my_param; - __DorisMysql* _my_conn; - __DorisMysqlRes* _my_result; + __DorisMysql* _my_conn = nullptr; + __DorisMysqlRes* _my_result = nullptr; std::string _sql_str; bool _is_open; int _field_num; diff --git a/be/src/vec/exec/scan/new_es_scan_node.h b/be/src/vec/exec/scan/new_es_scan_node.h index 5185ead56b..91abe0b48b 100644 --- a/be/src/vec/exec/scan/new_es_scan_node.h +++ b/be/src/vec/exec/scan/new_es_scan_node.h @@ -65,7 +65,7 @@ protected: private: TupleId _tuple_id; - TupleDescriptor* _tuple_desc; + TupleDescriptor* _tuple_desc = nullptr; std::map _properties; std::map _fields_context; @@ -75,17 +75,17 @@ private: std::vector _column_names; // Profile - std::unique_ptr _es_profile; + std::unique_ptr _es_profile = nullptr; // FIXME: non-static data member '_rows_read_counter' of 'NewEsScanNode' shadows member inherited from type 'VScanNode' #ifdef __clang__ #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wshadow-field" #endif - RuntimeProfile::Counter* _rows_read_counter; + RuntimeProfile::Counter* _rows_read_counter = nullptr; #ifdef __clang__ #pragma clang diagnostic pop #endif - RuntimeProfile::Counter* _read_timer; - RuntimeProfile::Counter* _materialize_timer; + RuntimeProfile::Counter* _read_timer = nullptr; + RuntimeProfile::Counter* _materialize_timer = nullptr; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/scan/new_es_scanner.h b/be/src/vec/exec/scan/new_es_scanner.h index 247fb3ff33..dd62e2d035 100644 --- a/be/src/vec/exec/scan/new_es_scanner.h +++ b/be/src/vec/exec/scan/new_es_scanner.h @@ -82,7 +82,7 @@ private: bool _batch_eof; TupleId _tuple_id; - const TupleDescriptor* _tuple_desc; + const TupleDescriptor* _tuple_desc = nullptr; std::unique_ptr _es_reader; std::unique_ptr _es_scroll_parser; diff --git a/be/src/vec/exec/scan/new_jdbc_scanner.h b/be/src/vec/exec/scan/new_jdbc_scanner.h index cca0a29fe7..3d60c7a656 100644 --- a/be/src/vec/exec/scan/new_jdbc_scanner.h +++ b/be/src/vec/exec/scan/new_jdbc_scanner.h @@ -80,7 +80,7 @@ private: // SQL std::string _query_string; // Descriptor of tuples read from JDBC table. - const TupleDescriptor* _tuple_desc; + const TupleDescriptor* _tuple_desc = nullptr; // the sql query database type: like mysql, PG... TOdbcTableType::type _table_type; // Scanner of JDBC. diff --git a/be/src/vec/exec/scan/new_odbc_scanner.h b/be/src/vec/exec/scan/new_odbc_scanner.h index e2388c86a0..57f67bac25 100644 --- a/be/src/vec/exec/scan/new_odbc_scanner.h +++ b/be/src/vec/exec/scan/new_odbc_scanner.h @@ -73,7 +73,7 @@ private: TupleId _tuple_id; // Descriptor of tuples read from ODBC table. - const TupleDescriptor* _tuple_desc; + const TupleDescriptor* _tuple_desc = nullptr; // Scanner of ODBC. std::unique_ptr _odbc_connector; diff --git a/be/src/vec/exec/scan/new_olap_scanner.h b/be/src/vec/exec/scan/new_olap_scanner.h index ccb5572846..e4864b67f4 100644 --- a/be/src/vec/exec/scan/new_olap_scanner.h +++ b/be/src/vec/exec/scan/new_olap_scanner.h @@ -55,8 +55,8 @@ class NewOlapScanner : public VScanner { public: struct Params { - RuntimeState* state; - RuntimeProfile* profile; + RuntimeState* state = nullptr; + RuntimeProfile* profile = nullptr; std::vector key_ranges; BaseTabletSPtr tablet; int64_t version; diff --git a/be/src/vec/exec/scan/scan_task_queue.h b/be/src/vec/exec/scan/scan_task_queue.h index c694859e3c..aeda9a4adc 100644 --- a/be/src/vec/exec/scan/scan_task_queue.h +++ b/be/src/vec/exec/scan/scan_task_queue.h @@ -42,7 +42,7 @@ struct ScanTask { } WorkFunction scan_func; - vectorized::ScannerContext* scanner_context; + vectorized::ScannerContext* scanner_context = nullptr; TGSTEntityPtr scan_entity; int priority; }; diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index 08dcb9679b..6afe03d535 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -188,12 +188,12 @@ protected: void _set_scanner_done(); - RuntimeState* _state; - VScanNode* _parent; - pipeline::ScanLocalStateBase* _local_state; + RuntimeState* _state = nullptr; + VScanNode* _parent = nullptr; + pipeline::ScanLocalStateBase* _local_state = nullptr; // the comment of same fields in VScanNode - const TupleDescriptor* _output_tuple_desc; + const TupleDescriptor* _output_tuple_desc = nullptr; // _transfer_lock is used to protect the critical section // where the ScanNode and ScannerScheduler interact. @@ -282,8 +282,8 @@ protected: RuntimeProfile::Counter* _newly_create_free_blocks_num = nullptr; RuntimeProfile::Counter* _scanner_wait_batch_timer = nullptr; - std::shared_ptr _dependency = nullptr; - std::shared_ptr _finish_dependency = nullptr; + std::shared_ptr _dependency; + std::shared_ptr _finish_dependency; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/exec/scan/scanner_scheduler.h b/be/src/vec/exec/scan/scanner_scheduler.h index 27befc4328..c561dba227 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.h +++ b/be/src/vec/exec/scan/scanner_scheduler.h @@ -102,7 +102,7 @@ private: // and put it to the _scheduling_map. // If any scanner finish, it will take ctx from and put it to pending queue again. std::atomic_uint _queue_idx = {0}; - BlockingQueue** _pending_queues; + BlockingQueue** _pending_queues = nullptr; // scheduling thread pool std::unique_ptr _scheduler_pool; @@ -132,7 +132,7 @@ struct SimplifiedScanTask { } std::function scan_func; - vectorized::ScannerContext* scanner_context; + vectorized::ScannerContext* scanner_context = nullptr; }; // used for cpu hard limit @@ -186,7 +186,7 @@ private: std::unique_ptr _scan_thread_pool; std::unique_ptr> _scan_task_queue; std::atomic _is_stop; - CgroupCpuCtl* _cgroup_cpu_ctl; + CgroupCpuCtl* _cgroup_cpu_ctl = nullptr; std::string _wg_name; }; diff --git a/be/src/vec/exec/scan/vfile_scanner.h b/be/src/vec/exec/scan/vfile_scanner.h index eeed2145ee..76aac26ad1 100644 --- a/be/src/vec/exec/scan/vfile_scanner.h +++ b/be/src/vec/exec/scan/vfile_scanner.h @@ -92,13 +92,13 @@ protected: Status _cast_src_block(Block* block) { return Status::OK(); } protected: - const TFileScanRangeParams* _params; + const TFileScanRangeParams* _params = nullptr; const std::vector& _ranges; int _next_range; std::unique_ptr _cur_reader; bool _cur_reader_eof; - std::unordered_map* _colname_to_value_range; + std::unordered_map* _colname_to_value_range = nullptr; // File source slot descriptors std::vector _file_slot_descs; // col names from _file_slot_descs @@ -144,7 +144,7 @@ protected: // row desc for default exprs std::unique_ptr _default_val_row_desc; // owned by scan node - ShardedKVCache* _kv_cache; + ShardedKVCache* _kv_cache = nullptr; bool _scanner_eof = false; int _rows = 0; @@ -154,7 +154,7 @@ protected: bool _strict_mode; bool _src_block_init = false; - Block* _src_block_ptr; + Block* _src_block_ptr = nullptr; Block _src_block; VExprContextSPtrs _push_down_conjuncts; @@ -178,7 +178,7 @@ private: RuntimeProfile::Counter* _file_counter = nullptr; RuntimeProfile::Counter* _has_fully_rf_file_counter = nullptr; - const std::unordered_map* _col_name_to_slot_id; + const std::unordered_map* _col_name_to_slot_id = nullptr; // single slot filter conjuncts std::unordered_map _slot_id_to_filter_conjuncts; // not single(zero or multi) slot filter conjuncts diff --git a/be/src/vec/exec/scan/vmeta_scanner.h b/be/src/vec/exec/scan/vmeta_scanner.h index e6eb32e81b..66e594231f 100644 --- a/be/src/vec/exec/scan/vmeta_scanner.h +++ b/be/src/vec/exec/scan/vmeta_scanner.h @@ -88,7 +88,7 @@ private: bool _meta_eos; TupleId _tuple_id; TUserIdentity _user_identity; - const TupleDescriptor* _tuple_desc; + const TupleDescriptor* _tuple_desc = nullptr; std::vector _batch_data; const TScanRange& _scan_range; }; diff --git a/be/src/vec/exec/scan/vscan_node.h b/be/src/vec/exec/scan/vscan_node.h index 3fe6a8483e..7ef0656956 100644 --- a/be/src/vec/exec/scan/vscan_node.h +++ b/be/src/vec/exec/scan/vscan_node.h @@ -316,11 +316,11 @@ protected: std::shared_ptr _scanner_profile; // rows read from the scanner (including those discarded by (pre)filters) - RuntimeProfile::Counter* _rows_read_counter; - RuntimeProfile::Counter* _byte_read_counter; + RuntimeProfile::Counter* _rows_read_counter = nullptr; + RuntimeProfile::Counter* _byte_read_counter = nullptr; // Wall based aggregate read throughput [rows/sec] - RuntimeProfile::Counter* _total_throughput_counter; - RuntimeProfile::Counter* _num_scanners; + RuntimeProfile::Counter* _total_throughput_counter = nullptr; + RuntimeProfile::Counter* _num_scanners = nullptr; RuntimeProfile::Counter* _get_next_timer = nullptr; RuntimeProfile::Counter* _open_timer = nullptr; @@ -346,9 +346,9 @@ protected: // Max num of scanner thread RuntimeProfile::Counter* _max_scanner_thread_num = nullptr; - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::HighWaterMarkCounter* _queued_blocks_memory_usage; - RuntimeProfile::HighWaterMarkCounter* _free_blocks_memory_usage; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; + RuntimeProfile::HighWaterMarkCounter* _queued_blocks_memory_usage = nullptr; + RuntimeProfile::HighWaterMarkCounter* _free_blocks_memory_usage = nullptr; std::unordered_map _colname_to_slot_id; std::vector _col_distribute_ids; diff --git a/be/src/vec/exec/scan/vscanner.h b/be/src/vec/exec/scan/vscanner.h index 1abd20ba27..c358e2932d 100644 --- a/be/src/vec/exec/scan/vscanner.h +++ b/be/src/vec/exec/scan/vscanner.h @@ -160,13 +160,13 @@ protected: _conjuncts.clear(); } - RuntimeState* _state; - VScanNode* _parent; - pipeline::ScanLocalStateBase* _local_state; + RuntimeState* _state = nullptr; + VScanNode* _parent = nullptr; + pipeline::ScanLocalStateBase* _local_state = nullptr; // Set if scan node has sort limit info int64_t _limit = -1; - RuntimeProfile* _profile; + RuntimeProfile* _profile = nullptr; const TupleDescriptor* _output_tuple_desc = nullptr; diff --git a/be/src/vec/exec/vaggregation_node.h b/be/src/vec/exec/vaggregation_node.h index f57eb6f10d..cf49817955 100644 --- a/be/src/vec/exec/vaggregation_node.h +++ b/be/src/vec/exec/vaggregation_node.h @@ -252,7 +252,7 @@ public: using Container = std::conditional_t; - Container* container; + Container* container = nullptr; uint32_t index; uint32_t sub_container_index; uint32_t index_in_sub_container; @@ -339,7 +339,7 @@ private: std::vector _key_containers; std::vector _value_containers; AggregateDataPtr _current_agg_data; - char* _current_keys; + char* _current_keys = nullptr; size_t _size_of_key {}; size_t _size_of_aggregate_states {}; uint32_t _index_in_sub_container {}; @@ -354,7 +354,7 @@ struct AggSpillContext { /// stream ids of writers/readers std::vector stream_ids; std::vector readers; - RuntimeProfile* runtime_profile; + RuntimeProfile* runtime_profile = nullptr; size_t read_cursor {}; @@ -419,11 +419,11 @@ protected: // left / full join will change the key nullable make output/input solt // nullable diff. so we need make nullable of it. std::vector _make_nullable_keys; - RuntimeProfile::Counter* _hash_table_compute_timer; - RuntimeProfile::Counter* _hash_table_emplace_timer; - RuntimeProfile::Counter* _hash_table_input_counter; - RuntimeProfile::Counter* _build_timer; - RuntimeProfile::Counter* _expr_timer; + RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; + RuntimeProfile::Counter* _hash_table_emplace_timer = nullptr; + RuntimeProfile::Counter* _hash_table_input_counter = nullptr; + RuntimeProfile::Counter* _build_timer = nullptr; + RuntimeProfile::Counter* _expr_timer = nullptr; private: friend class pipeline::AggSinkOperator; @@ -436,10 +436,10 @@ private: // may be we don't have to know the tuple id TupleId _intermediate_tuple_id; - TupleDescriptor* _intermediate_tuple_desc; + TupleDescriptor* _intermediate_tuple_desc = nullptr; TupleId _output_tuple_id; - TupleDescriptor* _output_tuple_desc; + TupleDescriptor* _output_tuple_desc = nullptr; bool _needs_finalize; bool _is_merge; @@ -458,21 +458,21 @@ private: AggSpillContext _spill_context; std::unique_ptr _spill_partition_helper; - RuntimeProfile::Counter* _build_table_convert_timer; - RuntimeProfile::Counter* _serialize_key_timer; - RuntimeProfile::Counter* _merge_timer; - RuntimeProfile::Counter* _get_results_timer; - RuntimeProfile::Counter* _serialize_data_timer; - RuntimeProfile::Counter* _serialize_result_timer; - RuntimeProfile::Counter* _deserialize_data_timer; - RuntimeProfile::Counter* _hash_table_iterate_timer; - RuntimeProfile::Counter* _insert_keys_to_column_timer; - RuntimeProfile::Counter* _streaming_agg_timer; - RuntimeProfile::Counter* _hash_table_size_counter; - RuntimeProfile::Counter* _max_row_size_counter; - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::Counter* _hash_table_memory_usage; - RuntimeProfile::HighWaterMarkCounter* _serialize_key_arena_memory_usage; + RuntimeProfile::Counter* _build_table_convert_timer = nullptr; + RuntimeProfile::Counter* _serialize_key_timer = nullptr; + RuntimeProfile::Counter* _merge_timer = nullptr; + RuntimeProfile::Counter* _get_results_timer = nullptr; + RuntimeProfile::Counter* _serialize_data_timer = nullptr; + RuntimeProfile::Counter* _serialize_result_timer = nullptr; + RuntimeProfile::Counter* _deserialize_data_timer = nullptr; + RuntimeProfile::Counter* _hash_table_iterate_timer = nullptr; + RuntimeProfile::Counter* _insert_keys_to_column_timer = nullptr; + RuntimeProfile::Counter* _streaming_agg_timer = nullptr; + RuntimeProfile::Counter* _hash_table_size_counter = nullptr; + RuntimeProfile::Counter* _max_row_size_counter = nullptr; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; + RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; + RuntimeProfile::HighWaterMarkCounter* _serialize_key_arena_memory_usage = nullptr; bool _should_expand_hash_table = true; bool _should_limit_output = false; diff --git a/be/src/vec/exec/vanalytic_eval_node.h b/be/src/vec/exec/vanalytic_eval_node.h index cec6b9eeda..8c26c1ad79 100644 --- a/be/src/vec/exec/vanalytic_eval_node.h +++ b/be/src/vec/exec/vanalytic_eval_node.h @@ -74,7 +74,7 @@ public: ~VAnalyticEvalNode() override = default; VAnalyticEvalNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; + Status init(const TPlanNode& tnode, RuntimeState* state) override; Status prepare(RuntimeState* state) override; Status open(RuntimeState* state) override; Status get_next(RuntimeState* state, vectorized::Block* block, bool* eos) override; diff --git a/be/src/vec/exec/vdata_gen_scan_node.h b/be/src/vec/exec/vdata_gen_scan_node.h index 0faa5d0fe3..3f6735957f 100644 --- a/be/src/vec/exec/vdata_gen_scan_node.h +++ b/be/src/vec/exec/vdata_gen_scan_node.h @@ -61,7 +61,7 @@ protected: TupleId _tuple_id; // Descriptor of tuples generated - const TupleDescriptor* _tuple_desc; + const TupleDescriptor* _tuple_desc = nullptr; std::vector _runtime_filter_descs; }; diff --git a/be/src/vec/exec/vjdbc_connector.h b/be/src/vec/exec/vjdbc_connector.h index a62e5e6efa..ba4fb98e0e 100644 --- a/be/src/vec/exec/vjdbc_connector.h +++ b/be/src/vec/exec/vjdbc_connector.h @@ -55,7 +55,7 @@ struct JdbcConnectorParam { bool use_transaction; TOdbcTableType::type table_type; - const TupleDescriptor* tuple_desc; + const TupleDescriptor* tuple_desc = nullptr; }; class JdbcConnector : public TableConnector { diff --git a/be/src/vec/exec/vmysql_scan_node.h b/be/src/vec/exec/vmysql_scan_node.h index 85ecca784e..4f63ed4d35 100644 --- a/be/src/vec/exec/vmysql_scan_node.h +++ b/be/src/vec/exec/vmysql_scan_node.h @@ -70,7 +70,7 @@ private: std::vector _filters; // Descriptor of tuples read from MySQL table. - const TupleDescriptor* _tuple_desc; + const TupleDescriptor* _tuple_desc = nullptr; // Tuple index in tuple row. int _slot_num; // Jni helper for scanning an HBase table. diff --git a/be/src/vec/exec/vpartition_sort_node.h b/be/src/vec/exec/vpartition_sort_node.h index 73b21ee676..acf18bad7b 100644 --- a/be/src/vec/exec/vpartition_sort_node.h +++ b/be/src/vec/exec/vpartition_sort_node.h @@ -221,19 +221,19 @@ private: int _num_partition = 0; int64_t _partition_inner_limit = 0; int _sort_idx = 0; - std::unique_ptr _previous_row = nullptr; + std::unique_ptr _previous_row; std::queue _blocks_buffer; int64_t child_input_rows = 0; std::mutex _buffer_mutex; TPartTopNPhase::type _topn_phase; - RuntimeProfile::Counter* _build_timer; - RuntimeProfile::Counter* _emplace_key_timer; - RuntimeProfile::Counter* _partition_sort_timer; - RuntimeProfile::Counter* _get_sorted_timer; - RuntimeProfile::Counter* _selector_block_timer; + RuntimeProfile::Counter* _build_timer = nullptr; + RuntimeProfile::Counter* _emplace_key_timer = nullptr; + RuntimeProfile::Counter* _partition_sort_timer = nullptr; + RuntimeProfile::Counter* _get_sorted_timer = nullptr; + RuntimeProfile::Counter* _selector_block_timer = nullptr; - RuntimeProfile::Counter* _hash_table_size_counter; + RuntimeProfile::Counter* _hash_table_size_counter = nullptr; //only for profile record std::vector partition_profile_output_rows; }; diff --git a/be/src/vec/exec/vschema_scan_node.h b/be/src/vec/exec/vschema_scan_node.h index 1b551704fb..cb3085ad4f 100644 --- a/be/src/vec/exec/vschema_scan_node.h +++ b/be/src/vec/exec/vschema_scan_node.h @@ -69,7 +69,7 @@ private: TupleId _tuple_id; // Descriptor of dest tuples - const TupleDescriptor* _dest_tuple_desc; + const TupleDescriptor* _dest_tuple_desc = nullptr; // Tuple index in tuple row. int _tuple_idx; // slot num need to fill in and return diff --git a/be/src/vec/exec/vset_operation_node.h b/be/src/vec/exec/vset_operation_node.h index ff016469f4..4e68965a04 100644 --- a/be/src/vec/exec/vset_operation_node.h +++ b/be/src/vec/exec/vset_operation_node.h @@ -123,9 +123,9 @@ private: bool _build_finished; std::vector _probe_finished_children_index; MutableBlock _mutable_block; - RuntimeProfile::Counter* _build_timer; // time to build hash table - RuntimeProfile::Counter* _probe_timer; // time to probe - RuntimeProfile::Counter* _pull_timer; // time to pull data + RuntimeProfile::Counter* _build_timer = nullptr; // time to build hash table + RuntimeProfile::Counter* _probe_timer = nullptr; // time to probe + RuntimeProfile::Counter* _pull_timer = nullptr; // time to pull data Arena _arena; template diff --git a/be/src/vec/exec/vsort_node.h b/be/src/vec/exec/vsort_node.h index 37c781bde2..5b13eb8fa9 100644 --- a/be/src/vec/exec/vsort_node.h +++ b/be/src/vec/exec/vsort_node.h @@ -86,8 +86,8 @@ private: std::vector _is_asc_order; std::vector _nulls_first; - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::Counter* _sort_blocks_memory_usage; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; + RuntimeProfile::Counter* _sort_blocks_memory_usage = nullptr; bool _use_topn_opt = false; // topn top value diff --git a/be/src/vec/exprs/table_function/vexplode_bitmap.h b/be/src/vec/exprs/table_function/vexplode_bitmap.h index 18cdc6bb22..bc11dd7ca8 100644 --- a/be/src/vec/exprs/table_function/vexplode_bitmap.h +++ b/be/src/vec/exprs/table_function/vexplode_bitmap.h @@ -54,7 +54,7 @@ private: // Not own object, just a reference const BitmapValue* _cur_bitmap = nullptr; // iterator of _cur_bitmap - std::unique_ptr _cur_iter = nullptr; + std::unique_ptr _cur_iter; ColumnPtr _value_column; }; diff --git a/be/src/vec/exprs/vectorized_agg_fn.h b/be/src/vec/exprs/vectorized_agg_fn.h index 831b65d09a..b3fb4f6d5e 100644 --- a/be/src/vec/exprs/vectorized_agg_fn.h +++ b/be/src/vec/exprs/vectorized_agg_fn.h @@ -120,12 +120,12 @@ private: const TypeDescriptor _return_type; - const SlotDescriptor* _intermediate_slot_desc; - const SlotDescriptor* _output_slot_desc; + const SlotDescriptor* _intermediate_slot_desc = nullptr; + const SlotDescriptor* _output_slot_desc = nullptr; - RuntimeProfile::Counter* _exec_timer; - RuntimeProfile::Counter* _merge_timer; - RuntimeProfile::Counter* _expr_timer; + RuntimeProfile::Counter* _exec_timer = nullptr; + RuntimeProfile::Counter* _merge_timer = nullptr; + RuntimeProfile::Counter* _expr_timer = nullptr; // input context VExprContextSPtrs _input_exprs_ctxs; diff --git a/be/src/vec/exprs/vslot_ref.h b/be/src/vec/exprs/vslot_ref.h index bda4d9a3e3..2084ae1871 100644 --- a/be/src/vec/exprs/vslot_ref.h +++ b/be/src/vec/exprs/vslot_ref.h @@ -51,7 +51,7 @@ public: private: int _slot_id; int _column_id; - const std::string* _column_name; + const std::string* _column_name = nullptr; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/functions/array/function_array_remove.h b/be/src/vec/functions/array/function_array_remove.h index e1560d06d2..5a0ee49369 100644 --- a/be/src/vec/functions/array/function_array_remove.h +++ b/be/src/vec/functions/array/function_array_remove.h @@ -102,7 +102,7 @@ private: PaddedPODArray* dst_null_map = nullptr; MutableColumnPtr array_nested_column = nullptr; - IColumn* dst_column; + IColumn* dst_column = nullptr; if (nested_null_map) { auto dst_nested_column = ColumnNullable::create(nested_column.clone_empty(), ColumnUInt8::create()); @@ -174,7 +174,7 @@ private: PaddedPODArray* dst_null_map = nullptr; MutableColumnPtr array_nested_column = nullptr; - IColumn* dst_column; + IColumn* dst_column = nullptr; if (nested_null_map) { auto dst_nested_column = ColumnNullable::create(nested_column.clone_empty(), ColumnUInt8::create()); diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index f7cf95861b..5b355cc266 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -1711,7 +1711,7 @@ public: bool is_use_default_implementation_for_constants() const override { return true; } private: - const char* name; + const char* name = nullptr; MonotonicityForRange monotonicity_for_range; DataTypes argument_types; diff --git a/be/src/vec/functions/regexps.h b/be/src/vec/functions/regexps.h index 378dd3ec68..d2963d853f 100644 --- a/be/src/vec/functions/regexps.h +++ b/be/src/vec/functions/regexps.h @@ -129,7 +129,7 @@ Regexps constructRegexps(const std::vector& str_patterns, } } hs_database_t* db = nullptr; - hs_compile_error_t* compile_error; + hs_compile_error_t* compile_error = nullptr; std::unique_ptr ids; diff --git a/be/src/vec/io/reader_buffer.h b/be/src/vec/io/reader_buffer.h index b6c4424747..abee924647 100644 --- a/be/src/vec/io/reader_buffer.h +++ b/be/src/vec/io/reader_buffer.h @@ -38,8 +38,8 @@ public: std::string to_string() { return std::string(_start, (_end - _start)); } private: - char* _start; - char* _end; + char* _start = nullptr; + char* _end = nullptr; }; } // namespace doris::vectorized diff --git a/be/src/vec/olap/vcollect_iterator.h b/be/src/vec/olap/vcollect_iterator.h index 06cd14346c..2c53d98713 100644 --- a/be/src/vec/olap/vcollect_iterator.h +++ b/be/src/vec/olap/vcollect_iterator.h @@ -153,7 +153,7 @@ private: protected: const TabletSchema& _schema; IteratorRowRef _ref; - std::vector* _compare_columns; + std::vector* _compare_columns = nullptr; }; // Compare row cursors between multiple merge elements, diff --git a/be/src/vec/runtime/vcsv_transformer.h b/be/src/vec/runtime/vcsv_transformer.h index 035a89e1ad..8821b85718 100644 --- a/be/src/vec/runtime/vcsv_transformer.h +++ b/be/src/vec/runtime/vcsv_transformer.h @@ -63,7 +63,7 @@ private: std::string_view _column_separator; std::string_view _line_delimiter; - doris::io::FileWriter* _file_writer; + doris::io::FileWriter* _file_writer = nullptr; // Used to buffer the export data of plain text // TODO(cmy): I simply use a fmt::memmory_buffer to buffer the data, to avoid calling // file writer's write() for every single row. diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h index bfdd1dd351..d6881f3d6f 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.h +++ b/be/src/vec/runtime/vdata_stream_recvr.h @@ -136,10 +136,10 @@ private: friend struct BlockSupplierSortCursorImpl; // DataStreamMgr instance used to create this recvr. (Not owned) - VDataStreamMgr* _mgr; + VDataStreamMgr* _mgr = nullptr; #ifdef USE_MEM_TRACKER - std::shared_ptr _query_mem_tracker; + std::shared_ptr _query_mem_tracker = nullptr; TUniqueId _query_id; #endif @@ -162,25 +162,25 @@ private: std::unique_ptr _merger; ObjectPool _sender_queue_pool; - RuntimeProfile* _profile; + RuntimeProfile* _profile = nullptr; - RuntimeProfile::Counter* _bytes_received_counter; - RuntimeProfile::Counter* _local_bytes_received_counter; - RuntimeProfile::Counter* _deserialize_row_batch_timer; - RuntimeProfile::Counter* _first_batch_wait_total_timer; - RuntimeProfile::Counter* _buffer_full_total_timer; - RuntimeProfile::Counter* _data_arrival_timer; - RuntimeProfile::Counter* _decompress_timer; - RuntimeProfile::Counter* _decompress_bytes; - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage; + RuntimeProfile::Counter* _bytes_received_counter = nullptr; + RuntimeProfile::Counter* _local_bytes_received_counter = nullptr; + RuntimeProfile::Counter* _deserialize_row_batch_timer = nullptr; + RuntimeProfile::Counter* _first_batch_wait_total_timer = nullptr; + RuntimeProfile::Counter* _buffer_full_total_timer = nullptr; + RuntimeProfile::Counter* _data_arrival_timer = nullptr; + RuntimeProfile::Counter* _decompress_timer = nullptr; + RuntimeProfile::Counter* _decompress_bytes = nullptr; + RuntimeProfile::Counter* _memory_usage_counter = nullptr; + RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage = nullptr; std::atomic _blocks_memory_usage_current_value = 0; - RuntimeProfile::Counter* _peak_memory_usage_counter; + RuntimeProfile::Counter* _peak_memory_usage_counter = nullptr; // Number of rows received - RuntimeProfile::Counter* _rows_produced_counter; + RuntimeProfile::Counter* _rows_produced_counter = nullptr; // Number of blocks received - RuntimeProfile::Counter* _blocks_produced_counter; + RuntimeProfile::Counter* _blocks_produced_counter = nullptr; std::shared_ptr _sub_plan_query_statistics_recvr; @@ -240,7 +240,7 @@ protected: Status _inner_get_batch_without_lock(Block* block, bool* eos); // Not managed by this class - VDataStreamRecvr* _recvr; + VDataStreamRecvr* _recvr = nullptr; std::mutex _lock; bool _is_cancelled; Status _cancel_status; @@ -257,7 +257,7 @@ protected: std::deque> _pending_closures; std::unordered_map> _local_closure; - std::shared_ptr _dependency = nullptr; + std::shared_ptr _dependency; std::shared_ptr _local_channel_dependency; }; diff --git a/be/src/vec/runtime/vfile_format_transformer.h b/be/src/vec/runtime/vfile_format_transformer.h index f8c3bde271..19f873a632 100644 --- a/be/src/vec/runtime/vfile_format_transformer.h +++ b/be/src/vec/runtime/vfile_format_transformer.h @@ -53,7 +53,7 @@ public: virtual int64_t written_len() = 0; protected: - RuntimeState* _state; // not owned, set when init + RuntimeState* _state = nullptr; // not owned, set when init const VExprContextSPtrs& _output_vexpr_ctxs; int64_t _cur_written_rows; bool _output_object_data; diff --git a/be/src/vec/runtime/vorc_transformer.h b/be/src/vec/runtime/vorc_transformer.h index 7dd17dfd1a..bc2ff05c88 100644 --- a/be/src/vec/runtime/vorc_transformer.h +++ b/be/src/vec/runtime/vorc_transformer.h @@ -64,8 +64,8 @@ public: void set_written_len(int64_t written_len); private: - doris::io::FileWriter* _file_writer; // not owned - int64_t _cur_pos = 0; // current write position + doris::io::FileWriter* _file_writer = nullptr; // not owned + int64_t _cur_pos = 0; // current write position bool _is_closed = false; int64_t _written_len = 0; const std::string _name; @@ -91,7 +91,7 @@ public: private: std::unique_ptr _create_row_batch(size_t sz); - doris::io::FileWriter* _file_writer; + doris::io::FileWriter* _file_writer = nullptr; std::unique_ptr _output_stream; std::unique_ptr _write_options; const std::string& _schema_str; diff --git a/be/src/vec/runtime/vparquet_transformer.h b/be/src/vec/runtime/vparquet_transformer.h index ad306c29e0..f825dcf1cc 100644 --- a/be/src/vec/runtime/vparquet_transformer.h +++ b/be/src/vec/runtime/vparquet_transformer.h @@ -60,8 +60,8 @@ public: void set_written_len(int64_t written_len); private: - doris::io::FileWriter* _file_writer; // not owned - int64_t _cur_pos = 0; // current write position + doris::io::FileWriter* _file_writer = nullptr; // not owned + int64_t _cur_pos = 0; // current write position bool _is_closed = false; int64_t _written_len = 0; }; diff --git a/be/src/vec/runtime/vsorted_run_merger.h b/be/src/vec/runtime/vsorted_run_merger.h index 2f9ebe04a6..00fe44e7d6 100644 --- a/be/src/vec/runtime/vsorted_run_merger.h +++ b/be/src/vec/runtime/vsorted_run_merger.h @@ -88,10 +88,10 @@ protected: Block _empty_block; // Times calls to get_next(). - RuntimeProfile::Counter* _get_next_timer; + RuntimeProfile::Counter* _get_next_timer = nullptr; // Times calls to get the next batch of rows from the input run. - RuntimeProfile::Counter* _get_next_block_timer; + RuntimeProfile::Counter* _get_next_block_timer = nullptr; private: void init_timers(RuntimeProfile* profile); diff --git a/be/src/vec/sink/delta_writer_v2_pool.h b/be/src/vec/sink/delta_writer_v2_pool.h index f05b144200..b2e267bcfd 100644 --- a/be/src/vec/sink/delta_writer_v2_pool.h +++ b/be/src/vec/sink/delta_writer_v2_pool.h @@ -87,7 +87,7 @@ private: UniqueId _load_id; TabletToDeltaWriterV2Map _map; std::atomic _use_cnt; - DeltaWriterV2Pool* _pool; + DeltaWriterV2Pool* _pool = nullptr; }; class DeltaWriterV2Pool { diff --git a/be/src/vec/sink/load_stream_stub.h b/be/src/vec/sink/load_stream_stub.h index 37dba96118..b8c3be255c 100644 --- a/be/src/vec/sink/load_stream_stub.h +++ b/be/src/vec/sink/load_stream_stub.h @@ -137,7 +137,7 @@ private: std::vector _success_tablets; std::vector _failed_tablets; - LoadStreamStub* _stub; + LoadStreamStub* _stub = nullptr; }; public: diff --git a/be/src/vec/sink/load_stream_stub_pool.h b/be/src/vec/sink/load_stream_stub_pool.h index 2cf55be491..0eb14fc935 100644 --- a/be/src/vec/sink/load_stream_stub_pool.h +++ b/be/src/vec/sink/load_stream_stub_pool.h @@ -87,7 +87,7 @@ private: UniqueId _load_id; int64_t _dst_id; std::atomic _use_cnt; - LoadStreamStubPool* _pool; + LoadStreamStubPool* _pool = nullptr; }; class LoadStreamStubPool { diff --git a/be/src/vec/sink/varrow_flight_result_writer.h b/be/src/vec/sink/varrow_flight_result_writer.h index 02faebfddb..b9b44d1dfd 100644 --- a/be/src/vec/sink/varrow_flight_result_writer.h +++ b/be/src/vec/sink/varrow_flight_result_writer.h @@ -52,11 +52,11 @@ public: private: void _init_profile(); - BufferControlBlock* _sinker; + BufferControlBlock* _sinker = nullptr; const VExprContextSPtrs& _output_vexpr_ctxs; - RuntimeProfile* _parent_profile; // parent profile from result sink. not owned + RuntimeProfile* _parent_profile = nullptr; // parent profile from result sink. not owned // total time cost on append batch operation RuntimeProfile::Counter* _append_row_batch_timer = nullptr; // tuple convert timer, child timer of _append_row_batch_timer diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index 0458798354..d25385d6b2 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -171,8 +171,8 @@ protected: // Sender instance id, unique within a fragment. int _sender_id; - RuntimeState* _state; - ObjectPool* _pool; + RuntimeState* _state = nullptr; + ObjectPool* _pool = nullptr; int _current_channel_idx; // index of current channel to send to if _random == true @@ -211,9 +211,9 @@ protected: std::unique_ptr _mem_tracker; // Throughput per total time spent in sender - RuntimeProfile::Counter* _overall_throughput; + RuntimeProfile::Counter* _overall_throughput = nullptr; // Used to counter send bytes under local data exchange - RuntimeProfile::Counter* _local_bytes_send_counter; + RuntimeProfile::Counter* _local_bytes_send_counter = nullptr; // Identifier of the destination plan node. PlanNodeId _dest_node_id; @@ -356,7 +356,7 @@ protected: Status send_current_batch(bool eos = false); Status close_internal(Status exec_status); - Parent* _parent; + Parent* _parent = nullptr; const RowDescriptor& _row_desc; TUniqueId _fragment_instance_id; @@ -376,14 +376,14 @@ protected: PUniqueId _query_id; PBlock _pb_block; std::shared_ptr _brpc_request; - std::shared_ptr _brpc_stub = nullptr; - std::shared_ptr> _send_remote_block_callback = nullptr; + std::shared_ptr _brpc_stub; + std::shared_ptr> _send_remote_block_callback; Status _receiver_status; int32_t _brpc_timeout_ms = 500; // whether the dest can be treated as query statistics transfer chain. bool _is_transfer_chain; bool _send_query_statistics_with_every_batch; - RuntimeState* _state; + RuntimeState* _state = nullptr; bool _is_local; std::shared_ptr _local_recvr; @@ -559,7 +559,7 @@ private: pipeline::ExchangeSinkBuffer* _buffer = nullptr; bool _eos_send = false; - std::shared_ptr> _send_callback = nullptr; + std::shared_ptr> _send_callback; std::unique_ptr _pblock; }; diff --git a/be/src/vec/sink/vmysql_result_writer.h b/be/src/vec/sink/vmysql_result_writer.h index 2e1ae53fff..10a0b7e9e0 100644 --- a/be/src/vec/sink/vmysql_result_writer.h +++ b/be/src/vec/sink/vmysql_result_writer.h @@ -66,7 +66,7 @@ private: int _add_one_cell(const ColumnPtr& column_ptr, size_t row_idx, const DataTypePtr& type, MysqlRowBuffer& buffer, int scale = -1); - BufferControlBlock* _sinker; + BufferControlBlock* _sinker = nullptr; const VExprContextSPtrs& _output_vexpr_ctxs; diff --git a/be/src/vec/sink/vresult_file_sink.h b/be/src/vec/sink/vresult_file_sink.h index fdf4843a52..b0d05823a5 100644 --- a/be/src/vec/sink/vresult_file_sink.h +++ b/be/src/vec/sink/vresult_file_sink.h @@ -75,7 +75,7 @@ private: // Owned by the RuntimeState. RowDescriptor _output_row_descriptor; - std::unique_ptr _output_block = nullptr; + std::unique_ptr _output_block; std::shared_ptr _sender; std::unique_ptr _stream_sender; int _buf_size = 1024; // Allocated from _pool diff --git a/be/src/vec/sink/vrow_distribution.h b/be/src/vec/sink/vrow_distribution.h index 77104ef26f..158a13abaa 100644 --- a/be/src/vec/sink/vrow_distribution.h +++ b/be/src/vec/sink/vrow_distribution.h @@ -59,17 +59,17 @@ class VRowDistribution { public: // only used to pass parameters for VRowDistribution struct VRowDistributionContext { - RuntimeState* state; - OlapTableBlockConvertor* block_convertor; - OlapTabletFinder* tablet_finder; - VOlapTablePartitionParam* vpartition; - RuntimeProfile::Counter* add_partition_request_timer; + RuntimeState* state = nullptr; + OlapTableBlockConvertor* block_convertor = nullptr; + OlapTabletFinder* tablet_finder = nullptr; + VOlapTablePartitionParam* vpartition = nullptr; + RuntimeProfile::Counter* add_partition_request_timer = nullptr; int64_t txn_id = -1; - ObjectPool* pool; - OlapTableLocationParam* location; - const VExprContextSPtrs* vec_output_expr_ctxs; + ObjectPool* pool = nullptr; + OlapTableLocationParam* location = nullptr; + const VExprContextSPtrs* vec_output_expr_ctxs = nullptr; std::shared_ptr schema; - void* caller; + void* caller = nullptr; CreatePartitionCallback create_partition_callback; }; friend class VTabletWriter; @@ -170,12 +170,12 @@ private: VOlapTablePartitionParam* _vpartition = nullptr; RuntimeProfile::Counter* _add_partition_request_timer = nullptr; int64_t _txn_id = -1; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; OlapTableLocationParam* _location = nullptr; // int64_t _number_output_rows = 0; - const VExprContextSPtrs* _vec_output_expr_ctxs; + const VExprContextSPtrs* _vec_output_expr_ctxs = nullptr; CreatePartitionCallback _create_partition_callback = nullptr; - void* _caller; + void* _caller = nullptr; std::shared_ptr _schema; // reuse for find_tablet. diff --git a/be/src/vec/sink/vtablet_block_convertor.h b/be/src/vec/sink/vtablet_block_convertor.h index 3ee3d58265..963ebd7d58 100644 --- a/be/src/vec/sink/vtablet_block_convertor.h +++ b/be/src/vec/sink/vtablet_block_convertor.h @@ -103,7 +103,7 @@ private: size_t _batch_size; std::optional _auto_inc_col_idx; - AutoIncIDBuffer* _auto_inc_id_buffer; + AutoIncIDBuffer* _auto_inc_id_buffer = nullptr; AutoIncIDAllocator _auto_inc_id_allocator; }; diff --git a/be/src/vec/sink/vtablet_finder.h b/be/src/vec/sink/vtablet_finder.h index 3759284733..537e38d86c 100644 --- a/be/src/vec/sink/vtablet_finder.h +++ b/be/src/vec/sink/vtablet_finder.h @@ -64,7 +64,7 @@ public: Bitmap& filter_bitmap() { return _filter_bitmap; } private: - VOlapTablePartitionParam* _vpartition; + VOlapTablePartitionParam* _vpartition = nullptr; FindTabletMode _find_tablet_mode; std::map _partition_to_tablet_map; vectorized::flat_hash_set _partition_ids; diff --git a/be/src/vec/sink/vtablet_sink.h b/be/src/vec/sink/vtablet_sink.h index 7aa6c5b321..dc406ac157 100644 --- a/be/src/vec/sink/vtablet_sink.h +++ b/be/src/vec/sink/vtablet_sink.h @@ -93,7 +93,7 @@ public: Status close(RuntimeState* state, Status exec_status) override; private: - ObjectPool* _pool; + ObjectPool* _pool = nullptr; bool _group_commit = false; diff --git a/be/src/vec/sink/vtablet_sink_v2.h b/be/src/vec/sink/vtablet_sink_v2.h index cef4659bdd..c7811a0171 100644 --- a/be/src/vec/sink/vtablet_sink_v2.h +++ b/be/src/vec/sink/vtablet_sink_v2.h @@ -61,7 +61,7 @@ public: Status close(RuntimeState* state, Status exec_status) override; private: - ObjectPool* _pool; + ObjectPool* _pool = nullptr; bool _group_commit = false; diff --git a/be/src/vec/sink/writer/vfile_result_writer.h b/be/src/vec/sink/writer/vfile_result_writer.h index b2d423eede..0e7e0ccefc 100644 --- a/be/src/vec/sink/writer/vfile_result_writer.h +++ b/be/src/vec/sink/writer/vfile_result_writer.h @@ -97,7 +97,7 @@ private: Status _delete_dir(); RuntimeState* _state; // not owned, set when init - const ResultFileOptions* _file_opts; + const ResultFileOptions* _file_opts = nullptr; TStorageBackendType::type _storage_type; TUniqueId _fragment_instance_id; diff --git a/be/src/vec/sink/writer/vmysql_table_writer.h b/be/src/vec/sink/writer/vmysql_table_writer.h index b3cd65057c..a88c5730cb 100644 --- a/be/src/vec/sink/writer/vmysql_table_writer.h +++ b/be/src/vec/sink/writer/vmysql_table_writer.h @@ -59,7 +59,7 @@ private: Status _insert_row(vectorized::Block& block, size_t row); MysqlConnInfo _conn_info; fmt::memory_buffer _insert_stmt_buffer; - MYSQL* _mysql_conn; + MYSQL* _mysql_conn = nullptr; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/sink/writer/vtablet_writer.h b/be/src/vec/sink/writer/vtablet_writer.h index c6190c2675..6c1f0757fc 100644 --- a/be/src/vec/sink/writer/vtablet_writer.h +++ b/be/src/vec/sink/writer/vtablet_writer.h @@ -368,7 +368,7 @@ protected: std::mutex _pending_batches_lock; // reuse for vectorized std::atomic _pending_batches_num {0}; // reuse for vectorized - std::shared_ptr _stub = nullptr; + std::shared_ptr _stub; // because we have incremantal open, we should keep one relative closure for one request. it's similarly for adding block. std::vector>> _open_callbacks; @@ -394,7 +394,7 @@ protected: std::mutex _closed_lock; bool _is_closed = false; - RuntimeState* _state; + RuntimeState* _state = nullptr; // rows number received per tablet, tablet_id -> rows_num std::vector> _tablets_received_rows; // rows number filtered per tablet, tablet_id -> filtered_rows_num @@ -476,7 +476,7 @@ private: friend class VTabletWriter; friend class VRowDistribution; - VTabletWriter* _parent; + VTabletWriter* _parent = nullptr; int64_t _index_id; vectorized::VExprContextSPtr _where_clause; @@ -566,7 +566,7 @@ private: std::shared_ptr _mem_tracker; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; bthread_t _sender_thread = 0; @@ -665,6 +665,6 @@ private: VRowDistribution _row_distribution; // reuse to avoid frequent memory allocation and release. std::vector _row_part_tablet_ids; - std::shared_ptr _v_wal_writer = nullptr; + std::shared_ptr _v_wal_writer; }; } // namespace doris::vectorized diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.h b/be/src/vec/sink/writer/vtablet_writer_v2.h index 9f55aa1b0a..69918b37a6 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.h +++ b/be/src/vec/sink/writer/vtablet_writer_v2.h @@ -150,7 +150,7 @@ private: std::shared_ptr _mem_tracker; TDataSink _t_sink; - ObjectPool* _pool; + ObjectPool* _pool = nullptr; // unique load id PUniqueId _load_id; diff --git a/be/src/vec/sink/writer/vwal_writer.h b/be/src/vec/sink/writer/vwal_writer.h index 017f447c32..d33f3f015a 100644 --- a/be/src/vec/sink/writer/vwal_writer.h +++ b/be/src/vec/sink/writer/vwal_writer.h @@ -102,7 +102,7 @@ private: std::string _label; RuntimeState* _state = nullptr; TupleDescriptor* _output_tuple_desc = nullptr; - std::shared_ptr _wal_writer = nullptr; + std::shared_ptr _wal_writer; }; } // namespace vectorized } // namespace doris \ No newline at end of file diff --git a/be/test/testutil/desc_tbl_builder.h b/be/test/testutil/desc_tbl_builder.h index 4ee9fc39dc..c29ef9acd4 100644 --- a/be/test/testutil/desc_tbl_builder.h +++ b/be/test/testutil/desc_tbl_builder.h @@ -52,7 +52,7 @@ public: private: // Owned by caller. - ObjectPool* _obj_pool; + ObjectPool* _obj_pool = nullptr; std::vector _tuples_descs; diff --git a/samples/connect/cpp/doris_client.h b/samples/connect/cpp/doris_client.h index 4b17a3e318..095689d07e 100644 --- a/samples/connect/cpp/doris_client.h +++ b/samples/connect/cpp/doris_client.h @@ -36,7 +36,7 @@ public: bool exec(const string& sql); private: // mysql handle - MYSQL* _client; + MYSQL* _client = nullptr; // doris result MYSQL_RES* _result = nullptr; //doris result as row