[refactor](standard)BE:Initialize pointer variables in the class to nullptr by default (#27587)

This commit is contained in:
ShowCode
2023-11-28 13:02:30 +08:00
committed by GitHub
parent fc2129a09f
commit f565f60bc3
290 changed files with 1112 additions and 1109 deletions

View File

@ -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);

View File

@ -227,7 +227,7 @@ protected:
// Reference to the ExecEnv::_master_info
const TMasterInfo& _master_info;
std::unique_ptr<AgentUtils> _agent_utils;
ExecEnv* _env;
ExecEnv* _env = nullptr;
// Protect task queue
std::mutex _worker_thread_lock;
@ -242,7 +242,7 @@ protected:
std::atomic<bool> _is_doing_work;
std::shared_ptr<MetricEntity> _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;

View File

@ -32,6 +32,6 @@ public:
void handle_topic_info(const std::vector<TopicInfo>& topic_info_list) override;
private:
ExecEnv* _exec_env;
ExecEnv* _exec_env = nullptr;
};
} // namespace doris

View File

@ -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 {

View File

@ -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;
};

View File

@ -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;
};

View File

@ -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_;
};

View File

@ -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> elf;
};

View File

@ -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;
};

View File

@ -252,7 +252,7 @@ protected:
int _id; // unique w/in single plan tree
TPlanNodeType::type _type;
ObjectPool* _pool;
ObjectPool* _pool = nullptr;
std::vector<TupleId> _tuple_ids;
vectorized::VExprContextSPtrs _conjuncts;
@ -276,15 +276,15 @@ protected:
// which will providea reference for operator memory.
std::unique_ptr<MemTracker> _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

View File

@ -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

View File

@ -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

View File

@ -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<ColumnDesc> _columns;

View File

@ -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;
};

View File

@ -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;
};

View File

@ -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);

View File

@ -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

View File

@ -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;
};

View File

@ -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<uint64_t, HybridSetBase*>::const_iterator it = _map.find(dst);
if (it == _map.end()) {

View File

@ -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

View File

@ -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

View File

@ -39,5 +39,5 @@ struct ToBinaryContext {
unsigned char buf[8];
std::ostream* outStream;
std::ostream* outStream = nullptr;
};

View File

@ -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 <class C2, class GP>

View File

@ -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&); \

View File

@ -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;
};

View File

@ -312,7 +312,7 @@ public:
void reset(T* p = NULL) { *this = p; }
protected:
T* ptr_;
T* ptr_ = nullptr;
private:
template <typename U>

View File

@ -244,7 +244,7 @@ inline void STLAssignToVectorChar(vector<char>* 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 <class STLContainer>
@ -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)

View File

@ -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;

View File

@ -127,7 +127,7 @@
class StringPiece {
private:
const char* ptr_;
const char* ptr_ = nullptr;
int length_;
public:

View File

@ -131,7 +131,7 @@ public:
private:
inline SubstituteArg() : text_(NULL), size_(-1) {}
const char* text_;
const char* text_ = nullptr;
int size_;
char scratch_[kFastToBufferSize];
};

View File

@ -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);
};

View File

@ -71,7 +71,7 @@ private:
int32_t _num_workers;
std::unique_ptr<ThreadPool> _download_workers;
std::shared_ptr<bufferevent_rate_limit_group> _rate_limit_group {nullptr};
std::shared_ptr<bufferevent_rate_limit_group> _rate_limit_group;
}; // end class DownloadAction
} // end namespace doris

View File

@ -44,7 +44,7 @@ private:
private:
ExecEnv* _exec_env;
std::shared_ptr<bufferevent_rate_limit_group> _rate_limit_group {nullptr};
std::shared_ptr<bufferevent_rate_limit_group> _rate_limit_group;
};
} // namespace doris

View File

@ -186,7 +186,7 @@ private:
mutable std::mutex _download_mutex;
Key _file_key;
IFileCache* _cache;
IFileCache* _cache = nullptr;
std::atomic<bool> _is_downloaded {false};
CacheType _cache_type;

View File

@ -205,7 +205,7 @@ private:
size_t _num_hit_segments = 0;
size_t _num_removed_segments = 0;
std::shared_ptr<MetricEntity> _entity = nullptr;
std::shared_ptr<MetricEntity> _entity;
DoubleGauge* file_cache_hits_ratio = nullptr;
UIntGauge* file_cache_removed_elements = nullptr;

View File

@ -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<std::string, std::string>& _properties;
int64_t _cur_offset;

View File

@ -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<PrefetchBuffer> {
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<char[]> _buf;
BufferStatus _buffer_status {BufferStatus::RESET};
std::mutex _lock;
@ -420,7 +420,7 @@ private:
io::FileReaderSPtr _reader;
PrefetchRange _file_range;
const std::vector<PrefetchRange>* _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<std::shared_ptr<PrefetchBuffer>> _pre_buffers;
int64_t _whole_pre_buffer_size;
@ -457,7 +457,7 @@ protected:
private:
Status _close_internal();
io::FileReaderSPtr _reader;
std::unique_ptr<char[]> _data = nullptr;
std::unique_ptr<char[]> _data;
size_t _size;
bool _closed = false;
};

View File

@ -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<bool> _closed = false;
RuntimeProfile* _profile;
RuntimeProfile* _profile = nullptr;
#ifdef USE_HADOOP_HDFS
HDFSProfile _hdfs_profile;
#endif

View File

@ -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

View File

@ -50,7 +50,7 @@ private:
private:
hdfsFile _hdfs_file = nullptr;
// A convenient pointer to _fs
HdfsFileSystem* _hdfs_fs;
HdfsFileSystem* _hdfs_fs = nullptr;
};
} // namespace io

View File

@ -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;

View File

@ -76,7 +76,7 @@ private:
std::vector<std::unique_ptr<Aws::S3::Model::CompletedPart>> _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};

View File

@ -94,11 +94,11 @@ protected:
std::shared_ptr<MetricEntity> _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<int64_t> published_count = 0;
};

View File

@ -126,7 +126,7 @@ public:
}
private:
const ColumnPredicate* _predicate;
const ColumnPredicate* _predicate = nullptr;
};
class MutilColumnBlockPredicate : public BlockColumnPredicate {

View File

@ -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<TExpr> expr;
const TabletColumn* new_column;
const TabletColumn* new_column = nullptr;
};
using SchemaMapping = std::vector<ColumnMapping>;

View File

@ -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<MetricEntity> _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

View File

@ -38,7 +38,7 @@ struct WriteRequest {
PUniqueId load_id;
TupleDescriptor* tuple_desc = nullptr;
// slots are in order of tablet's schema
const std::vector<SlotDescriptor*>* slots;
const std::vector<SlotDescriptor*>* slots = nullptr;
OlapTableSchemaParam* table_schema_param = nullptr;
bool is_high_priority = false;
bool write_file_cache = false;

View File

@ -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<SparseIndexType, SparseValueType> _sparse_map;
SparseLengthValueType* _sparse_count;

View File

@ -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<InvertedIndexCtx>;

View File

@ -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;
};

View File

@ -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

View File

@ -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<uint64_t> _last_id;
size_t _total_capacity;
std::unique_ptr<MemTrackerLimiter> _mem_tracker;
std::shared_ptr<MetricEntity> _entity = nullptr;
std::shared_ptr<MetricEntity> _entity;
IntGauge* cache_capacity = nullptr;
IntGauge* cache_usage = nullptr;
DoubleGauge* cache_usage_ratio = nullptr;

View File

@ -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<RowInBlockComparator> _vec_row_comparator;

View File

@ -91,7 +91,7 @@ private:
FlushStatistic _stats;
RowsetWriter* _rowset_writer;
RowsetWriter* _rowset_writer = nullptr;
MemTableStat _memtable_stat;
};

View File

@ -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<DataPageCache> _data_page_cache = nullptr;
std::unique_ptr<IndexPageCache> _index_page_cache = nullptr;
std::unique_ptr<DataPageCache> _data_page_cache;
std::unique_ptr<IndexPageCache> _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<PKIndexPageCache> _pk_index_page_cache = nullptr;
std::unique_ptr<PKIndexPageCache> _pk_index_page_cache;
Cache* _get_page_cache(segment_v2::PageTypePB page_type) {
switch (page_type) {

View File

@ -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<std::string, TypeDescriptor> _name_to_col_type;
@ -126,10 +126,10 @@ private:
std::vector<SlotDescriptor*> _src_slot_descs;
std::unique_ptr<RowDescriptor> _row_desc;
const TupleDescriptor* _dest_tuple_desc;
const TupleDescriptor* _dest_tuple_desc = nullptr;
std::unique_ptr<RuntimeState> _runtime_state;
RuntimeProfile* _runtime_profile;
RuntimeProfile* _runtime_profile = nullptr;
std::unique_ptr<vectorized::GenericReader> _cur_reader;
bool _cur_reader_eof;
const TBrokerScanRangeParams& _params;

View File

@ -138,7 +138,7 @@ public:
std::vector<RowSetSplits> 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<uint32_t> return_columns;

View File

@ -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);
};

View File

@ -30,7 +30,7 @@ struct RowCursorCell {
void* mutable_cell_ptr() const { return (char*)_ptr + 1; }
private:
void* _ptr;
void* _ptr = nullptr;
};
} // namespace doris

View File

@ -98,11 +98,11 @@ private:
std::pair<int, int> _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<RowwiseIterator> _iterator;

View File

@ -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

View File

@ -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<bool> _cancelled = false;
};

View File

@ -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_v2::SegmentWriter>& segment_writer);
SegmentFlusher* _flusher;
SegmentFlusher* _flusher = nullptr;
std::unique_ptr<segment_v2::SegmentWriter> _writer;
};

View File

@ -85,7 +85,8 @@ private:
std::unique_ptr<PageBuilder> _data_page_builder;
std::unique_ptr<BinaryPlainPageBuilder<FieldType::OLAP_FIELD_TYPE_VARCHAR>> _dict_builder;
std::unique_ptr<BinaryPlainPageBuilder<FieldType::OLAP_FIELD_TYPE_VARCHAR>> _dict_builder =
nullptr;
EncodingTypePB _encoding_type;
struct HashOfSlice {

View File

@ -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<Status> _load_once;
std::unique_ptr<IndexedColumnReader> _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;

View File

@ -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<void(const void*, const int, const uint64_t, void*)> _hash_func;

View File

@ -64,8 +64,8 @@ private:
io::FileReaderSPtr _file_reader;
DorisCallOnce<Status> _load_once;
const TypeInfo* _type_info;
std::unique_ptr<BloomFilterIndexPB> _bloom_filter_index_meta;
const TypeInfo* _type_info = nullptr;
std::unique_ptr<BloomFilterIndexPB> _bloom_filter_index_meta = nullptr;
std::unique_ptr<IndexedColumnReader> _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;
};

View File

@ -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;

View File

@ -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<ColumnIterator> _null_iterator;
std::unique_ptr<OffsetFileColumnIterator> _offsets_iterator; //OffsetFileIterator
std::unique_ptr<ColumnIterator> _key_iterator;
@ -502,7 +502,7 @@ public:
}
private:
ColumnReader* _struct_reader;
ColumnReader* _struct_reader = nullptr;
std::unique_ptr<ColumnIterator> _null_iterator;
std::vector<std::unique_ptr<ColumnIterator>> _sub_column_iterators;
};
@ -537,7 +537,7 @@ public:
}
private:
ColumnReader* _array_reader;
ColumnReader* _array_reader = nullptr;
std::unique_ptr<OffsetFileColumnIterator> _offset_iterator;
std::unique_ptr<ColumnIterator> _null_iterator;
std::unique_ptr<ColumnIterator> _item_iterator;

View File

@ -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

View File

@ -84,7 +84,7 @@ private:
FieldType _type;
EncodingTypePB _encoding;
std::unique_ptr<DataPagePreDecoder> _data_page_pre_decoder = nullptr;
std::unique_ptr<DataPagePreDecoder> _data_page_pre_decoder;
};
} // namespace segment_v2

View File

@ -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;
};

View File

@ -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

View File

@ -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<IndexPageBuilder> _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);
};

View File

@ -128,7 +128,7 @@ private:
Cache::Handle* _insert(const InvertedIndexSearcherCache::CacheKey& key, CacheValue* value);
private:
std::unique_ptr<MemTracker> _mem_tracker = nullptr;
std::unique_ptr<MemTracker> _mem_tracker;
};
using IndexCacheValuePtr = std::unique_ptr<InvertedIndexSearcherCache::CacheValue>;

View File

@ -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) {

View File

@ -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<char*, ReaderFileEntry*, lucene::util::Compare::Char,
lucene::util::Equals::Char, lucene::util::Deletor::acArray,
lucene::util::Deletor::Object<ReaderFileEntry>>;
EntriesType* entries;
EntriesType* entries = nullptr;
std::mutex _this_lock;

View File

@ -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:

View File

@ -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;
};

View File

@ -83,7 +83,7 @@ struct ParsedPage {
bool has_null;
Slice null_bitmap;
RleDecoder<bool> null_decoder;
std::unique_ptr<PageDecoder> data_decoder = nullptr;
std::unique_ptr<PageDecoder> data_decoder;
// ordinal of the first value in this page
ordinal_t first_ordinal = 0;

View File

@ -139,7 +139,7 @@ private:
size_t _count;
bool _finished;
int _bit_width;
RleEncoder<CppType>* _rle_encoder;
RleEncoder<CppType>* _rle_encoder = nullptr;
faststring _buf;
CppType _first_value;
CppType _last_value;

View File

@ -401,7 +401,7 @@ private:
_column_pred_in_remaining_vconjunct;
std::set<ColumnId> _not_apply_index_pred;
std::shared_ptr<ColumnPredicate> _runtime_predicate {nullptr};
std::shared_ptr<ColumnPredicate> _runtime_predicate;
// row schema of the key to seek
// only used in `_get_row_ranges_by_keys`

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -118,7 +118,7 @@ public:
private:
SegmentLoader();
std::unique_ptr<SegmentCache> _segment_cache = nullptr;
std::unique_ptr<SegmentCache> _segment_cache;
};
// A handle for a single rowset from segment lru cache.

View File

@ -158,7 +158,7 @@ public:
}
private:
SelectionVector* _sel_vec;
SelectionVector* _sel_vec = nullptr;
size_t _row_offset;
};

View File

@ -123,7 +123,7 @@ public:
ssize_t ordinal() const { return _ordinal; }
private:
const ShortKeyIndexDecoder* _decoder;
const ShortKeyIndexDecoder* _decoder = nullptr;
ssize_t _ordinal;
};

View File

@ -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
};

View File

@ -435,7 +435,7 @@ private:
// Type of new loaded data
RowsetTypePB _default_rowset_type;
HeartbeatFlags* _heartbeat_flags;
HeartbeatFlags* _heartbeat_flags = nullptr;
std::unique_ptr<ThreadPool> _base_compaction_thread_pool;
std::unique_ptr<ThreadPool> _cumu_compaction_thread_pool;

View File

@ -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<Status> _init_once;

View File

@ -88,7 +88,7 @@ private:
private:
const TCloneReq& _clone_req;
vector<TTabletInfo>* _tablet_infos;
vector<TTabletInfo>* _tablet_infos = nullptr;
int64_t _signature;
const TMasterInfo& _master_info;
int64_t _copy_size;

View File

@ -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<TTabletId>* _error_tablet_ids;
std::set<TTabletId>* _error_tablet_ids = nullptr;
std::map<TTabletId, TVersion>* _succ_tablets;
std::vector<std::tuple<int64_t, int64_t, int64_t>>* _discontinuous_version_tablets;
std::map<TTableId, int64_t>* _table_id_to_num_delta_rows;
std::vector<std::tuple<int64_t, int64_t, int64_t>>* _discontinuous_version_tablets = nullptr;
std::map<TTableId, int64_t>* _table_id_to_num_delta_rows = nullptr;
};
class AsyncTabletPublishTask {

View File

@ -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<PendingRowsetGuard> _pending_rs_guards;
}; // EngineTask

View File

@ -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

View File

@ -72,7 +72,7 @@ public:
Status get_wal_column_index(int64_t wal_id, std::vector<size_t>& column_index);
private:
ExecEnv* _exec_env;
ExecEnv* _exec_env = nullptr;
std::shared_mutex _lock;
scoped_refptr<Thread> _replay_thread;
CountDownLatch _stop_background_threads_latch;

View File

@ -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<Status(vectorized::Block* block)>;
using vectorized_update_memusage = std::function<void()>;
@ -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<size_t> _make_nullable_keys;
size_t _spill_partition_count_bits;
int64_t _limit; // -1: no limit

View File

@ -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<Status(
RuntimeState* state, vectorized::Block* block, SourceState& source_state)>;
@ -137,7 +137,7 @@ protected:
executor _executor;
vectorized::AggregatedDataVariants* _agg_data;
vectorized::AggregatedDataVariants* _agg_data = nullptr;
};
class AggSourceOperatorX : public OperatorX<AggLocalState> {

View File

@ -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<vectorized::VExprContextSPtrs> _agg_expr_ctxs;
};

View File

@ -120,9 +120,9 @@ private:
std::unique_ptr<vectorized::Arena> _agg_arena_pool;
std::vector<vectorized::AggFnEvaluator*> _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<void(int64_t peer_group_start, int64_t peer_group_end,
int64_t frame_start, int64_t frame_end)>;
@ -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<size_t> _offsets_of_aggregate_states;

Some files were not shown because too many files have changed in this diff Show More