[Enhance](BE) Add -Wshadow-field compile option to avoid unexpected shadowing behavior (#25698)

* Fix `Tablet::_meta_lock` shadows member inherited from `BaseTablet`

* Add -Wshadow-field compile option to avoid unexpected shadowing behavior
This commit is contained in:
plat1ko
2023-10-26 10:00:28 +08:00
committed by GitHub
parent da4de17d5c
commit 6dd60c6ebb
36 changed files with 138 additions and 88 deletions

View File

@ -271,6 +271,7 @@ if (COMPILER_CLANG)
add_compile_options(-fcolor-diagnostics
-Wpedantic
-Wshadow-field
-Wunused
-Wunused-command-line-argument
-Wunused-exception-parameter

View File

@ -64,7 +64,6 @@ private:
std::string _bucket;
std::string _key;
bool _closed = false;
bool _aborted = false;
std::shared_ptr<Aws::S3::S3Client> _client;

View File

@ -17,7 +17,14 @@
#pragma once
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshadow-field"
#endif
#include <CLucene.h> // IWYU pragma: keep
#ifdef __clang__
#pragma clang diagnostic pop
#endif
#include <CLucene/config/repl_wchar.h>
#include <CLucene/util/Misc.h>
#include <butil/macros.h>

View File

@ -47,7 +47,14 @@
#include <roaring/roaring.hh>
#include <set>
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshadow-field"
#endif
#include "CLucene/analysis/standard95/StandardAnalyzer.h"
#ifdef __clang__
#pragma clang diagnostic pop
#endif
#include "common/config.h"
#include "common/logging.h"
#include "common/status.h"

View File

@ -30,7 +30,14 @@
#include <roaring/roaring.hh>
#include <vector>
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshadow-field"
#endif
#include "CLucene/analysis/standard95/StandardAnalyzer.h"
#ifdef __clang__
#pragma clang diagnostic pop
#endif
#include "common/config.h"
#include "olap/field.h"
#include "olap/inverted_index_parser.h"

View File

@ -48,7 +48,15 @@ public:
bool is_ngram_bf() const override { return true; }
private:
// FIXME: non-static data member '_size' of 'NGramBloomFilter' shadows member inherited from type 'BloomFilter'
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshadow-field"
#endif
size_t _size;
#ifdef __clang__
#pragma clang diagnostic pop
#endif
size_t words;
std::vector<uint64_t> filter;
};

View File

@ -55,7 +55,15 @@ private:
std::vector<std::unique_ptr<TEsScanRange>> _scan_ranges;
std::unique_ptr<RuntimeProfile> _es_profile;
// FIXME: non-static data member '_rows_read_counter' of 'EsScanLocalState' shadows member inherited from type 'ScanLocalStateBase'
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshadow-field"
#endif
RuntimeProfile::Counter* _rows_read_counter;
#ifdef __clang__
#pragma clang diagnostic pop
#endif
RuntimeProfile::Counter* _read_timer;
RuntimeProfile::Counter* _materialize_timer;
};

View File

@ -182,7 +182,15 @@ private:
const int _consumer_id;
const TDataStreamSink _t_data_stream_sink;
vectorized::VExprContextSPtrs _output_expr_contexts;
// FIXME: non-static data member '_row_descriptor' of 'MultiCastDataStreamerSourceOperatorX' shadows member inherited from type 'OperatorXBase'
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshadow-field"
#endif
const RowDescriptor& _row_descriptor;
#ifdef __clang__
#pragma clang diagnostic pop
#endif
const RowDescriptor& _row_desc() { return _row_descriptor; }
};

View File

@ -169,9 +169,6 @@ private:
RuntimeProfile::Counter* _output_index_result_column_timer = nullptr;
// number of created olap scanners
RuntimeProfile::Counter* _num_scanners = nullptr;
// number of segment filtered by column stat when creating seg iterator
RuntimeProfile::Counter* _filtered_segment_counter = nullptr;
// total number of segment related to this scan node

View File

@ -58,8 +58,7 @@ public:
PartitionSortSourceLocalState(RuntimeState* state, OperatorXBase* parent)
: PipelineXLocalState<PartitionSortDependency>(state, parent),
_get_sorted_timer(nullptr),
_get_next_timer(nullptr),
_num_rows_returned(0) {}
_get_next_timer(nullptr) {}
Status init(RuntimeState* state, LocalStateInfo& info) override;
@ -67,7 +66,6 @@ private:
friend class PartitionSortSourceOperatorX;
RuntimeProfile::Counter* _get_sorted_timer;
RuntimeProfile::Counter* _get_next_timer;
int64_t _num_rows_returned;
int _sort_idx = 0;
};

View File

@ -94,10 +94,6 @@ private:
doris::vectorized::Block* out_block);
bool _should_expand_preagg_hash_tables();
vectorized::Block _preagg_block = vectorized::Block();
vectorized::PODArray<vectorized::AggregateDataPtr> _places;
RuntimeProfile::Counter* _queue_byte_size_counter;
RuntimeProfile::Counter* _queue_size_counter;
RuntimeProfile::Counter* _streaming_agg_timer;

View File

@ -103,7 +103,7 @@ protected:
friend class OperatorXBase;
ObjectPool* _pool;
int64_t _num_rows_returned;
int64_t _num_rows_returned {0};
std::unique_ptr<RuntimeProfile> _runtime_profile;

View File

@ -157,7 +157,14 @@ private:
// TODO: remove the _sink and _multi_cast_stream_sink_senders to set both
// of it in pipeline task not the fragment_context
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshadow-field"
#endif
DataSinkOperatorXPtr _sink;
#ifdef __clang__
#pragma clang diagnostic pop
#endif
std::atomic_bool _canceled = false;

View File

@ -432,12 +432,12 @@ class AggregateFunctionCollect
public:
using BaseHelper = IAggregateFunctionHelper<AggregateFunctionCollect<Data, HasLimit, ShowNull>>;
AggregateFunctionCollect(const DataTypes& argument_types,
AggregateFunctionCollect(const DataTypes& argument_types_,
UInt64 max_size_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<Data,
AggregateFunctionCollect<Data, HasLimit, ShowNull>>(
{argument_types}),
return_type(argument_types[0]) {}
{argument_types_}),
return_type(argument_types_[0]) {}
std::string get_name() const override {
if constexpr (ShowNull::value) {

View File

@ -109,8 +109,9 @@ template <typename Data>
class AggregateFunctionHLLUnion
: public IAggregateFunctionDataHelper<Data, AggregateFunctionHLLUnion<Data>> {
public:
AggregateFunctionHLLUnion(const DataTypes& argument_types)
: IAggregateFunctionDataHelper<Data, AggregateFunctionHLLUnion<Data>>(argument_types) {}
AggregateFunctionHLLUnion(const DataTypes& argument_types_)
: IAggregateFunctionDataHelper<Data, AggregateFunctionHLLUnion<Data>>(argument_types_) {
}
String get_name() const override { return Data::name(); }

View File

@ -247,18 +247,18 @@ class AggregateJavaUdaf final
: public IAggregateFunctionDataHelper<AggregateJavaUdafData, AggregateJavaUdaf> {
public:
ENABLE_FACTORY_CREATOR(AggregateJavaUdaf);
AggregateJavaUdaf(const TFunction& fn, const DataTypes& argument_types,
AggregateJavaUdaf(const TFunction& fn, const DataTypes& argument_types_,
const DataTypePtr& return_type)
: IAggregateFunctionDataHelper(argument_types),
: IAggregateFunctionDataHelper(argument_types_),
_fn(fn),
_return_type(return_type),
_first_created(true),
_exec_place(nullptr) {}
~AggregateJavaUdaf() override = default;
static AggregateFunctionPtr create(const TFunction& fn, const DataTypes& argument_types,
static AggregateFunctionPtr create(const TFunction& fn, const DataTypes& argument_types_,
const DataTypePtr& return_type) {
return std::make_shared<AggregateJavaUdaf>(fn, argument_types, return_type);
return std::make_shared<AggregateJavaUdaf>(fn, argument_types_, return_type);
}
//Note: The condition is added because maybe the BE can't find java-udaf impl jar
//So need to check as soon as possible, before call Data function

View File

@ -195,9 +195,9 @@ template <typename Data>
class ReaderFunctionData final
: public IAggregateFunctionDataHelper<Data, ReaderFunctionData<Data>> {
public:
ReaderFunctionData(const DataTypes& argument_types)
: IAggregateFunctionDataHelper<Data, ReaderFunctionData<Data>>(argument_types),
_argument_type(argument_types[0]) {}
ReaderFunctionData(const DataTypes& argument_types_)
: IAggregateFunctionDataHelper<Data, ReaderFunctionData<Data>>(argument_types_),
_argument_type(argument_types_[0]) {}
String get_name() const override { return Data::name(); }

View File

@ -336,14 +336,14 @@ public:
class AggregateRpcUdaf final
: public IAggregateFunctionDataHelper<AggregateRpcUdafData, AggregateRpcUdaf> {
public:
AggregateRpcUdaf(const TFunction& fn, const DataTypes& argument_types,
AggregateRpcUdaf(const TFunction& fn, const DataTypes& argument_types_,
const DataTypePtr& return_type)
: IAggregateFunctionDataHelper(argument_types), _fn(fn), _return_type(return_type) {}
: IAggregateFunctionDataHelper(argument_types_), _fn(fn), _return_type(return_type) {}
~AggregateRpcUdaf() = default;
static AggregateFunctionPtr create(const TFunction& fn, const DataTypes& argument_types,
static AggregateFunctionPtr create(const TFunction& fn, const DataTypes& argument_types_,
const DataTypePtr& return_type) {
return std::make_shared<AggregateRpcUdaf>(fn, argument_types, return_type);
return std::make_shared<AggregateRpcUdaf>(fn, argument_types_, return_type);
}
void create(AggregateDataPtr __restrict place) const override {

View File

@ -26,18 +26,18 @@ class AggregateStateMerge : public AggregateStateUnion {
public:
using AggregateStateUnion::create;
AggregateStateMerge(AggregateFunctionPtr function, const DataTypes& argument_types,
AggregateStateMerge(AggregateFunctionPtr function, const DataTypes& argument_types_,
const DataTypePtr& return_type)
: AggregateStateUnion(function, argument_types, return_type) {}
: AggregateStateUnion(function, argument_types_, return_type) {}
static AggregateFunctionPtr create(AggregateFunctionPtr function,
const DataTypes& argument_types,
const DataTypes& argument_types_,
const DataTypePtr& return_type) {
CHECK(argument_types.size() == 1);
CHECK(argument_types_.size() == 1);
if (function == nullptr) {
return nullptr;
}
return std::make_shared<AggregateStateMerge>(function, argument_types, return_type);
return std::make_shared<AggregateStateMerge>(function, argument_types_, return_type);
}
void set_version(const int version_) override {

View File

@ -25,21 +25,21 @@ const static std::string AGG_UNION_SUFFIX = "_union";
class AggregateStateUnion : public IAggregateFunctionHelper<AggregateStateUnion> {
public:
AggregateStateUnion(AggregateFunctionPtr function, const DataTypes& argument_types,
AggregateStateUnion(AggregateFunctionPtr function, const DataTypes& argument_types_,
const DataTypePtr& return_type)
: IAggregateFunctionHelper(argument_types),
: IAggregateFunctionHelper(argument_types_),
_function(function),
_return_type(return_type) {}
~AggregateStateUnion() override = default;
static AggregateFunctionPtr create(AggregateFunctionPtr function,
const DataTypes& argument_types,
const DataTypes& argument_types_,
const DataTypePtr& return_type) {
CHECK(argument_types.size() == 1);
CHECK(argument_types_.size() == 1);
if (function == nullptr) {
return nullptr;
}
return std::make_shared<AggregateStateUnion>(function, argument_types, return_type);
return std::make_shared<AggregateStateUnion>(function, argument_types_, return_type);
}
void set_version(const int version_) override {

View File

@ -389,9 +389,9 @@ template <typename Data>
class WindowFunctionData final
: public IAggregateFunctionDataHelper<Data, WindowFunctionData<Data>> {
public:
WindowFunctionData(const DataTypes& argument_types)
: IAggregateFunctionDataHelper<Data, WindowFunctionData<Data>>(argument_types),
_argument_type(argument_types[0]) {}
WindowFunctionData(const DataTypes& argument_types_)
: IAggregateFunctionDataHelper<Data, WindowFunctionData<Data>>(argument_types_),
_argument_type(argument_types_[0]) {}
String get_name() const override { return Data::name(); }

View File

@ -113,17 +113,17 @@ struct creator_without_type {
}
template <typename AggregateFunctionTemplate, typename... TArgs>
static AggregateFunctionPtr create(const DataTypes& argument_types,
static AggregateFunctionPtr create(const DataTypes& argument_types_,
const bool result_is_nullable, TArgs&&... args) {
IAggregateFunction* result(new AggregateFunctionTemplate(std::forward<TArgs>(args)...,
remove_nullable(argument_types)));
if (have_nullable(argument_types)) {
remove_nullable(argument_types_)));
if (have_nullable(argument_types_)) {
std::visit(
[&](auto multi_arguments, auto result_is_nullable) {
result = new NullableT<multi_arguments, result_is_nullable,
AggregateFunctionTemplate>(result, argument_types);
AggregateFunctionTemplate>(result, argument_types_);
},
make_bool_variant(argument_types.size() > 1),
make_bool_variant(argument_types_.size() > 1),
make_bool_variant(result_is_nullable));
}
@ -134,11 +134,11 @@ struct creator_without_type {
/// AggregateFunctionTemplate will handle the nullable arguments, no need to use
/// AggregateFunctionNullVariadicInline/AggregateFunctionNullUnaryInline
template <typename AggregateFunctionTemplate, typename... TArgs>
static AggregateFunctionPtr create_ignore_nullable(const DataTypes& argument_types,
static AggregateFunctionPtr create_ignore_nullable(const DataTypes& argument_types_,
const bool /*result_is_nullable*/,
TArgs&&... args) {
IAggregateFunction* result(
new AggregateFunctionTemplate(std::forward<TArgs>(args)..., argument_types));
new AggregateFunctionTemplate(std::forward<TArgs>(args)..., argument_types_));
CHECK_AGG_FUNCTION_SERIALIZED_TYPE(AggregateFunctionTemplate);
return AggregateFunctionPtr(result);
}

View File

@ -134,6 +134,11 @@ struct MethodBase {
size_t num_rows) = 0;
};
// FIXME: parameter 'keys' shadows member inherited from type `MethodBase`
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshadow-field"
#endif
template <typename TData>
struct MethodSerialized : public MethodBase<TData> {
using Base = MethodBase<TData>;
@ -464,7 +469,6 @@ struct MethodSingleNullableColumn : public SingleColumnMethod {
using Base = SingleColumnMethod;
using State = ColumnsHashing::HashMethodSingleLowNullableColumn<typename Base::State,
typename Base::Mapped>;
void insert_keys_into_columns(std::vector<typename Base::Key>& keys,
MutableColumns& key_columns, const size_t num_rows) override {
auto* col = key_columns[0].get();
@ -476,6 +480,9 @@ struct MethodSingleNullableColumn : public SingleColumnMethod {
}
}
};
#ifdef __clang__
#pragma clang diagnostic pop
#endif
template <typename RowRefListType>
using SerializedHashTableContext = MethodSerialized<PartitionedHashMap<StringRef, RowRefListType>>;

View File

@ -75,12 +75,12 @@ void EncloseCsvTextFieldSplitter::do_split(const Slice& line, std::vector<Slice>
const auto& column_sep_positions = _text_line_reader_ctx->column_sep_positions();
size_t value_start_offset = 0;
for (auto idx : column_sep_positions) {
process_value_func(data, value_start_offset, idx - value_start_offset, trimming_char,
process_value_func(data, value_start_offset, idx - value_start_offset, _trimming_char,
splitted_values);
value_start_offset = idx + value_sep_len;
value_start_offset = idx + _value_sep_len;
}
// process the last column
process_value_func(data, value_start_offset, line.size - value_start_offset, trimming_char,
process_value_func(data, value_start_offset, line.size - value_start_offset, _trimming_char,
splitted_values);
}
@ -91,11 +91,11 @@ void PlainCsvTextFieldSplitter::_split_field_single_char(const Slice& line,
size_t value_start = 0;
for (size_t i = 0; i < size; ++i) {
if (data[i] == _value_sep[0]) {
process_value_func(data, value_start, i - value_start, trimming_char, splitted_values);
value_start = i + value_sep_len;
process_value_func(data, value_start, i - value_start, _trimming_char, splitted_values);
value_start = i + _value_sep_len;
}
}
process_value_func(data, value_start, size - value_start, trimming_char, splitted_values);
process_value_func(data, value_start, size - value_start, _trimming_char, splitted_values);
}
void PlainCsvTextFieldSplitter::_split_field_multi_char(const Slice& line,
@ -114,9 +114,9 @@ void PlainCsvTextFieldSplitter::_split_field_multi_char(const Slice& line,
// curpos curpos
//kmp
vector<int> next(value_sep_len);
vector<int> next(_value_sep_len);
next[0] = -1;
for (int i = 1, j = -1; i < value_sep_len; i++) {
for (int i = 1, j = -1; i < _value_sep_len; i++) {
while (j > -1 && _value_sep[i] != _value_sep[j + 1]) {
j = next[j];
}
@ -135,8 +135,8 @@ void PlainCsvTextFieldSplitter::_split_field_multi_char(const Slice& line,
if (line[i] == _value_sep[j + 1]) {
j++;
}
if (j == value_sep_len - 1) {
curpos = i - value_sep_len + 1;
if (j == _value_sep_len - 1) {
curpos = i - _value_sep_len + 1;
/*
* column_separator : "xx"
@ -154,7 +154,7 @@ void PlainCsvTextFieldSplitter::_split_field_multi_char(const Slice& line,
*/
if (curpos >= start) {
process_value_func(line.data, start, curpos - start, trimming_char,
process_value_func(line.data, start, curpos - start, _trimming_char,
splitted_values);
start = i + 1;
}
@ -162,7 +162,7 @@ void PlainCsvTextFieldSplitter::_split_field_multi_char(const Slice& line,
j = next[j];
}
}
process_value_func(line.data, start, line.size - start, trimming_char, splitted_values);
process_value_func(line.data, start, line.size - start, _trimming_char, splitted_values);
}
void PlainCsvTextFieldSplitter::do_split(const Slice& line, std::vector<Slice>* splitted_values) {

View File

@ -90,7 +90,7 @@ class BaseCsvTextFieldSplitter : public BaseLineFieldSplitter<BaseCsvTextFieldSp
public:
explicit BaseCsvTextFieldSplitter(bool trim_tailing_space, bool trim_ends,
size_t value_sep_len = 1, char trimming_char = 0)
: trimming_char(trimming_char), value_sep_len(value_sep_len) {
: _trimming_char(trimming_char), _value_sep_len(value_sep_len) {
if (trim_tailing_space) {
if (trim_ends) {
process_value_func = &BaseCsvTextFieldSplitter::_process_value<true, true>;
@ -111,8 +111,8 @@ public:
}
protected:
const char trimming_char;
const size_t value_sep_len;
const char _trimming_char;
const size_t _value_sep_len;
ProcessValueFunc process_value_func;
private:

View File

@ -75,7 +75,15 @@ private:
// Profile
std::unique_ptr<RuntimeProfile> _es_profile;
// FIXME: non-static data member '_rows_read_counter' of 'NewEsScanNode' shadows member inherited from type 'VScanNode'
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshadow-field"
#endif
RuntimeProfile::Counter* _rows_read_counter;
#ifdef __clang__
#pragma clang diagnostic pop
#endif
RuntimeProfile::Counter* _read_timer;
RuntimeProfile::Counter* _materialize_timer;
};

View File

@ -47,7 +47,6 @@ NewEsScanner::NewEsScanner(RuntimeState* state, NewEsScanNode* parent, int64_t l
const std::map<std::string, std::string>& docvalue_context,
bool doc_value_mode, RuntimeProfile* profile)
: VScanner(state, static_cast<VScanNode*>(parent), limit, profile),
_is_init(false),
_es_eof(false),
_properties(properties),
_line_eof(false),
@ -57,7 +56,9 @@ NewEsScanner::NewEsScanner(RuntimeState* state, NewEsScanNode* parent, int64_t l
_es_reader(nullptr),
_es_scroll_parser(nullptr),
_docvalue_context(docvalue_context),
_doc_value_mode(doc_value_mode) {}
_doc_value_mode(doc_value_mode) {
_is_init = false;
}
NewEsScanner::NewEsScanner(RuntimeState* state, pipeline::ScanLocalStateBase* local_state,
int64_t limit, TupleId tuple_id,
@ -65,7 +66,6 @@ NewEsScanner::NewEsScanner(RuntimeState* state, pipeline::ScanLocalStateBase* lo
const std::map<std::string, std::string>& docvalue_context,
bool doc_value_mode, RuntimeProfile* profile)
: VScanner(state, local_state, limit, profile),
_is_init(false),
_es_eof(false),
_properties(properties),
_line_eof(false),
@ -75,7 +75,9 @@ NewEsScanner::NewEsScanner(RuntimeState* state, pipeline::ScanLocalStateBase* lo
_es_reader(nullptr),
_es_scroll_parser(nullptr),
_docvalue_context(docvalue_context),
_doc_value_mode(doc_value_mode) {}
_doc_value_mode(doc_value_mode) {
_is_init = false;
}
Status NewEsScanner::prepare(RuntimeState* state, const VExprContextSPtrs& conjuncts) {
VLOG_CRITICAL << NEW_SCANNER_TYPE << "::prepare";

View File

@ -74,7 +74,6 @@ private:
Status _get_next(std::vector<vectorized::MutableColumnPtr>& columns);
private:
bool _is_init;
bool _es_eof;
const std::map<std::string, std::string>& _properties;

View File

@ -40,12 +40,12 @@ NewJdbcScanner::NewJdbcScanner(RuntimeState* state, NewJdbcScanNode* parent, int
const TupleId& tuple_id, const std::string& query_string,
TOdbcTableType::type table_type, RuntimeProfile* profile)
: VScanner(state, static_cast<VScanNode*>(parent), limit, profile),
_is_init(false),
_jdbc_eos(false),
_tuple_id(tuple_id),
_query_string(query_string),
_tuple_desc(nullptr),
_table_type(table_type) {
_is_init = false;
_load_jar_timer = ADD_TIMER(get_parent()->_scanner_profile, "LoadJarTime");
_init_connector_timer = ADD_TIMER(get_parent()->_scanner_profile, "InitConnectorTime");
_check_type_timer = ADD_TIMER(get_parent()->_scanner_profile, "CheckTypeTime");
@ -63,12 +63,12 @@ NewJdbcScanner::NewJdbcScanner(RuntimeState* state,
const TupleId& tuple_id, const std::string& query_string,
TOdbcTableType::type table_type, RuntimeProfile* profile)
: VScanner(state, local_state, limit, profile),
_is_init(false),
_jdbc_eos(false),
_tuple_id(tuple_id),
_query_string(query_string),
_tuple_desc(nullptr),
_table_type(table_type) {
_is_init = false;
_load_jar_timer = ADD_TIMER(local_state->_scanner_profile, "LoadJarTime");
_init_connector_timer = ADD_TIMER(local_state->_scanner_profile, "InitConnectorTime");
_check_type_timer = ADD_TIMER(local_state->_scanner_profile, "CheckTypeTime");

View File

@ -72,8 +72,6 @@ protected:
private:
void _update_profile();
bool _is_init;
bool _jdbc_eos;
// Tuple id resolved in prepare() to set _tuple_desc;

View File

@ -52,13 +52,14 @@ namespace doris::vectorized {
NewOdbcScanner::NewOdbcScanner(RuntimeState* state, NewOdbcScanNode* parent, int64_t limit,
const TOdbcScanNode& odbc_scan_node, RuntimeProfile* profile)
: VScanner(state, static_cast<VScanNode*>(parent), limit, profile),
_is_init(false),
_odbc_eof(false),
_table_name(odbc_scan_node.table_name),
_connect_string(odbc_scan_node.connect_string),
_query_string(odbc_scan_node.query_string),
_tuple_id(odbc_scan_node.tuple_id),
_tuple_desc(nullptr) {}
_tuple_desc(nullptr) {
_is_init = false;
}
Status NewOdbcScanner::prepare(RuntimeState* state, const VExprContextSPtrs& conjuncts) {
VLOG_CRITICAL << NEW_SCANNER_TYPE << "::prepare";

View File

@ -61,8 +61,6 @@ protected:
Status _get_block_impl(RuntimeState* state, Block* block, bool* eos) override;
private:
bool _is_init;
// Indicates whether there are more rows to process. Set in _odbc_connector.next().
bool _odbc_eof;

View File

@ -193,9 +193,6 @@ private:
RuntimeProfile::Counter* _output_index_result_column_timer = nullptr;
// number of created olap scanners
RuntimeProfile::Counter* _num_scanners = nullptr;
// number of segment filtered by column stat when creating seg iterator
RuntimeProfile::Counter* _filtered_segment_counter = nullptr;
// total number of segment related to this scan node

View File

@ -31,20 +31,20 @@ class PipScannerContext : public vectorized::ScannerContext {
public:
PipScannerContext(RuntimeState* state, vectorized::VScanNode* parent,
const TupleDescriptor* output_tuple_desc,
const std::list<vectorized::VScannerSPtr>& scanners, int64_t limit,
const std::list<vectorized::VScannerSPtr>& scanners, int64_t limit_,
int64_t max_bytes_in_blocks_queue, const std::vector<int>& col_distribute_ids,
const int num_parallel_instances)
: vectorized::ScannerContext(state, parent, output_tuple_desc, scanners, limit,
: vectorized::ScannerContext(state, parent, output_tuple_desc, scanners, limit_,
max_bytes_in_blocks_queue, num_parallel_instances),
_col_distribute_ids(col_distribute_ids),
_need_colocate_distribute(!_col_distribute_ids.empty()) {}
PipScannerContext(RuntimeState* state, ScanLocalStateBase* local_state,
const TupleDescriptor* output_tuple_desc,
const std::list<vectorized::VScannerSPtr>& scanners, int64_t limit,
const std::list<vectorized::VScannerSPtr>& scanners, int64_t limit_,
int64_t max_bytes_in_blocks_queue, const std::vector<int>& col_distribute_ids,
const int num_parallel_instances)
: vectorized::ScannerContext(state, nullptr, output_tuple_desc, scanners, limit,
: vectorized::ScannerContext(state, nullptr, output_tuple_desc, scanners, limit_,
max_bytes_in_blocks_queue, num_parallel_instances,
local_state),
_col_distribute_ids(col_distribute_ids),

View File

@ -306,8 +306,6 @@ protected:
VExprContextSPtrs _stale_expr_ctxs;
VExprContextSPtrs _common_expr_ctxs_push_down;
RuntimeState* _state;
// If sort info is set, push limit to each scanner;
int64_t _limit_per_scanner = -1;

View File

@ -69,8 +69,6 @@ private:
BlockQueueSharedPtr _queue;
RuntimeProfile* _profile; // Allocated from _pool
// Owned by the RuntimeState.
const std::vector<TExpr>& _t_output_expr;
VExprContextSPtrs _output_vexpr_ctxs;