From 86abd532ca975a6b7f62971068a96868e6413893 Mon Sep 17 00:00:00 2001 From: Zhao Chun Date: Thu, 23 Sep 2021 21:18:41 +0800 Subject: [PATCH] Fix override by clang-tidy (#358) --- be/src/agent/heartbeat_server.h | 4 +- be/src/agent/multi_worker_pool.h | 6 +- be/src/column/column.h | 6 +- be/src/column/decimalv3_column.h | 2 +- be/src/column/object_column.h | 2 +- be/src/env/env_posix.cpp | 4 +- be/src/env/writable_file_wrapper.h | 2 +- be/src/exec/broker_writer.h | 8 +- be/src/exec/buffered_reader.h | 20 +- be/src/exec/decompressor.h | 38 +- be/src/exec/es/es_predicate.h | 6 +- be/src/exec/es/es_query_builder.h | 2 +- be/src/exec/exchange_node.h | 2 +- be/src/exec/local_file_reader.h | 20 +- be/src/exec/local_file_writer.h | 6 +- be/src/exec/parquet/column_reader.cpp | 2 +- be/src/exec/parquet/stored_column_reader.cpp | 6 +- be/src/exec/parquet_reader.h | 2 +- be/src/exec/parquet_writer.h | 2 +- .../aggregate_blocking_sink_operator.h | 2 +- .../aggregate_blocking_source_operator.h | 2 +- ...ggregate_distinct_blocking_sink_operator.h | 2 +- ...regate_distinct_blocking_source_operator.h | 2 +- ...gregate_distinct_streaming_sink_operator.h | 2 +- ...egate_distinct_streaming_source_operator.h | 2 +- .../aggregate_streaming_sink_operator.h | 2 +- .../aggregate_streaming_source_operator.h | 2 +- be/src/exec/pipeline/pipeline_driver_queue.h | 2 +- .../exec/pipeline/sort/sort_sink_operator.h | 2 +- .../exec/pipeline/sort/sort_source_operator.h | 2 +- be/src/exec/plain_text_line_reader.h | 6 +- be/src/exec/scan_node.h | 6 +- be/src/exec/select_node.h | 6 +- be/src/exec/tablet_sink.h | 2 +- be/src/exec/vectorized/analytic_node.h | 2 +- be/src/exec/vectorized/assert_num_rows_node.h | 2 +- be/src/exec/vectorized/es_http_scan_node.h | 2 +- be/src/exec/vectorized/hdfs_scanner.h | 2 +- be/src/exec/vectorized/hdfs_scanner_orc.cpp | 6 +- be/src/exec/vectorized/hdfs_scanner_orc.h | 2 +- be/src/exec/vectorized/json_scanner.h | 2 +- be/src/exec/vectorized/parquet_reader.h | 2 +- be/src/exec/vectorized/parquet_scanner.h | 2 +- be/src/exec/vectorized/schema_scan_node.h | 2 +- .../schema_scanner/schema_charsets_scanner.h | 4 +- .../schema_collations_scanner.h | 4 +- .../schema_scanner/schema_columns_scanner.h | 6 +- .../schema_scanner/schema_dummy_scanner.h | 4 +- .../schema_scanner/schema_events_scanner.h | 2 +- .../schema_schema_privileges_scanner.h | 2 +- .../schema_scanner/schema_schemata_scanner.h | 6 +- .../schema_statistics_scanner.h | 2 +- .../schema_table_privileges_scanner.h | 2 +- .../schema_scanner/schema_tables_scanner.h | 6 +- .../schema_scanner/schema_triggers_scanner.h | 2 +- .../schema_user_privileges_scanner.h | 2 +- .../schema_scanner/schema_variables_scanner.h | 6 +- .../schema_scanner/schema_views_scanner.h | 2 +- be/src/exec/vectorized/topn_node.h | 2 +- be/src/exprs/agg/avg.h | 2 +- be/src/exprs/agg/count.h | 4 +- be/src/exprs/agg/hll_ndv.h | 2 +- be/src/exprs/agg/hll_union_count.h | 2 +- be/src/exprs/agg/maxmin.h | 4 +- be/src/exprs/agg/nullable_aggregate.h | 2 +- be/src/exprs/agg/sum.h | 2 +- be/src/exprs/agg/variance.h | 4 +- be/src/exprs/agg/window.h | 20 +- be/src/exprs/slot_ref.h | 38 +- be/src/exprs/vectorized/arithmetic_expr.cpp | 2 +- be/src/exprs/vectorized/cast_expr.cpp | 2 +- be/src/exprs/vectorized/column_ref.h | 2 +- .../exprs/vectorized/in_const_predicate.hpp | 4 +- .../vectorized/in_iterator_predicate.hpp | 4 +- be/src/exprs/vectorized/info_func.h | 4 +- be/src/exprs/vectorized/literal.h | 4 +- be/src/exprs/vectorized/runtime_filter.h | 6 +- .../apache-orc/c++/include/orc/Exceptions.hh | 6 +- .../apache-orc/c++/include/orc/Statistics.hh | 18 +- .../orc/apache-orc/c++/include/orc/Vector.hh | 136 +++--- .../formats/orc/apache-orc/c++/src/ByteRLE.cc | 34 +- .../orc/apache-orc/c++/src/ColumnPrinter.cc | 10 +- .../orc/apache-orc/c++/src/ColumnWriter.cc | 206 ++++----- .../orc/apache-orc/c++/src/ColumnWriter.hh | 4 +- .../orc/apache-orc/c++/src/Compression.cc | 82 ++-- .../orc/apache-orc/c++/src/LzoDecompressor.cc | 2 +- .../orc/apache-orc/c++/src/MemoryPool.cc | 2 +- .../orc/apache-orc/c++/src/Statistics.hh | 28 +- .../orc/apache-orc/c++/src/StripeStream.hh | 12 +- .../orc/apache-orc/c++/src/Timezone.cc | 24 +- .../orc/apache-orc/c++/src/Timezone.hh | 2 +- .../orc/apache-orc/c++/src/io/InputStream.hh | 30 +- .../orc/apache-orc/c++/src/io/OutputStream.hh | 12 +- be/src/geo/geo_types.h | 2 +- be/src/gutil/stl_util.h | 4 +- .../gutil/threading/thread_collision_warner.h | 2 +- be/src/http/action/checksum_action.h | 2 +- be/src/http/action/compaction_action.h | 2 +- be/src/http/action/health_action.h | 2 +- be/src/http/action/metrics_action.cpp | 6 +- be/src/http/action/metrics_action.h | 2 +- be/src/http/action/pprof_actions.cpp | 28 +- be/src/http/action/reload_tablet_action.h | 2 +- be/src/http/action/restore_tablet_action.h | 2 +- be/src/http/action/snapshot_action.h | 2 +- be/src/http/action/update_config_action.h | 2 +- be/src/http/download_action.h | 2 +- be/src/http/monitor_action.h | 2 +- be/src/http/web_page_handler.h | 2 +- be/src/plugin/plugin_loader.h | 12 +- be/src/runtime/buffered_block_mgr2.h | 2 +- be/src/runtime/data_stream_sender.h | 2 +- be/src/runtime/descriptors.h | 16 +- be/src/runtime/disk_io_mgr.h | 4 +- be/src/runtime/file_result_writer.h | 8 +- be/src/runtime/fragment_mgr.h | 4 +- be/src/runtime/memory_scratch_sink.h | 14 +- be/src/runtime/message_body_sink.h | 2 +- be/src/runtime/mysql_table_sink.h | 14 +- be/src/runtime/routine_load/data_consumer.h | 14 +- .../routine_load/data_consumer_group.h | 4 +- .../routine_load/kafka_consumer_pipe.h | 2 +- be/src/runtime/row_batch.h | 2 +- be/src/runtime/stream_load/stream_load_pipe.h | 2 +- be/src/service/internal_service.h | 2 +- be/src/storage/decimal_type_info.h | 2 +- be/src/storage/fs/file_block_manager.cpp | 38 +- be/src/storage/lru_cache.h | 23 +- be/src/storage/memory/mem_tablet.h | 2 +- be/src/storage/memory/typed_column_reader.h | 12 +- be/src/storage/memory/typed_column_writer.h | 18 +- be/src/storage/null_predicate.h | 8 +- be/src/storage/primary_index.cpp | 6 +- be/src/storage/rowset/beta_rowset_writer.h | 5 +- .../rowset/segment_v2/bitmap_index_writer.cpp | 2 +- .../rowset/segment_v2/column_writer.cpp | 2 +- .../rowset/segment_v2/segment_iterator.h | 2 +- .../rowset/unique_rowset_id_generator.h | 2 +- .../rowset/vectorized/rowset_writer_adapter.h | 3 +- be/src/storage/schema_change.h | 14 +- be/src/storage/tablet.h | 2 +- .../task/engine_publish_version_task.h | 2 +- be/src/storage/vectorized/rowset_merger.cpp | 2 +- be/src/util/broker_load_error_hub.h | 10 +- be/src/util/callback_closure.h | 2 +- be/src/util/core_local.cpp | 2 +- be/src/util/metrics.h | 12 +- be/src/util/mysql_load_error_hub.h | 10 +- be/src/util/null_load_error_hub.h | 10 +- be/src/util/ref_count_closure.h | 2 +- be/src/util/thrift_server.cpp | 14 +- licenses-binary/LICENSE-joda-time.txt | 404 +++++++++--------- 152 files changed, 868 insertions(+), 885 deletions(-) diff --git a/be/src/agent/heartbeat_server.h b/be/src/agent/heartbeat_server.h index ebcd2c195cb..e3cd30e5c0e 100644 --- a/be/src/agent/heartbeat_server.h +++ b/be/src/agent/heartbeat_server.h @@ -41,7 +41,7 @@ class ThriftServer; class HeartbeatServer : public HeartbeatServiceIf { public: explicit HeartbeatServer(TMasterInfo* master_info); - virtual ~HeartbeatServer(){}; + ~HeartbeatServer() override{}; virtual void init_cluster_id(); @@ -52,7 +52,7 @@ public: // // Output parameters: // * heartbeat_result: The result of heartbeat set - virtual void heartbeat(THeartbeatResult& heartbeat_result, const TMasterInfo& master_info); + void heartbeat(THeartbeatResult& heartbeat_result, const TMasterInfo& master_info) override; private: Status _heartbeat(const TMasterInfo& master_info); diff --git a/be/src/agent/multi_worker_pool.h b/be/src/agent/multi_worker_pool.h index 727f98697ad..e3ae7855976 100644 --- a/be/src/agent/multi_worker_pool.h +++ b/be/src/agent/multi_worker_pool.h @@ -14,12 +14,12 @@ class MultiWorkerPool : public TaskWorkerPool { public: MultiWorkerPool(const TaskWorkerType worker_type, ExecEnv* env, const TMasterInfo& master_info, int worker_num); - virtual ~MultiWorkerPool(){}; + ~MultiWorkerPool() override{}; - virtual void start(); + void start() override; // submit task to queue and wait to be executed - virtual void submit_task(const TAgentTaskRequest& task); + void submit_task(const TAgentTaskRequest& task) override; private: void submit_publish_version_task(const TAgentTaskRequest& task); diff --git a/be/src/column/column.h b/be/src/column/column.h index a80d1bd11a6..54cd7b331cf 100644 --- a/be/src/column/column.h +++ b/be/src/column/column.h @@ -358,11 +358,13 @@ public: return MutablePtr(new Derived(std::forward>(arg))); } - typename AncestorBaseType::MutablePtr clone() const { + typename AncestorBaseType::MutablePtr clone() const override { return typename AncestorBase::MutablePtr(new Derived(*derived())); } - typename AncestorBaseType::Ptr clone_shared() const { return typename AncestorBase::Ptr(new Derived(*derived())); } + typename AncestorBaseType::Ptr clone_shared() const override { + return typename AncestorBase::Ptr(new Derived(*derived())); + } }; } // namespace vectorized diff --git a/be/src/column/decimalv3_column.h b/be/src/column/decimalv3_column.h index 6c462a48eda..ce9c1f8024d 100644 --- a/be/src/column/decimalv3_column.h +++ b/be/src/column/decimalv3_column.h @@ -30,7 +30,7 @@ public: MutableColumnPtr clone_empty() const override; - void put_mysql_row_buffer(MysqlRowBuffer* buf, size_t idx) const; + void put_mysql_row_buffer(MysqlRowBuffer* buf, size_t idx) const override; std::string debug_item(uint32_t idx) const override; void crc32_hash(uint32_t* hash, uint32_t from, uint32_t to) const override; diff --git a/be/src/column/object_column.h b/be/src/column/object_column.h index 64c19cc0dea..72ddb02a937 100644 --- a/be/src/column/object_column.h +++ b/be/src/column/object_column.h @@ -179,7 +179,7 @@ public: const std::vector& get_pool() const { return _pool; } - std::string debug_item(uint32_t idx) const; + std::string debug_item(uint32_t idx) const override; std::string debug_string() const override { std::stringstream ss; diff --git a/be/src/env/env_posix.cpp b/be/src/env/env_posix.cpp index 3b7aa55e215..01db7ac2150 100644 --- a/be/src/env/env_posix.cpp +++ b/be/src/env/env_posix.cpp @@ -409,9 +409,9 @@ public: PosixRandomRWFile(string fname, int fd, bool sync_on_close) : _filename(std::move(fname)), _fd(fd), _sync_on_close(sync_on_close), _closed(false) {} - ~PosixRandomRWFile() { WARN_IF_ERROR(close(), "Failed to close " + _filename); } + ~PosixRandomRWFile() override { WARN_IF_ERROR(close(), "Failed to close " + _filename); } - virtual Status read_at(uint64_t offset, const Slice& result) const override { + Status read_at(uint64_t offset, const Slice& result) const override { return do_readv_at(_fd, _filename, offset, &result, 1, nullptr); } diff --git a/be/src/env/writable_file_wrapper.h b/be/src/env/writable_file_wrapper.h index 3fd00407eca..72fe8fb3c3b 100644 --- a/be/src/env/writable_file_wrapper.h +++ b/be/src/env/writable_file_wrapper.h @@ -12,7 +12,7 @@ public: // |file| must outlive WritableFileWrapper. explicit WritableFileWrapper(WritableFile* file, Ownership ownership) : _file(file), _ownership(ownership) {} - ~WritableFileWrapper() { + ~WritableFileWrapper() override { if (_ownership == kTakesOwnership) { delete _file; } diff --git a/be/src/exec/broker_writer.h b/be/src/exec/broker_writer.h index 3f913d52e3c..befc4cd93d6 100644 --- a/be/src/exec/broker_writer.h +++ b/be/src/exec/broker_writer.h @@ -43,13 +43,13 @@ class BrokerWriter : public FileWriter { public: BrokerWriter(ExecEnv* env, const std::vector& broker_addresses, const std::map& properties, const std::string& path, int64_t start_offset); - virtual ~BrokerWriter(); + ~BrokerWriter() override; - virtual Status open() override; + Status open() override; - virtual Status write(const uint8_t* buf, size_t buf_len, size_t* written_len) override; + Status write(const uint8_t* buf, size_t buf_len, size_t* written_len) override; - virtual Status close() override; + Status close() override; private: ExecEnv* _env; diff --git a/be/src/exec/buffered_reader.h b/be/src/exec/buffered_reader.h index 06454246c8d..ba573e99d72 100644 --- a/be/src/exec/buffered_reader.h +++ b/be/src/exec/buffered_reader.h @@ -37,19 +37,19 @@ public: // If the reader need the file size, set it when construct FileReader. // There is no other way to set the file size. BufferedReader(FileReader* reader, int64_t = 1024 * 1024); - virtual ~BufferedReader(); + ~BufferedReader() override; - virtual Status open() override; + Status open() override; // Read - virtual Status read(uint8_t* buf, size_t* buf_len, bool* eof) override; - virtual Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) override; - virtual Status read_one_message(std::unique_ptr* buf, size_t* length) override; - virtual int64_t size() override; - virtual Status seek(int64_t position) override; - virtual Status tell(int64_t* position) override; - virtual void close() override; - virtual bool closed() override; + Status read(uint8_t* buf, size_t* buf_len, bool* eof) override; + Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) override; + Status read_one_message(std::unique_ptr* buf, size_t* length) override; + int64_t size() override; + Status seek(int64_t position) override; + Status tell(int64_t* position) override; + void close() override; + bool closed() override; private: Status _fill(); diff --git a/be/src/exec/decompressor.h b/be/src/exec/decompressor.h index 520dbbd0b95..d59280fc20b 100644 --- a/be/src/exec/decompressor.h +++ b/be/src/exec/decompressor.h @@ -68,17 +68,17 @@ protected: class GzipDecompressor : public Decompressor { public: - virtual ~GzipDecompressor(); + ~GzipDecompressor() override; - virtual Status decompress(uint8_t* input, size_t input_len, size_t* input_bytes_read, uint8_t* output, - size_t output_len, size_t* output_bytes_written, bool* stream_end) override; + Status decompress(uint8_t* input, size_t input_len, size_t* input_bytes_read, uint8_t* output, size_t output_len, + size_t* output_bytes_written, bool* stream_end) override; - virtual std::string debug_info() override; + std::string debug_info() override; private: friend class Decompressor; GzipDecompressor(bool is_deflate); - virtual Status init() override; + Status init() override; private: bool _is_deflate; @@ -92,17 +92,17 @@ private: class Bzip2Decompressor : public Decompressor { public: - virtual ~Bzip2Decompressor(); + ~Bzip2Decompressor() override; - virtual Status decompress(uint8_t* input, size_t input_len, size_t* input_bytes_read, uint8_t* output, - size_t output_len, size_t* output_bytes_written, bool* stream_end) override; + Status decompress(uint8_t* input, size_t input_len, size_t* input_bytes_read, uint8_t* output, size_t output_len, + size_t* output_bytes_written, bool* stream_end) override; - virtual std::string debug_info() override; + std::string debug_info() override; private: friend class Decompressor; Bzip2Decompressor() : Decompressor(CompressionTypePB::BZIP2) {} - virtual Status init() override; + Status init() override; private: bz_stream _bz_strm; @@ -110,17 +110,17 @@ private: class Lz4FrameDecompressor : public Decompressor { public: - virtual ~Lz4FrameDecompressor(); + ~Lz4FrameDecompressor() override; - virtual Status decompress(uint8_t* input, size_t input_len, size_t* input_bytes_read, uint8_t* output, - size_t output_len, size_t* output_bytes_written, bool* stream_end) override; + Status decompress(uint8_t* input, size_t input_len, size_t* input_bytes_read, uint8_t* output, size_t output_len, + size_t* output_bytes_written, bool* stream_end) override; - virtual std::string debug_info() override; + std::string debug_info() override; private: friend class Decompressor; Lz4FrameDecompressor() : Decompressor(CompressionTypePB::LZ4_FRAME) {} - virtual Status init() override; + Status init() override; size_t get_block_size(const LZ4F_frameInfo_t* info); @@ -134,12 +134,12 @@ private: /// It offers a very wide range of compression/speed trade-off. class ZstandardDecompressor : public Decompressor { public: - virtual ~ZstandardDecompressor(); + ~ZstandardDecompressor() override; - virtual Status decompress(uint8_t* input, size_t input_len, size_t* input_bytes_read, uint8_t* output, - size_t output_len, size_t* output_bytes_write, bool* stream_end) override; + Status decompress(uint8_t* input, size_t input_len, size_t* input_bytes_read, uint8_t* output, size_t output_len, + size_t* output_bytes_write, bool* stream_end) override; - virtual std::string debug_info() override; + std::string debug_info() override; private: friend class Decompressor; diff --git a/be/src/exec/es/es_predicate.h b/be/src/exec/es/es_predicate.h index f489d82616d..5c95c750863 100644 --- a/be/src/exec/es/es_predicate.h +++ b/be/src/exec/es/es_predicate.h @@ -50,8 +50,8 @@ public: class SExtLiteral : public ExtLiteral { public: SExtLiteral(PrimitiveType type, void* value) : _type(type), _value(value) { _str = value_to_string(); } - ~SExtLiteral(); - const std::string& to_string() const { return _str; } + ~SExtLiteral() override; + const std::string& to_string() const override { return _str; } private: int8_t get_byte(); @@ -82,7 +82,7 @@ public: _value = _value_to_string(column); } VExtLiteral() = default; - const std::string& to_string() const { return _value; } + const std::string& to_string() const override { return _value; } private: static std::string _value_to_string(ColumnPtr& column); diff --git a/be/src/exec/es/es_query_builder.h b/be/src/exec/es/es_query_builder.h index 88dbc063352..e3b37e569a0 100644 --- a/be/src/exec/es/es_query_builder.h +++ b/be/src/exec/es/es_query_builder.h @@ -118,7 +118,7 @@ class BooleanQueryBuilder : public QueryBuilder { public: BooleanQueryBuilder(const std::vector& predicates); BooleanQueryBuilder(); - virtual ~BooleanQueryBuilder(); + ~BooleanQueryBuilder() override; // class method for transfer predicate to es query value, invoker should enclose this value with `query` static void to_query(const std::vector& predicates, rapidjson::Document* root, rapidjson::Value* query); diff --git a/be/src/exec/exchange_node.h b/be/src/exec/exchange_node.h index ff0d3440358..ff8e7f28b86 100644 --- a/be/src/exec/exchange_node.h +++ b/be/src/exec/exchange_node.h @@ -50,7 +50,7 @@ class RuntimeProfile; class ExchangeNode final : public ExecNode { public: ExchangeNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - virtual ~ExchangeNode() {} + ~ExchangeNode() override {} Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; Status prepare(RuntimeState* state) override; diff --git a/be/src/exec/local_file_reader.h b/be/src/exec/local_file_reader.h index 052e592905c..2bcbb230c8d 100644 --- a/be/src/exec/local_file_reader.h +++ b/be/src/exec/local_file_reader.h @@ -31,22 +31,22 @@ namespace starrocks { class LocalFileReader : public FileReader { public: LocalFileReader(const std::string& path, int64_t start_offset); - virtual ~LocalFileReader(); + ~LocalFileReader() override; - virtual Status open() override; + Status open() override; // Read content to 'buf', 'buf_len' is the max size of this buffer. // Return ok when read success, and 'buf_len' is set to size of read content // If reach to end of file, the eof is set to true. meanwhile 'buf_len' // is set to zero. - virtual Status read(uint8_t* buf, size_t* buf_len, bool* eof) override; - virtual Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) override; - virtual Status read_one_message(std::unique_ptr* buf, size_t* length) override; - virtual int64_t size() override; - virtual Status seek(int64_t position) override; - virtual Status tell(int64_t* position) override; - virtual void close() override; - virtual bool closed() override; + Status read(uint8_t* buf, size_t* buf_len, bool* eof) override; + Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) override; + Status read_one_message(std::unique_ptr* buf, size_t* length) override; + int64_t size() override; + Status seek(int64_t position) override; + Status tell(int64_t* position) override; + void close() override; + bool closed() override; private: std::string _path; diff --git a/be/src/exec/local_file_writer.h b/be/src/exec/local_file_writer.h index 03b906905bc..20c6ae37c06 100644 --- a/be/src/exec/local_file_writer.h +++ b/be/src/exec/local_file_writer.h @@ -33,13 +33,13 @@ class RuntimeState; class LocalFileWriter : public FileWriter { public: LocalFileWriter(const std::string& path, int64_t start_offset); - virtual ~LocalFileWriter(); + ~LocalFileWriter() override; Status open() override; - virtual Status write(const uint8_t* buf, size_t buf_len, size_t* written_len) override; + Status write(const uint8_t* buf, size_t buf_len, size_t* written_len) override; - virtual Status close() override; + Status close() override; private: std::string _path; diff --git a/be/src/exec/parquet/column_reader.cpp b/be/src/exec/parquet/column_reader.cpp index 075d338380f..f7515b5e8dd 100644 --- a/be/src/exec/parquet/column_reader.cpp +++ b/be/src/exec/parquet/column_reader.cpp @@ -630,7 +630,7 @@ public: return Status::OK(); } - void get_levels(level_t** def_levels, level_t** rep_levels, size_t* num_levels) { + void get_levels(level_t** def_levels, level_t** rep_levels, size_t* num_levels) override { _element_reader->get_levels(def_levels, rep_levels, num_levels); } diff --git a/be/src/exec/parquet/stored_column_reader.cpp b/be/src/exec/parquet/stored_column_reader.cpp index 91e7a4b31da..dfb8fcbb845 100644 --- a/be/src/exec/parquet/stored_column_reader.cpp +++ b/be/src/exec/parquet/stored_column_reader.cpp @@ -68,7 +68,7 @@ class OptionalStoredColumnReader : public StoredColumnReader { public: OptionalStoredColumnReader(const StoredColumnReaderOptions& opts) : _opts(opts) {} - ~OptionalStoredColumnReader() = default; + ~OptionalStoredColumnReader() override = default; Status init(const ParquetField* field, const tparquet::ColumnChunk* chunk_metadata, RandomAccessFile* file) { _field = field; @@ -93,7 +93,7 @@ public: } } - void set_needs_levels(bool needs_levels) { _needs_levels = needs_levels; } + void set_needs_levels(bool needs_levels) override { _needs_levels = needs_levels; } void get_levels(level_t** def_levels, level_t** rep_levels, size_t* num_levels) override { // _needs_levels must be true @@ -150,7 +150,7 @@ public: Status read_records(size_t* num_rows, ColumnContentType content_type, vectorized::Column* dst) override; - void get_levels(level_t** def_levels, level_t** rep_levels, size_t* num_levels) { + void get_levels(level_t** def_levels, level_t** rep_levels, size_t* num_levels) override { *def_levels = nullptr; *rep_levels = nullptr; *num_levels = 0; diff --git a/be/src/exec/parquet_reader.h b/be/src/exec/parquet_reader.h index a6c03053788..d4474ae4f2e 100644 --- a/be/src/exec/parquet_reader.h +++ b/be/src/exec/parquet_reader.h @@ -55,7 +55,7 @@ class FileReader; class ParquetFile : public arrow::io::RandomAccessFile { public: ParquetFile(FileReader* file); - virtual ~ParquetFile(); + ~ParquetFile() override; arrow::Result Read(int64_t nbytes, void* buffer) override; arrow::Result ReadAt(int64_t position, int64_t nbytes, void* out) override; arrow::Result GetSize() override; diff --git a/be/src/exec/parquet_writer.h b/be/src/exec/parquet_writer.h index 164cbc579ff..b724cf10cc0 100644 --- a/be/src/exec/parquet_writer.h +++ b/be/src/exec/parquet_writer.h @@ -54,7 +54,7 @@ class RowBatch; class ParquetOutputStream : public arrow::io::OutputStream { public: ParquetOutputStream(FileWriter* file_writer); - virtual ~ParquetOutputStream(); + ~ParquetOutputStream() override; arrow::Status Write(const void* data, int64_t nbytes) override; // return the current write position of the stream diff --git a/be/src/exec/pipeline/aggregate/aggregate_blocking_sink_operator.h b/be/src/exec/pipeline/aggregate/aggregate_blocking_sink_operator.h index 4a9cea89acb..7e67a47bf7f 100644 --- a/be/src/exec/pipeline/aggregate/aggregate_blocking_sink_operator.h +++ b/be/src/exec/pipeline/aggregate/aggregate_blocking_sink_operator.h @@ -12,7 +12,7 @@ public: : Operator(id, "aggregate_blocking_sink", plan_node_id), _aggregator(aggregator) { _aggregator->set_aggr_phase(AggrPhase2); } - ~AggregateBlockingSinkOperator() = default; + ~AggregateBlockingSinkOperator() override = default; bool has_output() const override { return false; } bool need_input() const override { return true; } diff --git a/be/src/exec/pipeline/aggregate/aggregate_blocking_source_operator.h b/be/src/exec/pipeline/aggregate/aggregate_blocking_source_operator.h index 7135766f823..80a99e5f2f8 100644 --- a/be/src/exec/pipeline/aggregate/aggregate_blocking_source_operator.h +++ b/be/src/exec/pipeline/aggregate/aggregate_blocking_source_operator.h @@ -10,7 +10,7 @@ class AggregateBlockingSourceOperator : public SourceOperator { public: AggregateBlockingSourceOperator(int32_t id, int32_t plan_node_id, AggregatorPtr aggregator) : SourceOperator(id, "aggregate_blocking_source", plan_node_id), _aggregator(aggregator) {} - ~AggregateBlockingSourceOperator() = default; + ~AggregateBlockingSourceOperator() override = default; bool has_output() const override; bool is_finished() const override; diff --git a/be/src/exec/pipeline/aggregate/aggregate_distinct_blocking_sink_operator.h b/be/src/exec/pipeline/aggregate/aggregate_distinct_blocking_sink_operator.h index e1737a673fc..db3cdea8305 100644 --- a/be/src/exec/pipeline/aggregate/aggregate_distinct_blocking_sink_operator.h +++ b/be/src/exec/pipeline/aggregate/aggregate_distinct_blocking_sink_operator.h @@ -12,7 +12,7 @@ public: : Operator(id, "aggregate_distinct_blocking_sink", plan_node_id), _aggregator(aggregator) { _aggregator->set_aggr_phase(AggrPhase2); } - ~AggregateDistinctBlockingSinkOperator() = default; + ~AggregateDistinctBlockingSinkOperator() override = default; bool has_output() const override { return false; } bool need_input() const override { return true; } diff --git a/be/src/exec/pipeline/aggregate/aggregate_distinct_blocking_source_operator.h b/be/src/exec/pipeline/aggregate/aggregate_distinct_blocking_source_operator.h index 94373af7d70..f1e108cb4f7 100644 --- a/be/src/exec/pipeline/aggregate/aggregate_distinct_blocking_source_operator.h +++ b/be/src/exec/pipeline/aggregate/aggregate_distinct_blocking_source_operator.h @@ -10,7 +10,7 @@ class AggregateDistinctBlockingSourceOperator : public SourceOperator { public: AggregateDistinctBlockingSourceOperator(int32_t id, int32_t plan_node_id, AggregatorPtr aggregator) : SourceOperator(id, "aggregate_distinct_blocking_source", plan_node_id), _aggregator(aggregator) {} - ~AggregateDistinctBlockingSourceOperator() = default; + ~AggregateDistinctBlockingSourceOperator() override = default; bool has_output() const override; bool is_finished() const override; diff --git a/be/src/exec/pipeline/aggregate/aggregate_distinct_streaming_sink_operator.h b/be/src/exec/pipeline/aggregate/aggregate_distinct_streaming_sink_operator.h index 9428a3407c5..79b041a978d 100644 --- a/be/src/exec/pipeline/aggregate/aggregate_distinct_streaming_sink_operator.h +++ b/be/src/exec/pipeline/aggregate/aggregate_distinct_streaming_sink_operator.h @@ -12,7 +12,7 @@ public: : Operator(id, "aggregate_distinct_streaming_sink", plan_node_id), _aggregator(aggregator) { _aggregator->set_aggr_phase(AggrPhase1); } - ~AggregateDistinctStreamingSinkOperator() = default; + ~AggregateDistinctStreamingSinkOperator() override = default; bool has_output() const override { return false; } bool need_input() const override { return true; } diff --git a/be/src/exec/pipeline/aggregate/aggregate_distinct_streaming_source_operator.h b/be/src/exec/pipeline/aggregate/aggregate_distinct_streaming_source_operator.h index 40b7f9643cc..920f8aba848 100644 --- a/be/src/exec/pipeline/aggregate/aggregate_distinct_streaming_source_operator.h +++ b/be/src/exec/pipeline/aggregate/aggregate_distinct_streaming_source_operator.h @@ -10,7 +10,7 @@ class AggregateDistinctStreamingSourceOperator : public SourceOperator { public: AggregateDistinctStreamingSourceOperator(int32_t id, int32_t plan_node_id, AggregatorPtr aggregator) : SourceOperator(id, "aggregate_distinct_streaming_source", plan_node_id), _aggregator(aggregator) {} - ~AggregateDistinctStreamingSourceOperator() = default; + ~AggregateDistinctStreamingSourceOperator() override = default; bool has_output() const override; bool is_finished() const override; diff --git a/be/src/exec/pipeline/aggregate/aggregate_streaming_sink_operator.h b/be/src/exec/pipeline/aggregate/aggregate_streaming_sink_operator.h index 638a1b954d2..81f7a3118fb 100644 --- a/be/src/exec/pipeline/aggregate/aggregate_streaming_sink_operator.h +++ b/be/src/exec/pipeline/aggregate/aggregate_streaming_sink_operator.h @@ -12,7 +12,7 @@ public: : Operator(id, "aggregate_streaming_sink", plan_node_id), _aggregator(aggregator) { _aggregator->set_aggr_phase(AggrPhase1); } - ~AggregateStreamingSinkOperator() = default; + ~AggregateStreamingSinkOperator() override = default; bool has_output() const override { return false; } bool need_input() const override { return true; } diff --git a/be/src/exec/pipeline/aggregate/aggregate_streaming_source_operator.h b/be/src/exec/pipeline/aggregate/aggregate_streaming_source_operator.h index 00099a3177a..bc3082058fe 100644 --- a/be/src/exec/pipeline/aggregate/aggregate_streaming_source_operator.h +++ b/be/src/exec/pipeline/aggregate/aggregate_streaming_source_operator.h @@ -10,7 +10,7 @@ class AggregateStreamingSourceOperator : public SourceOperator { public: AggregateStreamingSourceOperator(int32_t id, int32_t plan_node_id, AggregatorPtr aggregator) : SourceOperator(id, "aggregate_streaming_source", plan_node_id), _aggregator(aggregator) {} - ~AggregateStreamingSourceOperator() = default; + ~AggregateStreamingSourceOperator() override = default; bool has_output() const override; bool is_finished() const override; diff --git a/be/src/exec/pipeline/pipeline_driver_queue.h b/be/src/exec/pipeline/pipeline_driver_queue.h index f41800dc96a..d021894a737 100644 --- a/be/src/exec/pipeline/pipeline_driver_queue.h +++ b/be/src/exec/pipeline/pipeline_driver_queue.h @@ -54,7 +54,7 @@ public: static const size_t QUEUE_SIZE = 8; // maybe other value for ratio. static constexpr double RATIO_OF_ADJACENT_QUEUE = 1.7; - void put_back(const DriverPtr& driver); + void put_back(const DriverPtr& driver) override; DriverPtr take(size_t* queue_index) override; SubQuerySharedDriverQueue* get_sub_queue(size_t) override; diff --git a/be/src/exec/pipeline/sort/sort_sink_operator.h b/be/src/exec/pipeline/sort/sort_sink_operator.h index 78341a0816d..3241d7b0637 100644 --- a/be/src/exec/pipeline/sort/sort_sink_operator.h +++ b/be/src/exec/pipeline/sort/sort_sink_operator.h @@ -50,7 +50,7 @@ public: Status push_chunk(RuntimeState* state, const vectorized::ChunkPtr& chunk) override; - virtual void finish(RuntimeState* state) override; + void finish(RuntimeState* state) override; private: // This method is the same as topn node. diff --git a/be/src/exec/pipeline/sort/sort_source_operator.h b/be/src/exec/pipeline/sort/sort_source_operator.h index 80086a3a7f4..a1b870d68c6 100644 --- a/be/src/exec/pipeline/sort/sort_source_operator.h +++ b/be/src/exec/pipeline/sort/sort_source_operator.h @@ -34,7 +34,7 @@ public: void add_morsel(Morsel* morsel) {} - virtual void finish(RuntimeState* state) override; + void finish(RuntimeState* state) override; private: std::shared_ptr _chunks_sorter; diff --git a/be/src/exec/plain_text_line_reader.h b/be/src/exec/plain_text_line_reader.h index db39bafb2f6..9853cee287a 100644 --- a/be/src/exec/plain_text_line_reader.h +++ b/be/src/exec/plain_text_line_reader.h @@ -39,11 +39,11 @@ public: PlainTextLineReader(RuntimeProfile* profile, FileReader* file_reader, Decompressor* decompressor, size_t length, uint8_t row_delimiter); - virtual ~PlainTextLineReader(); + ~PlainTextLineReader() override; - virtual Status read_line(const uint8_t** ptr, size_t* size, bool* eof) override; + Status read_line(const uint8_t** ptr, size_t* size, bool* eof) override; - virtual void close() override; + void close() override; private: bool update_eof(); diff --git a/be/src/exec/scan_node.h b/be/src/exec/scan_node.h index 40cb9283412..5746f97afd2 100644 --- a/be/src/exec/scan_node.h +++ b/be/src/exec/scan_node.h @@ -85,16 +85,16 @@ class TScanRange; class ScanNode : public ExecNode { public: ScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : ExecNode(pool, tnode, descs) {} - virtual ~ScanNode() {} + ~ScanNode() override {} // Set up counters - virtual Status prepare(RuntimeState* state); + Status prepare(RuntimeState* state) override; // Convert scan_ranges into node-specific scan restrictions. This should be // called after prepare() virtual Status set_scan_ranges(const std::vector& scan_ranges) = 0; - virtual bool is_scan_node() const { return true; } + bool is_scan_node() const override { return true; } RuntimeProfile::Counter* bytes_read_counter() const { return _bytes_read_counter; } RuntimeProfile::Counter* rows_read_counter() const { return _rows_read_counter; } diff --git a/be/src/exec/select_node.h b/be/src/exec/select_node.h index 8414099b715..bba581c3100 100644 --- a/be/src/exec/select_node.h +++ b/be/src/exec/select_node.h @@ -37,10 +37,10 @@ class SelectNode : public ExecNode { public: SelectNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - virtual Status prepare(RuntimeState* state); - virtual Status open(RuntimeState* state); + Status prepare(RuntimeState* state) override; + Status open(RuntimeState* state) override; Status get_next(RuntimeState* state, ChunkPtr* chunk, bool* eos) override; - virtual Status close(RuntimeState* state); + Status close(RuntimeState* state) override; private: // true if last get_next() call on child signalled eos diff --git a/be/src/exec/tablet_sink.h b/be/src/exec/tablet_sink.h index c7b640bdbd7..70f6f29b713 100644 --- a/be/src/exec/tablet_sink.h +++ b/be/src/exec/tablet_sink.h @@ -87,7 +87,7 @@ template class ReusableClosure : public google::protobuf::Closure { public: ReusableClosure() : cid(INVALID_BTHREAD_ID) {} - ~ReusableClosure() { + ~ReusableClosure() override { // shouldn't delete when Run() is calling or going to be called, wait for current Run() done. join(); } diff --git a/be/src/exec/vectorized/analytic_node.h b/be/src/exec/vectorized/analytic_node.h index ad3982b8da8..ce14eca2f5d 100644 --- a/be/src/exec/vectorized/analytic_node.h +++ b/be/src/exec/vectorized/analytic_node.h @@ -26,7 +26,7 @@ struct FrameRange { class AnalyticNode : public ExecNode { public: - ~AnalyticNode() {} + ~AnalyticNode() override {} AnalyticNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; diff --git a/be/src/exec/vectorized/assert_num_rows_node.h b/be/src/exec/vectorized/assert_num_rows_node.h index e2ee3562e28..2e125381be1 100644 --- a/be/src/exec/vectorized/assert_num_rows_node.h +++ b/be/src/exec/vectorized/assert_num_rows_node.h @@ -24,7 +24,7 @@ namespace starrocks::vectorized { class AssertNumRowsNode : public ExecNode { public: AssertNumRowsNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - virtual ~AssertNumRowsNode(){}; + ~AssertNumRowsNode() override{}; Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; Status prepare(RuntimeState* state) override; diff --git a/be/src/exec/vectorized/es_http_scan_node.h b/be/src/exec/vectorized/es_http_scan_node.h index b9c3639b2e6..63f69019f92 100644 --- a/be/src/exec/vectorized/es_http_scan_node.h +++ b/be/src/exec/vectorized/es_http_scan_node.h @@ -20,7 +20,7 @@ namespace vectorized { class EsHttpScanNode final : public starrocks::ScanNode { public: EsHttpScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - ~EsHttpScanNode(); + ~EsHttpScanNode() override; Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; Status prepare(RuntimeState* state) override; diff --git a/be/src/exec/vectorized/hdfs_scanner.h b/be/src/exec/vectorized/hdfs_scanner.h index 137f786cae0..126042a1f4e 100644 --- a/be/src/exec/vectorized/hdfs_scanner.h +++ b/be/src/exec/vectorized/hdfs_scanner.h @@ -177,7 +177,7 @@ protected: class HdfsParquetScanner final : public HdfsScanner { public: HdfsParquetScanner() = default; - virtual ~HdfsParquetScanner() = default; + ~HdfsParquetScanner() override = default; void update_counter(); Status do_open(RuntimeState* runtime_state) override; diff --git a/be/src/exec/vectorized/hdfs_scanner_orc.cpp b/be/src/exec/vectorized/hdfs_scanner_orc.cpp index 65831556941..a673ec4c470 100644 --- a/be/src/exec/vectorized/hdfs_scanner_orc.cpp +++ b/be/src/exec/vectorized/hdfs_scanner_orc.cpp @@ -52,11 +52,11 @@ public: const std::map& bloomFilters) override; bool filterMinMax(size_t rowGroupIdx, const std::unordered_map& rowIndexes, const std::map& bloomFilter); - bool filterOnPickStringDictionary(const std::unordered_map& sdicts); + bool filterOnPickStringDictionary(const std::unordered_map& sdicts) override; bool is_slot_evaluated(SlotId id) { return _dict_filter_eval_cache.find(id) != _dict_filter_eval_cache.end(); } - virtual void onStartingPickRowGroups() override; - virtual void onEndingPickRowGroups() override; + void onStartingPickRowGroups() override; + void onEndingPickRowGroups() override; private: const HdfsScannerParams& _scanner_params; diff --git a/be/src/exec/vectorized/hdfs_scanner_orc.h b/be/src/exec/vectorized/hdfs_scanner_orc.h index 7dcc2481dfb..e7fe4d1faa5 100644 --- a/be/src/exec/vectorized/hdfs_scanner_orc.h +++ b/be/src/exec/vectorized/hdfs_scanner_orc.h @@ -14,7 +14,7 @@ class OrcRowReaderFilter; class HdfsOrcScanner final : public HdfsScanner { public: HdfsOrcScanner() = default; - virtual ~HdfsOrcScanner() = default; + ~HdfsOrcScanner() override = default; void update_counter(); Status do_open(RuntimeState* runtime_state) override; diff --git a/be/src/exec/vectorized/json_scanner.h b/be/src/exec/vectorized/json_scanner.h index 000edf367a9..d3c87852144 100644 --- a/be/src/exec/vectorized/json_scanner.h +++ b/be/src/exec/vectorized/json_scanner.h @@ -28,7 +28,7 @@ class JsonScanner : public FileScanner { public: JsonScanner(RuntimeState* state, RuntimeProfile* profile, const TBrokerScanRange& scan_range, ScannerCounter* counter); - ~JsonScanner(); + ~JsonScanner() override; // Open this scanner, will initialize information needed Status open() override; diff --git a/be/src/exec/vectorized/parquet_reader.h b/be/src/exec/vectorized/parquet_reader.h index f7085c372ad..c19daa734cc 100644 --- a/be/src/exec/vectorized/parquet_reader.h +++ b/be/src/exec/vectorized/parquet_reader.h @@ -32,7 +32,7 @@ using RecordBatchPtr = std::shared_ptr; class ParquetChunkFile : public arrow::io::RandomAccessFile { public: ParquetChunkFile(std::shared_ptr file, uint64_t pos); - virtual ~ParquetChunkFile(); + ~ParquetChunkFile() override; arrow::Result Read(int64_t nbytes, void* buffer) override; arrow::Result ReadAt(int64_t position, int64_t nbytes, void* out) override; arrow::Result GetSize() override; diff --git a/be/src/exec/vectorized/parquet_scanner.h b/be/src/exec/vectorized/parquet_scanner.h index 1cf53528d4d..7ece343272f 100644 --- a/be/src/exec/vectorized/parquet_scanner.h +++ b/be/src/exec/vectorized/parquet_scanner.h @@ -32,7 +32,7 @@ public: ParquetScanner(RuntimeState* state, RuntimeProfile* profile, const TBrokerScanRange& scan_range, ScannerCounter* counter); - ~ParquetScanner(); + ~ParquetScanner() override; Status open() override; diff --git a/be/src/exec/vectorized/schema_scan_node.h b/be/src/exec/vectorized/schema_scan_node.h index 695b9006b92..9f19b7c3efb 100644 --- a/be/src/exec/vectorized/schema_scan_node.h +++ b/be/src/exec/vectorized/schema_scan_node.h @@ -19,7 +19,7 @@ namespace starrocks::vectorized { class SchemaScanNode : public ScanNode { public: SchemaScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - ~SchemaScanNode(); + ~SchemaScanNode() override; // Prepare conjuncts, create Schema columns to slots mapping // initialize _schema_scanner diff --git a/be/src/exec/vectorized/schema_scanner/schema_charsets_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_charsets_scanner.h index 5374a9875de..e6e80f0263f 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_charsets_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_charsets_scanner.h @@ -12,9 +12,9 @@ namespace starrocks::vectorized { class SchemaCharsetsScanner : public SchemaScanner { public: SchemaCharsetsScanner(); - virtual ~SchemaCharsetsScanner(); + ~SchemaCharsetsScanner() override; - virtual Status get_next(ChunkPtr* chunk, bool* eos); + Status get_next(ChunkPtr* chunk, bool* eos) override; private: struct CharsetStruct { diff --git a/be/src/exec/vectorized/schema_scanner/schema_collations_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_collations_scanner.h index cdf0b5df7cc..08f1aeff488 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_collations_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_collations_scanner.h @@ -12,9 +12,9 @@ namespace starrocks::vectorized { class SchemaCollationsScanner : public SchemaScanner { public: SchemaCollationsScanner(); - virtual ~SchemaCollationsScanner(); + ~SchemaCollationsScanner() override; - virtual Status get_next(ChunkPtr* chunk, bool* eos); + Status get_next(ChunkPtr* chunk, bool* eos) override; private: struct CollationStruct { diff --git a/be/src/exec/vectorized/schema_scanner/schema_columns_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_columns_scanner.h index 5494a4f04a0..eb0224f12b1 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_columns_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_columns_scanner.h @@ -12,9 +12,9 @@ namespace starrocks::vectorized { class SchemaColumnsScanner : public SchemaScanner { public: SchemaColumnsScanner(); - virtual ~SchemaColumnsScanner(); - virtual Status start(RuntimeState* state); - virtual Status get_next(ChunkPtr* chunk, bool* eos); + ~SchemaColumnsScanner() override; + Status start(RuntimeState* state) override; + Status get_next(ChunkPtr* chunk, bool* eos) override; std::string to_mysql_data_type_string(TColumnDesc& desc); std::string type_to_string(TColumnDesc& desc); diff --git a/be/src/exec/vectorized/schema_scanner/schema_dummy_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_dummy_scanner.h index 2e28a75ec44..4f115c1caac 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_dummy_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_dummy_scanner.h @@ -10,9 +10,9 @@ namespace starrocks::vectorized { class SchemaDummyScanner : public SchemaScanner { public: SchemaDummyScanner(); - virtual ~SchemaDummyScanner(); + ~SchemaDummyScanner() override; virtual Status start(); - virtual Status get_next(ChunkPtr* chunk, bool* eos); + Status get_next(ChunkPtr* chunk, bool* eos) override; private: static SchemaScanner::ColumnDesc _s_dummy_columns[]; diff --git a/be/src/exec/vectorized/schema_scanner/schema_events_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_events_scanner.h index d8e268e22b3..5c22f8273a0 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_events_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_events_scanner.h @@ -10,7 +10,7 @@ namespace starrocks::vectorized { class SchemaEventsScanner : public SchemaScanner { public: SchemaEventsScanner(); - virtual ~SchemaEventsScanner(); + ~SchemaEventsScanner() override; private: static SchemaScanner::ColumnDesc _s_cols_events[]; diff --git a/be/src/exec/vectorized/schema_scanner/schema_schema_privileges_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_schema_privileges_scanner.h index 114bd1547fe..9288114f46e 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_schema_privileges_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_schema_privileges_scanner.h @@ -12,7 +12,7 @@ namespace starrocks::vectorized { class SchemaSchemaPrivilegesScanner : public SchemaScanner { public: SchemaSchemaPrivilegesScanner(); - virtual ~SchemaSchemaPrivilegesScanner(); + ~SchemaSchemaPrivilegesScanner() override; Status start(RuntimeState* state) override; Status get_next(ChunkPtr* chunk, bool* eos) override; diff --git a/be/src/exec/vectorized/schema_scanner/schema_schemata_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_schemata_scanner.h index bf04bbfd2d1..eba79fff51a 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_schemata_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_schemata_scanner.h @@ -10,10 +10,10 @@ namespace starrocks::vectorized { class SchemaSchemataScanner : public SchemaScanner { public: SchemaSchemataScanner(); - virtual ~SchemaSchemataScanner(); + ~SchemaSchemataScanner() override; - virtual Status start(RuntimeState* state); - virtual Status get_next(ChunkPtr* chunk, bool* eos); + Status start(RuntimeState* state) override; + Status get_next(ChunkPtr* chunk, bool* eos) override; private: Status fill_chunk(ChunkPtr* chunk); diff --git a/be/src/exec/vectorized/schema_scanner/schema_statistics_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_statistics_scanner.h index 3d81aab02d7..3b76ea75f9a 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_statistics_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_statistics_scanner.h @@ -10,7 +10,7 @@ namespace starrocks::vectorized { class SchemaStatisticsScanner : public SchemaScanner { public: SchemaStatisticsScanner(); - virtual ~SchemaStatisticsScanner(); + ~SchemaStatisticsScanner() override; private: static SchemaScanner::ColumnDesc _s_cols_statistics[]; diff --git a/be/src/exec/vectorized/schema_scanner/schema_table_privileges_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_table_privileges_scanner.h index 8204839f507..7f02ea086f6 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_table_privileges_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_table_privileges_scanner.h @@ -12,7 +12,7 @@ namespace starrocks::vectorized { class SchemaTablePrivilegesScanner : public SchemaScanner { public: SchemaTablePrivilegesScanner(); - virtual ~SchemaTablePrivilegesScanner(); + ~SchemaTablePrivilegesScanner() override; Status start(RuntimeState* state) override; Status get_next(ChunkPtr* chunk, bool* eos) override; diff --git a/be/src/exec/vectorized/schema_scanner/schema_tables_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_tables_scanner.h index b09020236ac..8b8bc50d747 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_tables_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_tables_scanner.h @@ -10,10 +10,10 @@ namespace starrocks::vectorized { class SchemaTablesScanner : public SchemaScanner { public: SchemaTablesScanner(); - virtual ~SchemaTablesScanner(); + ~SchemaTablesScanner() override; - virtual Status start(RuntimeState* state); - virtual Status get_next(ChunkPtr* chunk, bool* eos); + Status start(RuntimeState* state) override; + Status get_next(ChunkPtr* chunk, bool* eos) override; private: Status get_new_table(); diff --git a/be/src/exec/vectorized/schema_scanner/schema_triggers_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_triggers_scanner.h index 8bd80bfee92..f5708f0943e 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_triggers_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_triggers_scanner.h @@ -10,7 +10,7 @@ namespace starrocks::vectorized { class SchemaTriggersScanner : public SchemaScanner { public: SchemaTriggersScanner(); - virtual ~SchemaTriggersScanner(); + ~SchemaTriggersScanner() override; private: static SchemaScanner::ColumnDesc _s_cols_triggers[]; diff --git a/be/src/exec/vectorized/schema_scanner/schema_user_privileges_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_user_privileges_scanner.h index 58ef9ac95d0..8f5a33f01f3 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_user_privileges_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_user_privileges_scanner.h @@ -12,7 +12,7 @@ namespace starrocks::vectorized { class SchemaUserPrivilegesScanner : public SchemaScanner { public: SchemaUserPrivilegesScanner(); - virtual ~SchemaUserPrivilegesScanner(); + ~SchemaUserPrivilegesScanner() override; Status start(RuntimeState* state) override; Status get_next(ChunkPtr* chunk, bool* eos) override; diff --git a/be/src/exec/vectorized/schema_scanner/schema_variables_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_variables_scanner.h index ce4890a7b9b..abc33bd89d9 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_variables_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_variables_scanner.h @@ -13,10 +13,10 @@ namespace starrocks::vectorized { class SchemaVariablesScanner : public SchemaScanner { public: SchemaVariablesScanner(TVarType::type type); - virtual ~SchemaVariablesScanner(); + ~SchemaVariablesScanner() override; - virtual Status start(RuntimeState* state); - virtual Status get_next(ChunkPtr* chunk, bool* eos); + Status start(RuntimeState* state) override; + Status get_next(ChunkPtr* chunk, bool* eos) override; private: Status fill_chunk(ChunkPtr* chunk); diff --git a/be/src/exec/vectorized/schema_scanner/schema_views_scanner.h b/be/src/exec/vectorized/schema_scanner/schema_views_scanner.h index 9e7524b1b8e..75340c24d52 100644 --- a/be/src/exec/vectorized/schema_scanner/schema_views_scanner.h +++ b/be/src/exec/vectorized/schema_scanner/schema_views_scanner.h @@ -10,7 +10,7 @@ namespace starrocks::vectorized { class SchemaViewsScanner : public SchemaScanner { public: SchemaViewsScanner(); - virtual ~SchemaViewsScanner(); + ~SchemaViewsScanner() override; Status start(RuntimeState* state) override; Status get_next(ChunkPtr* chunk, bool* eos) override; diff --git a/be/src/exec/vectorized/topn_node.h b/be/src/exec/vectorized/topn_node.h index 5f222361c42..60a142e56ca 100644 --- a/be/src/exec/vectorized/topn_node.h +++ b/be/src/exec/vectorized/topn_node.h @@ -16,7 +16,7 @@ class ChunksSorter; class TopNNode final : public ::starrocks::ExecNode { public: TopNNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - virtual ~TopNNode(); + ~TopNNode() override; // overridden methods defined in ::starrocks::ExecNode Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; diff --git a/be/src/exprs/agg/avg.h b/be/src/exprs/agg/avg.h index 2f556ad6cde..f03b272acb3 100644 --- a/be/src/exprs/agg/avg.h +++ b/be/src/exprs/agg/avg.h @@ -168,7 +168,7 @@ public: column->append(result); } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); ResultColumnType* column = down_cast(dst); diff --git a/be/src/exprs/agg/count.h b/be/src/exprs/agg/count.h index bfc1bae5498..579df67885c 100644 --- a/be/src/exprs/agg/count.h +++ b/be/src/exprs/agg/count.h @@ -41,7 +41,7 @@ public: data(state).count += input_column->get_data()[row_num]; } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); Int64Column* column = down_cast(dst); for (size_t i = start; i < end; ++i) { @@ -134,7 +134,7 @@ public: data(state).count += input_column->get_data()[row_num]; } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); Int64Column* column = down_cast(dst); for (size_t i = start; i < end; ++i) { diff --git a/be/src/exprs/agg/hll_ndv.h b/be/src/exprs/agg/hll_ndv.h index dc4fc1983fe..806ff84ab51 100644 --- a/be/src/exprs/agg/hll_ndv.h +++ b/be/src/exprs/agg/hll_ndv.h @@ -78,7 +78,7 @@ public: this->data(state).merge(hll); } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); Int64Column* column = down_cast(dst); int64_t result = this->data(state).estimate_cardinality(); diff --git a/be/src/exprs/agg/hll_union_count.h b/be/src/exprs/agg/hll_union_count.h index 70d16122261..2a8224ef32e 100644 --- a/be/src/exprs/agg/hll_union_count.h +++ b/be/src/exprs/agg/hll_union_count.h @@ -43,7 +43,7 @@ public: this->data(state).merge(*(hll_column->get_object(row_num))); } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); Int64Column* column = down_cast(dst); int64_t result = this->data(state).estimate_cardinality(); diff --git a/be/src/exprs/agg/maxmin.h b/be/src/exprs/agg/maxmin.h index 6ccb3c9601e..69da157a0db 100644 --- a/be/src/exprs/agg/maxmin.h +++ b/be/src/exprs/agg/maxmin.h @@ -215,7 +215,7 @@ public: down_cast(to)->append(this->data(state).result); } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); InputColumnType* column = down_cast(dst); for (size_t i = start; i < end; ++i) { @@ -270,7 +270,7 @@ public: column->append(this->data(state).slice()); } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); BinaryColumn* column = down_cast(dst); for (size_t i = start; i < end; ++i) { diff --git a/be/src/exprs/agg/nullable_aggregate.h b/be/src/exprs/agg/nullable_aggregate.h index 6d4f71df8af..e09a5bc3df1 100644 --- a/be/src/exprs/agg/nullable_aggregate.h +++ b/be/src/exprs/agg/nullable_aggregate.h @@ -105,7 +105,7 @@ public: } void batch_finalize(FunctionContext* ctx, size_t batch_size, const Buffer& agg_states, - size_t state_offset, Column* to) const { + size_t state_offset, Column* to) const override { for (size_t i = 0; i < batch_size; i++) { finalize_to_column(ctx, agg_states[i] + state_offset, to); } diff --git a/be/src/exprs/agg/sum.h b/be/src/exprs/agg/sum.h index dace143096c..c80897493af 100644 --- a/be/src/exprs/agg/sum.h +++ b/be/src/exprs/agg/sum.h @@ -74,7 +74,7 @@ public: this->data(state).sum += input_column->get_data()[row_num]; } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); ResultType result = this->data(state).sum; ResultColumnType* column = down_cast(dst); diff --git a/be/src/exprs/agg/variance.h b/be/src/exprs/agg/variance.h index 55a45e83968..408aea8e5bf 100644 --- a/be/src/exprs/agg/variance.h +++ b/be/src/exprs/agg/variance.h @@ -233,7 +233,7 @@ public: } } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); TResult result; @@ -359,7 +359,7 @@ public: } } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); TResult result; diff --git a/be/src/exprs/agg/window.h b/be/src/exprs/agg/window.h index b5bd087eedc..6a762ca88d1 100644 --- a/be/src/exprs/agg/window.h +++ b/be/src/exprs/agg/window.h @@ -56,7 +56,7 @@ class WindowFunction : public AggregateFunctionStateHelper { } void batch_finalize(FunctionContext* ctx, size_t batch_size, const Buffer& agg_states, - size_t state_offset, Column* to) const { + size_t state_offset, Column* to) const override { DCHECK(false) << "Shouldn't call this method for window function!"; } @@ -129,7 +129,7 @@ class RowNumberWindowFunction final : public WindowFunction { this->data(state).cur_positon++; } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); Int64Column* column = down_cast(dst); column->get_data()[start] = this->data(state).cur_positon; @@ -162,7 +162,7 @@ class RankWindowFunction final : public WindowFunction { this->data(state).count = peer_group_count; } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); Int64Column* column = down_cast(dst); for (size_t i = start; i < end; ++i) { @@ -193,7 +193,7 @@ class DenseRankWindowFunction final : public WindowFunction { } } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { DCHECK_GT(end, start); Int64Column* column = down_cast(dst); for (size_t i = start; i < end; ++i) { @@ -241,7 +241,7 @@ class FirstValueWindowFunction final : public ValueWindowFunctiondata(state).has_value = true; } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { this->get_values_helper(state, dst, start, end); } @@ -278,7 +278,7 @@ class LastValueWindowFunction final : public ValueWindowFunctiondata(state).value = column->get_data()[frame_end - 1]; } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { this->get_values_helper(state, dst, start, end); } @@ -336,7 +336,7 @@ class LeadLagWindowFunction final : public ValueWindowFunctiondata(state).value = column->get_data()[frame_end - 1]; } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { this->get_values_helper(state, dst, start, end); } @@ -381,7 +381,7 @@ class FirstValueWindowFunction> final : public Wind this->data(state).has_value = true; } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { this->get_slice_values(state, dst, start, end); } @@ -420,7 +420,7 @@ class LastValueWindowFunction> final : public Windo this->data(state).buffer.insert(this->data(state).buffer.end(), p, p + slice.size); } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { this->get_slice_values(state, dst, start, end); } @@ -481,7 +481,7 @@ class LeadLagWindowFunction> final : public WindowF this->data(state).value.insert(this->data(state).value.end(), p, p + slice.size); } - void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const { + void get_values(FunctionContext* ctx, ConstAggDataPtr state, Column* dst, size_t start, size_t end) const override { this->get_slice_values(state, dst, start, end); } diff --git a/be/src/exprs/slot_ref.h b/be/src/exprs/slot_ref.h index ff9032b5714..30765ee53e0 100644 --- a/be/src/exprs/slot_ref.h +++ b/be/src/exprs/slot_ref.h @@ -40,7 +40,7 @@ class SlotRef final : public Expr { public: SlotRef(const TExprNode& node); SlotRef(const SlotDescriptor* desc); - virtual Expr* clone(ObjectPool* pool) const override { return pool->add(new SlotRef(*this)); } + Expr* clone(ObjectPool* pool) const override { return pool->add(new SlotRef(*this)); } // TODO: this is a hack to allow aggregation nodes to work around NULL slot // descriptors. Ideally the FE would dictate the type of the intermediate SlotRefs. @@ -51,34 +51,34 @@ public: Status prepare(const SlotDescriptor* slot_desc, const RowDescriptor& row_desc); - virtual Status prepare(RuntimeState* state, const RowDescriptor& row_desc, ExprContext* ctx); + Status prepare(RuntimeState* state, const RowDescriptor& row_desc, ExprContext* ctx) override; static void* get_value(Expr* expr, TupleRow* row); void* get_slot(TupleRow* row); Tuple* get_tuple(TupleRow* row); bool is_null_bit_set(TupleRow* row); static bool vector_compute_fn(Expr* expr, VectorizedRowBatch* batch); static bool is_nullable(Expr* expr); - virtual std::string debug_string() const; - virtual bool is_constant() const { return false; } - virtual bool is_vectorized() const { return true; } - virtual bool is_bound(const std::vector& tuple_ids) const; - virtual int get_slot_ids(std::vector* slot_ids) const; + std::string debug_string() const override; + bool is_constant() const override { return false; } + bool is_vectorized() const override { return true; } + bool is_bound(const std::vector& tuple_ids) const override; + int get_slot_ids(std::vector* slot_ids) const override; SlotId slot_id() const { return _slot_id; } TupleId tuple_id() const { return _tuple_id; } inline NullIndicatorOffset null_indicator_offset() const { return _null_indicator_offset; } - virtual starrocks_udf::BooleanVal get_boolean_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::TinyIntVal get_tiny_int_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::SmallIntVal get_small_int_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::IntVal get_int_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::BigIntVal get_big_int_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::LargeIntVal get_large_int_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::FloatVal get_float_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::DoubleVal get_double_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::StringVal get_string_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::DateTimeVal get_datetime_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::DecimalVal get_decimal_val(ExprContext* context, TupleRow*); - virtual starrocks_udf::DecimalV2Val get_decimalv2_val(ExprContext* context, TupleRow*); + starrocks_udf::BooleanVal get_boolean_val(ExprContext* context, TupleRow*) override; + starrocks_udf::TinyIntVal get_tiny_int_val(ExprContext* context, TupleRow*) override; + starrocks_udf::SmallIntVal get_small_int_val(ExprContext* context, TupleRow*) override; + starrocks_udf::IntVal get_int_val(ExprContext* context, TupleRow*) override; + starrocks_udf::BigIntVal get_big_int_val(ExprContext* context, TupleRow*) override; + starrocks_udf::LargeIntVal get_large_int_val(ExprContext* context, TupleRow*) override; + starrocks_udf::FloatVal get_float_val(ExprContext* context, TupleRow*) override; + starrocks_udf::DoubleVal get_double_val(ExprContext* context, TupleRow*) override; + starrocks_udf::StringVal get_string_val(ExprContext* context, TupleRow*) override; + starrocks_udf::DateTimeVal get_datetime_val(ExprContext* context, TupleRow*) override; + starrocks_udf::DecimalVal get_decimal_val(ExprContext* context, TupleRow*) override; + starrocks_udf::DecimalV2Val get_decimalv2_val(ExprContext* context, TupleRow*) override; // virtual starrocks_udf::ArrayVal GetArrayVal(ExprContext* context, TupleRow*); // vector query engine diff --git a/be/src/exprs/vectorized/arithmetic_expr.cpp b/be/src/exprs/vectorized/arithmetic_expr.cpp index ce5c3973694..817cbaad5f4 100644 --- a/be/src/exprs/vectorized/arithmetic_expr.cpp +++ b/be/src/exprs/vectorized/arithmetic_expr.cpp @@ -35,7 +35,7 @@ public: return VectorizedStrictBinaryFunction::template evaluate(l, r); } } - std::string debug_string() const { + std::string debug_string() const override { std::stringstream out; auto expr_debug_string = Expr::debug_string(); out << "VectorizedArithmeticExpr (" diff --git a/be/src/exprs/vectorized/cast_expr.cpp b/be/src/exprs/vectorized/cast_expr.cpp index 2a3ad85f47f..84e5a5b8c84 100644 --- a/be/src/exprs/vectorized/cast_expr.cpp +++ b/be/src/exprs/vectorized/cast_expr.cpp @@ -655,7 +655,7 @@ public: return cast_fn(column); } }; - std::string debug_string() const { + std::string debug_string() const override { std::stringstream out; auto expr_debug_string = Expr::debug_string(); out << "VectorizedCastExpr (" diff --git a/be/src/exprs/vectorized/column_ref.h b/be/src/exprs/vectorized/column_ref.h index fa3a58a8e0c..bca6d6d0299 100644 --- a/be/src/exprs/vectorized/column_ref.h +++ b/be/src/exprs/vectorized/column_ref.h @@ -24,7 +24,7 @@ public: // FixMe(kks): currenly, join runtime filter need this method // we should find a way remove this method - bool is_bound(const std::vector& tuple_ids) const; + bool is_bound(const std::vector& tuple_ids) const override; Expr* clone(ObjectPool* pool) const override { return pool->add(new ColumnRef(*this)); } diff --git a/be/src/exprs/vectorized/in_const_predicate.hpp b/be/src/exprs/vectorized/in_const_predicate.hpp index 688ce6bb453..52faa2db286 100644 --- a/be/src/exprs/vectorized/in_const_predicate.hpp +++ b/be/src/exprs/vectorized/in_const_predicate.hpp @@ -44,9 +44,9 @@ public: VectorizedInConstPredicate(const VectorizedInConstPredicate& other) : Predicate(other), _is_not_in(other._is_not_in), _null_in_set(false) {} - ~VectorizedInConstPredicate() {} + ~VectorizedInConstPredicate() override {} - virtual Expr* clone(ObjectPool* pool) const override { return pool->add(new VectorizedInConstPredicate(*this)); } + Expr* clone(ObjectPool* pool) const override { return pool->add(new VectorizedInConstPredicate(*this)); } Status prepare([[maybe_unused]] RuntimeState* state) { if (_is_prepare) { diff --git a/be/src/exprs/vectorized/in_iterator_predicate.hpp b/be/src/exprs/vectorized/in_iterator_predicate.hpp index 5f08ba47b72..448dbb0eb4a 100644 --- a/be/src/exprs/vectorized/in_iterator_predicate.hpp +++ b/be/src/exprs/vectorized/in_iterator_predicate.hpp @@ -22,9 +22,9 @@ public: VectorizedInIteratorPredicate(const VectorizedInIteratorPredicate& other) : Predicate(other), _is_not_in(other._is_not_in) {} - ~VectorizedInIteratorPredicate() {} + ~VectorizedInIteratorPredicate() override {} - virtual Expr* clone(ObjectPool* pool) const override { + Expr* clone(ObjectPool* pool) const override { return pool->add(new VectorizedInIteratorPredicate(*this)); } diff --git a/be/src/exprs/vectorized/info_func.h b/be/src/exprs/vectorized/info_func.h index 14b23954c88..c11ccc6196e 100644 --- a/be/src/exprs/vectorized/info_func.h +++ b/be/src/exprs/vectorized/info_func.h @@ -12,9 +12,9 @@ class VectorizedInfoFunc final : public Expr { public: VectorizedInfoFunc(const TExprNode& node); - virtual ~VectorizedInfoFunc(); + ~VectorizedInfoFunc() override; - virtual Expr* clone(ObjectPool* pool) const override { return pool->add(new VectorizedInfoFunc(*this)); } + Expr* clone(ObjectPool* pool) const override { return pool->add(new VectorizedInfoFunc(*this)); } ColumnPtr evaluate(ExprContext* context, vectorized::Chunk* ptr) override; diff --git a/be/src/exprs/vectorized/literal.h b/be/src/exprs/vectorized/literal.h index f3d83620c2f..8c048b0bcad 100644 --- a/be/src/exprs/vectorized/literal.h +++ b/be/src/exprs/vectorized/literal.h @@ -11,9 +11,9 @@ class VectorizedLiteral final : public Expr { public: VectorizedLiteral(const TExprNode& node); - virtual ~VectorizedLiteral(); + ~VectorizedLiteral() override; - virtual Expr* clone(ObjectPool* pool) const override { return pool->add(new VectorizedLiteral(*this)); } + Expr* clone(ObjectPool* pool) const override { return pool->add(new VectorizedLiteral(*this)); } ColumnPtr evaluate(ExprContext* context, vectorized::Chunk* ptr) override; diff --git a/be/src/exprs/vectorized/runtime_filter.h b/be/src/exprs/vectorized/runtime_filter.h index 3c98cabb182..cd80311287c 100644 --- a/be/src/exprs/vectorized/runtime_filter.h +++ b/be/src/exprs/vectorized/runtime_filter.h @@ -231,9 +231,9 @@ public: using ColumnType = RunTimeColumnType; RuntimeBloomFilter() = default; - ~RuntimeBloomFilter() = default; + ~RuntimeBloomFilter() override = default; - virtual RuntimeBloomFilter* create_empty(ObjectPool* pool) override { return pool->add(new RuntimeBloomFilter()); }; + RuntimeBloomFilter* create_empty(ObjectPool* pool) override { return pool->add(new RuntimeBloomFilter()); }; void init_min_max() { _has_min_max = false; @@ -566,7 +566,7 @@ public: return offset; } - virtual bool check_equal(const JoinRuntimeFilter& base_rf) const override { + bool check_equal(const JoinRuntimeFilter& base_rf) const override { if (!JoinRuntimeFilter::check_equal(base_rf)) return false; const auto& rf = static_cast&>(base_rf); if constexpr (!IsSlice) { diff --git a/be/src/formats/orc/apache-orc/c++/include/orc/Exceptions.hh b/be/src/formats/orc/apache-orc/c++/include/orc/Exceptions.hh index fac66fe73e0..cb725801298 100644 --- a/be/src/formats/orc/apache-orc/c++/include/orc/Exceptions.hh +++ b/be/src/formats/orc/apache-orc/c++/include/orc/Exceptions.hh @@ -34,7 +34,7 @@ class NotImplementedYet : public std::logic_error { public: explicit NotImplementedYet(const std::string& what_arg); explicit NotImplementedYet(const char* what_arg); - virtual ~NotImplementedYet() ORC_NOEXCEPT; + ~NotImplementedYet() ORC_NOEXCEPT override; NotImplementedYet(const NotImplementedYet&); private: @@ -45,7 +45,7 @@ class ParseError : public std::runtime_error { public: explicit ParseError(const std::string& what_arg); explicit ParseError(const char* what_arg); - virtual ~ParseError() ORC_NOEXCEPT; + ~ParseError() ORC_NOEXCEPT override; ParseError(const ParseError&); private: @@ -56,7 +56,7 @@ class InvalidArgument : public std::runtime_error { public: explicit InvalidArgument(const std::string& what_arg); explicit InvalidArgument(const char* what_arg); - virtual ~InvalidArgument() ORC_NOEXCEPT; + ~InvalidArgument() ORC_NOEXCEPT override; InvalidArgument(const InvalidArgument&); private: diff --git a/be/src/formats/orc/apache-orc/c++/include/orc/Statistics.hh b/be/src/formats/orc/apache-orc/c++/include/orc/Statistics.hh index e215a0627bb..262970cb10a 100644 --- a/be/src/formats/orc/apache-orc/c++/include/orc/Statistics.hh +++ b/be/src/formats/orc/apache-orc/c++/include/orc/Statistics.hh @@ -60,7 +60,7 @@ public: */ class BinaryColumnStatistics : public ColumnStatistics { public: - virtual ~BinaryColumnStatistics(); + ~BinaryColumnStatistics() override; /** * Check whether column has total length. @@ -76,7 +76,7 @@ public: */ class BooleanColumnStatistics : public ColumnStatistics { public: - virtual ~BooleanColumnStatistics(); + ~BooleanColumnStatistics() override; /** * Check whether column has true/false count. @@ -93,7 +93,7 @@ public: */ class DateColumnStatistics : public ColumnStatistics { public: - virtual ~DateColumnStatistics(); + ~DateColumnStatistics() override; /** * Check whether column has minimum. @@ -125,7 +125,7 @@ public: */ class DecimalColumnStatistics : public ColumnStatistics { public: - virtual ~DecimalColumnStatistics(); + ~DecimalColumnStatistics() override; /** * Check whether column has minimum. @@ -169,7 +169,7 @@ public: */ class DoubleColumnStatistics : public ColumnStatistics { public: - virtual ~DoubleColumnStatistics(); + ~DoubleColumnStatistics() override; /** * Check whether column has minimum. @@ -216,7 +216,7 @@ public: */ class IntegerColumnStatistics : public ColumnStatistics { public: - virtual ~IntegerColumnStatistics(); + ~IntegerColumnStatistics() override; /** * Check whether column has minimum. @@ -262,7 +262,7 @@ public: */ class StringColumnStatistics : public ColumnStatistics { public: - virtual ~StringColumnStatistics(); + ~StringColumnStatistics() override; /** * Check whether column has minimum. @@ -306,7 +306,7 @@ public: */ class TimestampColumnStatistics : public ColumnStatistics { public: - virtual ~TimestampColumnStatistics(); + ~TimestampColumnStatistics() override; /** * Check whether minimum timestamp exists. @@ -389,7 +389,7 @@ public: class StripeStatistics : public Statistics { public: - virtual ~StripeStatistics(); + ~StripeStatistics() override; /** * Get the statistics of a given RowIndex entry in a given column. diff --git a/be/src/formats/orc/apache-orc/c++/include/orc/Vector.hh b/be/src/formats/orc/apache-orc/c++/include/orc/Vector.hh index 8e717db511c..fb6ef093543 100644 --- a/be/src/formats/orc/apache-orc/c++/include/orc/Vector.hh +++ b/be/src/formats/orc/apache-orc/c++/include/orc/Vector.hh @@ -100,35 +100,35 @@ private: struct LongVectorBatch : public ColumnVectorBatch { LongVectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~LongVectorBatch(); + ~LongVectorBatch() override; DataBuffer data; - std::string toString() const; - void resize(uint64_t capacity); - void clear(); - uint64_t getMemoryUsage(); - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + std::string toString() const override; + void resize(uint64_t capacity) override; + void clear() override; + uint64_t getMemoryUsage() override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; }; struct DoubleVectorBatch : public ColumnVectorBatch { DoubleVectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~DoubleVectorBatch(); - std::string toString() const; - void resize(uint64_t capacity); - void clear(); - uint64_t getMemoryUsage(); + ~DoubleVectorBatch() override; + std::string toString() const override; + void resize(uint64_t capacity) override; + void clear() override; + uint64_t getMemoryUsage() override; DataBuffer data; - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; }; struct StringVectorBatch : public ColumnVectorBatch { StringVectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~StringVectorBatch(); - std::string toString() const; - void resize(uint64_t capacity); - void clear(); - uint64_t getMemoryUsage(); + ~StringVectorBatch() override; + std::string toString() const override; + void resize(uint64_t capacity) override; + void clear() override; + uint64_t getMemoryUsage() override; // pointers to the start of each string DataBuffer data; @@ -139,7 +139,7 @@ struct StringVectorBatch : public ColumnVectorBatch { // dict codes, iff. there is dictionary. DataBuffer codes; bool use_codes; - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; }; struct StringDictionary { @@ -168,37 +168,37 @@ struct StringDictionary { */ struct EncodedStringVectorBatch : public StringVectorBatch { EncodedStringVectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~EncodedStringVectorBatch(); - std::string toString() const; - void resize(uint64_t capacity); + ~EncodedStringVectorBatch() override; + std::string toString() const override; + void resize(uint64_t capacity) override; std::shared_ptr dictionary; // index for dictionary entry DataBuffer index; - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; }; struct StructVectorBatch : public ColumnVectorBatch { StructVectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~StructVectorBatch(); - std::string toString() const; - void resize(uint64_t capacity); - void clear(); - uint64_t getMemoryUsage(); - bool hasVariableLength(); + ~StructVectorBatch() override; + std::string toString() const override; + void resize(uint64_t capacity) override; + void clear() override; + uint64_t getMemoryUsage() override; + bool hasVariableLength() override; std::vector fields; - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; }; struct ListVectorBatch : public ColumnVectorBatch { ListVectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~ListVectorBatch(); - std::string toString() const; - void resize(uint64_t capacity); - void clear(); - uint64_t getMemoryUsage(); - bool hasVariableLength(); + ~ListVectorBatch() override; + std::string toString() const override; + void resize(uint64_t capacity) override; + void clear() override; + uint64_t getMemoryUsage() override; + bool hasVariableLength() override; /** * The offset of the first element of each list. @@ -208,17 +208,17 @@ struct ListVectorBatch : public ColumnVectorBatch { // the concatenated elements ORC_UNIQUE_PTR elements; - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; }; struct MapVectorBatch : public ColumnVectorBatch { MapVectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~MapVectorBatch(); - std::string toString() const; - void resize(uint64_t capacity); - void clear(); - uint64_t getMemoryUsage(); - bool hasVariableLength(); + ~MapVectorBatch() override; + std::string toString() const override; + void resize(uint64_t capacity) override; + void clear() override; + uint64_t getMemoryUsage() override; + bool hasVariableLength() override; /** * The offset of the first element of each map. @@ -230,17 +230,17 @@ struct MapVectorBatch : public ColumnVectorBatch { ORC_UNIQUE_PTR keys; // the concatenated elements ORC_UNIQUE_PTR elements; - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; }; struct UnionVectorBatch : public ColumnVectorBatch { UnionVectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~UnionVectorBatch(); - std::string toString() const; - void resize(uint64_t capacity); - void clear(); - uint64_t getMemoryUsage(); - bool hasVariableLength(); + ~UnionVectorBatch() override; + std::string toString() const override; + void resize(uint64_t capacity) override; + void clear() override; + uint64_t getMemoryUsage() override; + bool hasVariableLength() override; /** * For each value, which element of children has the value. @@ -254,7 +254,7 @@ struct UnionVectorBatch : public ColumnVectorBatch { // the sub-columns std::vector children; - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; }; struct Decimal { @@ -269,11 +269,11 @@ struct Decimal { struct Decimal64VectorBatch : public ColumnVectorBatch { Decimal64VectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~Decimal64VectorBatch(); - std::string toString() const; - void resize(uint64_t capacity); - void clear(); - uint64_t getMemoryUsage(); + ~Decimal64VectorBatch() override; + std::string toString() const override; + void resize(uint64_t capacity) override; + void clear() override; + uint64_t getMemoryUsage() override; // total number of digits int32_t precision; @@ -282,7 +282,7 @@ struct Decimal64VectorBatch : public ColumnVectorBatch { // the numeric values DataBuffer values; - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; protected: /** @@ -296,11 +296,11 @@ protected: struct Decimal128VectorBatch : public ColumnVectorBatch { Decimal128VectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~Decimal128VectorBatch(); - std::string toString() const; - void resize(uint64_t capacity); - void clear(); - uint64_t getMemoryUsage(); + ~Decimal128VectorBatch() override; + std::string toString() const override; + void resize(uint64_t capacity) override; + void clear() override; + uint64_t getMemoryUsage() override; // total number of digits int32_t precision; @@ -309,7 +309,7 @@ struct Decimal128VectorBatch : public ColumnVectorBatch { // the numeric values DataBuffer values; - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; protected: /** @@ -329,11 +329,11 @@ protected: */ struct TimestampVectorBatch : public ColumnVectorBatch { TimestampVectorBatch(uint64_t capacity, MemoryPool& pool); - virtual ~TimestampVectorBatch(); - std::string toString() const; - void resize(uint64_t capacity); - void clear(); - uint64_t getMemoryUsage(); + ~TimestampVectorBatch() override; + std::string toString() const override; + void resize(uint64_t capacity) override; + void clear() override; + uint64_t getMemoryUsage() override; // the number of seconds past 1 Jan 1970 00:00 UTC (aka time_t) // Note that we always assume data is in GMT timezone; therefore it is @@ -344,7 +344,7 @@ struct TimestampVectorBatch : public ColumnVectorBatch { // the nanoseconds of each value DataBuffer nanoseconds; - virtual void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; + void filter(uint8_t* f_data, uint32_t f_size, uint32_t true_size) override; }; } // namespace orc diff --git a/be/src/formats/orc/apache-orc/c++/src/ByteRLE.cc b/be/src/formats/orc/apache-orc/c++/src/ByteRLE.cc index 205ef92ffd2..269d9fd0d2d 100644 --- a/be/src/formats/orc/apache-orc/c++/src/ByteRLE.cc +++ b/be/src/formats/orc/apache-orc/c++/src/ByteRLE.cc @@ -43,7 +43,7 @@ ByteRleEncoder::~ByteRleEncoder() { class ByteRleEncoderImpl : public ByteRleEncoder { public: ByteRleEncoderImpl(std::unique_ptr output); - virtual ~ByteRleEncoderImpl() override; + ~ByteRleEncoderImpl() override; /** * Encode the next batch of values. @@ -52,19 +52,19 @@ public: * @param notNull If the pointer is null, all values are read. If the * pointer is not null, positions that are false are skipped. */ - virtual void add(const char* data, uint64_t numValues, const char* notNull) override; + void add(const char* data, uint64_t numValues, const char* notNull) override; /** * Get size of buffer used so far. */ - virtual uint64_t getBufferSize() const override; + uint64_t getBufferSize() const override; /** * Flush underlying BufferedOutputStream. */ - virtual uint64_t flush() override; + uint64_t flush() override; - virtual void recordPosition(PositionRecorder* recorder) const override; + void recordPosition(PositionRecorder* recorder) const override; protected: std::unique_ptr outputStream; @@ -209,7 +209,7 @@ std::unique_ptr createByteRleEncoder(std::unique_ptr output); - virtual ~BooleanRleEncoderImpl() override; + ~BooleanRleEncoderImpl() override; /** * Encode the next batch of values @@ -218,14 +218,14 @@ public: * @param notNull If the pointer is null, all values are read. If the * pointer is not null, positions that are false are skipped. */ - virtual void add(const char* data, uint64_t numValues, const char* notNull) override; + void add(const char* data, uint64_t numValues, const char* notNull) override; /** * Flushing underlying BufferedOutputStream */ - virtual uint64_t flush() override; + uint64_t flush() override; - virtual void recordPosition(PositionRecorder* recorder) const override; + void recordPosition(PositionRecorder* recorder) const override; private: int bitsRemained; @@ -290,22 +290,22 @@ class ByteRleDecoderImpl : public ByteRleDecoder { public: ByteRleDecoderImpl(std::unique_ptr input); - virtual ~ByteRleDecoderImpl(); + ~ByteRleDecoderImpl() override; /** * Seek to a particular spot. */ - virtual void seek(PositionProvider&); + void seek(PositionProvider&) override; /** * Seek over a given number of values. */ - virtual void skip(uint64_t numValues); + void skip(uint64_t numValues) override; /** * Read a number of values into the batch. */ - virtual void next(char* data, uint64_t numValues, char* notNull); + void next(char* data, uint64_t numValues, char* notNull) override; protected: inline void nextBuffer(); @@ -465,22 +465,22 @@ class BooleanRleDecoderImpl : public ByteRleDecoderImpl { public: BooleanRleDecoderImpl(std::unique_ptr input); - virtual ~BooleanRleDecoderImpl(); + ~BooleanRleDecoderImpl() override; /** * Seek to a particular spot. */ - virtual void seek(PositionProvider&); + void seek(PositionProvider&) override; /** * Seek over a given number of values. */ - virtual void skip(uint64_t numValues); + void skip(uint64_t numValues) override; /** * Read a number of values into the batch. */ - virtual void next(char* data, uint64_t numValues, char* notNull); + void next(char* data, uint64_t numValues, char* notNull) override; protected: size_t remainingBits; diff --git a/be/src/formats/orc/apache-orc/c++/src/ColumnPrinter.cc b/be/src/formats/orc/apache-orc/c++/src/ColumnPrinter.cc index 6deba288e75..66dc2ed40c4 100644 --- a/be/src/formats/orc/apache-orc/c++/src/ColumnPrinter.cc +++ b/be/src/formats/orc/apache-orc/c++/src/ColumnPrinter.cc @@ -75,7 +75,7 @@ private: public: DoubleColumnPrinter(std::string&, const Type& type); - virtual ~DoubleColumnPrinter() override {} + ~DoubleColumnPrinter() override {} void printRow(uint64_t rowId) override; void reset(const ColumnVectorBatch& batch) override; }; @@ -134,7 +134,7 @@ private: public: StringColumnPrinter(std::string&); - virtual ~StringColumnPrinter() override {} + ~StringColumnPrinter() override {} void printRow(uint64_t rowId) override; void reset(const ColumnVectorBatch& batch) override; }; @@ -146,7 +146,7 @@ private: public: BinaryColumnPrinter(std::string&); - virtual ~BinaryColumnPrinter() override {} + ~BinaryColumnPrinter() override {} void printRow(uint64_t rowId) override; void reset(const ColumnVectorBatch& batch) override; }; @@ -158,7 +158,7 @@ private: public: ListColumnPrinter(std::string&, const Type& type); - virtual ~ListColumnPrinter() override {} + ~ListColumnPrinter() override {} void printRow(uint64_t rowId) override; void reset(const ColumnVectorBatch& batch) override; }; @@ -171,7 +171,7 @@ private: public: MapColumnPrinter(std::string&, const Type& type); - virtual ~MapColumnPrinter() override {} + ~MapColumnPrinter() override {} void printRow(uint64_t rowId) override; void reset(const ColumnVectorBatch& batch) override; }; diff --git a/be/src/formats/orc/apache-orc/c++/src/ColumnWriter.cc b/be/src/formats/orc/apache-orc/c++/src/ColumnWriter.cc index 1b826689a8f..89697436092 100644 --- a/be/src/formats/orc/apache-orc/c++/src/ColumnWriter.cc +++ b/be/src/formats/orc/apache-orc/c++/src/ColumnWriter.cc @@ -39,7 +39,7 @@ public: StreamsFactoryImpl(const WriterOptions& writerOptions, OutputStream* outputStream) : options(writerOptions), outStream(outputStream) {} - virtual std::unique_ptr createStream(proto::Stream_Kind kind) const override; + std::unique_ptr createStream(proto::Stream_Kind kind) const override; private: const WriterOptions& options; @@ -224,29 +224,28 @@ class StructColumnWriter : public ColumnWriter { public: StructColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + uint64_t getEstimatedSize() const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void getStripeStatistics(std::vector& stats) const override; + void getStripeStatistics(std::vector& stats) const override; - virtual void getFileStatistics(std::vector& stats) const override; + void getFileStatistics(std::vector& stats) const override; - virtual void mergeStripeStatsIntoFileStats() override; + void mergeStripeStatsIntoFileStats() override; - virtual void mergeRowGroupStatsIntoStripeStats() override; + void mergeRowGroupStatsIntoStripeStats() override; - virtual void createRowIndexEntry() override; + void createRowIndexEntry() override; - virtual void writeIndex(std::vector& streams) const override; + void writeIndex(std::vector& streams) const override; - virtual void writeDictionary() override; + void writeDictionary() override; - virtual void reset() override; + void reset() override; private: std::vector> children; @@ -384,16 +383,15 @@ class IntegerColumnWriter : public ColumnWriter { public: IntegerColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; + uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void recordPosition() const override; + void recordPosition() const override; protected: std::unique_ptr rleEncoder; @@ -482,16 +480,15 @@ class ByteColumnWriter : public ColumnWriter { public: ByteColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; + uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void recordPosition() const override; + void recordPosition() const override; private: std::unique_ptr byteRleEncoder; @@ -579,16 +576,15 @@ class BooleanColumnWriter : public ColumnWriter { public: BooleanColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; + uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void recordPosition() const override; + void recordPosition() const override; private: std::unique_ptr rleEncoder; @@ -677,16 +673,15 @@ class DoubleColumnWriter : public ColumnWriter { public: DoubleColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options, bool isFloat); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; + uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void recordPosition() const override; + void recordPosition() const override; private: bool isFloat; @@ -917,22 +912,21 @@ class StringColumnWriter : public ColumnWriter { public: StringColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; + uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void recordPosition() const override; + void recordPosition() const override; - virtual void createRowIndexEntry() override; + void createRowIndexEntry() override; - virtual void writeDictionary() override; + void writeDictionary() override; - virtual void reset() override; + void reset() override; private: /** @@ -1328,8 +1322,7 @@ public: padBuffer.resize(maxLength * 6); } - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; private: uint64_t maxLength; @@ -1402,8 +1395,7 @@ public: // PASS } - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; private: uint64_t maxLength; @@ -1465,8 +1457,7 @@ public: // PASS } - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; }; void BinaryColumnWriter::add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, @@ -1512,16 +1503,15 @@ public: TimestampColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options, bool isInstantType); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; + uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void recordPosition() const override; + void recordPosition() const override; protected: std::unique_ptr secRleEncoder, nanoRleEncoder; @@ -1662,8 +1652,7 @@ class DateColumnWriter : public IntegerColumnWriter { public: DateColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; }; DateColumnWriter::DateColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options) @@ -1712,16 +1701,15 @@ public: Decimal64ColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; + uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void recordPosition() const override; + void recordPosition() const override; protected: RleVersion rleVersion; @@ -1841,8 +1829,7 @@ class Decimal128ColumnWriter : public Decimal64ColumnWriter { public: Decimal128ColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; private: char buffer[20]; @@ -1923,32 +1910,31 @@ public: ListColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); ~ListColumnWriter() override; - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; + uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void getStripeStatistics(std::vector& stats) const override; + void getStripeStatistics(std::vector& stats) const override; - virtual void getFileStatistics(std::vector& stats) const override; + void getFileStatistics(std::vector& stats) const override; - virtual void mergeStripeStatsIntoFileStats() override; + void mergeStripeStatsIntoFileStats() override; - virtual void mergeRowGroupStatsIntoStripeStats() override; + void mergeRowGroupStatsIntoStripeStats() override; - virtual void createRowIndexEntry() override; + void createRowIndexEntry() override; - virtual void writeIndex(std::vector& streams) const override; + void writeIndex(std::vector& streams) const override; - virtual void recordPosition() const override; + void recordPosition() const override; - virtual void writeDictionary() override; + void writeDictionary() override; - virtual void reset() override; + void reset() override; private: std::unique_ptr lengthEncoder; @@ -2122,32 +2108,31 @@ public: MapColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); ~MapColumnWriter() override; - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; + uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void getStripeStatistics(std::vector& stats) const override; + void getStripeStatistics(std::vector& stats) const override; - virtual void getFileStatistics(std::vector& stats) const override; + void getFileStatistics(std::vector& stats) const override; - virtual void mergeStripeStatsIntoFileStats() override; + void mergeStripeStatsIntoFileStats() override; - virtual void mergeRowGroupStatsIntoStripeStats() override; + void mergeRowGroupStatsIntoStripeStats() override; - virtual void createRowIndexEntry() override; + void createRowIndexEntry() override; - virtual void writeIndex(std::vector& streams) const override; + void writeIndex(std::vector& streams) const override; - virtual void recordPosition() const override; + void recordPosition() const override; - virtual void writeDictionary() override; + void writeDictionary() override; - virtual void reset() override; + void reset() override; private: std::unique_ptr keyWriter; @@ -2362,32 +2347,31 @@ class UnionColumnWriter : public ColumnWriter { public: UnionColumnWriter(const Type& type, const StreamsFactory& factory, const WriterOptions& options); - virtual void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, - const char* incomingMask) override; + void add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues, const char* incomingMask) override; - virtual void flush(std::vector& streams) override; + void flush(std::vector& streams) override; - virtual uint64_t getEstimatedSize() const override; + uint64_t getEstimatedSize() const override; - virtual void getColumnEncoding(std::vector& encodings) const override; + void getColumnEncoding(std::vector& encodings) const override; - virtual void getStripeStatistics(std::vector& stats) const override; + void getStripeStatistics(std::vector& stats) const override; - virtual void getFileStatistics(std::vector& stats) const override; + void getFileStatistics(std::vector& stats) const override; - virtual void mergeStripeStatsIntoFileStats() override; + void mergeStripeStatsIntoFileStats() override; - virtual void mergeRowGroupStatsIntoStripeStats() override; + void mergeRowGroupStatsIntoStripeStats() override; - virtual void createRowIndexEntry() override; + void createRowIndexEntry() override; - virtual void writeIndex(std::vector& streams) const override; + void writeIndex(std::vector& streams) const override; - virtual void recordPosition() const override; + void recordPosition() const override; - virtual void writeDictionary() override; + void writeDictionary() override; - virtual void reset() override; + void reset() override; private: std::unique_ptr rleEncoder; diff --git a/be/src/formats/orc/apache-orc/c++/src/ColumnWriter.hh b/be/src/formats/orc/apache-orc/c++/src/ColumnWriter.hh index 1348738b738..c419ac9c699 100644 --- a/be/src/formats/orc/apache-orc/c++/src/ColumnWriter.hh +++ b/be/src/formats/orc/apache-orc/c++/src/ColumnWriter.hh @@ -52,11 +52,11 @@ std::unique_ptr createStreamsFactory(const WriterOptions& option */ class RowIndexPositionRecorder : public PositionRecorder { public: - virtual ~RowIndexPositionRecorder() override; + ~RowIndexPositionRecorder() override; RowIndexPositionRecorder(proto::RowIndexEntry& entry) : rowIndexEntry(entry) {} - virtual void add(uint64_t pos) override { rowIndexEntry.add_positions(pos); } + void add(uint64_t pos) override { rowIndexEntry.add_positions(pos); } private: proto::RowIndexEntry& rowIndexEntry; diff --git a/be/src/formats/orc/apache-orc/c++/src/Compression.cc b/be/src/formats/orc/apache-orc/c++/src/Compression.cc index eb2f39dda01..5d303fa0ab2 100644 --- a/be/src/formats/orc/apache-orc/c++/src/Compression.cc +++ b/be/src/formats/orc/apache-orc/c++/src/Compression.cc @@ -55,14 +55,14 @@ public: CompressionStreamBase(OutputStream* outStream, int compressionLevel, uint64_t capacity, uint64_t blockSize, MemoryPool& pool); - virtual bool Next(void** data, int* size) override = 0; - virtual void BackUp(int count) override; + bool Next(void** data, int* size) override = 0; + void BackUp(int count) override; - virtual std::string getName() const override = 0; - virtual uint64_t flush() override; + std::string getName() const override = 0; + uint64_t flush() override; - virtual bool isCompressed() const override { return true; } - virtual uint64_t getSize() const override; + bool isCompressed() const override { return true; } + uint64_t getSize() const override; protected: void writeHeader(char* buffer, size_t compressedSize, bool original) { @@ -148,8 +148,8 @@ public: CompressionStream(OutputStream* outStream, int compressionLevel, uint64_t capacity, uint64_t blockSize, MemoryPool& pool); - virtual bool Next(void** data, int* size) override; - virtual std::string getName() const override = 0; + bool Next(void** data, int* size) override; + std::string getName() const override = 0; protected: // return total compressed size @@ -194,12 +194,12 @@ public: ZlibCompressionStream(OutputStream* outStream, int compressionLevel, uint64_t capacity, uint64_t blockSize, MemoryPool& pool); - virtual ~ZlibCompressionStream() override { end(); } + ~ZlibCompressionStream() override { end(); } - virtual std::string getName() const override; + std::string getName() const override; protected: - virtual uint64_t doStreamingCompression() override; + uint64_t doStreamingCompression() override; private: void init(); @@ -278,13 +278,13 @@ enum DecompressState { DECOMPRESS_HEADER, DECOMPRESS_START, DECOMPRESS_CONTINUE, class DecompressionStream : public SeekableInputStream { public: DecompressionStream(std::unique_ptr inStream, size_t bufferSize, MemoryPool& pool); - virtual ~DecompressionStream() override {} - virtual bool Next(const void** data, int* size) override; - virtual void BackUp(int count) override; - virtual bool Skip(int count) override; - virtual int64_t ByteCount() const override; - virtual void seek(PositionProvider& position) override; - virtual std::string getName() const override = 0; + ~DecompressionStream() override {} + bool Next(const void** data, int* size) override; + void BackUp(int count) override; + bool Skip(int count) override; + int64_t ByteCount() const override; + void seek(PositionProvider& position) override; + std::string getName() const override = 0; protected: virtual void NextDecompress(const void** data, int* size, size_t availableSize) = 0; @@ -522,11 +522,11 @@ void DecompressionStream::seek(PositionProvider& position) { class ZlibDecompressionStream : public DecompressionStream { public: ZlibDecompressionStream(std::unique_ptr inStream, size_t blockSize, MemoryPool& pool); - virtual ~ZlibDecompressionStream() override; - virtual std::string getName() const override; + ~ZlibDecompressionStream() override; + std::string getName() const override; protected: - virtual void NextDecompress(const void** data, int* size, size_t availableSize) override; + void NextDecompress(const void** data, int* size, size_t availableSize) override; private: z_stream zstream; @@ -634,11 +634,11 @@ class BlockDecompressionStream : public DecompressionStream { public: BlockDecompressionStream(std::unique_ptr inStream, size_t blockSize, MemoryPool& pool); - virtual ~BlockDecompressionStream() override {} - virtual std::string getName() const override = 0; + ~BlockDecompressionStream() override {} + std::string getName() const override = 0; protected: - virtual void NextDecompress(const void** data, int* size, size_t availableSize) override; + void NextDecompress(const void** data, int* size, size_t availableSize) override; virtual uint64_t decompress(const char* input, uint64_t length, char* output, size_t maxOutputLength) = 0; @@ -697,7 +697,7 @@ public: } protected: - virtual uint64_t decompress(const char* input, uint64_t length, char* output, size_t maxOutputLength) override; + uint64_t decompress(const char* input, uint64_t length, char* output, size_t maxOutputLength) override; }; uint64_t SnappyDecompressionStream::decompress(const char* _input, uint64_t length, char* output, @@ -731,7 +731,7 @@ public: } protected: - virtual uint64_t decompress(const char* input, uint64_t length, char* output, size_t maxOutputLength) override; + uint64_t decompress(const char* input, uint64_t length, char* output, size_t maxOutputLength) override; }; uint64_t LzoDecompressionStream::decompress(const char* inputPtr, uint64_t length, char* output, @@ -753,7 +753,7 @@ public: } protected: - virtual uint64_t decompress(const char* input, uint64_t length, char* output, size_t maxOutputLength) override; + uint64_t decompress(const char* input, uint64_t length, char* output, size_t maxOutputLength) override; }; uint64_t Lz4DecompressionStream::decompress(const char* inputPtr, uint64_t length, char* output, @@ -776,8 +776,8 @@ public: // PASS } - virtual bool Next(void** data, int* size) override; - virtual std::string getName() const override = 0; + bool Next(void** data, int* size) override; + std::string getName() const override = 0; protected: // compresses a block and returns the compressed size @@ -854,16 +854,14 @@ public: this->init(); } - virtual std::string getName() const override { return "Lz4CompressionStream"; } + std::string getName() const override { return "Lz4CompressionStream"; } - virtual ~Lz4CompressionSteam() override { this->end(); } + ~Lz4CompressionSteam() override { this->end(); } protected: - virtual uint64_t doBlockCompression() override; + uint64_t doBlockCompression() override; - virtual uint64_t estimateMaxCompressionSize() override { - return static_cast(LZ4_compressBound(bufferSize)); - } + uint64_t estimateMaxCompressionSize() override { return static_cast(LZ4_compressBound(bufferSize)); } private: void init(); @@ -905,16 +903,14 @@ public: this->init(); } - virtual std::string getName() const override { return "ZstdCompressionStream"; } + std::string getName() const override { return "ZstdCompressionStream"; } - virtual ~ZSTDCompressionStream() override { this->end(); } + ~ZSTDCompressionStream() override { this->end(); } protected: - virtual uint64_t doBlockCompression() override; + uint64_t doBlockCompression() override; - virtual uint64_t estimateMaxCompressionSize() override { - return ZSTD_compressBound(static_cast(bufferSize)); - } + uint64_t estimateMaxCompressionSize() override { return ZSTD_compressBound(static_cast(bufferSize)); } private: void init(); @@ -957,7 +953,7 @@ public: this->init(); } - virtual ~ZSTDDecompressionStream() override { this->end(); } + ~ZSTDDecompressionStream() override { this->end(); } std::string getName() const override { std::ostringstream result; @@ -966,7 +962,7 @@ public: } protected: - virtual uint64_t decompress(const char* input, uint64_t length, char* output, size_t maxOutputLength) override; + uint64_t decompress(const char* input, uint64_t length, char* output, size_t maxOutputLength) override; private: void init(); diff --git a/be/src/formats/orc/apache-orc/c++/src/LzoDecompressor.cc b/be/src/formats/orc/apache-orc/c++/src/LzoDecompressor.cc index 4654c653ffe..15361e0bbf6 100644 --- a/be/src/formats/orc/apache-orc/c++/src/LzoDecompressor.cc +++ b/be/src/formats/orc/apache-orc/c++/src/LzoDecompressor.cc @@ -52,7 +52,7 @@ public: MalformedInputException(const MalformedInputException& other) : ParseError(other.what()) {} - virtual ~MalformedInputException() noexcept; + ~MalformedInputException() noexcept override; }; MalformedInputException::~MalformedInputException() noexcept { diff --git a/be/src/formats/orc/apache-orc/c++/src/MemoryPool.cc b/be/src/formats/orc/apache-orc/c++/src/MemoryPool.cc index 41942f12915..23f002e63a8 100644 --- a/be/src/formats/orc/apache-orc/c++/src/MemoryPool.cc +++ b/be/src/formats/orc/apache-orc/c++/src/MemoryPool.cc @@ -39,7 +39,7 @@ MemoryPool::~MemoryPool() { class MemoryPoolImpl : public MemoryPool { public: - virtual ~MemoryPoolImpl() override; + ~MemoryPoolImpl() override; char* malloc(uint64_t size) override; void free(char* p) override; diff --git a/be/src/formats/orc/apache-orc/c++/src/Statistics.hh b/be/src/formats/orc/apache-orc/c++/src/Statistics.hh index db64f5f5bbd..52bb4e1580d 100644 --- a/be/src/formats/orc/apache-orc/c++/src/Statistics.hh +++ b/be/src/formats/orc/apache-orc/c++/src/Statistics.hh @@ -208,7 +208,7 @@ private: public: ColumnStatisticsImpl() { reset(); } ColumnStatisticsImpl(const proto::ColumnStatistics& stats); - virtual ~ColumnStatisticsImpl() override; + ~ColumnStatisticsImpl() override; uint64_t getNumberOfValues() const override { return _stats.getNumberOfValues(); } @@ -246,7 +246,7 @@ private: public: BinaryColumnStatisticsImpl() { reset(); } BinaryColumnStatisticsImpl(const proto::ColumnStatistics& stats, const StatContext& statContext); - virtual ~BinaryColumnStatisticsImpl() override; + ~BinaryColumnStatisticsImpl() override; uint64_t getNumberOfValues() const override { return _stats.getNumberOfValues(); } @@ -316,7 +316,7 @@ private: public: BooleanColumnStatisticsImpl() { reset(); } BooleanColumnStatisticsImpl(const proto::ColumnStatistics& stats, const StatContext& statContext); - virtual ~BooleanColumnStatisticsImpl() override; + ~BooleanColumnStatisticsImpl() override; bool hasCount() const override { return _hasCount; } @@ -406,7 +406,7 @@ private: public: DateColumnStatisticsImpl() { reset(); } DateColumnStatisticsImpl(const proto::ColumnStatistics& stats, const StatContext& statContext); - virtual ~DateColumnStatisticsImpl() override; + ~DateColumnStatisticsImpl() override; bool hasMinimum() const override { return _stats.hasMinimum(); } @@ -498,7 +498,7 @@ private: public: DecimalColumnStatisticsImpl() { reset(); } DecimalColumnStatisticsImpl(const proto::ColumnStatistics& stats, const StatContext& statContext); - virtual ~DecimalColumnStatisticsImpl() override; + ~DecimalColumnStatisticsImpl() override; bool hasMinimum() const override { return _stats.hasMinimum(); } @@ -660,7 +660,7 @@ private: public: DoubleColumnStatisticsImpl() { reset(); } DoubleColumnStatisticsImpl(const proto::ColumnStatistics& stats); - virtual ~DoubleColumnStatisticsImpl() override; + ~DoubleColumnStatisticsImpl() override; bool hasMinimum() const override { return _stats.hasMinimum(); } @@ -789,7 +789,7 @@ private: public: IntegerColumnStatisticsImpl() { reset(); } IntegerColumnStatisticsImpl(const proto::ColumnStatistics& stats); - virtual ~IntegerColumnStatisticsImpl() override; + ~IntegerColumnStatisticsImpl() override; bool hasMinimum() const override { return _stats.hasMinimum(); } @@ -921,7 +921,7 @@ private: public: StringColumnStatisticsImpl() { reset(); } StringColumnStatisticsImpl(const proto::ColumnStatistics& stats, const StatContext& statContext); - virtual ~StringColumnStatisticsImpl() override; + ~StringColumnStatisticsImpl() override; bool hasMinimum() const override { return _stats.hasMinimum(); } @@ -1076,7 +1076,7 @@ private: public: TimestampColumnStatisticsImpl() { reset(); } TimestampColumnStatisticsImpl(const proto::ColumnStatistics& stats, const StatContext& statContext); - virtual ~TimestampColumnStatisticsImpl() override; + ~TimestampColumnStatisticsImpl() override; bool hasMinimum() const override { return _stats.hasMinimum(); } @@ -1303,9 +1303,9 @@ public: StatisticsImpl(const proto::Footer& footer, const StatContext& statContext); - virtual const ColumnStatistics* getColumnStatistics(uint32_t columnId) const override { return colStats[columnId]; } + const ColumnStatistics* getColumnStatistics(uint32_t columnId) const override { return colStats[columnId]; } - virtual ~StatisticsImpl() override; + ~StatisticsImpl() override; uint32_t getNumberOfColumns() const override { return static_cast(colStats.size()); } }; @@ -1324,18 +1324,18 @@ public: std::vector >& indexStats, const StatContext& statContext); - virtual const ColumnStatistics* getColumnStatistics(uint32_t columnId) const override { + const ColumnStatistics* getColumnStatistics(uint32_t columnId) const override { return columnStats->getColumnStatistics(columnId); } uint32_t getNumberOfColumns() const override { return columnStats->getNumberOfColumns(); } - virtual const ColumnStatistics* getRowIndexStatistics(uint32_t columnId, uint32_t rowIndex) const override { + const ColumnStatistics* getRowIndexStatistics(uint32_t columnId, uint32_t rowIndex) const override { // check id indices are valid return rowIndexStats[columnId][rowIndex].get(); } - virtual ~StripeStatisticsImpl() override; + ~StripeStatisticsImpl() override; uint32_t getNumberOfRowIndexStats(uint32_t columnId) const override { return static_cast(rowIndexStats[columnId].size()); diff --git a/be/src/formats/orc/apache-orc/c++/src/StripeStream.hh b/be/src/formats/orc/apache-orc/c++/src/StripeStream.hh index 44a1811450e..cb2d3eef1c1 100644 --- a/be/src/formats/orc/apache-orc/c++/src/StripeStream.hh +++ b/be/src/formats/orc/apache-orc/c++/src/StripeStream.hh @@ -53,14 +53,14 @@ public: const proto::StripeFooter& footer, uint64_t stripeStart, InputStream& input, const Timezone& writerTimezone, const Timezone& readerTimezone); - virtual ~StripeStreamsImpl() override; + ~StripeStreamsImpl() override; - virtual const std::vector getSelectedColumns() const override; + const std::vector getSelectedColumns() const override; - virtual proto::ColumnEncoding getEncoding(uint64_t columnId) const override; + proto::ColumnEncoding getEncoding(uint64_t columnId) const override; - virtual std::unique_ptr getStream(uint64_t columnId, proto::Stream_Kind kind, - bool shouldStream) const override; + std::unique_ptr getStream(uint64_t columnId, proto::Stream_Kind kind, + bool shouldStream) const override; MemoryPool& getMemoryPool() const override; @@ -143,7 +143,7 @@ public: // PASS } - virtual ~StripeInformationImpl() override { + ~StripeInformationImpl() override { // PASS } diff --git a/be/src/formats/orc/apache-orc/c++/src/Timezone.cc b/be/src/formats/orc/apache-orc/c++/src/Timezone.cc index c9d672ff677..ccba2814c9b 100644 --- a/be/src/formats/orc/apache-orc/c++/src/Timezone.cc +++ b/be/src/formats/orc/apache-orc/c++/src/Timezone.cc @@ -231,7 +231,7 @@ class FutureRuleImpl : public FutureRule { } public: - virtual ~FutureRuleImpl() override; + ~FutureRuleImpl() override; bool isDefined() const override; const TimezoneVariant& getVariant(int64_t clk) const override; void print(std::ostream& out) const override; @@ -501,24 +501,24 @@ static uint32_t decode32(const unsigned char* ptr) { class Version1Parser : public VersionParser { public: - virtual ~Version1Parser() override; + ~Version1Parser() override; - virtual uint64_t getVersion() const override { return 1; } + uint64_t getVersion() const override { return 1; } /** * Get the number of bytes */ - virtual uint64_t getTimeSize() const override { return 4; } + uint64_t getTimeSize() const override { return 4; } /** * Parse the time at the given location. */ - virtual int64_t parseTime(const unsigned char* ptr) const override { + int64_t parseTime(const unsigned char* ptr) const override { // sign extend from 32 bits return static_cast(decode32(ptr)); } - virtual std::string parseFutureString(const unsigned char*, uint64_t, uint64_t) const override { return ""; } + std::string parseFutureString(const unsigned char*, uint64_t, uint64_t) const override { return ""; } }; Version1Parser::~Version1Parser() { @@ -527,23 +527,23 @@ Version1Parser::~Version1Parser() { class Version2Parser : public VersionParser { public: - virtual ~Version2Parser() override; + ~Version2Parser() override; - virtual uint64_t getVersion() const override { return 2; } + uint64_t getVersion() const override { return 2; } /** * Get the number of bytes */ - virtual uint64_t getTimeSize() const override { return 8; } + uint64_t getTimeSize() const override { return 8; } /** * Parse the time at the given location. */ - virtual int64_t parseTime(const unsigned char* ptr) const override { + int64_t parseTime(const unsigned char* ptr) const override { return static_cast(decode32(ptr)) << 32 | decode32(ptr + 4); } - virtual std::string parseFutureString(const unsigned char* ptr, uint64_t offset, uint64_t length) const override { + std::string parseFutureString(const unsigned char* ptr, uint64_t offset, uint64_t length) const override { return std::string(reinterpret_cast(ptr) + offset + 1, length - 2); } }; @@ -555,7 +555,7 @@ Version2Parser::~Version2Parser() { class TimezoneImpl : public Timezone { public: TimezoneImpl(const std::string& name, const std::vector bytes); - virtual ~TimezoneImpl() override; + ~TimezoneImpl() override; /** * Get the variant for the given time (time_t). diff --git a/be/src/formats/orc/apache-orc/c++/src/Timezone.hh b/be/src/formats/orc/apache-orc/c++/src/Timezone.hh index fbf5165b3a5..623e497fb9e 100644 --- a/be/src/formats/orc/apache-orc/c++/src/Timezone.hh +++ b/be/src/formats/orc/apache-orc/c++/src/Timezone.hh @@ -113,7 +113,7 @@ class TimezoneError : public std::runtime_error { public: TimezoneError(const std::string& what); TimezoneError(const TimezoneError&); - virtual ~TimezoneError() ORC_NOEXCEPT; + ~TimezoneError() ORC_NOEXCEPT override; }; /** diff --git a/be/src/formats/orc/apache-orc/c++/src/io/InputStream.hh b/be/src/formats/orc/apache-orc/c++/src/io/InputStream.hh index a4ba01c7ac3..84ebb5b40b6 100644 --- a/be/src/formats/orc/apache-orc/c++/src/io/InputStream.hh +++ b/be/src/formats/orc/apache-orc/c++/src/io/InputStream.hh @@ -54,7 +54,7 @@ public: */ class SeekableInputStream : public google::protobuf::io::ZeroCopyInputStream { public: - virtual ~SeekableInputStream(); + ~SeekableInputStream() override; virtual void seek(PositionProvider& position) = 0; virtual std::string getName() const = 0; }; @@ -72,13 +72,13 @@ private: public: SeekableArrayInputStream(const unsigned char* list, uint64_t length, uint64_t block_size = 0); SeekableArrayInputStream(const char* list, uint64_t length, uint64_t block_size = 0); - virtual ~SeekableArrayInputStream() override; - virtual bool Next(const void** data, int* size) override; - virtual void BackUp(int count) override; - virtual bool Skip(int count) override; - virtual google::protobuf::int64 ByteCount() const override; - virtual void seek(PositionProvider& position) override; - virtual std::string getName() const override; + ~SeekableArrayInputStream() override; + bool Next(const void** data, int* size) override; + void BackUp(int count) override; + bool Skip(int count) override; + google::protobuf::int64 ByteCount() const override; + void seek(PositionProvider& position) override; + std::string getName() const override; }; /** @@ -98,14 +98,14 @@ private: public: SeekableFileInputStream(InputStream* input, uint64_t offset, uint64_t byteCount, MemoryPool& pool, uint64_t blockSize = 0); - virtual ~SeekableFileInputStream() override; + ~SeekableFileInputStream() override; - virtual bool Next(const void** data, int* size) override; - virtual void BackUp(int count) override; - virtual bool Skip(int count) override; - virtual int64_t ByteCount() const override; - virtual void seek(PositionProvider& position) override; - virtual std::string getName() const override; + bool Next(const void** data, int* size) override; + void BackUp(int count) override; + bool Skip(int count) override; + int64_t ByteCount() const override; + void seek(PositionProvider& position) override; + std::string getName() const override; }; } // namespace orc diff --git a/be/src/formats/orc/apache-orc/c++/src/io/OutputStream.hh b/be/src/formats/orc/apache-orc/c++/src/io/OutputStream.hh index fe4e4bf1315..8419f5f25c2 100644 --- a/be/src/formats/orc/apache-orc/c++/src/io/OutputStream.hh +++ b/be/src/formats/orc/apache-orc/c++/src/io/OutputStream.hh @@ -52,13 +52,13 @@ private: public: BufferedOutputStream(MemoryPool& pool, OutputStream* outStream, uint64_t capacity, uint64_t block_size); - virtual ~BufferedOutputStream() override; + ~BufferedOutputStream() override; - virtual bool Next(void** data, int* size) override; - virtual void BackUp(int count) override; - virtual google::protobuf::int64 ByteCount() const override; - virtual bool WriteAliasedRaw(const void* data, int size) override; - virtual bool AllowsAliasing() const override; + bool Next(void** data, int* size) override; + void BackUp(int count) override; + google::protobuf::int64 ByteCount() const override; + bool WriteAliasedRaw(const void* data, int size) override; + bool AllowsAliasing() const override; virtual std::string getName() const; virtual uint64_t getSize() const; diff --git a/be/src/geo/geo_types.h b/be/src/geo/geo_types.h index 3a21f0d3785..25fe3df44e2 100644 --- a/be/src/geo/geo_types.h +++ b/be/src/geo/geo_types.h @@ -137,7 +137,7 @@ private: class GeoCircle : public GeoShape { public: GeoCircle(); - ~GeoCircle(); + ~GeoCircle() override; GeoParseStatus init(double lng, double lat, double radius); diff --git a/be/src/gutil/stl_util.h b/be/src/gutil/stl_util.h index a275ce41e6b..c2b206829a3 100644 --- a/be/src/gutil/stl_util.h +++ b/be/src/gutil/stl_util.h @@ -437,7 +437,7 @@ class TemplatedElementDeleter : public BaseDeleter { public: explicit TemplatedElementDeleter(STLContainer* ptr) : container_ptr_(ptr) {} - virtual ~TemplatedElementDeleter() { STLDeleteElements(container_ptr_); } + ~TemplatedElementDeleter() override { STLDeleteElements(container_ptr_); } private: STLContainer* container_ptr_; @@ -469,7 +469,7 @@ class TemplatedValueDeleter : public BaseDeleter { public: explicit TemplatedValueDeleter(STLContainer* ptr) : container_ptr_(ptr) {} - virtual ~TemplatedValueDeleter() { STLDeleteValues(container_ptr_); } + ~TemplatedValueDeleter() override { STLDeleteValues(container_ptr_); } private: STLContainer* container_ptr_; diff --git a/be/src/gutil/threading/thread_collision_warner.h b/be/src/gutil/threading/thread_collision_warner.h index a7c7c1975a6..a33d4503a34 100644 --- a/be/src/gutil/threading/thread_collision_warner.h +++ b/be/src/gutil/threading/thread_collision_warner.h @@ -135,7 +135,7 @@ struct BASE_EXPORT AsserterBase { }; struct BASE_EXPORT DCheckAsserter : public AsserterBase { - virtual ~DCheckAsserter() {} + ~DCheckAsserter() override {} void warn(int64_t previous_thread_id, int64_t current_thread_id) override; }; diff --git a/be/src/http/action/checksum_action.h b/be/src/http/action/checksum_action.h index b2a86c46157..f7ae2c5e6cd 100644 --- a/be/src/http/action/checksum_action.h +++ b/be/src/http/action/checksum_action.h @@ -34,7 +34,7 @@ class ChecksumAction : public HttpHandler { public: explicit ChecksumAction(ExecEnv* exec_env); - virtual ~ChecksumAction() {} + ~ChecksumAction() override {} void handle(HttpRequest* req) override; diff --git a/be/src/http/action/compaction_action.h b/be/src/http/action/compaction_action.h index 7afdd6027d2..45e54379ab8 100644 --- a/be/src/http/action/compaction_action.h +++ b/be/src/http/action/compaction_action.h @@ -34,7 +34,7 @@ class CompactionAction : public HttpHandler { public: CompactionAction(CompactionActionType type) : _type(type) {} - virtual ~CompactionAction() {} + ~CompactionAction() override {} void handle(HttpRequest* req) override; diff --git a/be/src/http/action/health_action.h b/be/src/http/action/health_action.h index 36504b26c61..d5407a9e850 100644 --- a/be/src/http/action/health_action.h +++ b/be/src/http/action/health_action.h @@ -33,7 +33,7 @@ class HealthAction : public HttpHandler { public: HealthAction(ExecEnv* exec_env); - virtual ~HealthAction(){}; + ~HealthAction() override{}; void handle(HttpRequest* req) override; diff --git a/be/src/http/action/metrics_action.cpp b/be/src/http/action/metrics_action.cpp index 8988faec911..7ab5f38de47 100644 --- a/be/src/http/action/metrics_action.cpp +++ b/be/src/http/action/metrics_action.cpp @@ -39,7 +39,7 @@ namespace starrocks { class PrometheusMetricsVisitor : public MetricsVisitor { public: - virtual ~PrometheusMetricsVisitor() {} + ~PrometheusMetricsVisitor() override {} void visit(const std::string& prefix, const std::string& name, MetricCollector* collector) override; std::string to_string() const { return _ss.str(); } @@ -55,7 +55,7 @@ private: // starrocks_be_process_thread_num LONG 240 class SimpleCoreMetricsVisitor : public MetricsVisitor { public: - virtual ~SimpleCoreMetricsVisitor() {} + ~SimpleCoreMetricsVisitor() override {} void visit(const std::string& prefix, const std::string& name, MetricCollector* collector) override; std::string to_string() const { return _ss.str(); } @@ -146,7 +146,7 @@ void SimpleCoreMetricsVisitor::visit(const std::string& prefix, const std::strin class JsonMetricsVisitor : public MetricsVisitor { public: JsonMetricsVisitor() {} - virtual ~JsonMetricsVisitor() {} + ~JsonMetricsVisitor() override {} void visit(const std::string& prefix, const std::string& name, MetricCollector* collector) override; std::string to_string() { rapidjson::StringBuffer strBuf; diff --git a/be/src/http/action/metrics_action.h b/be/src/http/action/metrics_action.h index 155ae139e9c..cd4b0e1ba58 100644 --- a/be/src/http/action/metrics_action.h +++ b/be/src/http/action/metrics_action.h @@ -33,7 +33,7 @@ class MetricRegistry; class MetricsAction : public HttpHandler { public: MetricsAction(MetricRegistry* metrics) : _metrics(metrics) {} - virtual ~MetricsAction() {} + ~MetricsAction() override {} void handle(HttpRequest* req) override; diff --git a/be/src/http/action/pprof_actions.cpp b/be/src/http/action/pprof_actions.cpp index 7ba6eae206f..3c1e4cd7c2e 100644 --- a/be/src/http/action/pprof_actions.cpp +++ b/be/src/http/action/pprof_actions.cpp @@ -53,9 +53,9 @@ static std::mutex kPprofActionMutex; class HeapAction : public HttpHandler { public: HeapAction() {} - virtual ~HeapAction() {} + ~HeapAction() override {} - virtual void handle(HttpRequest* req) override; + void handle(HttpRequest* req) override; }; void HeapAction::handle(HttpRequest* req) { @@ -93,9 +93,9 @@ void HeapAction::handle(HttpRequest* req) { class GrowthAction : public HttpHandler { public: GrowthAction() {} - virtual ~GrowthAction() {} + ~GrowthAction() override {} - virtual void handle(HttpRequest* req) override; + void handle(HttpRequest* req) override; }; void GrowthAction::handle(HttpRequest* req) { @@ -115,9 +115,9 @@ void GrowthAction::handle(HttpRequest* req) { class ProfileAction : public HttpHandler { public: ProfileAction() {} - virtual ~ProfileAction() {} + ~ProfileAction() override {} - virtual void handle(HttpRequest* req) override; + void handle(HttpRequest* req) override; }; void ProfileAction::handle(HttpRequest* req) { @@ -158,23 +158,23 @@ void ProfileAction::handle(HttpRequest* req) { class PmuProfileAction : public HttpHandler { public: PmuProfileAction() {} - virtual ~PmuProfileAction() {} - virtual void handle(HttpRequest* req) override {} + ~PmuProfileAction() override {} + void handle(HttpRequest* req) override {} }; class ContentionAction : public HttpHandler { public: ContentionAction() {} - virtual ~ContentionAction() {} + ~ContentionAction() override {} - virtual void handle(HttpRequest* req) override {} + void handle(HttpRequest* req) override {} }; class CmdlineAction : public HttpHandler { public: CmdlineAction() {} - virtual ~CmdlineAction() {} - virtual void handle(HttpRequest* req) override; + ~CmdlineAction() override {} + void handle(HttpRequest* req) override; }; void CmdlineAction::handle(HttpRequest* req) { @@ -196,9 +196,9 @@ void CmdlineAction::handle(HttpRequest* req) { class SymbolAction : public HttpHandler { public: SymbolAction(BfdParser* parser) : _parser(parser) {} - virtual ~SymbolAction() {} + ~SymbolAction() override {} - virtual void handle(HttpRequest* req) override; + void handle(HttpRequest* req) override; private: BfdParser* _parser; diff --git a/be/src/http/action/reload_tablet_action.h b/be/src/http/action/reload_tablet_action.h index 0fc684e42d5..acf527cec42 100644 --- a/be/src/http/action/reload_tablet_action.h +++ b/be/src/http/action/reload_tablet_action.h @@ -33,7 +33,7 @@ class ReloadTabletAction : public HttpHandler { public: ReloadTabletAction(ExecEnv* exec_env); - virtual ~ReloadTabletAction() {} + ~ReloadTabletAction() override {} void handle(HttpRequest* req) override; diff --git a/be/src/http/action/restore_tablet_action.h b/be/src/http/action/restore_tablet_action.h index 1afcd11cc66..49f8065a588 100644 --- a/be/src/http/action/restore_tablet_action.h +++ b/be/src/http/action/restore_tablet_action.h @@ -37,7 +37,7 @@ class RestoreTabletAction : public HttpHandler { public: RestoreTabletAction(ExecEnv* exec_env); - virtual ~RestoreTabletAction() {} + ~RestoreTabletAction() override {} void handle(HttpRequest* req) override; diff --git a/be/src/http/action/snapshot_action.h b/be/src/http/action/snapshot_action.h index 370330e3988..cc4c5005f78 100644 --- a/be/src/http/action/snapshot_action.h +++ b/be/src/http/action/snapshot_action.h @@ -37,7 +37,7 @@ class SnapshotAction : public HttpHandler { public: explicit SnapshotAction(ExecEnv* exec_env); - virtual ~SnapshotAction() {} + ~SnapshotAction() override {} void handle(HttpRequest* req) override; diff --git a/be/src/http/action/update_config_action.h b/be/src/http/action/update_config_action.h index 391fa8d7ed1..47128049729 100644 --- a/be/src/http/action/update_config_action.h +++ b/be/src/http/action/update_config_action.h @@ -29,7 +29,7 @@ namespace starrocks { class UpdateConfigAction : public HttpHandler { public: UpdateConfigAction() {} - virtual ~UpdateConfigAction() {} + ~UpdateConfigAction() override {} void handle(HttpRequest* req) override; }; diff --git a/be/src/http/download_action.h b/be/src/http/download_action.h index 1adf73cef81..861d9190753 100644 --- a/be/src/http/download_action.h +++ b/be/src/http/download_action.h @@ -42,7 +42,7 @@ public: // for load error DownloadAction(ExecEnv* exec_env, const std::string& error_log_root_dir); - virtual ~DownloadAction() {} + ~DownloadAction() override {} void handle(HttpRequest* req) override; diff --git a/be/src/http/monitor_action.h b/be/src/http/monitor_action.h index 8a44072b3be..f08339d5d82 100644 --- a/be/src/http/monitor_action.h +++ b/be/src/http/monitor_action.h @@ -37,7 +37,7 @@ class MonitorAction : public HttpHandler { public: MonitorAction(); - virtual ~MonitorAction() {} + ~MonitorAction() override {} void register_module(const std::string& name, RestMonitorIface* module); diff --git a/be/src/http/web_page_handler.h b/be/src/http/web_page_handler.h index 236986d2bcc..95795908453 100644 --- a/be/src/http/web_page_handler.h +++ b/be/src/http/web_page_handler.h @@ -45,7 +45,7 @@ public: typedef std::function TemplatePageHandlerCallback; WebPageHandler(EvHttpServer* http_server); - virtual ~WebPageHandler(); + ~WebPageHandler() override; void handle(HttpRequest* req) override; diff --git a/be/src/plugin/plugin_loader.h b/be/src/plugin/plugin_loader.h index 6d11d65da88..37f133e7d2f 100644 --- a/be/src/plugin/plugin_loader.h +++ b/be/src/plugin/plugin_loader.h @@ -73,14 +73,14 @@ public: _install_path(install_path), _plugin_handler(nullptr){}; - virtual ~DynamicPluginLoader() { + ~DynamicPluginLoader() override { // just close plugin, but don't clean install path (maybe other plugin has used) WARN_IF_ERROR(close_plugin(), "close plugin failed."); }; - virtual Status install(); + Status install() override; - virtual Status uninstall(); + Status uninstall() override; private: Status open_plugin(); @@ -101,11 +101,11 @@ class BuiltinPluginLoader : public PluginLoader { public: BuiltinPluginLoader(const std::string& name, int type, const Plugin* plugin); - virtual ~BuiltinPluginLoader() { WARN_IF_ERROR(uninstall(), "close plugin failed."); } + ~BuiltinPluginLoader() override { WARN_IF_ERROR(uninstall(), "close plugin failed."); } - virtual Status install(); + Status install() override; - virtual Status uninstall(); + Status uninstall() override; }; } // namespace starrocks diff --git a/be/src/runtime/buffered_block_mgr2.h b/be/src/runtime/buffered_block_mgr2.h index adb8ca5b7b5..ca238184e66 100644 --- a/be/src/runtime/buffered_block_mgr2.h +++ b/be/src/runtime/buffered_block_mgr2.h @@ -143,7 +143,7 @@ public: class Block : public InternalQueue::Node { public: // A null dtor to pass codestyle check - ~Block() {} + ~Block() override {} // Pins a block in memory--assigns a free buffer to a block and reads it from disk if // necessary. If there are no free blocks and no unpinned blocks, '*pinned' is set to diff --git a/be/src/runtime/data_stream_sender.h b/be/src/runtime/data_stream_sender.h index d67b55be43a..e82788884c5 100644 --- a/be/src/runtime/data_stream_sender.h +++ b/be/src/runtime/data_stream_sender.h @@ -71,7 +71,7 @@ public: DataStreamSender(ObjectPool* pool, bool is_vectorized, int sender_id, const RowDescriptor& row_desc, const TDataStreamSink& sink, const std::vector& destinations, int per_channel_buffer_size, bool send_query_statistics_with_every_batch); - virtual ~DataStreamSender(); + ~DataStreamSender() override; Status init(const TDataSink& thrift_sink) override; diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h index 4ec9a838652..8fa36a08b71 100644 --- a/be/src/runtime/descriptors.h +++ b/be/src/runtime/descriptors.h @@ -214,14 +214,14 @@ private: class OlapTableDescriptor : public TableDescriptor { public: OlapTableDescriptor(const TTableDescriptor& tdesc); - virtual std::string debug_string() const; + std::string debug_string() const override; }; class SchemaTableDescriptor : public TableDescriptor { public: SchemaTableDescriptor(const TTableDescriptor& tdesc); - virtual ~SchemaTableDescriptor(); - virtual std::string debug_string() const; + ~SchemaTableDescriptor() override; + std::string debug_string() const override; TSchemaTableType::type schema_table_type() const { return _schema_table_type; } private: @@ -231,8 +231,8 @@ private: class BrokerTableDescriptor : public TableDescriptor { public: BrokerTableDescriptor(const TTableDescriptor& tdesc); - virtual ~BrokerTableDescriptor(); - virtual std::string debug_string() const; + ~BrokerTableDescriptor() override; + std::string debug_string() const override; private: }; @@ -240,8 +240,8 @@ private: class EsTableDescriptor : public TableDescriptor { public: EsTableDescriptor(const TTableDescriptor& tdesc); - virtual ~EsTableDescriptor(); - virtual std::string debug_string() const; + ~EsTableDescriptor() override; + std::string debug_string() const override; private: }; @@ -249,7 +249,7 @@ private: class MySQLTableDescriptor : public TableDescriptor { public: MySQLTableDescriptor(const TTableDescriptor& tdesc); - virtual std::string debug_string() const; + std::string debug_string() const override; const std::string mysql_db() const { return _mysql_db; } const std::string mysql_table() const { return _mysql_table; } const std::string host() const { return _host; } diff --git a/be/src/runtime/disk_io_mgr.h b/be/src/runtime/disk_io_mgr.h index 0ea44436906..258d2d68f46 100644 --- a/be/src/runtime/disk_io_mgr.h +++ b/be/src/runtime/disk_io_mgr.h @@ -345,7 +345,7 @@ public: ScanRange() : ScanRange(-1) {} ScanRange(int initial_capacity); - virtual ~ScanRange(); + ~ScanRange() override; // Resets this scan range object with the scan range description. The scan range // must fall within the file bounds (offset >= 0 and offset + len <= file_length). @@ -504,7 +504,7 @@ public: class WriteRange : public RequestRange { public: // a null dtor to pass codestyle check - ~WriteRange() {} + ~WriteRange() override {} // This callback is invoked on each WriteRange after the write is complete or the // context is cancelled. The status returned by the callback parameter indicates diff --git a/be/src/runtime/file_result_writer.h b/be/src/runtime/file_result_writer.h index f6f0c1e1f9d..2c2e88ea712 100644 --- a/be/src/runtime/file_result_writer.h +++ b/be/src/runtime/file_result_writer.h @@ -67,12 +67,12 @@ class FileResultWriter final : public ResultWriter { public: FileResultWriter(const ResultFileOptions* file_option, const std::vector& output_expr_ctxs, RuntimeProfile* parent_profile); - virtual ~FileResultWriter(); + ~FileResultWriter() override; - virtual Status init(RuntimeState* state) override; - virtual Status append_row_batch(const RowBatch* batch) override; + Status init(RuntimeState* state) override; + Status append_row_batch(const RowBatch* batch) override; Status append_chunk(vectorized::Chunk* chunk) override; - virtual Status close() override; + Status close() override; private: Status _write_csv_file(const RowBatch& batch); diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h index b18ae8af01b..315dc9e6859 100644 --- a/be/src/runtime/fragment_mgr.h +++ b/be/src/runtime/fragment_mgr.h @@ -56,7 +56,7 @@ public: typedef std::function FinishCallback; FragmentMgr(ExecEnv* exec_env); - virtual ~FragmentMgr(); + ~FragmentMgr() override; // execute one plan fragment Status exec_plan_fragment(const TExecPlanFragmentParams& params); @@ -75,7 +75,7 @@ public: void cancel_worker(); - virtual void debug(std::stringstream& ss); + void debug(std::stringstream& ss) override; Status trigger_profile_report(const PTriggerProfileReportRequest* request); diff --git a/be/src/runtime/memory_scratch_sink.h b/be/src/runtime/memory_scratch_sink.h index 30f1e7b82ae..614e0e0b5de 100644 --- a/be/src/runtime/memory_scratch_sink.h +++ b/be/src/runtime/memory_scratch_sink.h @@ -59,21 +59,21 @@ public: MemoryScratchSink(const RowDescriptor& row_desc, const std::vector& select_exprs, const TMemoryScratchSink& sink); - virtual ~MemoryScratchSink(); + ~MemoryScratchSink() override; - virtual Status prepare(RuntimeState* state); + Status prepare(RuntimeState* state) override; - virtual Status open(RuntimeState* state); + Status open(RuntimeState* state) override; // send data in 'batch' to this backend queue mgr // Blocks until all rows in batch are pushed to the queue - virtual Status send(RuntimeState* state, RowBatch* batch); + Status send(RuntimeState* state, RowBatch* batch) override; - virtual Status send_chunk(RuntimeState* state, vectorized::Chunk* chunk) override; + Status send_chunk(RuntimeState* state, vectorized::Chunk* chunk) override; - virtual Status close(RuntimeState* state, Status exec_status); + Status close(RuntimeState* state, Status exec_status) override; - virtual RuntimeProfile* profile() { return _profile; } + RuntimeProfile* profile() override { return _profile; } private: Status prepare_exprs(RuntimeState* state); diff --git a/be/src/runtime/message_body_sink.h b/be/src/runtime/message_body_sink.h index 715ee9b33aa..ab6f64b83a0 100644 --- a/be/src/runtime/message_body_sink.h +++ b/be/src/runtime/message_body_sink.h @@ -41,7 +41,7 @@ public: class MessageBodyFileSink : public MessageBodySink { public: MessageBodyFileSink(const std::string& path) : _path(path) {} - virtual ~MessageBodyFileSink(); + ~MessageBodyFileSink() override; Status open(); diff --git a/be/src/runtime/mysql_table_sink.h b/be/src/runtime/mysql_table_sink.h index 49849963481..364430df9d5 100644 --- a/be/src/runtime/mysql_table_sink.h +++ b/be/src/runtime/mysql_table_sink.h @@ -43,23 +43,23 @@ class MysqlTableSink : public DataSink { public: MysqlTableSink(ObjectPool* pool, const RowDescriptor& row_desc, const std::vector& t_exprs); - virtual ~MysqlTableSink(); + ~MysqlTableSink() override; - virtual Status init(const TDataSink& thrift_sink); + Status init(const TDataSink& thrift_sink) override; - virtual Status prepare(RuntimeState* state); + Status prepare(RuntimeState* state) override; - virtual Status open(RuntimeState* state); + Status open(RuntimeState* state) override; // send data in 'batch' to this backend stream mgr // Blocks until all rows in batch are placed in the buffer - virtual Status send(RuntimeState* state, RowBatch* batch); + Status send(RuntimeState* state, RowBatch* batch) override; // Flush all buffered data and close all existing channels to destination // hosts. Further send() calls are illegal after calling close(). - virtual Status close(RuntimeState* state, Status exec_status); + Status close(RuntimeState* state, Status exec_status) override; - virtual RuntimeProfile* profile() { return _profile; } + RuntimeProfile* profile() override { return _profile; } private: // owned by RuntimeState diff --git a/be/src/runtime/routine_load/data_consumer.h b/be/src/runtime/routine_load/data_consumer.h index 0632f584392..604995f1a74 100644 --- a/be/src/runtime/routine_load/data_consumer.h +++ b/be/src/runtime/routine_load/data_consumer.h @@ -83,7 +83,7 @@ protected: class KafkaEventCb : public RdKafka::EventCb { public: - void event_cb(RdKafka::Event& event) { + void event_cb(RdKafka::Event& event) override { switch (event.type()) { case RdKafka::Event::EVENT_ERROR: LOG(INFO) << "kafka error: " << RdKafka::err2str(event.err()) << ", event: " << event.str(); @@ -114,7 +114,7 @@ public: KafkaDataConsumer(StreamLoadContext* ctx) : DataConsumer(ctx), _brokers(ctx->kafka_info->brokers), _topic(ctx->kafka_info->topic) {} - virtual ~KafkaDataConsumer() { + ~KafkaDataConsumer() override { VLOG(3) << "deconstruct consumer"; if (_k_consumer) { _k_consumer->close(); @@ -123,13 +123,13 @@ public: } } - virtual Status init(StreamLoadContext* ctx) override; + Status init(StreamLoadContext* ctx) override; // TODO(cmy): currently do not implement single consumer start method, using group_consume - virtual Status consume(StreamLoadContext* ctx) override { return Status::OK(); } - virtual Status cancel(StreamLoadContext* ctx) override; + Status consume(StreamLoadContext* ctx) override { return Status::OK(); } + Status cancel(StreamLoadContext* ctx) override; // reassign partition topics - virtual Status reset() override; - virtual bool match(StreamLoadContext* ctx) override; + Status reset() override; + bool match(StreamLoadContext* ctx) override; // commit kafka offset Status commit(std::vector& offset); diff --git a/be/src/runtime/routine_load/data_consumer_group.h b/be/src/runtime/routine_load/data_consumer_group.h index 0a5d61c5e95..e4af0676952 100644 --- a/be/src/runtime/routine_load/data_consumer_group.h +++ b/be/src/runtime/routine_load/data_consumer_group.h @@ -69,9 +69,9 @@ class KafkaDataConsumerGroup : public DataConsumerGroup { public: KafkaDataConsumerGroup() : DataConsumerGroup(), _queue(500) {} - virtual ~KafkaDataConsumerGroup(); + ~KafkaDataConsumerGroup() override; - virtual Status start_all(StreamLoadContext* ctx) override; + Status start_all(StreamLoadContext* ctx) override; // assign topic partitions to all consumers equally Status assign_topic_partitions(StreamLoadContext* ctx); diff --git a/be/src/runtime/routine_load/kafka_consumer_pipe.h b/be/src/runtime/routine_load/kafka_consumer_pipe.h index 934b7cecd98..4d6a46c13da 100644 --- a/be/src/runtime/routine_load/kafka_consumer_pipe.h +++ b/be/src/runtime/routine_load/kafka_consumer_pipe.h @@ -39,7 +39,7 @@ public: KafkaConsumerPipe(size_t max_buffered_bytes = 1024 * 1024, size_t min_chunk_size = 64 * 1024) : StreamLoadPipe(max_buffered_bytes, min_chunk_size) {} - virtual ~KafkaConsumerPipe() {} + ~KafkaConsumerPipe() override {} Status append_with_row_delimiter(const char* data, size_t size, char row_delimiter) { Status st = append(data, size); diff --git a/be/src/runtime/row_batch.h b/be/src/runtime/row_batch.h index 57b4fc3ab74..0a3a5c3f257 100644 --- a/be/src/runtime/row_batch.h +++ b/be/src/runtime/row_batch.h @@ -100,7 +100,7 @@ public: // - tuple_ptrs // - tuple mem pool data // - buffer handles from the io mgr - virtual ~RowBatch(); + ~RowBatch() override; // used to c void clear(); diff --git a/be/src/runtime/stream_load/stream_load_pipe.h b/be/src/runtime/stream_load/stream_load_pipe.h index f0147535bf6..b024de0ddc8 100644 --- a/be/src/runtime/stream_load/stream_load_pipe.h +++ b/be/src/runtime/stream_load/stream_load_pipe.h @@ -44,7 +44,7 @@ public: _total_length(total_length), _finished(false), _cancelled(false) {} - virtual ~StreamLoadPipe() {} + ~StreamLoadPipe() override {} Status open() override { return Status::OK(); } diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h index fb13856b65a..39f182b534f 100644 --- a/be/src/service/internal_service.h +++ b/be/src/service/internal_service.h @@ -38,7 +38,7 @@ template class PInternalServiceImpl : public T { public: PInternalServiceImpl(ExecEnv* exec_env); - virtual ~PInternalServiceImpl(); + ~PInternalServiceImpl() override; void transmit_data(::google::protobuf::RpcController* controller, const ::starrocks::PTransmitDataParams* request, ::starrocks::PTransmitDataResult* response, ::google::protobuf::Closure* done) override; diff --git a/be/src/storage/decimal_type_info.h b/be/src/storage/decimal_type_info.h index 9440a5f2de9..3d5797878d6 100644 --- a/be/src/storage/decimal_type_info.h +++ b/be/src/storage/decimal_type_info.h @@ -162,7 +162,7 @@ public: std::string to_zone_map_string(const void* src) { return _delegate->to_string(src); } protected: - int _datum_cmp_impl(const Datum& left, const Datum& right) const { + int _datum_cmp_impl(const Datum& left, const Datum& right) const override { const CppType& lhs = left.get(); const CppType& rhs = right.get(); return (lhs < rhs) ? -1 : (lhs > rhs) ? 1 : 0; diff --git a/be/src/storage/fs/file_block_manager.cpp b/be/src/storage/fs/file_block_manager.cpp index c9124371ccb..f5729574b40 100644 --- a/be/src/storage/fs/file_block_manager.cpp +++ b/be/src/storage/fs/file_block_manager.cpp @@ -69,26 +69,26 @@ class FileWritableBlock : public WritableBlock { public: FileWritableBlock(FileBlockManager* block_manager, string path, shared_ptr writer); - virtual ~FileWritableBlock(); + ~FileWritableBlock() override; - virtual Status close() override; + Status close() override; - virtual Status abort() override; + Status abort() override; - virtual BlockManager* block_manager() const override; + BlockManager* block_manager() const override; - virtual const BlockId& id() const override; - virtual const std::string& path() const override; + const BlockId& id() const override; + const std::string& path() const override; - virtual Status append(const Slice& data) override; + Status append(const Slice& data) override; - virtual Status appendv(const Slice* data, size_t data_cnt) override; + Status appendv(const Slice* data, size_t data_cnt) override; - virtual Status finalize() override; + Status finalize() override; - virtual size_t bytes_appended() const override; + size_t bytes_appended() const override; - virtual State state() const override; + State state() const override; void handle_error(const Status& s) const; @@ -255,20 +255,20 @@ public: FileReadableBlock(FileBlockManager* block_manager, string path, std::shared_ptr> file_handle); - virtual ~FileReadableBlock(); + ~FileReadableBlock() override; - virtual Status close() override; + Status close() override; - virtual BlockManager* block_manager() const override; + BlockManager* block_manager() const override; - virtual const BlockId& id() const override; - virtual const std::string& path() const override; + const BlockId& id() const override; + const std::string& path() const override; - virtual Status size(uint64_t* sz) const override; + Status size(uint64_t* sz) const override; - virtual Status read(uint64_t offset, Slice result) const override; + Status read(uint64_t offset, Slice result) const override; - virtual Status readv(uint64_t offset, const Slice* results, size_t res_cnt) const override; + Status readv(uint64_t offset, const Slice* results, size_t res_cnt) const override; void handle_error(const Status& s) const; diff --git a/be/src/storage/lru_cache.h b/be/src/storage/lru_cache.h index c49b01ef6a8..7d3c10dc559 100644 --- a/be/src/storage/lru_cache.h +++ b/be/src/storage/lru_cache.h @@ -325,19 +325,18 @@ static const int kNumShards = 1 << kNumShardBits; class ShardedLRUCache : public Cache { public: explicit ShardedLRUCache(size_t capacity); - virtual ~ShardedLRUCache() {} - virtual Handle* insert(const CacheKey& key, void* value, size_t charge, - void (*deleter)(const CacheKey& key, void* value), - CachePriority priority = CachePriority::NORMAL); - virtual Handle* lookup(const CacheKey& key); - virtual void release(Handle* handle); - virtual void erase(const CacheKey& key); - virtual void* value(Handle* handle); + ~ShardedLRUCache() override {} + Handle* insert(const CacheKey& key, void* value, size_t charge, void (*deleter)(const CacheKey& key, void* value), + CachePriority priority = CachePriority::NORMAL) override; + Handle* lookup(const CacheKey& key) override; + void release(Handle* handle) override; + void erase(const CacheKey& key) override; + void* value(Handle* handle) override; Slice value_slice(Handle* handle) override; - virtual uint64_t new_id(); - virtual void prune(); - virtual size_t get_memory_usage(); - virtual void get_cache_status(rapidjson::Document* document); + uint64_t new_id() override; + void prune() override; + size_t get_memory_usage() override; + void get_cache_status(rapidjson::Document* document) override; private: static inline uint32_t _hash_slice(const CacheKey& s); diff --git a/be/src/storage/memory/mem_tablet.h b/be/src/storage/memory/mem_tablet.h index 28f5d6cdbc0..3e1d3bb1c3a 100644 --- a/be/src/storage/memory/mem_tablet.h +++ b/be/src/storage/memory/mem_tablet.h @@ -51,7 +51,7 @@ public: MemTablet(MemTracker* mem_tracker, TabletMetaSharedPtr tablet_meta, DataDir* data_dir); - virtual ~MemTablet(); + ~MemTablet() override; // Initialize Status init(); diff --git a/be/src/storage/memory/typed_column_reader.h b/be/src/storage/memory/typed_column_reader.h index e9ff3936453..e63b4c8b7b6 100644 --- a/be/src/storage/memory/typed_column_reader.h +++ b/be/src/storage/memory/typed_column_reader.h @@ -143,7 +143,7 @@ public: _real_version(real_version), _deltas(std::move(deltas)) {} - const void* get(const uint32_t rid) const { + const void* get(const uint32_t rid) const override { return TypedColumnGet, T, Nullable, ST>(*this, rid); } @@ -153,7 +153,7 @@ public: // if the block is not the last block, nrows should be less or equal to 64K if the block // is the last block, and since Column object doesn't store information about number of // rows(it is stored in MemSubTablet), so it needs to be provided by caller. - Status get_block(size_t nrows, size_t bid, ColumnBlockHolder* cbh) const { + Status get_block(size_t nrows, size_t bid, ColumnBlockHolder* cbh) const override { // Check if this block has any updates in delta bool base_only = true; for (size_t i = 0; i < _deltas.size(); ++i) { @@ -216,13 +216,15 @@ public: return Status::OK(); } - uint64_t hashcode(const void* rhs, size_t rhs_idx) const { return TypedColumnHashcode(rhs, rhs_idx); } + uint64_t hashcode(const void* rhs, size_t rhs_idx) const override { + return TypedColumnHashcode(rhs, rhs_idx); + } - bool equals(const uint32_t rid, const void* rhs, size_t rhs_idx) const { + bool equals(const uint32_t rid, const void* rhs, size_t rhs_idx) const override { return TypedColumnEquals, T, Nullable, ST>(*this, rid, rhs, rhs_idx); } - string debug_string() const { + string debug_string() const override { return StringPrintf("%s version=%zu(real=%zu) ndelta=%zu", _column->debug_string().c_str(), _version, _real_version, _deltas.size()); } diff --git a/be/src/storage/memory/typed_column_writer.h b/be/src/storage/memory/typed_column_writer.h index acbb9a98504..8925f4bc709 100644 --- a/be/src/storage/memory/typed_column_writer.h +++ b/be/src/storage/memory/typed_column_writer.h @@ -71,23 +71,25 @@ public: _real_version(real_version), _deltas(std::move(deltas)) {} - const void* get(const uint32_t rid) const { + const void* get(const uint32_t rid) const override { return TypedColumnGet, T, Nullable, ST>(*this, rid); } - uint64_t hashcode(const void* rhs, size_t rhs_idx) const { return TypedColumnHashcode(rhs, rhs_idx); } + uint64_t hashcode(const void* rhs, size_t rhs_idx) const override { + return TypedColumnHashcode(rhs, rhs_idx); + } - bool equals(const uint32_t rid, const void* rhs, size_t rhs_idx) const { + bool equals(const uint32_t rid, const void* rhs, size_t rhs_idx) const override { return TypedColumnEquals, T, Nullable, ST>(*this, rid, rhs, rhs_idx); } - string debug_string() const { + string debug_string() const override { return StringPrintf("%s version=%zu(real=%zu) ndelta=%zu insert:%zu update:%zu", _column->debug_string().c_str(), _version, _real_version, _deltas.size(), _num_insert, _num_update); } - Status insert(uint32_t rid, const void* value) { + Status insert(uint32_t rid, const void* value) override { uint32_t bid = Column::block_id(rid); if (bid >= _base->size()) { RETURN_IF_ERROR(add_block()); @@ -122,7 +124,7 @@ public: return Status::OK(); } - Status update(uint32_t rid, const void* value) { + Status update(uint32_t rid, const void* value) override { DCHECK_LT(rid, _base->size() * Column::BLOCK_SIZE); if (Nullable) { auto& uv = _updates[rid]; @@ -155,7 +157,7 @@ public: return Status::OK(); } - Status finalize(uint64_t version) { + Status finalize(uint64_t version) override { if (_updates.size() == 0) { // insert(append) only return Status::OK(); @@ -200,7 +202,7 @@ public: return Status::OK(); } - Status get_new_column(scoped_refptr* ret) { + Status get_new_column(scoped_refptr* ret) override { if (ret->get() != _column.get()) { DLOG(INFO) << StringPrintf("%s switch new column", _column->debug_string().c_str()); (*ret).swap(_column); diff --git a/be/src/storage/null_predicate.h b/be/src/storage/null_predicate.h index c931893fe2f..b9d78ca11c6 100644 --- a/be/src/storage/null_predicate.h +++ b/be/src/storage/null_predicate.h @@ -35,14 +35,14 @@ class VectorizedRowBatch; class NullPredicate : public ColumnPredicate { public: NullPredicate(uint32_t column_id, bool is_null); - virtual ~NullPredicate(); + ~NullPredicate() override; - virtual void evaluate(VectorizedRowBatch* batch) const override; + void evaluate(VectorizedRowBatch* batch) const override; void evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const override; - virtual Status evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, - Roaring* roaring) const override; + Status evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, + Roaring* roaring) const override; Status convert_to(const ColumnPredicate** outpout, FieldType from_type, FieldType to_type, ObjectPool* obj_pool) const override { diff --git a/be/src/storage/primary_index.cpp b/be/src/storage/primary_index.cpp index 7f0ece7fc1a..39cee5ab2aa 100644 --- a/be/src/storage/primary_index.cpp +++ b/be/src/storage/primary_index.cpp @@ -251,7 +251,7 @@ public: return _map.capacity() * (1 + (sizeof(Key) + 3) / 4 * 4 + sizeof(RowIdPack4)); } - std::string memory_info() const { + std::string memory_info() const override { auto caps = _map.capacities(); string caps_str; for (auto e : caps) { @@ -527,7 +527,7 @@ public: std::size_t memory_usage() const final { return _map.capacity() * (1 + S * 4 + sizeof(RowIdPack4)); } - std::string memory_info() const { + std::string memory_info() const override { auto caps = _map.capacities(); string caps_str; for (auto e : caps) { @@ -745,7 +745,7 @@ public: return ret; } - std::string memory_info() const { + std::string memory_info() const override { auto caps = _map.capacities(); string caps_str; for (auto e : caps) { diff --git a/be/src/storage/rowset/beta_rowset_writer.h b/be/src/storage/rowset/beta_rowset_writer.h index b03dfb58041..19e60f30682 100644 --- a/be/src/storage/rowset/beta_rowset_writer.h +++ b/be/src/storage/rowset/beta_rowset_writer.h @@ -52,12 +52,11 @@ public: OLAPStatus add_chunk(const vectorized::Chunk& chunk) override; - OLAPStatus add_chunk_with_rssid(const vectorized::Chunk& chunk, const vector& rssid); + OLAPStatus add_chunk_with_rssid(const vectorized::Chunk& chunk, const vector& rssid) override; OLAPStatus flush_chunk(const vectorized::Chunk& chunk) override; - virtual OLAPStatus flush_chunk_with_deletes(const vectorized::Chunk& upserts, - const vectorized::Column& deletes) override; + OLAPStatus flush_chunk_with_deletes(const vectorized::Chunk& upserts, const vectorized::Column& deletes) override; // add rowset by create hard link OLAPStatus add_rowset(RowsetSharedPtr rowset) override; diff --git a/be/src/storage/rowset/segment_v2/bitmap_index_writer.cpp b/be/src/storage/rowset/segment_v2/bitmap_index_writer.cpp index a76b9df1015..45ca054e26f 100644 --- a/be/src/storage/rowset/segment_v2/bitmap_index_writer.cpp +++ b/be/src/storage/rowset/segment_v2/bitmap_index_writer.cpp @@ -71,7 +71,7 @@ public: explicit BitmapIndexWriterImpl(const TypeInfoPtr& type_info) : _typeinfo(type_info), _reverted_index_size(0), _tracker(), _pool(&_tracker) {} - ~BitmapIndexWriterImpl() = default; + ~BitmapIndexWriterImpl() override = default; void add_values(const void* values, size_t count) override { auto p = reinterpret_cast(values); diff --git a/be/src/storage/rowset/segment_v2/column_writer.cpp b/be/src/storage/rowset/segment_v2/column_writer.cpp index e457cf7fcb6..ec23561d5d8 100644 --- a/be/src/storage/rowset/segment_v2/column_writer.cpp +++ b/be/src/storage/rowset/segment_v2/column_writer.cpp @@ -182,7 +182,7 @@ public: return _scalar_column_writer->append(data, null_flags, count, has_null); }; - Status append(const RowCursorCell& cell) { + Status append(const RowCursorCell& cell) override { if (!_is_speculated) { _scalar_column_writer->set_encoding(DEFAULT_ENCODING); _is_speculated = true; diff --git a/be/src/storage/rowset/segment_v2/segment_iterator.h b/be/src/storage/rowset/segment_v2/segment_iterator.h index b9aa0aae28c..a2376f963f0 100644 --- a/be/src/storage/rowset/segment_v2/segment_iterator.h +++ b/be/src/storage/rowset/segment_v2/segment_iterator.h @@ -58,7 +58,7 @@ public: Status next_batch(RowBlockV2* row_block) override; const Schema& schema() const override { return _schema; } bool is_lazy_materialization_read() const override { return _lazy_materialization_read; } - uint64_t data_id() const { return _segment->id(); } + uint64_t data_id() const override { return _segment->id(); } private: Status _init(); diff --git a/be/src/storage/rowset/unique_rowset_id_generator.h b/be/src/storage/rowset/unique_rowset_id_generator.h index b1c74b87d83..2220b348574 100644 --- a/be/src/storage/rowset/unique_rowset_id_generator.h +++ b/be/src/storage/rowset/unique_rowset_id_generator.h @@ -32,7 +32,7 @@ namespace starrocks { class UniqueRowsetIdGenerator : public RowsetIdGenerator { public: UniqueRowsetIdGenerator(const UniqueId& backend_uid); - ~UniqueRowsetIdGenerator() {} + ~UniqueRowsetIdGenerator() override {} RowsetId next_id() override; diff --git a/be/src/storage/rowset/vectorized/rowset_writer_adapter.h b/be/src/storage/rowset/vectorized/rowset_writer_adapter.h index 3daed2016c2..6d5a8791214 100644 --- a/be/src/storage/rowset/vectorized/rowset_writer_adapter.h +++ b/be/src/storage/rowset/vectorized/rowset_writer_adapter.h @@ -36,8 +36,7 @@ public: OLAPStatus flush_chunk(const vectorized::Chunk& chunk) override; - virtual OLAPStatus flush_chunk_with_deletes(const vectorized::Chunk& upserts, - const vectorized::Column& deletes) override; + OLAPStatus flush_chunk_with_deletes(const vectorized::Chunk& upserts, const vectorized::Column& deletes) override; OLAPStatus add_rowset(RowsetSharedPtr rowset) override { return _writer->add_rowset(rowset); } diff --git a/be/src/storage/schema_change.h b/be/src/storage/schema_change.h index 6e54b157404..9a2f6094af7 100644 --- a/be/src/storage/schema_change.h +++ b/be/src/storage/schema_change.h @@ -126,7 +126,7 @@ class LinkedSchemaChange : public SchemaChange { public: explicit LinkedSchemaChange(MemTracker* mem_tracker, const RowBlockChanger& row_block_changer) : SchemaChange(mem_tracker), _row_block_changer(row_block_changer) {} - ~LinkedSchemaChange() { _mem_tracker->release(_mem_tracker->consumption()); } + ~LinkedSchemaChange() override { _mem_tracker->release(_mem_tracker->consumption()); } bool process(RowsetReaderSharedPtr rowset_reader, RowsetWriter* new_rowset_writer, TabletSharedPtr new_tablet, TabletSharedPtr base_tablet) override; @@ -142,10 +142,10 @@ public: // @params tablet the instance of tablet which has new schema. // @params row_block_changer changer to modifiy the data of RowBlock explicit SchemaChangeDirectly(MemTracker* mem_tracker, const RowBlockChanger& row_block_changer); - virtual ~SchemaChangeDirectly(); + ~SchemaChangeDirectly() override; - virtual bool process(RowsetReaderSharedPtr rowset_reader, RowsetWriter* new_rowset_writer, - TabletSharedPtr new_tablet, TabletSharedPtr base_tablet) override; + bool process(RowsetReaderSharedPtr rowset_reader, RowsetWriter* new_rowset_writer, TabletSharedPtr new_tablet, + TabletSharedPtr base_tablet) override; private: const RowBlockChanger& _row_block_changer; @@ -162,10 +162,10 @@ class SchemaChangeWithSorting : public SchemaChange { public: explicit SchemaChangeWithSorting(MemTracker* mem_tracker, const RowBlockChanger& row_block_changer, size_t memory_limitation); - virtual ~SchemaChangeWithSorting(); + ~SchemaChangeWithSorting() override; - virtual bool process(RowsetReaderSharedPtr rowset_reader, RowsetWriter* new_rowset_builder, - TabletSharedPtr new_tablet, TabletSharedPtr base_tablet) override; + bool process(RowsetReaderSharedPtr rowset_reader, RowsetWriter* new_rowset_builder, TabletSharedPtr new_tablet, + TabletSharedPtr base_tablet) override; private: bool _internal_sorting(const std::vector& row_block_arr, const Version& temp_delta_versions, diff --git a/be/src/storage/tablet.h b/be/src/storage/tablet.h index 5704dc1be0d..3eb2fb9893c 100644 --- a/be/src/storage/tablet.h +++ b/be/src/storage/tablet.h @@ -69,7 +69,7 @@ public: Tablet(MemTracker* mem_tracker, TabletMetaSharedPtr tablet_meta, DataDir* data_dir); - ~Tablet(); + ~Tablet() override; OLAPStatus init(); inline bool init_succeeded(); diff --git a/be/src/storage/task/engine_publish_version_task.h b/be/src/storage/task/engine_publish_version_task.h index eaf9a01863c..4ca3a4a6406 100644 --- a/be/src/storage/task/engine_publish_version_task.h +++ b/be/src/storage/task/engine_publish_version_task.h @@ -31,7 +31,7 @@ namespace starrocks { class EnginePublishVersionTask : public EngineTask { public: EnginePublishVersionTask(TPublishVersionRequest& publish_version_req, vector* error_tablet_ids); - virtual ~EnginePublishVersionTask() override {} + ~EnginePublishVersionTask() override {} OLAPStatus finish() override; diff --git a/be/src/storage/vectorized/rowset_merger.cpp b/be/src/storage/vectorized/rowset_merger.cpp index 0cdd45244d9..5de15209f15 100644 --- a/be/src/storage/vectorized/rowset_merger.cpp +++ b/be/src/storage/vectorized/rowset_merger.cpp @@ -221,7 +221,7 @@ public: } Status do_merge(Tablet& tablet, int64_t version, const Schema& schema, const vector& rowsets, - RowsetWriter* writer, const MergeConfig& cfg) { + RowsetWriter* writer, const MergeConfig& cfg) override { MonotonicStopWatch timer; timer.start(); _chunk_size = cfg.chunk_size; diff --git a/be/src/util/broker_load_error_hub.h b/be/src/util/broker_load_error_hub.h index 7ec2dabc750..a4524ab1b9d 100644 --- a/be/src/util/broker_load_error_hub.h +++ b/be/src/util/broker_load_error_hub.h @@ -56,15 +56,15 @@ public: BrokerLoadErrorHub(ExecEnv* env, const TBrokerErrorHubInfo& info, const std::string& error_log_file_name); - virtual ~BrokerLoadErrorHub(); + ~BrokerLoadErrorHub() override; - virtual Status prepare(); + Status prepare() override; - virtual Status export_error(const ErrorMsg& error_msg); + Status export_error(const ErrorMsg& error_msg) override; - virtual Status close(); + Status close() override; - virtual std::string debug_string() const; + std::string debug_string() const override; private: Status write_to_broker(); diff --git a/be/src/util/callback_closure.h b/be/src/util/callback_closure.h index 9b17922b119..96cb2662783 100644 --- a/be/src/util/callback_closure.h +++ b/be/src/util/callback_closure.h @@ -34,7 +34,7 @@ template class CallBackClosure : public google::protobuf::Closure { public: CallBackClosure() : _refs(0) {} - ~CallBackClosure() {} + ~CallBackClosure() override {} // Disallow copy and assignment. CallBackClosure(const CallBackClosure& other) = delete; diff --git a/be/src/util/core_local.cpp b/be/src/util/core_local.cpp index 03bf77a0bca..06994799f53 100644 --- a/be/src/util/core_local.cpp +++ b/be/src/util/core_local.cpp @@ -48,7 +48,7 @@ struct alignas(CACHE_LINE_SIZE) CoreDataBlock { template class CoreDataAllocatorImpl : public CoreDataAllocator { public: - virtual ~CoreDataAllocatorImpl(); + ~CoreDataAllocatorImpl() override; void* get_or_create(size_t id) override { size_t block_id = id / ELEMENTS_PER_BLOCK; { diff --git a/be/src/util/metrics.h b/be/src/util/metrics.h index 9b967e27136..d8bf5ade848 100644 --- a/be/src/util/metrics.h +++ b/be/src/util/metrics.h @@ -94,7 +94,7 @@ template class AtomicMetric : public Metric { public: AtomicMetric(MetricType type, MetricUnit unit) : Metric(type, unit), _value(T()) {} - virtual ~AtomicMetric() {} + ~AtomicMetric() override {} std::string to_string() const override { return std::to_string(value()); } @@ -116,7 +116,7 @@ template class LockSimpleMetric : public Metric { public: LockSimpleMetric(MetricType type, MetricUnit unit) : Metric(type, unit), _value(T()) {} - virtual ~LockSimpleMetric() {} + ~LockSimpleMetric() override {} std::string to_string() const override { return std::to_string(value()); } @@ -155,7 +155,7 @@ class CoreLocalCounter : public Metric { public: CoreLocalCounter(MetricUnit unit) : Metric(MetricType::COUNTER, unit), _value() {} - virtual ~CoreLocalCounter() {} + ~CoreLocalCounter() override {} std::string to_string() const override { std::stringstream ss; @@ -185,14 +185,14 @@ template class AtomicCounter : public AtomicMetric { public: AtomicCounter(MetricUnit unit) : AtomicMetric(MetricType::COUNTER, unit) {} - virtual ~AtomicCounter() {} + ~AtomicCounter() override {} }; template class AtomicGauge : public AtomicMetric { public: AtomicGauge(MetricUnit unit) : AtomicMetric(MetricType::GAUGE, unit) {} - virtual ~AtomicGauge() {} + ~AtomicGauge() override {} }; template @@ -396,7 +396,7 @@ class TcmallocMetric final : public UIntGauge { public: TcmallocMetric(const std::string& tcmalloc_var) : UIntGauge(MetricUnit::BYTES), _tcmalloc_var(tcmalloc_var) {} - virtual uint64_t value() const override { + uint64_t value() const override { uint64_t val = 0; #if !defined(ADDRESS_SANITIZER) && !defined(THREAD_SANITIZER) && !defined(LEAK_SANITIZER) MallocExtension::instance()->GetNumericProperty(_tcmalloc_var.c_str(), reinterpret_cast(&val)); diff --git a/be/src/util/mysql_load_error_hub.h b/be/src/util/mysql_load_error_hub.h index 2b5777b527b..ad824c8445f 100644 --- a/be/src/util/mysql_load_error_hub.h +++ b/be/src/util/mysql_load_error_hub.h @@ -65,15 +65,15 @@ public: MysqlLoadErrorHub(const TMysqlErrorHubInfo& info); - virtual ~MysqlLoadErrorHub(); + ~MysqlLoadErrorHub() override; - virtual Status prepare(); + Status prepare() override; - virtual Status export_error(const ErrorMsg& error_msg); + Status export_error(const ErrorMsg& error_msg) override; - virtual Status close(); + Status close() override; - virtual std::string debug_string() const; + std::string debug_string() const override; private: Status open_mysql_conn(__StarRocksMysql** my_conn); diff --git a/be/src/util/null_load_error_hub.h b/be/src/util/null_load_error_hub.h index 62576efbeb0..bf9c959d43a 100644 --- a/be/src/util/null_load_error_hub.h +++ b/be/src/util/null_load_error_hub.h @@ -38,15 +38,15 @@ class NullLoadErrorHub : public LoadErrorHub { public: NullLoadErrorHub(); - virtual ~NullLoadErrorHub(); + ~NullLoadErrorHub() override; - virtual Status prepare(); + Status prepare() override; - virtual Status export_error(const ErrorMsg& error_msg); + Status export_error(const ErrorMsg& error_msg) override; - virtual Status close(); + Status close() override; - virtual std::string debug_string() const; + std::string debug_string() const override; private: std::mutex _mtx; diff --git a/be/src/util/ref_count_closure.h b/be/src/util/ref_count_closure.h index fb1a1556b1e..5f852f91370 100644 --- a/be/src/util/ref_count_closure.h +++ b/be/src/util/ref_count_closure.h @@ -33,7 +33,7 @@ template class RefCountClosure : public google::protobuf::Closure { public: RefCountClosure() : _refs(0) {} - ~RefCountClosure() {} + ~RefCountClosure() override {} void ref() { _refs.fetch_add(1); } diff --git a/be/src/util/thrift_server.cpp b/be/src/util/thrift_server.cpp index 356b2332750..c378444e710 100644 --- a/be/src/util/thrift_server.cpp +++ b/be/src/util/thrift_server.cpp @@ -46,24 +46,24 @@ public: ThriftServerEventProcessor(ThriftServer* thrift_server) : _thrift_server(thrift_server), _signal_fired(false) {} // friendly to code style - virtual ~ThriftServerEventProcessor() {} + ~ThriftServerEventProcessor() override {} // Called by TNonBlockingServer when server has acquired its resources and is ready to // serve, and signals to StartAndWaitForServer that start-up is finished. // From TServerEventHandler. - virtual void preServe(); + void preServe() override; // Called when a client connects; we create per-client state and call any // SessionHandlerIf handler. - virtual void* createContext(std::shared_ptr input, - std::shared_ptr output); + void* createContext(std::shared_ptr input, + std::shared_ptr output) override; // Called when a client starts an RPC; we set the thread-local session key. - virtual void processContext(void* context, std::shared_ptr output); + void processContext(void* context, std::shared_ptr output) override; // Called when a client disconnects; we call any SessionHandlerIf handler. - virtual void deleteContext(void* serverContext, std::shared_ptr input, - std::shared_ptr output); + void deleteContext(void* serverContext, std::shared_ptr input, + std::shared_ptr output) override; // Waits for a timeout of TIMEOUT_MS for a server to signal that it has started // correctly. diff --git a/licenses-binary/LICENSE-joda-time.txt b/licenses-binary/LICENSE-joda-time.txt index 75b52484ea4..d6456956733 100644 --- a/licenses-binary/LICENSE-joda-time.txt +++ b/licenses-binary/LICENSE-joda-time.txt @@ -1,202 +1,202 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License.