[fix][refactor] fix bugs and refactor some code by lint (#7871)

1. Fix some `passedByValue` issues.
2. Fix some `dereferenceBeforeCheck` issues.
3. Fix some `uninitMemberVar` issues.
4. Fix some iterator `eraseDereference` issues.
5. Fix compile issue introduced from #7923 #7905 #7848
This commit is contained in:
Mingyu Chen
2022-02-01 14:31:14 +08:00
committed by GitHub
parent 82f421a019
commit c0e59e59aa
38 changed files with 92 additions and 86 deletions

View File

@ -150,7 +150,7 @@ Status HDFSWriter::_connect() {
// set other conf
if (!_properties.empty()) {
std::map<std::string, std::string>::iterator iter;
for (iter = _properties.begin(); iter != _properties.end(); iter++) {
for (iter = _properties.begin(); iter != _properties.end(); ++iter) {
hdfsBuilderConfSetStr(hdfs_builder, iter->first.c_str(), iter->second.c_str());
}
}
@ -165,32 +165,29 @@ Status HDFSWriter::_connect() {
Status HDFSWriter::_parse_properties(std::map<std::string, std::string>& prop) {
std::map<std::string, std::string>::iterator iter;
for (iter = prop.begin(); iter != prop.end(); iter++) {
for (iter = prop.begin(); iter != prop.end();) {
if (iter->first.compare(FS_KEY) == 0) {
_namenode = iter->second;
prop.erase(iter);
}
if (iter->first.compare(USER) == 0) {
iter = prop.erase(iter);
} else if (iter->first.compare(USER) == 0) {
_user = iter->second;
prop.erase(iter);
}
if (iter->first.compare(KERBEROS_PRINCIPAL) == 0) {
iter = prop.erase(iter);
} else if (iter->first.compare(KERBEROS_PRINCIPAL) == 0) {
_kerb_principal = iter->second;
prop.erase(iter);
}
if (iter->first.compare(KERB_TICKET_CACHE_PATH) == 0) {
iter = prop.erase(iter);
} else if (iter->first.compare(KERB_TICKET_CACHE_PATH) == 0) {
_kerb_ticket_cache_path = iter->second;
prop.erase(iter);
}
if (iter->first.compare(TOKEN) == 0) {
iter = prop.erase(iter);
} else if (iter->first.compare(TOKEN) == 0) {
_token = iter->second;
prop.erase(iter);
iter = prop.erase(iter);
} else {
++iter;
}
}
if (_namenode.empty()) {
DCHECK(false) << "hdfs properties is incorrect.";
LOG(ERROR) << "hdfs properties is incorrect.";
LOG(WARNING) << "hdfs properties is incorrect.";
return Status::InternalError("hdfs properties is incorrect");
}

View File

@ -33,7 +33,7 @@ OdbcScanNode::OdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const Descr
std::string scan_node_type)
: ScanNode(pool, tnode, descs),
_is_init(false),
_scan_node_type(scan_node_type),
_scan_node_type(std::move(scan_node_type)),
_table_name(tnode.odbc_scan_node.table_name),
_connect_string(std::move(tnode.odbc_scan_node.connect_string)),
_query_string(std::move(tnode.odbc_scan_node.query_string)),

View File

@ -145,6 +145,7 @@ FunctionContext::TypeDesc AnyValUtil::column_type_to_type_desc(const TypeDescrip
break;
case TYPE_OBJECT:
out.type = FunctionContext::TYPE_OBJECT;
// FIXME(cmy): is this fallthrough meaningful?
case TYPE_CHAR:
out.type = FunctionContext::TYPE_CHAR;
out.len = type.len;

View File

@ -900,7 +900,7 @@ private:
std::unique_ptr<IBloomFilterFuncBase> _bloomfilter_func;
bool _is_bloomfilter = false;
bool _is_ignored_in_filter = false;
std::string *_ignored_in_filter_msg;
std::string* _ignored_in_filter_msg = nullptr;
UniqueId _fragment_instance_id;
uint32_t _filter_id;
};

View File

@ -77,7 +77,7 @@ enum class RuntimeFilterRole { PRODUCER = 0, CONSUMER = 1 };
struct RuntimeFilterParams {
RuntimeFilterParams() : filter_type(RuntimeFilterType::UNKNOWN_FILTER),
bloom_filter_size(-1), filter_id(0), fragment_instance_id(0, 0) {}
bloom_filter_size(-1), max_in_num(0), filter_id(0), fragment_instance_id(0, 0) {}
RuntimeFilterType filter_type;
PrimitiveType column_return_type;
@ -240,10 +240,9 @@ protected:
RuntimeState* _state;
MemTracker* _mem_tracker;
ObjectPool* _pool;
int32_t _fragment_id;
// _wrapper is a runtime filter function wrapper
// _wrapper should alloc from _pool
RuntimePredicateWrapper* _wrapper;
RuntimePredicateWrapper* _wrapper = nullptr;
// runtime filter type
RuntimeFilterType _runtime_filter_type;
// runtime filter id

View File

@ -226,7 +226,7 @@ void CmdlineAction::handle(HttpRequest* req) {
}
char buf[1024];
// Ignore unused return value
if (fscanf(fp, "%s ", buf))
if (fscanf(fp, "%1023s ", buf))
;
fclose(fp);
std::string str = buf;

View File

@ -63,9 +63,11 @@ DeltaWriter::~DeltaWriter() {
// cancel and wait all memtables in flush queue to be finished
_flush_token->cancel();
const FlushStatistic& stat = _flush_token->get_stats();
_tablet->flush_bytes->increment(stat.flush_size_bytes);
_tablet->flush_count->increment(stat.flush_count);
if (_tablet != nullptr) {
const FlushStatistic& stat = _flush_token->get_stats();
_tablet->flush_bytes->increment(stat.flush_size_bytes);
_tablet->flush_count->increment(stat.flush_count);
}
}
if (_tablet != nullptr) {

View File

@ -295,7 +295,7 @@ FileReadableBlock::FileReadableBlock(
std::shared_ptr<OpenedFileHandle<RandomAccessFile>> file_handle)
: _block_manager(block_manager),
_path_desc(path_desc),
_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

@ -111,8 +111,10 @@ private:
};
RemoteWritableBlock::RemoteWritableBlock(RemoteBlockManager* block_manager, const FilePathDesc& path_desc,
shared_ptr<WritableFile> local_writer) : _block_manager(block_manager),
_path_desc(path_desc), _local_writer(local_writer) {
shared_ptr<WritableFile> local_writer)
: _block_manager(block_manager),
_path_desc(path_desc),
_local_writer(std::move(local_writer)) {
}
RemoteWritableBlock::~RemoteWritableBlock() {
@ -209,7 +211,7 @@ private:
// The underlying opened file backing this block.
std::shared_ptr<OpenedFileHandle<RandomAccessFile>> _file_handle;
// the backing file of OpenedFileHandle, not owned.
RandomAccessFile* _file;
RandomAccessFile* _file = nullptr;
// Whether or not this block has been closed. Close() is thread-safe, so
// this must be an atomic primitive.
@ -336,4 +338,4 @@ Status RemoteBlockManager::link_file(const FilePathDesc& src_path_desc, const Fi
}
} // namespace fs
} // namespace doris
} // namespace doris

View File

@ -210,7 +210,7 @@ public:
MergeIterator(std::vector<RowwiseIterator*> iters, std::shared_ptr<MemTracker> parent, int sequence_id_idx)
: _origin_iters(std::move(iters)), _sequence_id_idx(sequence_id_idx), _merge_heap(MergeContextComparator(_sequence_id_idx)) {
// use for count the mem use of Block use in Merge
_mem_tracker = MemTracker::CreateTracker(-1, "MergeIterator", parent, false);
_mem_tracker = MemTracker::CreateTracker(-1, "MergeIterator", std::move(parent), false);
}
~MergeIterator() override {

View File

@ -272,7 +272,7 @@ OLAPStatus RunLengthIntegerReader::_read_patched_base_values(uint8_t first_byte)
_literals[_num_literals++] = base + patched_val;
// increment the patch to point to next entry in patch list
patch_idx++;
++patch_idx;
if (patch_idx < pl) {
// read the next gap and patch
@ -284,7 +284,7 @@ OLAPStatus RunLengthIntegerReader::_read_patched_base_values(uint8_t first_byte)
// <=255 then patch 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

@ -154,6 +154,9 @@ private:
class BinaryPlainPageDecoder : public PageDecoder {
public:
BinaryPlainPageDecoder(Slice data) : BinaryPlainPageDecoder(data, PageDecoderOptions()) {}
BinaryPlainPageDecoder(Slice data, const PageDecoderOptions& options)
: _data(data),
_options(options),

View File

@ -652,7 +652,6 @@ Status FileColumnIterator::_read_data_page(const OrdinalPageIndexIterator& iter)
RETURN_IF_ERROR(_reader->read_page(_opts, iter.page(), &handle, &page_body, &footer));
// parse data page
RETURN_IF_ERROR(ParsedPage::create(std::move(handle), page_body, footer.data_page_footer(),
_opts.stats,
_reader->encoding_info(), iter.page(), iter.page_index(),
&_page));
@ -673,7 +672,7 @@ Status FileColumnIterator::_read_data_page(const OrdinalPageIndexIterator& iter)
&_dict_page_handle, &dict_data, &dict_footer));
// ignore dict_footer.dict_page_footer().encoding() due to only
// PLAIN_ENCODING is supported for dict page right now
_dict_decoder.reset(new BinaryPlainPageDecoder(dict_data, _opts.stats));
_dict_decoder.reset(new BinaryPlainPageDecoder(dict_data));
RETURN_IF_ERROR(_dict_decoder->init());
auto* pd_decoder = (BinaryPlainPageDecoder*)_dict_decoder.get();

View File

@ -444,11 +444,11 @@ Status ScalarColumnWriter::finish_current_page() {
body.push_back(encoded_values.slice());
OwnedSlice nullmap;
if (is_nullable() && _null_bitmap_builder->has_null()) {
nullmap = _null_bitmap_builder->finish();
body.push_back(nullmap.slice());
}
if (_null_bitmap_builder != nullptr) {
if (is_nullable() && _null_bitmap_builder->has_null()) {
nullmap = _null_bitmap_builder->finish();
body.push_back(nullmap.slice());
}
_null_bitmap_builder->reset();
}

View File

@ -102,7 +102,7 @@ Status IndexedColumnIterator::_read_data_page(const PagePointer& pp) {
// parse data page
// note that page_index is not used in IndexedColumnIterator, so we pass 0
return ParsedPage::create(std::move(handle), body, footer.data_page_footer(),
nullptr, _reader->encoding_info(), pp, 0, &_data_page);
_reader->encoding_info(), pp, 0, &_data_page);
}
Status IndexedColumnIterator::seek_to_ordinal(ordinal_t idx) {

View File

@ -32,10 +32,7 @@ struct PageBuilderOptions {
size_t dict_page_size = DEFAULT_PAGE_SIZE;
};
struct PageDecoderOptions {
struct OlapReaderStatistics* stats;
PageDecoderOptions(struct OlapReaderStatistics* stats) : stats(stats) {}
};
struct PageDecoderOptions { };
} // namespace segment_v2
} // namespace doris

View File

@ -36,7 +36,6 @@ namespace segment_v2 {
// this object
struct ParsedPage {
static Status create(PageHandle handle, const Slice& body, const DataPageFooterPB& footer,
struct OlapReaderStatistics* stats,
const EncodingInfo* encoding, const PagePointer& page_pointer,
uint32_t page_index, std::unique_ptr<ParsedPage>* result) {
std::unique_ptr<ParsedPage> page(new ParsedPage);
@ -52,7 +51,7 @@ struct ParsedPage {
}
Slice data_slice(body.data, body.size - null_size);
PageDecoderOptions opts(stats);
PageDecoderOptions opts;
RETURN_IF_ERROR(encoding->create_page_decoder(data_slice, opts, &page->data_decoder));
RETURN_IF_ERROR(page->data_decoder->init());

View File

@ -280,9 +280,9 @@ void TimestampedVersionTracker::recover_versioned_tracker(
while (version_path_iter != timestamped_versions.end()) {
// add version to _version_graph
_version_graph.add_version_to_graph((*version_path_iter)->version());
version_path_iter++;
++version_path_iter;
}
_path_map_iter++;
++_path_map_iter;
}
LOG(INFO) << "recover_versioned_tracker current map info " << get_current_path_map_str();
}
@ -335,7 +335,7 @@ void TimestampedVersionTracker::capture_expired_paths(
int64_t path_version = iter->first;
path_version_vec->push_back(path_version);
}
iter++;
++iter;
}
}
@ -388,11 +388,11 @@ std::string TimestampedVersionTracker::get_current_path_map_str() {
tracker_info << (*version_path_iter)->version().second;
tracker_info << "]";
version_path_iter++;
++version_path_iter;
}
tracker_info << std::endl;
iter++;
++iter;
}
return tracker_info.str();
}

View File

@ -159,7 +159,7 @@ private:
std::mutex _status_lock;
Status _exec_status;
bool _set_rsc_info;
bool _set_rsc_info = false;
std::string _user;
std::string _group;
@ -525,7 +525,7 @@ void FragmentMgr::set_pipe(const TUniqueId& fragment_instance_id,
std::lock_guard<std::mutex> lock(_lock);
auto iter = _fragment_map.find(fragment_instance_id);
if (iter != _fragment_map.end()) {
_fragment_map[fragment_instance_id]->set_pipe(pipe);
_fragment_map[fragment_instance_id]->set_pipe(std::move(pipe));
}
}
}

View File

@ -152,7 +152,7 @@ Status ResultFileSink::close(RuntimeState* state, Status exec_status) {
if (_is_top_sink) {
// close sender, this is normal path end
if (_sender) {
_sender->update_num_written_rows(_writer->get_written_rows());
_sender->update_num_written_rows(_writer == nullptr ? 0 : _writer->get_written_rows());
_sender->close(final_status);
}
state->exec_env()->result_mgr()->cancel_at_time(

View File

@ -63,9 +63,7 @@ Status ResultQueueMgr::fetch_result(const TUniqueId& fragment_instance_id,
if (*result == nullptr) {
*eos = true;
// put sentinel for consistency, avoid repeated invoking fetch result when have no rowbatch
if (queue != nullptr) {
queue->blocking_put(nullptr);
}
queue->blocking_put(nullptr);
} else {
*eos = false;
}

View File

@ -120,7 +120,7 @@ Status ResultSink::close(RuntimeState* state, Status exec_status) {
// close sender, this is normal path end
if (_sender) {
_sender->update_num_written_rows(_writer->get_written_rows());
_sender->update_num_written_rows(_writer == nullptr ? 0 : _writer->get_written_rows());
_sender->update_max_peak_memory_bytes();
_sender->close(final_status);
}

View File

@ -160,7 +160,7 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) {
<< ", job id: " << task.job_id
<< ", queue size: " << _thread_pool.get_queue_size()
<< ", current tasks num: " << _task_map.size();
return Status::TooManyTasks(UniqueId(task.id).to_string());
return Status::TooManyTasks(UniqueId(task.id).to_string() + "_" + BackendOptions::get_localhost());
}
// create the context

View File

@ -288,7 +288,8 @@ std::string StreamLoadContext::to_json_for_mini_load() const {
std::string StreamLoadContext::brief(bool detail) const {
std::stringstream ss;
ss << "id=" << id << ", job_id=" << job_id << ", txn_id=" << txn_id << ", label=" << label;
ss << "id=" << id << ", job_id=" << job_id << ", txn_id=" << txn_id << ", label=" << label
<< ", elapse(s)=" << (UnixMillis() - start_millis) / 1000;
if (detail) {
switch (load_src_type) {
case TLoadSourceType::KAFKA:

View File

@ -190,7 +190,7 @@ private:
std::shared_ptr<arrow::Schema> _schema;
arrow::MemoryPool* _pool;
size_t _cur_field_idx;
size_t _cur_field_idx = 0;
std::vector<std::shared_ptr<arrow::Array>> _arrays;
};

View File

@ -103,7 +103,7 @@ BfdParser* BfdParser::create() {
char prog_name[1024];
// Ignore unused return value
if (fscanf(file, "%s ", prog_name))
if (fscanf(file, "%1023s ", prog_name))
;
fclose(file);
std::unique_ptr<BfdParser> parser(new BfdParser(prog_name));

View File

@ -67,7 +67,7 @@ Status PprofUtils::get_self_cmdline(std::string* cmd) {
}
char buf[1024];
// Ignore unused return value
if (fscanf(fp, "%s ", buf))
if (fscanf(fp, "%1023s ", buf))
;
fclose(fp);
*cmd = buf;

View File

@ -315,8 +315,6 @@ void ColumnComplexType<T>::replicate(const uint32_t* counts, size_t target_size,
res_data.push_back(data[i]);
}
}
return res;
}
using ColumnBitmap = ColumnComplexType<BitmapValue>;

View File

@ -54,7 +54,7 @@ public:
int row_idx = 0;
while (_rows_returned < _num_rows) {
for (int j = 0; j < _schema.num_columns(); ++j) {
vectorized::ColumnWithTypeAndName vc = block->get_by_position(j);
vectorized::ColumnWithTypeAndName& vc = block->get_by_position(j);
vectorized::IColumn& vi = (vectorized::IColumn&)(*vc.column);
char data[16] = {};
@ -121,7 +121,7 @@ Status VAutoIncrementIterator::init(const StorageReadOptions& opts) {
// }
class VMergeIteratorContext {
public:
VMergeIteratorContext(RowwiseIterator* iter, std::shared_ptr<MemTracker> parent) : _iter(iter) {}
VMergeIteratorContext(RowwiseIterator* iter) : _iter(iter) {}
VMergeIteratorContext(const VMergeIteratorContext&) = delete;
VMergeIteratorContext(VMergeIteratorContext&&) = delete;
VMergeIteratorContext& operator=(const VMergeIteratorContext&) = delete;
@ -323,7 +323,7 @@ Status VMergeIterator::init(const StorageReadOptions& opts) {
_schema.reset(new Schema((*(_origin_iters.begin()))->schema()));
for (auto iter : _origin_iters) {
std::unique_ptr<VMergeIteratorContext> ctx(new VMergeIteratorContext(iter, _mem_tracker));
std::unique_ptr<VMergeIteratorContext> ctx(new VMergeIteratorContext(iter));
RETURN_IF_ERROR(ctx->init(opts));
if (!ctx->valid()) {
continue;

View File

@ -29,6 +29,7 @@
#include "util/logging.h"
using doris::segment_v2::PageBuilderOptions;
using doris::OlapReaderStatistics;
namespace doris {

View File

@ -102,10 +102,10 @@ static int seg_id = 0;
namespace doris {
class BaseBenchmark {
public:
BaseBenchmark(std::string name, int iterations) : _name(name), _iterations(iterations) {}
BaseBenchmark(const std::string& name, int iterations) : _name(name), _iterations(iterations) {}
virtual ~BaseBenchmark() {}
void add_name(std::string str) { _name += str; }
void add_name(const std::string& str) { _name += str; }
virtual void init() {}
virtual void run() {}
@ -135,7 +135,7 @@ private:
class BinaryDictPageBenchmark : public BaseBenchmark {
public:
BinaryDictPageBenchmark(std::string name, int iterations) : BaseBenchmark(name, iterations) {}
BinaryDictPageBenchmark(const std::string& name, int iterations) : BaseBenchmark(name, iterations) {}
virtual ~BinaryDictPageBenchmark() override {}
virtual void init() override {}
@ -215,7 +215,7 @@ private:
class BinaryDictPageEncodeBenchmark : public BinaryDictPageBenchmark {
public:
BinaryDictPageEncodeBenchmark(std::string name, int iterations, int rows_number)
BinaryDictPageEncodeBenchmark(const std::string& name, int iterations, int rows_number)
: BinaryDictPageBenchmark(name + "/rows_number:" + std::to_string(rows_number),
iterations),
_rows_number(rows_number) {}
@ -242,7 +242,7 @@ private:
class BinaryDictPageDecodeBenchmark : public BinaryDictPageBenchmark {
public:
BinaryDictPageDecodeBenchmark(std::string name, int iterations, int rows_number)
BinaryDictPageDecodeBenchmark(const std::string& name, int iterations, int rows_number)
: BinaryDictPageBenchmark(name + "/rows_number:" + std::to_string(rows_number),
iterations),
_rows_number(rows_number) {}
@ -271,7 +271,7 @@ private:
class SegmentBenchmark : public BaseBenchmark {
public:
SegmentBenchmark(std::string name, int iterations, std::string column_type)
SegmentBenchmark(const std::string& name, int iterations, const std::string& column_type)
: BaseBenchmark(name, iterations),
_tracker(std::make_shared<MemTracker>()),
_pool(_tracker.get()) {
@ -282,7 +282,7 @@ public:
init_schema(column_type);
}
SegmentBenchmark(std::string name, int iterations)
SegmentBenchmark(const std::string& name, int iterations)
: BaseBenchmark(name, iterations),
_tracker(std::make_shared<MemTracker>()),
_pool(_tracker.get()) {
@ -302,7 +302,7 @@ public:
virtual void init() override {}
virtual void run() override {}
void init_schema(std::string column_type) {
void init_schema(const std::string& column_type) {
std::string column_valid = "/column_type:";
std::vector<std::string> tokens = strings::Split(column_type, ",");
@ -411,7 +411,7 @@ private:
class SegmentWriteBenchmark : public SegmentBenchmark {
public:
SegmentWriteBenchmark(std::string name, int iterations, std::string column_type,
SegmentWriteBenchmark(const std::string& name, int iterations, const std::string& column_type,
int rows_number)
: SegmentBenchmark(name + "/rows_number:" + std::to_string(rows_number), iterations,
column_type),
@ -428,7 +428,7 @@ private:
class SegmentWriteByFileBenchmark : public SegmentBenchmark {
public:
SegmentWriteByFileBenchmark(std::string name, int iterations, std::string file_str)
SegmentWriteByFileBenchmark(const std::string& name, int iterations, const std::string& file_str)
: SegmentBenchmark(name + "/file_path:" + file_str, iterations) {
std::ifstream file(file_str);
assert(file.is_open());
@ -458,7 +458,7 @@ private:
class SegmentScanBenchmark : public SegmentBenchmark {
public:
SegmentScanBenchmark(std::string name, int iterations, std::string column_type, int rows_number)
SegmentScanBenchmark(const std::string& name, int iterations, const std::string& column_type, int rows_number)
: SegmentBenchmark(name + "/rows_number:" + std::to_string(rows_number), iterations,
column_type),
_dataset(generate_dataset(rows_number)) {}
@ -491,7 +491,7 @@ private:
class SegmentScanByFileBenchmark : public SegmentBenchmark {
public:
SegmentScanByFileBenchmark(std::string name, int iterations, std::string file_str)
SegmentScanByFileBenchmark(const std::string& name, int iterations, const std::string& file_str)
: SegmentBenchmark(name, iterations) {
std::ifstream file(file_str);
assert(file.is_open());
@ -540,7 +540,7 @@ private:
// Call method: ./benchmark_tool --operation=Custom
class CustomBenchmark : public BaseBenchmark {
public:
CustomBenchmark(std::string name, int iterations, std::function<void()> init_func,
CustomBenchmark(const std::string& name, int iterations, std::function<void()> init_func,
std::function<void()> run_func)
: BaseBenchmark(name, iterations), _init_func(init_func), _run_func(run_func) {}
virtual ~CustomBenchmark() override {}

View File

@ -2717,7 +2717,7 @@ show_param ::=
{:
RESULT = new ShowAlterStmt(type, db, parser.where, orderByClause, limitClause);
:}
| KW_SHOW KW_DATA KW_SKEW KW_FROM base_table_ref:table_ref
| KW_DATA KW_SKEW KW_FROM base_table_ref:table_ref
{:
RESULT = new ShowDataSkewStmt(table_ref);
:}

View File

@ -3933,7 +3933,7 @@ public class Catalog {
long totalReplicaNum = 0;
for (Map.Entry<String, Long> entry : partitionNameToId.entrySet()) {
long indexNum = olapTable.getIndexIdToMeta().size();
long bucketNum = distributionInfo.getBucketNum();
long bucketNum = defaultDistributionInfo.getBucketNum();
long replicaNum = partitionInfo.getReplicaAllocation(entry.getValue()).getTotalReplicaNum();
totalReplicaNum += indexNum * bucketNum * replicaNum;
}

View File

@ -40,6 +40,7 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;

View File

@ -18,6 +18,7 @@ package org.apache.doris.mysql.nio;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.qe.ConnectProcessor;
import org.xnio.StreamConnection;
import java.io.IOException;
@ -59,4 +60,10 @@ public class NConnectContext extends ConnectContext {
public void stopAcceptQuery() throws IOException {
mysqlChannel.stopAcceptQuery();
}
@Override
public String toString() {
return "[remote ip: " + mysqlChannel.getRemoteIp() + "]";
}
}

View File

@ -133,6 +133,7 @@ public class CatalogOperationTest {
Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState());
}
Thread.sleep(1000);
renameTblStmt = "alter table test.newNewTest rename r1";
alterTableStmt = (AlterTableStmt)UtFrameUtils.parseAndAnalyzeStmt(renameTblStmt, connectContext);
try {

View File

@ -37,7 +37,7 @@ if [ -f $pidfile ]; then
pidcomm=`ps -p $pid -o comm=`
if [ "java" != "$pidcomm" ]; then
echo "ERROR: pid process may not be fe. "
echo "ERROR: pid process may not broker. "
fi
if kill -9 $pid > /dev/null 2>&1; then

View File

@ -38,7 +38,7 @@ private:
// mysql handle
MYSQL* _client;
// doris result
MYSQL_RES* _result;
MYSQL_RES* _result = nullptr;
//doris result as row
MYSQL_ROW _row;
};