[BUG] Fixed some uninitialized variables (#5850)

Fixed some potential bugs caused by uninitialized variables
This commit is contained in:
stdpain
2021-05-25 10:34:35 +08:00
committed by GitHub
parent 63662194ab
commit 1ec615c562
23 changed files with 334 additions and 300 deletions

View File

@ -96,8 +96,13 @@ CsvScanNode::CsvScanNode(ObjectPool* pool, const TPlanNode& tnode, const Descrip
_default_values(tnode.csv_scan_node.default_values),
_is_init(false),
_tuple_desc(nullptr),
_slot_num(0),
_tuple_pool(nullptr),
_text_converter(nullptr),
_tuple(nullptr),
_runtime_state(nullptr),
_split_check_timer(nullptr),
_split_line_timer(nullptr),
_hll_column_num(0) {
// do nothing
LOG(INFO) << "csv scan node: " << apache::thrift::ThriftDebugString(tnode).c_str();
@ -589,8 +594,7 @@ bool CsvScanNode::split_check_fill(const std::string& line, RuntimeState* state)
}
}
for (std::map<std::string, TMiniLoadEtlFunction>::iterator iter = _column_function_map.begin();
iter != _column_function_map.end(); iter++) {
for (auto iter = _column_function_map.begin(); iter != _column_function_map.end(); ++iter) {
TMiniLoadEtlFunction& function = iter->second;
const std::string& column_name = iter->first;
const SlotDescriptor* slot = _column_slot_map[column_name];

View File

@ -51,7 +51,11 @@ const std::string ERROR_MEM_LIMIT_EXCEEDED =
"$1 bytes for $2.";
EsScanNode::EsScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
: ScanNode(pool, tnode, descs), _tuple_id(tnode.es_scan_node.tuple_id), _scan_range_idx(0) {
: ScanNode(pool, tnode, descs),
_tuple_id(tnode.es_scan_node.tuple_id),
_tuple_desc(nullptr),
_env(nullptr),
_scan_range_idx(0) {
if (tnode.es_scan_node.__isset.properties) {
_properties = tnode.es_scan_node.properties;
}
@ -217,7 +221,7 @@ Status EsScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos)
// convert
VLOG_CRITICAL << "begin to convert: scan_range_idx=" << _scan_range_idx
<< ", num_rows=" << result.rows.num_rows;
<< ", num_rows=" << result.rows.num_rows;
std::vector<TExtColumnData>& cols = result.rows.cols;
// indexes of the next non-null value in the row batch, per column.
std::vector<int> cols_next_val_idx(_tuple_desc->slots().size(), 0);
@ -420,7 +424,8 @@ bool EsScanNode::get_disjuncts(ExprContext* context, Expr* conjunct,
TExtLiteral literal;
if (!to_ext_literal(context, expr, &literal)) {
VLOG_CRITICAL << "get disjuncts fail: can't get literal, node_type=" << expr->node_type();
VLOG_CRITICAL << "get disjuncts fail: can't get literal, node_type="
<< expr->node_type();
return false;
}
@ -446,7 +451,7 @@ bool EsScanNode::get_disjuncts(ExprContext* context, Expr* conjunct,
TExtLiteral literal;
if (!to_ext_literal(context, conjunct->get_child(1), &literal)) {
VLOG_CRITICAL << "get disjuncts fail: can't get literal, node_type="
<< conjunct->get_child(1)->node_type();
<< conjunct->get_child(1)->node_type();
return false;
}
@ -497,7 +502,7 @@ bool EsScanNode::get_disjuncts(ExprContext* context, Expr* conjunct,
if (!to_ext_literal(slot_desc->type().type, const_cast<void*>(iter->get_value()),
&literal)) {
VLOG_CRITICAL << "get disjuncts fail: can't get literal, node_type="
<< slot_desc->type().type;
<< slot_desc->type().type;
return false;
}
in_pred_values.push_back(literal);
@ -523,7 +528,7 @@ bool EsScanNode::get_disjuncts(ExprContext* context, Expr* conjunct,
return true;
} else {
VLOG_CRITICAL << "get disjuncts fail: node type is " << conjunct->node_type()
<< ", should be BINARY_PRED or COMPOUND_PRED";
<< ", should be BINARY_PRED or COMPOUND_PRED";
return false;
}
}

View File

@ -36,7 +36,8 @@ MysqlScanNode::MysqlScanNode(ObjectPool* pool, const TPlanNode& tnode, const Des
_tuple_id(tnode.mysql_scan_node.tuple_id),
_columns(tnode.mysql_scan_node.columns),
_filters(tnode.mysql_scan_node.filters),
_tuple_desc(nullptr) {}
_tuple_desc(nullptr),
_slot_num(0) {}
MysqlScanNode::~MysqlScanNode() {}
@ -136,7 +137,8 @@ Status MysqlScanNode::write_text_slot(char* value, int value_length, SlotDescrip
if (!_text_converter->write_slot(slot, _tuple, value, value_length, true, false,
_tuple_pool.get())) {
std::stringstream ss;
ss << "Fail to convert mysql value:'" << value << "' to " << slot->type() << " on column:`" << slot->col_name() + "`";
ss << "Fail to convert mysql value:'" << value << "' to " << slot->type() << " on column:`"
<< slot->col_name() + "`";
return Status::InternalError(ss.str());
}

View File

@ -36,7 +36,8 @@ OdbcScanNode::OdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const Descr
_connect_string(std::move(tnode.odbc_scan_node.connect_string)),
_query_string(std::move(tnode.odbc_scan_node.query_string)),
_tuple_id(tnode.odbc_scan_node.tuple_id),
_tuple_desc(nullptr) {}
_tuple_desc(nullptr),
_slot_num(0) {}
OdbcScanNode::~OdbcScanNode() {}
@ -115,7 +116,8 @@ Status OdbcScanNode::write_text_slot(char* value, int value_length, SlotDescript
if (!_text_converter->write_slot(slot, _tuple, value, value_length, true, false,
_tuple_pool.get())) {
std::stringstream ss;
ss << "Fail to convert odbc value:'" << value << "' to " << slot->type() << " on column:`" << slot->col_name() + "`";
ss << "Fail to convert odbc value:'" << value << "' to " << slot->type() << " on column:`"
<< slot->col_name() + "`";
return Status::InternalError(ss.str());
}

View File

@ -43,18 +43,17 @@ OlapScanner::OlapScanner(RuntimeState* runtime_state, OlapScanNode* parent, bool
_tuple_desc(parent->_tuple_desc),
_profile(parent->runtime_profile()),
_string_slots(parent->_string_slots),
_id(-1),
_is_open(false),
_aggregation(aggregation),
_need_agg_finalize(need_agg_finalize),
_tuple_idx(parent->_tuple_idx),
_direct_conjunct_size(parent->_direct_conjunct_size),
_mem_tracker(MemTracker::CreateTracker(runtime_state->fragment_mem_tracker()->limit(),
"OlapScanner",
runtime_state->fragment_mem_tracker(),
true, true, MemTrackerLevel::DEBUG)) {
_reader.reset(new Reader());
DCHECK(_reader.get() != NULL);
_reader(new Reader()),
_version(-1),
_mem_tracker(MemTracker::CreateTracker(
runtime_state->fragment_mem_tracker()->limit(), "OlapScanner",
runtime_state->fragment_mem_tracker(), true, true, MemTrackerLevel::DEBUG)) {
_rows_read_counter = parent->rows_read_counter();
_rows_pushed_cond_filtered_counter = parent->_rows_pushed_cond_filtered_counter;
}
@ -358,9 +357,10 @@ Status OlapScanner::get_batch(RuntimeState* state, RowBatch* batch, bool* eof) {
config::doris_max_pushdown_conjuncts_return_rate) {
_use_pushdown_conjuncts = false;
VLOG_CRITICAL << "Stop Using PushDown Conjuncts. "
<< "PushDownReturnRate: " << pushdown_return_rate << "%"
<< " MaxPushDownReturnRate: "
<< config::doris_max_pushdown_conjuncts_return_rate << "%";
<< "PushDownReturnRate: " << pushdown_return_rate << "%"
<< " MaxPushDownReturnRate: "
<< config::doris_max_pushdown_conjuncts_return_rate
<< "%";
}
}
}

View File

@ -110,20 +110,14 @@ Status PartitionedHashTableCtx::Init(ObjectPool* pool, RuntimeState* state, int
// TODO chenhao replace ExprContext with ScalarFnEvaluator
for (int i = 0; i < build_exprs_.size(); i++) {
ExprContext* context = pool->add(new ExprContext(build_exprs_[i]));
context->prepare(state, row_desc, tracker_);
if (context == nullptr) {
return Status::InternalError("Hashtable init error.");
}
RETURN_IF_ERROR(context->prepare(state, row_desc, tracker_));
build_expr_evals_.push_back(context);
}
DCHECK_EQ(build_exprs_.size(), build_expr_evals_.size());
for (int i = 0; i < probe_exprs_.size(); i++) {
ExprContext* context = pool->add(new ExprContext(probe_exprs_[i]));
context->prepare(state, row_desc_probe, tracker_);
if (context == nullptr) {
return Status::InternalError("Hashtable init error.");
}
RETURN_IF_ERROR(context->prepare(state, row_desc_probe, tracker_));
probe_expr_evals_.push_back(context);
}
DCHECK_EQ(probe_exprs_.size(), probe_expr_evals_.size());
@ -474,7 +468,8 @@ Status PartitionedHashTable::ResizeBuckets(int64_t num_buckets,
DCHECK_GT(num_buckets, num_filled_buckets_)
<< "Cannot shrink the hash table to smaller number of buckets than the number of "
<< "filled buckets.";
VLOG_CRITICAL << "Resizing hash table from " << num_buckets_ << " to " << num_buckets << " buckets.";
VLOG_CRITICAL << "Resizing hash table from " << num_buckets_ << " to " << num_buckets
<< " buckets.";
if (max_num_buckets_ != -1 && num_buckets > max_num_buckets_) {
*got_memory = false;
return Status::OK();

View File

@ -32,7 +32,7 @@ SchemaScanner::ColumnDesc SchemaCharsetsScanner::_s_css_columns[] = {
SchemaCharsetsScanner::CharsetStruct SchemaCharsetsScanner::_s_charsets[] = {
{"utf8", "utf8_general_ci", "UTF-8 Unicode", 3},
{NULL, NULL, 0},
{nullptr, nullptr, 0},
};
SchemaCharsetsScanner::SchemaCharsetsScanner()
@ -48,7 +48,7 @@ Status SchemaCharsetsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
int len = strlen(_s_charsets[_index].charset);
str_slot->ptr = (char*)pool->allocate(len + 1);
if (NULL == str_slot->ptr) {
if (nullptr == str_slot->ptr) {
return Status::InternalError("No Memory.");
}
memcpy(str_slot->ptr, _s_charsets[_index].charset, len + 1);
@ -60,7 +60,7 @@ Status SchemaCharsetsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
int len = strlen(_s_charsets[_index].default_collation);
str_slot->ptr = (char*)pool->allocate(len + 1);
if (NULL == str_slot->ptr) {
if (nullptr == str_slot->ptr) {
return Status::InternalError("No Memory.");
}
memcpy(str_slot->ptr, _s_charsets[_index].default_collation, len + 1);
@ -72,7 +72,7 @@ Status SchemaCharsetsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
int len = strlen(_s_charsets[_index].description);
str_slot->ptr = (char*)pool->allocate(len + 1);
if (NULL == str_slot->ptr) {
if (nullptr == str_slot->ptr) {
return Status::InternalError("No Memory.");
}
memcpy(str_slot->ptr, _s_charsets[_index].description, len + 1);
@ -91,13 +91,13 @@ Status SchemaCharsetsScanner::get_next_row(Tuple* tuple, MemPool* pool, bool* eo
if (!_is_init) {
return Status::InternalError("call this before initial.");
}
if (NULL == _s_charsets[_index].charset) {
if (nullptr == tuple || nullptr == pool || nullptr == eos) {
return Status::InternalError("invalid parameter.");
}
if (nullptr == _s_charsets[_index].charset) {
*eos = true;
return Status::OK();
}
if (NULL == tuple || NULL == pool || NULL == eos) {
return Status::InternalError("invalid parameter.");
}
*eos = false;
return fill_one_row(tuple, pool);
}

View File

@ -34,7 +34,7 @@ SchemaScanner::ColumnDesc SchemaCollationsScanner::_s_cols_columns[] = {
SchemaCollationsScanner::CollationStruct SchemaCollationsScanner::_s_collations[] = {
{"utf8_general_ci", "utf8", 33, "Yes", "Yes", 1},
{NULL, NULL, 0, NULL, NULL, 0},
{nullptr, nullptr, 0, nullptr, nullptr, 0},
};
SchemaCollationsScanner::SchemaCollationsScanner()
@ -51,7 +51,7 @@ Status SchemaCollationsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
int len = strlen(_s_collations[_index].name);
str_slot->ptr = (char*)pool->allocate(len + 1);
if (NULL == str_slot->ptr) {
if (nullptr == str_slot->ptr) {
return Status::InternalError("No Memory.");
}
memcpy(str_slot->ptr, _s_collations[_index].name, len + 1);
@ -63,7 +63,7 @@ Status SchemaCollationsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
int len = strlen(_s_collations[_index].charset);
str_slot->ptr = (char*)pool->allocate(len + 1);
if (NULL == str_slot->ptr) {
if (nullptr == str_slot->ptr) {
return Status::InternalError("No Memory.");
}
memcpy(str_slot->ptr, _s_collations[_index].charset, len + 1);
@ -80,7 +80,7 @@ Status SchemaCollationsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
int len = strlen(_s_collations[_index].is_default);
str_slot->ptr = (char*)pool->allocate(len + 1);
if (NULL == str_slot->ptr) {
if (nullptr == str_slot->ptr) {
return Status::InternalError("No Memory.");
}
memcpy(str_slot->ptr, _s_collations[_index].is_default, len + 1);
@ -92,7 +92,7 @@ Status SchemaCollationsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
int len = strlen(_s_collations[_index].is_compile);
str_slot->ptr = (char*)pool->allocate(len + 1);
if (NULL == str_slot->ptr) {
if (nullptr == str_slot->ptr) {
return Status::InternalError("No Memory.");
}
memcpy(str_slot->ptr, _s_collations[_index].is_compile, len + 1);
@ -111,13 +111,14 @@ Status SchemaCollationsScanner::get_next_row(Tuple* tuple, MemPool* pool, bool*
if (!_is_init) {
return Status::InternalError("call this before initial.");
}
if (NULL == _s_collations[_index].name) {
if (nullptr == tuple || nullptr == pool || nullptr == eos) {
return Status::InternalError("invalid parameter.");
}
if (nullptr == _s_collations[_index].name) {
*eos = true;
return Status::OK();
}
if (NULL == tuple || NULL == pool || NULL == eos) {
return Status::InternalError("invalid parameter.");
}
*eos = false;
return fill_one_row(tuple, pool);
}

View File

@ -51,7 +51,7 @@ Status SchemaVariablesScanner::start(RuntimeState* state) {
}
var_params.__set_threadId(_param->thread_id);
if (NULL != _param->ip && 0 != _param->port) {
if (nullptr != _param->ip && 0 != _param->port) {
RETURN_IF_ERROR(SchemaHelper::show_variables(*(_param->ip), _param->port, var_params,
&_var_result));
} else {
@ -68,7 +68,7 @@ Status SchemaVariablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
int len = strlen(_begin->first.c_str());
str_slot->ptr = (char*)pool->allocate(len + 1);
if (NULL == str_slot->ptr) {
if (nullptr == str_slot->ptr) {
return Status::InternalError("No Memory.");
}
memcpy(str_slot->ptr, _begin->first.c_str(), len + 1);
@ -80,7 +80,7 @@ Status SchemaVariablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
int len = strlen(_begin->second.c_str());
str_slot->ptr = (char*)pool->allocate(len + 1);
if (NULL == str_slot->ptr) {
if (nullptr == str_slot->ptr) {
return Status::InternalError("No Memory.");
}
memcpy(str_slot->ptr, _begin->second.c_str(), len + 1);
@ -94,13 +94,13 @@ Status SchemaVariablesScanner::get_next_row(Tuple* tuple, MemPool* pool, bool* e
if (!_is_init) {
return Status::InternalError("call this before initial.");
}
if (nullptr == tuple || nullptr == pool || nullptr == eos) {
return Status::InternalError("invalid parameter.");
}
if (_begin == _var_result.variables.end()) {
*eos = true;
return Status::OK();
}
if (NULL == tuple || NULL == pool || NULL == eos) {
return Status::InternalError("invalid parameter.");
}
*eos = false;
return fill_one_row(tuple, pool);
}

View File

@ -22,12 +22,12 @@
#include "common/status.h"
#include "gutil/strings/substitute.h"
#include "http/http_channel.h"
#include "http/http_request.h"
#include "http/http_headers.h"
#include "http/http_request.h"
#include "http/http_status.h"
#include "service/backend_options.h"
#include "olap/storage_engine.h"
#include "olap/tablet_manager.h"
#include "service/backend_options.h"
#include "util/json_util.h"
namespace doris {
@ -38,7 +38,7 @@ TabletsDistributionAction::TabletsDistributionAction() {
_host = BackendOptions::get_localhost();
}
void TabletsDistributionAction::handle(HttpRequest *req) {
void TabletsDistributionAction::handle(HttpRequest* req) {
req->add_output_header(HttpHeaders::CONTENT_TYPE, HEADER_JSON.c_str());
std::string req_group_method = req->param("group_by");
@ -50,13 +50,16 @@ void TabletsDistributionAction::handle(HttpRequest *req) {
partition_id = std::stoull(req_partition_id);
} catch (const std::exception& e) {
LOG(WARNING) << "invalid argument. partition_id:" << req_partition_id;
Status status = Status::InternalError(strings::Substitute("invalid argument: partition_id"));
Status status = Status::InternalError(
strings::Substitute("invalid argument: partition_id"));
std::string status_result = to_json(status);
HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, status_result);
return;
}
}
HttpChannel::send_reply(req, HttpStatus::OK, get_tablets_distribution_group_by_partition(partition_id).ToString());
HttpChannel::send_reply(
req, HttpStatus::OK,
get_tablets_distribution_group_by_partition(partition_id).ToString());
return;
}
LOG(WARNING) << "invalid argument. group_by:" << req_group_method;
@ -65,11 +68,13 @@ void TabletsDistributionAction::handle(HttpRequest *req) {
HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, status_result);
}
EasyJson TabletsDistributionAction::get_tablets_distribution_group_by_partition(uint64_t partition_id) {
EasyJson TabletsDistributionAction::get_tablets_distribution_group_by_partition(
uint64_t partition_id) {
std::map<int64_t, std::map<DataDir*, int64_t>> tablets_num_on_disk;
std::map<int64_t, std::map<DataDir*, std::vector<TabletSize>>> tablets_info_on_disk;
TabletManager* tablet_manager = StorageEngine::instance()->tablet_manager();
tablet_manager->get_tablets_distribution_on_different_disks(tablets_num_on_disk, tablets_info_on_disk);
tablet_manager->get_tablets_distribution_on_different_disks(tablets_num_on_disk,
tablets_info_on_disk);
EasyJson tablets_distribution_ej;
tablets_distribution_ej["msg"] = "OK";
@ -78,8 +83,9 @@ EasyJson TabletsDistributionAction::get_tablets_distribution_group_by_partition(
data["host"] = _host;
EasyJson tablets_distribution = data.Set("tablets_distribution", EasyJson::kArray);
int64_t tablet_total_number = 0;
std::map<int64_t, std::map<DataDir*, int64_t>>::iterator partition_iter = tablets_num_on_disk.begin();
for (; partition_iter != tablets_num_on_disk.end(); partition_iter++) {
std::map<int64_t, std::map<DataDir*, int64_t>>::iterator partition_iter =
tablets_num_on_disk.begin();
for (; partition_iter != tablets_num_on_disk.end(); ++partition_iter) {
if (partition_id != 0 && partition_id != partition_iter->first) {
continue;
}
@ -87,18 +93,26 @@ EasyJson TabletsDistributionAction::get_tablets_distribution_group_by_partition(
partition["partition_id"] = partition_iter->first;
EasyJson disks = partition.Set("disks", EasyJson::kArray);
std::map<DataDir*, int64_t>::iterator disk_iter = (partition_iter->second).begin();
for (; disk_iter != (partition_iter->second).end(); disk_iter++) {
for (; disk_iter != (partition_iter->second).end(); ++disk_iter) {
EasyJson disk = disks.PushBack(EasyJson::kObject);
disk["disk_path"] = disk_iter->first->path();
disk["tablets_num"] = disk_iter->second;
tablet_total_number += disk_iter->second;
if (partition_id != 0) {
EasyJson tablets = disk.Set("tablets", EasyJson::kArray);
for (int64_t i = 0; i < tablets_info_on_disk[partition_iter->first][disk_iter->first].size(); i++) {
for (int64_t i = 0;
i < tablets_info_on_disk[partition_iter->first][disk_iter->first].size();
i++) {
EasyJson tablet = tablets.PushBack(EasyJson::kObject);
tablet["tablet_id"] = tablets_info_on_disk[partition_iter->first][disk_iter->first][i].tablet_id;
tablet["schema_hash"] = tablets_info_on_disk[partition_iter->first][disk_iter->first][i].schema_hash;
tablet["tablet_size"] = tablets_info_on_disk[partition_iter->first][disk_iter->first][i].tablet_size;
tablet["tablet_id"] =
tablets_info_on_disk[partition_iter->first][disk_iter->first][i]
.tablet_id;
tablet["schema_hash"] =
tablets_info_on_disk[partition_iter->first][disk_iter->first][i]
.schema_hash;
tablet["tablet_size"] =
tablets_info_on_disk[partition_iter->first][disk_iter->first][i]
.tablet_size;
}
}
}
@ -108,4 +122,3 @@ EasyJson TabletsDistributionAction::get_tablets_distribution_group_by_partition(
}
} // namespace doris

View File

@ -295,7 +295,7 @@ FileReadableBlock::FileReadableBlock(
std::shared_ptr<OpenedFileHandle<RandomAccessFile>> file_handle)
: _block_manager(block_manager),
_path(std::move(path)),
_file_handle(file_handle),
_file_handle(std::move(file_handle)),
_closed(false) {
if (_block_manager->_metrics) {
_block_manager->_metrics->blocks_open_reading->increment(1);

View File

@ -414,8 +414,7 @@ void HllSetHelper::set_sparse(char* result, const std::map<int, uint8_t>& index_
result[0] = HLL_DATA_SPARSE;
len = sizeof(HllSetResolver::SetTypeValueType) + sizeof(HllSetResolver::SparseLengthValueType);
char* write_value_pos = result + len;
for (std::map<int, uint8_t>::const_iterator iter = index_to_value.begin();
iter != index_to_value.end(); iter++) {
for (auto iter = index_to_value.begin(); iter != index_to_value.end(); ++iter) {
write_value_pos[0] = (char)(iter->first & 0xff);
write_value_pos[1] = (char)(iter->first >> 8 & 0xff);
write_value_pos[2] = iter->second;
@ -433,8 +432,7 @@ void HllSetHelper::set_explicit(char* result, const std::set<uint64_t>& hash_val
len = sizeof(HllSetResolver::SetTypeValueType) +
sizeof(HllSetResolver::ExplicitLengthValueType);
char* write_pos = result + len;
for (std::set<uint64_t>::const_iterator iter = hash_value_set.begin();
iter != hash_value_set.end(); iter++) {
for (auto iter = hash_value_set.begin(); iter != hash_value_set.end(); ++iter) {
uint64_t hash_value = *iter;
*(uint64_t*)write_pos = hash_value;
write_pos += 8;
@ -445,8 +443,7 @@ void HllSetHelper::set_explicit(char* result, const std::set<uint64_t>& hash_val
void HllSetHelper::set_full(char* result, const std::map<int, uint8_t>& index_to_value,
const int registers_len, int& len) {
result[0] = HLL_DATA_FULL;
for (std::map<int, uint8_t>::const_iterator iter = index_to_value.begin();
iter != index_to_value.end(); iter++) {
for (auto iter = index_to_value.begin(); iter != index_to_value.end(); ++iter) {
result[1 + iter->first] = iter->second;
}
len = registers_len + sizeof(HllSetResolver::SetTypeValueType);

View File

@ -30,8 +30,11 @@
namespace doris {
BetaRowsetReader::BetaRowsetReader(BetaRowsetSharedPtr rowset,
std::shared_ptr<MemTracker> parent_tracker)
: _rowset(std::move(rowset)), _stats(&_owned_stats), _parent_tracker(std::move(parent_tracker)) {
std::shared_ptr<MemTracker> parent_tracker)
: _context(nullptr),
_rowset(std::move(rowset)),
_stats(&_owned_stats),
_parent_tracker(std::move(parent_tracker)) {
_rowset->aquire();
}
@ -65,8 +68,9 @@ OLAPStatus BetaRowsetReader::init(RowsetReaderContext* read_context) {
}
}
if (read_context->delete_handler != nullptr) {
read_context->delete_handler->get_delete_conditions_after_version(_rowset->end_version(),
&read_options.delete_conditions, read_options.delete_condition_predicates.get());
read_context->delete_handler->get_delete_conditions_after_version(
_rowset->end_version(), &read_options.delete_conditions,
read_options.delete_condition_predicates.get());
}
if (read_context->predicates != nullptr) {
read_options.column_predicates.insert(read_options.column_predicates.end(),

View File

@ -254,7 +254,7 @@ OLAPStatus RunLengthIntegerReader::_read_patched_base_values(uint8_t first_byte)
// if gap is <=255 then patch value cannot be 0
while (curr_gap == 255 && curr_patch == 0) {
actual_gap += 255;
patch_idx++;
++patch_idx;
curr_gap = (uint64_t)unpacked_patch[patch_idx] >> pw;
curr_patch = unpacked_patch[patch_idx] & ((1L << pw) - 1);
}

View File

@ -25,9 +25,9 @@
#include <boost/algorithm/string.hpp>
#include <boost/algorithm/string/classification.hpp>
#include <boost/algorithm/string/split.hpp>
#include <filesystem>
#include <cstdio>
#include <cstdlib>
#include <filesystem>
#include "env/env.h"
#include "gutil/strings/strcat.h"
@ -71,19 +71,18 @@ static bool _cmp_tablet_by_create_time(const TabletSharedPtr& a, const TabletSha
}
TabletManager::TabletManager(int32_t tablet_map_lock_shard_size)
: _mem_tracker(MemTracker::CreateTracker(-1, "TabletManager", nullptr, false)),
_tablets_shards_size(tablet_map_lock_shard_size),
_tablets_shards_mask(tablet_map_lock_shard_size - 1),
_last_update_stat_ms(0) {
: _mem_tracker(MemTracker::CreateTracker(-1, "TabletManager", nullptr, false)),
_tablets_shards_size(tablet_map_lock_shard_size),
_tablets_shards_mask(tablet_map_lock_shard_size - 1),
_last_update_stat_ms(0) {
CHECK_GT(_tablets_shards_size, 0);
CHECK_EQ(_tablets_shards_size & _tablets_shards_mask, 0);
_tablets_shards.resize(_tablets_shards_size);
for (auto& tablets_shard : _tablets_shards) {
tablets_shard.lock = std::unique_ptr<RWMutex>(new RWMutex());
}
REGISTER_HOOK_METRIC(tablet_meta_mem_consumption, [this]() {
return _mem_tracker->consumption();
});
REGISTER_HOOK_METRIC(tablet_meta_mem_consumption,
[this]() { return _mem_tracker->consumption(); });
}
TabletManager::~TabletManager() {
@ -96,7 +95,8 @@ OLAPStatus TabletManager::_add_tablet_unlocked(TTabletId tablet_id, SchemaHash s
bool force) {
OLAPStatus res = OLAP_SUCCESS;
VLOG_NOTICE << "begin to add tablet to TabletManager. "
<< "tablet_id=" << tablet_id << ", schema_hash=" << schema_hash << ", force=" << force;
<< "tablet_id=" << tablet_id << ", schema_hash=" << schema_hash
<< ", force=" << force;
TabletSharedPtr existed_tablet = nullptr;
tablet_map_t& tablet_map = _get_tablet_map(tablet_id);
@ -204,7 +204,7 @@ OLAPStatus TabletManager::_add_tablet_to_map_unlocked(TTabletId tablet_id, Schem
_mem_tracker->Consume(tablet->tablet_meta()->mem_size() * 2);
VLOG_NOTICE << "add tablet to map successfully."
<< " tablet_id=" << tablet_id << ", schema_hash=" << schema_hash;
<< " tablet_id=" << tablet_id << ", schema_hash=" << schema_hash;
return res;
}
@ -571,7 +571,7 @@ OLAPStatus TabletManager::drop_tablets_on_error_root_path(
}
TSchemaHash schema_hash = tablet_info.schema_hash;
VLOG_NOTICE << "drop_tablet begin. tablet_id=" << tablet_id
<< ", schema_hash=" << schema_hash;
<< ", schema_hash=" << schema_hash;
TabletSharedPtr dropped_tablet = _get_tablet_unlocked(tablet_id, schema_hash);
if (dropped_tablet == nullptr) {
LOG(WARNING) << "dropping tablet not exist. "
@ -746,9 +746,9 @@ TabletSharedPtr TabletManager::find_best_tablet_to_compaction(
if (now_ms - last_failure_ms <=
config::min_compaction_failure_interval_sec * 1000) {
VLOG_CRITICAL << "Too often to check compaction, skip it. "
<< "compaction_type=" << compaction_type_str
<< ", last_failure_time_ms=" << last_failure_ms
<< ", tablet_id=" << tablet_ptr->tablet_id();
<< "compaction_type=" << compaction_type_str
<< ", last_failure_time_ms=" << last_failure_ms
<< ", tablet_id=" << tablet_ptr->tablet_id();
continue;
}
@ -764,8 +764,8 @@ TabletSharedPtr TabletManager::find_best_tablet_to_compaction(
}
}
uint32_t current_compaction_score =
tablet_ptr->calc_compaction_score(compaction_type, cumulative_compaction_policy);
uint32_t current_compaction_score = tablet_ptr->calc_compaction_score(
compaction_type, cumulative_compaction_policy);
double scan_frequency = 0.0;
if (config::compaction_tablet_scan_frequency_factor != 0) {
@ -788,11 +788,11 @@ TabletSharedPtr TabletManager::find_best_tablet_to_compaction(
if (best_tablet != nullptr) {
VLOG_CRITICAL << "Found the best tablet for compaction. "
<< "compaction_type=" << compaction_type_str
<< ", tablet_id=" << best_tablet->tablet_id() << ", path=" << data_dir->path()
<< ", compaction_score=" << compaction_score
<< ", tablet_scan_frequency=" << tablet_scan_frequency
<< ", highest_score=" << highest_score;
<< "compaction_type=" << compaction_type_str
<< ", tablet_id=" << best_tablet->tablet_id() << ", path=" << data_dir->path()
<< ", compaction_score=" << compaction_score
<< ", tablet_scan_frequency=" << tablet_scan_frequency
<< ", highest_score=" << highest_score;
*score = compaction_score;
}
return best_tablet;
@ -800,7 +800,8 @@ TabletSharedPtr TabletManager::find_best_tablet_to_compaction(
OLAPStatus TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tablet_id,
TSchemaHash schema_hash, const string& meta_binary,
bool update_meta, bool force, bool restore, bool check_path) {
bool update_meta, bool force, bool restore,
bool check_path) {
TabletMetaSharedPtr tablet_meta(new TabletMeta());
OLAPStatus status = tablet_meta->deserialize(meta_binary);
if (status != OLAP_SUCCESS) {
@ -915,9 +916,9 @@ OLAPStatus TabletManager::load_tablet_from_dir(DataDir* store, TTabletId tablet_
tablet_meta->set_shard_id(shard);
string meta_binary;
tablet_meta->serialize(&meta_binary);
RETURN_NOT_OK_LOG(
load_tablet_from_meta(store, tablet_id, schema_hash, meta_binary, true, force, restore, true),
strings::Substitute("fail to load tablet. header_path=$0", header_path));
RETURN_NOT_OK_LOG(load_tablet_from_meta(store, tablet_id, schema_hash, meta_binary, true, force,
restore, true),
strings::Substitute("fail to load tablet. header_path=$0", header_path));
return OLAP_SUCCESS;
}
@ -957,7 +958,8 @@ OLAPStatus TabletManager::report_tablet_info(TTabletInfo* tablet_info) {
return res;
}
OLAPStatus TabletManager::build_all_report_tablets_info(std::map<TTabletId, TTablet>* tablets_info) {
OLAPStatus TabletManager::build_all_report_tablets_info(
std::map<TTabletId, TTablet>* tablets_info) {
DCHECK(tablets_info != nullptr);
LOG(INFO) << "begin to build all report tablets info";
@ -999,7 +1001,8 @@ OLAPStatus TabletManager::build_all_report_tablets_info(std::map<TTabletId, TTab
}
}
}
DorisMetrics::instance()->tablet_version_num_distribution->set_histogram(tablet_version_num_hist);
DorisMetrics::instance()->tablet_version_num_distribution->set_histogram(
tablet_version_num_hist);
LOG(INFO) << "success to build all report tablets info. tablet_count=" << tablets_info->size();
return OLAP_SUCCESS;
}
@ -1127,8 +1130,8 @@ OLAPStatus TabletManager::start_trash_sweep() {
break;
}
}
// >= 200 means there may be more tablets need to be handled
// So continue
// >= 200 means there may be more tablets need to be handled
// So continue
} while (clean_num >= 200);
return OLAP_SUCCESS;
} // start_trash_sweep
@ -1256,7 +1259,7 @@ void TabletManager::do_tablet_meta_checkpoint(DataDir* data_dir) {
}
int64_t cost = watch.elapsed_time() / 1000 / 1000;
LOG(INFO) << "finish to do meta checkpoint on dir: " << data_dir->path()
<< ", number: " << counter << ", cost(ms): " << cost;
<< ", number: " << counter << ", cost(ms): " << cost;
return;
}
@ -1279,8 +1282,8 @@ void TabletManager::_build_tablet_stat() {
stat.__set_data_size(tablet->tablet_footprint());
stat.__set_row_num(tablet->num_rows());
VLOG_NOTICE << "building tablet stat. tablet_id=" << item.first
<< ", data_size=" << tablet->tablet_footprint()
<< ", row_num=" << tablet->num_rows();
<< ", data_size=" << tablet->tablet_footprint()
<< ", row_num=" << tablet->num_rows();
break;
}
@ -1454,7 +1457,8 @@ OLAPStatus TabletManager::_drop_tablet_directly_unlocked(TTabletId tablet_id,
}
TabletSharedPtr TabletManager::_get_tablet_unlocked(TTabletId tablet_id, SchemaHash schema_hash) {
VLOG_NOTICE << "begin to get tablet. tablet_id=" << tablet_id << ", schema_hash=" << schema_hash;
VLOG_NOTICE << "begin to get tablet. tablet_id=" << tablet_id
<< ", schema_hash=" << schema_hash;
tablet_map_t& tablet_map = _get_tablet_map(tablet_id);
tablet_map_t::iterator it = tablet_map.find(tablet_id);
if (it != tablet_map.end()) {
@ -1462,7 +1466,7 @@ TabletSharedPtr TabletManager::_get_tablet_unlocked(TTabletId tablet_id, SchemaH
CHECK(tablet != nullptr) << "tablet is nullptr. tablet_id=" << tablet_id;
if (tablet->equal(tablet_id, schema_hash)) {
VLOG_NOTICE << "get tablet success. tablet_id=" << tablet_id
<< ", schema_hash=" << schema_hash;
<< ", schema_hash=" << schema_hash;
return tablet;
}
}
@ -1487,7 +1491,7 @@ void TabletManager::_remove_tablet_from_partition(const Tablet& tablet) {
}
}
void TabletManager::obtain_specific_quantity_tablets(vector<TabletInfo> &tablets_info,
void TabletManager::obtain_specific_quantity_tablets(vector<TabletInfo>& tablets_info,
int64_t num) {
for (const auto& tablets_shard : _tablets_shards) {
ReadLock rlock(tablets_shard.lock.get());
@ -1521,8 +1525,8 @@ TabletManager::tablets_shard& TabletManager::_get_tablets_shard(TTabletId tablet
}
void TabletManager::get_tablets_distribution_on_different_disks(
std::map<int64_t, std::map<DataDir*, int64_t>> &tablets_num_on_disk,
std::map<int64_t, std::map<DataDir*, std::vector<TabletSize>>> &tablets_info_on_disk) {
std::map<int64_t, std::map<DataDir*, int64_t>>& tablets_num_on_disk,
std::map<int64_t, std::map<DataDir*, std::vector<TabletSize>>>& tablets_info_on_disk) {
std::vector<DataDir*> data_dirs = StorageEngine::instance()->get_stores();
std::map<int64_t, std::set<TabletInfo>> partition_tablet_map;
{
@ -1533,24 +1537,26 @@ void TabletManager::get_tablets_distribution_on_different_disks(
partition_tablet_map = _partition_tablet_map;
}
std::map<int64_t, std::set<TabletInfo>>::iterator partition_iter = partition_tablet_map.begin();
for (; partition_iter != partition_tablet_map.end(); partition_iter++) {
for (; partition_iter != partition_tablet_map.end(); ++partition_iter) {
std::map<DataDir*, int64_t> tablets_num;
std::map<DataDir*, std::vector<TabletSize>> tablets_info;
for(int i = 0; i < data_dirs.size(); i++) {
for (int i = 0; i < data_dirs.size(); i++) {
tablets_num[data_dirs[i]] = 0;
}
int64_t partition_id = partition_iter->first;
std::set<TabletInfo>::iterator tablet_info_iter = (partition_iter->second).begin();
for(; tablet_info_iter != (partition_iter->second).end(); tablet_info_iter++) {
for (; tablet_info_iter != (partition_iter->second).end(); ++tablet_info_iter) {
// get_tablet() will hold 'tablet_shard_lock'
TabletSharedPtr tablet = get_tablet(tablet_info_iter->tablet_id, tablet_info_iter->schema_hash);
TabletSharedPtr tablet =
get_tablet(tablet_info_iter->tablet_id, tablet_info_iter->schema_hash);
if (tablet == nullptr) {
continue;
}
DataDir* data_dir = tablet->data_dir();
size_t tablet_footprint = tablet->tablet_footprint();
tablets_num[data_dir]++;
TabletSize tablet_size(tablet_info_iter->tablet_id, tablet_info_iter->schema_hash, tablet_footprint);
TabletSize tablet_size(tablet_info_iter->tablet_id, tablet_info_iter->schema_hash,
tablet_footprint);
tablets_info[data_dir].push_back(tablet_size);
}
tablets_num_on_disk[partition_id] = tablets_num;

View File

@ -82,6 +82,7 @@ public:
_num_data_bytes_sent(0),
_packet_seq(0),
_need_close(false),
_be_number(0),
_brpc_dest_addr(brpc_dest),
_is_transfer_chain(is_transfer_chain),
_send_query_statistics_with_every_batch(send_query_statistics_with_every_batch) {
@ -468,8 +469,8 @@ Status DataStreamSender::prepare(RuntimeState* state) {
<< "])";
_profile = _pool->add(new RuntimeProfile(title.str()));
SCOPED_TIMER(_profile->total_time_counter());
_mem_tracker = MemTracker::CreateTracker(
_profile, -1, "DataStreamSender", state->instance_mem_tracker());
_mem_tracker = MemTracker::CreateTracker(_profile, -1, "DataStreamSender",
state->instance_mem_tracker());
if (_part_type == TPartitionType::UNPARTITIONED || _part_type == TPartitionType::RANDOM) {
// Randomize the order we open/transmit to channels to avoid thundering herd problems.

View File

@ -666,7 +666,7 @@ void HllDppSinkMerge::update_hll_set(TupleRow* agg_row, TupleRow* row, ExprConte
if (value->hash_set.size() > HLL_EXPLICIT_INT64_NUM) {
value->type = HLL_DATA_SPARSE;
for (std::set<uint64_t>::iterator iter = value->hash_set.begin();
iter != value->hash_set.end(); iter++) {
iter != value->hash_set.end(); ++iter) {
uint64_t hash = *iter;
int idx = hash % REGISTERS_SIZE;
uint8_t first_one_bit = __builtin_ctzl(hash >> HLL_COLUMN_PRECISION) + 1;

View File

@ -170,7 +170,8 @@ FragmentExecState::FragmentExecState(const TUniqueId& query_id,
this, std::placeholders::_1, std::placeholders::_2,
std::placeholders::_3)),
_timeout_second(-1),
_fragments_ctx(fragments_ctx) {
_fragments_ctx(std::move(fragments_ctx)),
_set_rsc_info(false) {
_start_time = DateTimeValue::local_time();
_coord_addr = _fragments_ctx->coord_addr;
}
@ -436,7 +437,7 @@ void FragmentMgr::_exec_actual(std::shared_ptr<FragmentExecState> exec_state, Fi
{
std::lock_guard<std::mutex> lock(_lock);
_fragment_map.erase(exec_state->fragment_instance_id());
if (all_done) {
if (all_done && fragments_ctx) {
_fragments_ctx_map.erase(fragments_ctx->query_id);
}
}

View File

@ -107,7 +107,7 @@ class SortedRunMerger::ParallelBatchedRowSupplier : public SortedRunMerger::Batc
public:
// Construct an instance from a sorted input run.
ParallelBatchedRowSupplier(SortedRunMerger* parent, const RunBatchSupplier& sorted_run)
: BatchedRowSupplier(parent, sorted_run) {}
: BatchedRowSupplier(parent, sorted_run), _input_row_batch_backup(nullptr) {}
~ParallelBatchedRowSupplier() {
// when have the limit clause need to wait the _pull_task_thread join terminate

View File

@ -314,16 +314,20 @@ void BackendService::close_scanner(TScanCloseResult& result_, const TScanClosePa
result_.status = t_status;
}
void BackendService::get_stream_load_record(TStreamLoadRecordResult& result, const int64_t last_stream_record_time) {
void BackendService::get_stream_load_record(TStreamLoadRecordResult& result,
const int64_t last_stream_record_time) {
auto stream_load_recorder = StorageEngine::instance()->get_stream_load_recorder();
if (stream_load_recorder != nullptr) {
std::map<std::string, std::string> records;
auto st = stream_load_recorder->get_batch(std::to_string(last_stream_record_time), config::stream_load_record_batch_size, &records);
auto st = stream_load_recorder->get_batch(std::to_string(last_stream_record_time),
config::stream_load_record_batch_size, &records);
if (st.ok()) {
LOG(INFO) << "get_batch stream_load_record rocksdb successfully. records size: " << records.size() << ", last_stream_load_timestamp: " << last_stream_record_time;
LOG(INFO) << "get_batch stream_load_record rocksdb successfully. records size: "
<< records.size()
<< ", last_stream_load_timestamp: " << last_stream_record_time;
std::map<std::string, TStreamLoadRecord> stream_load_record_batch;
std::map<std::string, std::string>::iterator it = records.begin();
for (; it != records.end(); it++) {
for (; it != records.end(); ++it) {
TStreamLoadRecord stream_load_item;
StreamLoadContext::parse_stream_load_record(it->second, stream_load_item);
stream_load_record_batch.emplace(it->first.c_str(), stream_load_item);

View File

@ -487,7 +487,7 @@ void HllVal::agg_parse_and_cal(FunctionContext* ctx, const HllVal& other) {
sparse_map = resolver.get_sparse_map();
for (std::map<doris::HllSetResolver::SparseIndexType,
doris::HllSetResolver::SparseValueType>::iterator iter = sparse_map.begin();
iter != sparse_map.end(); iter++) {
iter != sparse_map.end(); ++iter) {
pdata[iter->first] = std::max(pdata[iter->first], (uint8_t)iter->second);
}
} else if (resolver.get_hll_data_type() == doris::HLL_DATA_FULL) {

View File

@ -168,7 +168,7 @@ class FromRowBatchConverter : public arrow::TypeVisitor {
public:
FromRowBatchConverter(const RowBatch& batch, const std::shared_ptr<arrow::Schema>& schema,
arrow::MemoryPool* pool)
: _batch(batch), _schema(schema), _pool(pool) {
: _batch(batch), _schema(schema), _pool(pool), _cur_field_idx(-1) {
// obtain local time zone
time_t ts = 0;
struct tm t;

View File

@ -23,14 +23,14 @@
#include <cstdio>
#include <limits>
#include <map>
#include <sstream>
#include <vector>
#include <new>
#include <numeric>
#include <roaring/roaring.hh>
#include <sstream>
#include <stdexcept>
#include <string>
#include <utility>
#include <vector>
namespace doris_udf {
@ -72,7 +72,7 @@ inline uint8_t* encode_varint64(uint8_t* dst, uint64_t v) {
return dst;
}
inline const uint8_t* decode_varint64_ptr(const uint8_t* p, const uint8_t* limit, uint64_t* value) {
inline const uint8_t* decode_varint64_ptr(const uint8_t* p, const uint8_t* limit, uint64_t* value) {
uint64_t result = 0;
for (uint32_t shift = 0; shift <= 63 && p < limit; shift += 7) {
uint64_t byte = *p;
@ -109,7 +109,6 @@ inline int varint_length(uint64_t v) {
return len;
}
// serialized bitmap := TypeCode(1), Payload
// The format of payload depends on value of TypeCode which is defined below
struct BitmapTypeCode {
@ -431,7 +430,7 @@ public:
bool isSubset(const Roaring64Map& r) const {
for (const auto& map_entry : roarings) {
auto roaring_iter = r.roarings.find(map_entry.first);
if (roaring_iter == roarings.cend())
if (roaring_iter == r.roarings.cend())
return false;
else if (!map_entry.second.isSubset(roaring_iter->second))
return false;
@ -967,7 +966,7 @@ public:
type_of_iterator& operator++() { // ++i, must returned inc. value
if (i.has_value == true) roaring_advance_uint32_iterator(&i);
while (!i.has_value) {
map_iter++;
++map_iter;
if (map_iter == map_end) return *this;
roaring_init_iterator(&map_iter->second.roaring, &i);
}
@ -978,7 +977,7 @@ public:
Roaring64MapSetBitForwardIterator orig(*this);
roaring_advance_uint32_iterator(&i);
while (!i.has_value) {
map_iter++;
++map_iter;
if (map_iter == map_end) return orig;
roaring_init_iterator(&map_iter->second.roaring, &i);
}
@ -1048,36 +1047,36 @@ public:
// Construct a bitmap from given elements.
explicit BitmapValue(const std::vector<uint64_t>& bits) {
switch (bits.size()) {
case 0:
_type = EMPTY;
break;
case 1:
_type = SINGLE;
_sv = bits[0];
break;
default:
_type = BITMAP;
_bitmap.addMany(bits.size(), &bits[0]);
case 0:
_type = EMPTY;
break;
case 1:
_type = SINGLE;
_sv = bits[0];
break;
default:
_type = BITMAP;
_bitmap.addMany(bits.size(), &bits[0]);
}
}
void add(uint64_t value) {
switch (_type) {
case EMPTY:
_sv = value;
_type = SINGLE;
case EMPTY:
_sv = value;
_type = SINGLE;
break;
case SINGLE:
//there is no need to convert the type if two variables are equal
if (_sv == value) {
break;
case SINGLE:
//there is no need to convert the type if two variables are equal
if (_sv == value) {
break;
}
_bitmap.add(_sv);
_bitmap.add(value);
_type = BITMAP;
break;
case BITMAP:
_bitmap.add(value);
}
_bitmap.add(_sv);
_bitmap.add(value);
_type = BITMAP;
break;
case BITMAP:
_bitmap.add(value);
}
}
@ -1088,26 +1087,26 @@ public:
// SINGLE -> BITMAP
BitmapValue& operator|=(const BitmapValue& rhs) {
switch (rhs._type) {
case EMPTY:
break;
case SINGLE:
add(rhs._sv);
break;
case BITMAP:
switch (_type) {
case EMPTY:
_bitmap = rhs._bitmap;
_type = BITMAP;
break;
case SINGLE:
add(rhs._sv);
_bitmap = rhs._bitmap;
_bitmap.add(_sv);
_type = BITMAP;
break;
case BITMAP:
switch (_type) {
case EMPTY:
_bitmap = rhs._bitmap;
_type = BITMAP;
break;
case SINGLE:
_bitmap = rhs._bitmap;
_bitmap.add(_sv);
_type = BITMAP;
break;
case BITMAP:
_bitmap |= rhs._bitmap;
}
break;
_bitmap |= rhs._bitmap;
}
break;
}
return *this;
}
@ -1119,45 +1118,45 @@ public:
// BITMAP -> SINGLE
BitmapValue& operator&=(const BitmapValue& rhs) {
switch (rhs._type) {
case EMPTY:
_type = EMPTY;
_bitmap.clear();
break;
case SINGLE:
switch (_type) {
case EMPTY:
_type = EMPTY;
_bitmap.clear();
break;
case SINGLE:
switch (_type) {
case EMPTY:
break;
case SINGLE:
if (_sv != rhs._sv) {
_type = EMPTY;
}
break;
case BITMAP:
if (!_bitmap.contains(rhs._sv)) {
_type = EMPTY;
} else {
_type = SINGLE;
_sv = rhs._sv;
}
_bitmap.clear();
break;
if (_sv != rhs._sv) {
_type = EMPTY;
}
break;
case BITMAP:
switch (_type) {
case EMPTY:
break;
case SINGLE:
if (!rhs._bitmap.contains(_sv)) {
_type = EMPTY;
}
break;
case BITMAP:
_bitmap &= rhs._bitmap;
_convert_to_smaller_type();
break;
if (!_bitmap.contains(rhs._sv)) {
_type = EMPTY;
} else {
_type = SINGLE;
_sv = rhs._sv;
}
_bitmap.clear();
break;
}
break;
case BITMAP:
switch (_type) {
case EMPTY:
break;
case SINGLE:
if (!rhs._bitmap.contains(_sv)) {
_type = EMPTY;
}
break;
case BITMAP:
_bitmap &= rhs._bitmap;
_convert_to_smaller_type();
break;
}
break;
}
return *this;
}
@ -1165,12 +1164,12 @@ public:
// check if value x is present
bool contains(uint64_t x) {
switch (_type) {
case EMPTY:
return false;
case SINGLE:
return _sv == x;
case BITMAP:
return _bitmap.contains(x);
case EMPTY:
return false;
case SINGLE:
return _sv == x;
case BITMAP:
return _bitmap.contains(x);
}
return false;
}
@ -1178,12 +1177,12 @@ public:
// TODO should the return type be uint64_t?
int64_t cardinality() const {
switch (_type) {
case EMPTY:
return 0;
case SINGLE:
return 1;
case BITMAP:
return _bitmap.cardinality();
case EMPTY:
return 0;
case SINGLE:
return 1;
case BITMAP:
return _bitmap.cardinality();
}
return 0;
}
@ -1193,22 +1192,22 @@ public:
size_t getSizeInBytes() {
size_t res = 0;
switch (_type) {
case EMPTY:
res = 1;
break;
case SINGLE:
if (_sv <= std::numeric_limits<uint32_t>::max()) {
res = 1 + sizeof(uint32_t);
} else {
res = 1 + sizeof(uint64_t);
}
break;
case BITMAP:
//DCHECK(_bitmap.cardinality() > 1);
_bitmap.runOptimize();
_bitmap.shrinkToFit();
res = _bitmap.getSizeInBytes();
break;
case EMPTY:
res = 1;
break;
case SINGLE:
if (_sv <= std::numeric_limits<uint32_t>::max()) {
res = 1 + sizeof(uint32_t);
} else {
res = 1 + sizeof(uint64_t);
}
break;
case BITMAP:
//DCHECK(_bitmap.cardinality() > 1);
_bitmap.runOptimize();
_bitmap.shrinkToFit();
res = _bitmap.getSizeInBytes();
break;
}
return res;
}
@ -1217,21 +1216,21 @@ public:
// Client should call `getSizeInBytes` first to get the serialized size.
void write(char* dst) {
switch (_type) {
case EMPTY:
*dst = BitmapTypeCode::EMPTY;
break;
case SINGLE:
if (_sv <= std::numeric_limits<uint32_t>::max()) {
*(dst++) = BitmapTypeCode::SINGLE32;
encode_fixed32_le(reinterpret_cast<uint8_t*>(dst), static_cast<uint32_t>(_sv));
} else {
*(dst++) = BitmapTypeCode::SINGLE64;
encode_fixed64_le(reinterpret_cast<uint8_t*>(dst), _sv);
}
break;
case BITMAP:
_bitmap.write(dst);
break;
case EMPTY:
*dst = BitmapTypeCode::EMPTY;
break;
case SINGLE:
if (_sv <= std::numeric_limits<uint32_t>::max()) {
*(dst++) = BitmapTypeCode::SINGLE32;
encode_fixed32_le(reinterpret_cast<uint8_t*>(dst), static_cast<uint32_t>(_sv));
} else {
*(dst++) = BitmapTypeCode::SINGLE64;
encode_fixed64_le(reinterpret_cast<uint8_t*>(dst), _sv);
}
break;
case BITMAP:
_bitmap.write(dst);
break;
}
}
@ -1240,24 +1239,24 @@ public:
bool deserialize(const char* src) {
//DCHECK(*src >= BitmapTypeCode::EMPTY && *src <= BitmapTypeCode::BITMAP64);
switch (*src) {
case BitmapTypeCode::EMPTY:
_type = EMPTY;
break;
case BitmapTypeCode::SINGLE32:
_type = SINGLE;
_sv = decode_fixed32_le(reinterpret_cast<const uint8_t*>(src + 1));
break;
case BitmapTypeCode::SINGLE64:
_type = SINGLE;
_sv = decode_fixed64_le(reinterpret_cast<const uint8_t*>(src + 1));
break;
case BitmapTypeCode::BITMAP32:
case BitmapTypeCode::BITMAP64:
_type = BITMAP;
_bitmap = detail::Roaring64Map::read(src);
break;
default:
return false;
case BitmapTypeCode::EMPTY:
_type = EMPTY;
break;
case BitmapTypeCode::SINGLE32:
_type = SINGLE;
_sv = decode_fixed32_le(reinterpret_cast<const uint8_t*>(src + 1));
break;
case BitmapTypeCode::SINGLE64:
_type = SINGLE;
_sv = decode_fixed64_le(reinterpret_cast<const uint8_t*>(src + 1));
break;
case BitmapTypeCode::BITMAP32:
case BitmapTypeCode::BITMAP64:
_type = BITMAP;
_bitmap = detail::Roaring64Map::read(src);
break;
default:
return false;
}
return true;
}
@ -1266,32 +1265,32 @@ public:
std::string to_string() const {
std::stringstream ss;
switch (_type) {
case EMPTY:
break;
case SINGLE:
ss << _sv;
break;
case BITMAP: {
struct IterCtx {
std::stringstream* ss = nullptr;
bool first = true;
} iter_ctx;
iter_ctx.ss = &ss;
case EMPTY:
break;
case SINGLE:
ss << _sv;
break;
case BITMAP: {
struct IterCtx {
std::stringstream* ss = nullptr;
bool first = true;
} iter_ctx;
iter_ctx.ss = &ss;
_bitmap.iterate(
[](uint64_t value, void* c) -> bool {
auto ctx = reinterpret_cast<IterCtx*>(c);
if (ctx->first) {
ctx->first = false;
} else {
(*ctx->ss) << ",";
}
(*ctx->ss) << value;
return true;
},
&iter_ctx);
break;
}
_bitmap.iterate(
[](uint64_t value, void* c) -> bool {
auto ctx = reinterpret_cast<IterCtx*>(c);
if (ctx->first) {
ctx->first = false;
} else {
(*ctx->ss) << ",";
}
(*ctx->ss) << value;
return true;
},
&iter_ctx);
break;
}
}
return ss.str();
}
@ -1314,9 +1313,9 @@ private:
enum BitmapDataType {
EMPTY = 0,
SINGLE = 1, // single element
BITMAP = 2 // more than one elements
BITMAP = 2 // more than one elements
};
uint64_t _sv = 0; // store the single value when _type == SINGLE
uint64_t _sv = 0; // store the single value when _type == SINGLE
detail::Roaring64Map _bitmap; // used when _type == BITMAP
BitmapDataType _type;
};