From 6c6380969b40526600c6fd49db86213b166f5fd6 Mon Sep 17 00:00:00 2001 From: Zhengguo Yang Date: Wed, 17 Nov 2021 10:18:35 +0800 Subject: [PATCH] [refactor] replace boost smart ptr with stl (#6856) 1. replace all boost::shared_ptr to std::shared_ptr 2. replace all boost::scopted_ptr to std::unique_ptr 3. replace all boost::scoped_array to std::unique 4. replace all boost:thread to std::thread --- be/CMakeLists.txt | 9 +- be/src/agent/heartbeat_server.cpp | 2 +- be/src/agent/pusher.cpp | 12 +- be/src/agent/task_worker_pool.cpp | 11 +- be/src/agent/topic_subscriber.h | 2 +- be/src/agent/utils.cpp | 8 +- be/src/common/logconfig.cpp | 6 +- be/src/env/env.h | 4 +- be/src/exec/aggregation_node.cpp | 54 ++--- be/src/exec/aggregation_node.h | 7 +- be/src/exec/aggregation_node_ir.cpp | 2 +- be/src/exec/analytic_eval_node.cpp | 69 +++--- be/src/exec/analytic_eval_node.h | 16 +- be/src/exec/blocking_join_node.cpp | 17 +- be/src/exec/blocking_join_node.h | 16 +- be/src/exec/broker_scan_node.h | 4 +- be/src/exec/broker_scanner.cpp | 17 +- be/src/exec/cross_join_node.h | 5 +- be/src/exec/csv_scan_node.h | 7 +- be/src/exec/data_sink.cpp | 18 +- be/src/exec/data_sink.h | 7 +- be/src/exec/decompressor.cpp | 4 +- be/src/exec/es/es_predicate.cpp | 6 +- be/src/exec/es/es_scroll_parser.cpp | 10 +- be/src/exec/es_http_scan_node.cpp | 4 +- be/src/exec/es_scan_node.cpp | 6 +- be/src/exec/exchange_node.cpp | 19 +- be/src/exec/exchange_node.h | 6 +- be/src/exec/exec_node.cpp | 34 +-- be/src/exec/exec_node.h | 10 +- be/src/exec/file_reader.h | 2 +- be/src/exec/hash_join_node.cpp | 34 +-- be/src/exec/hash_join_node.h | 18 +- be/src/exec/hash_join_node_ir.cpp | 2 +- be/src/exec/hash_table.cpp | 18 +- be/src/exec/hash_table.h | 15 +- be/src/exec/merge_join_node.cpp | 26 +-- be/src/exec/merge_join_node.h | 9 +- be/src/exec/merge_node.cpp | 14 +- be/src/exec/merge_node.h | 4 +- be/src/exec/mysql_scan_node.cpp | 32 +-- be/src/exec/mysql_scanner.cpp | 36 +-- be/src/exec/odbc_connector.cpp | 21 +- be/src/exec/odbc_connector.h | 17 +- be/src/exec/odbc_scan_node.cpp | 24 +- be/src/exec/olap_scan_node.cpp | 48 ++-- be/src/exec/olap_utils.h | 2 +- be/src/exec/parquet_reader.cpp | 2 +- be/src/exec/partitioned_aggregation_node.cc | 124 +++++----- be/src/exec/partitioned_aggregation_node.h | 47 ++-- .../exec/partitioned_aggregation_node_ir.cc | 14 +- be/src/exec/partitioned_hash_table.cc | 54 ++--- be/src/exec/partitioned_hash_table.h | 47 ++-- be/src/exec/partitioned_hash_table.inline.h | 48 ++-- be/src/exec/pl_task_root.cpp | 12 +- be/src/exec/read_write_util.h | 1 - be/src/exec/repeat_node.cpp | 6 +- be/src/exec/row_batch_list.h | 2 +- be/src/exec/s3_reader.h | 2 +- be/src/exec/scanner_ir.cpp | 4 +- be/src/exec/schema_scan_node.cpp | 42 ++-- be/src/exec/schema_scan_node.h | 6 +- be/src/exec/schema_scanner.cpp | 16 +- be/src/exec/schema_scanner.h | 14 +- .../schema_scanner/schema_columns_scanner.cpp | 32 +-- be/src/exec/schema_scanner/schema_helper.cpp | 2 +- .../schema_schema_privileges_scanner.cpp | 20 +- .../schema_schemata_scanner.cpp | 18 +- .../schema_table_privileges_scanner.cpp | 20 +- .../schema_scanner/schema_tables_scanner.cpp | 34 +-- .../schema_user_privileges_scanner.cpp | 20 +- .../schema_scanner/schema_views_scanner.cpp | 38 +-- be/src/exec/select_node.cpp | 2 +- be/src/exec/select_node.h | 4 +- be/src/exec/set_operation_node.cpp | 6 +- be/src/exec/sort_exec_exprs.cpp | 9 +- be/src/exec/sort_exec_exprs.h | 2 +- be/src/exec/spill_sort_node.cc | 6 +- be/src/exec/spill_sort_node.h | 2 +- be/src/exec/tablet_info.cpp | 2 +- be/src/exec/topn_node.cpp | 16 +- be/src/exec/topn_node.h | 7 +- be/src/exec/union_node.h | 8 +- be/src/exprs/agg_fn.h | 2 +- be/src/exprs/agg_fn_evaluator.cpp | 76 +++--- be/src/exprs/agg_fn_evaluator.h | 21 +- be/src/exprs/aggregate_functions.cpp | 116 ++++++---- be/src/exprs/aggregate_functions.h | 9 +- be/src/exprs/anyval_util.cpp | 4 +- be/src/exprs/anyval_util.h | 6 +- be/src/exprs/arithmetic_expr.cpp | 4 +- be/src/exprs/binary_predicate.cpp | 18 +- be/src/exprs/bitmap_function.h | 21 +- be/src/exprs/block_bloom_filter_impl.cc | 2 +- be/src/exprs/bloomfilter_predicate.cpp | 4 +- be/src/exprs/case_expr.cpp | 4 +- be/src/exprs/cast_expr.cpp | 4 +- be/src/exprs/conditional_functions.h | 2 +- be/src/exprs/conditional_functions_ir.cpp | 4 +- be/src/exprs/encryption_functions.cpp | 14 +- be/src/exprs/expr.cpp | 60 ++--- be/src/exprs/expr.h | 15 +- be/src/exprs/expr_context.cpp | 42 ++-- be/src/exprs/expr_context.h | 6 +- be/src/exprs/expr_value.h | 14 +- be/src/exprs/in_predicate.cpp | 12 +- be/src/exprs/in_predicate.h | 3 +- be/src/exprs/info_func.cpp | 2 +- be/src/exprs/json_functions.cpp | 4 +- be/src/exprs/like_predicate.cpp | 4 +- be/src/exprs/math_functions.cpp | 60 ++--- be/src/exprs/new_agg_fn_evaluator.cc | 2 +- be/src/exprs/new_agg_fn_evaluator.h | 15 +- be/src/exprs/new_in_predicate.cpp | 4 +- be/src/exprs/new_in_predicate.h | 4 +- be/src/exprs/operators.cpp | 2 - be/src/exprs/runtime_filter.cpp | 2 +- be/src/exprs/scalar_fn_call.cpp | 76 +++--- be/src/exprs/scalar_fn_call.h | 2 +- be/src/exprs/slot_ref.cpp | 26 +-- be/src/exprs/slot_ref.h | 6 +- be/src/exprs/string_functions.cpp | 47 ++-- be/src/exprs/timestamp_functions.cpp | 27 ++- be/src/exprs/timestamp_functions.h | 30 ++- be/src/exprs/tuple_is_null_predicate.cpp | 2 +- be/src/gutil/gscoped_ptr.h | 2 +- be/src/http/action/checksum_action.h | 2 +- be/src/http/action/reload_tablet_action.h | 2 - be/src/http/action/restore_tablet_action.h | 1 - be/src/http/action/snapshot_action.h | 3 +- be/src/olap/aggregate_func.h | 4 +- be/src/olap/base_compaction.cpp | 2 +- be/src/olap/bloom_filter_writer.cpp | 8 +- be/src/olap/byte_buffer.cpp | 42 ++-- be/src/olap/byte_buffer.h | 6 +- be/src/olap/data_dir.cpp | 37 +-- be/src/olap/decimal12.h | 13 +- be/src/olap/field.h | 8 +- be/src/olap/file_helper.cpp | 46 ++-- be/src/olap/file_helper.h | 16 +- be/src/olap/file_stream.cpp | 20 +- be/src/olap/file_stream.h | 12 +- be/src/olap/fs/block_manager.h | 2 +- be/src/olap/hll.cpp | 5 +- be/src/olap/in_stream.cpp | 26 +-- be/src/olap/in_stream.h | 6 +- be/src/olap/lru_cache.cpp | 29 ++- be/src/olap/lru_cache.h | 27 +-- be/src/olap/memory/hash_index.cpp | 2 +- be/src/olap/olap_cond.cpp | 18 +- be/src/olap/olap_define.h | 20 +- be/src/olap/olap_index.cpp | 8 +- be/src/olap/olap_index.h | 2 +- be/src/olap/options.cpp | 4 +- be/src/olap/out_stream.cpp | 44 ++-- be/src/olap/out_stream.h | 2 +- be/src/olap/push_handler.cpp | 22 +- be/src/olap/push_handler.h | 5 +- be/src/olap/row_block2.cpp | 2 +- be/src/olap/row_cursor.cpp | 8 +- be/src/olap/rowset/bit_field_reader.cpp | 6 +- be/src/olap/rowset/bit_field_writer.cpp | 6 +- be/src/olap/rowset/column_data_writer.cpp | 14 +- be/src/olap/rowset/column_reader.cpp | 142 ++++++------ be/src/olap/rowset/column_reader.h | 42 ++-- be/src/olap/rowset/column_writer.cpp | 73 +++--- be/src/olap/rowset/column_writer.h | 4 +- be/src/olap/rowset/segment_group.cpp | 50 ++-- be/src/olap/rowset/segment_reader.cpp | 50 ++-- .../olap/rowset/segment_v2/column_reader.cpp | 2 +- .../olap/rowset/segment_v2/column_writer.cpp | 2 +- be/src/olap/rowset/segment_writer.cpp | 10 +- be/src/olap/short_key_index.h | 2 +- be/src/olap/skiplist.h | 40 ++-- be/src/olap/snapshot_manager.cpp | 37 ++- be/src/olap/storage_engine.cpp | 8 +- be/src/olap/stream_index_common.cpp | 2 +- be/src/olap/stream_index_reader.cpp | 8 +- be/src/olap/stream_index_reader.h | 2 +- be/src/olap/stream_index_writer.cpp | 8 +- be/src/olap/tablet_meta.cpp | 5 +- be/src/olap/task/engine_batch_load_task.cpp | 6 +- be/src/olap/task/engine_checksum_task.cpp | 6 +- be/src/olap/task/engine_clone_task.cpp | 53 +++-- be/src/olap/types.h | 8 +- be/src/olap/utils.cpp | 34 +-- be/src/olap/utils.h | 8 +- be/src/plugin/plugin.h | 4 +- be/src/runtime/buffered_block_mgr2.cc | 154 ++++++------ be/src/runtime/buffered_block_mgr2.h | 45 ++-- be/src/runtime/buffered_tuple_stream2.cc | 93 ++++---- be/src/runtime/buffered_tuple_stream2.h | 25 +- .../runtime/buffered_tuple_stream2.inline.h | 10 +- be/src/runtime/buffered_tuple_stream2_ir.cc | 16 +- be/src/runtime/buffered_tuple_stream3.cc | 6 +- be/src/runtime/buffered_tuple_stream3.h | 9 +- be/src/runtime/bufferpool/buffer_allocator.h | 6 +- be/src/runtime/bufferpool/buffer_pool.cc | 22 +- be/src/runtime/bufferpool/buffer_pool.h | 33 ++- .../runtime/bufferpool/buffer_pool_internal.h | 8 +- .../runtime/bufferpool/reservation_tracker.h | 17 +- be/src/runtime/cache/cache_utils.h | 4 +- be/src/runtime/cache/result_cache.cpp | 4 +- be/src/runtime/cache/result_cache.h | 3 +- be/src/runtime/cache/result_node.cpp | 16 +- be/src/runtime/cache/result_node.h | 13 +- be/src/runtime/data_spliter.cpp | 6 +- be/src/runtime/data_spliter.h | 2 +- be/src/runtime/data_stream_mgr.cpp | 14 +- be/src/runtime/data_stream_mgr.h | 11 +- be/src/runtime/data_stream_recvr.cc | 18 +- be/src/runtime/data_stream_recvr.h | 5 +- be/src/runtime/data_stream_sender.cpp | 41 ++-- be/src/runtime/data_stream_sender.h | 2 +- be/src/runtime/datetime_value.cpp | 18 +- be/src/runtime/datetime_value.h | 25 +- be/src/runtime/descriptors.cpp | 29 ++- be/src/runtime/disk_io_mgr.cc | 134 +++++------ be/src/runtime/disk_io_mgr.h | 24 +- be/src/runtime/disk_io_mgr_internal.h | 10 +- be/src/runtime/disk_io_mgr_reader_context.cc | 42 ++-- be/src/runtime/disk_io_mgr_scan_range.cc | 76 +++--- be/src/runtime/dpp_sink.cpp | 16 +- be/src/runtime/dpp_writer.cpp | 8 +- be/src/runtime/export_sink.cpp | 4 +- be/src/runtime/external_scan_context_mgr.cpp | 4 +- be/src/runtime/file_result_writer.cpp | 39 ++-- be/src/runtime/fold_constant_mgr.cpp | 51 ++-- be/src/runtime/fragment_mgr.cpp | 4 +- be/src/runtime/mem_pool.h | 12 +- be/src/runtime/mem_tracker.h | 22 +- be/src/runtime/memory_scratch_sink.cpp | 2 +- be/src/runtime/mysql_result_writer.cpp | 20 +- be/src/runtime/mysql_table_writer.cpp | 6 +- be/src/runtime/plan_fragment_executor.cpp | 53 ++--- be/src/runtime/plan_fragment_executor.h | 15 +- be/src/runtime/qsorter.cpp | 4 +- be/src/runtime/qsorter.h | 3 +- be/src/runtime/raw_value.cpp | 26 +-- be/src/runtime/raw_value.h | 16 +- be/src/runtime/raw_value_ir.cpp | 6 +- be/src/runtime/result_buffer_mgr.cpp | 17 +- be/src/runtime/result_buffer_mgr.h | 10 +- be/src/runtime/result_file_sink.cpp | 48 ++-- be/src/runtime/result_file_sink.h | 19 +- be/src/runtime/result_sink.cpp | 2 +- be/src/runtime/result_sink.h | 6 +- be/src/runtime/row_batch.cpp | 34 +-- be/src/runtime/row_batch.h | 9 +- be/src/runtime/runtime_state.cpp | 8 +- be/src/runtime/runtime_state.h | 35 ++- be/src/runtime/sorted_run_merger.cc | 32 +-- be/src/runtime/sorted_run_merger.h | 3 +- be/src/runtime/spill_sorter.cc | 115 ++++----- be/src/runtime/spill_sorter.h | 6 +- be/src/runtime/string_search.hpp | 14 +- be/src/runtime/string_value.h | 2 +- be/src/runtime/thread_resource_mgr.cpp | 6 +- be/src/runtime/thread_resource_mgr.h | 6 +- be/src/runtime/tuple.cpp | 2 +- be/src/runtime/tuple.h | 2 +- be/src/runtime/tuple_row.h | 12 +- be/src/service/backend_service.cpp | 4 +- be/src/service/doris_main.cpp | 4 +- be/src/testutil/desc_tbl_builder.cc | 2 +- be/src/udf/uda_test_harness.h | 6 +- be/src/udf/udf.cpp | 24 +- be/src/udf/udf.h | 28 +-- be/src/udf/udf_internal.h | 6 +- be/src/udf/udf_ir.cpp | 6 +- be/src/util/aes_util.cpp | 2 +- be/src/util/batch_process_thread_pool.hpp | 3 +- be/src/util/bit_stream_utils.h | 4 +- be/src/util/bitmap_value.h | 4 +- be/src/util/buffer_builder.hpp | 32 +-- be/src/util/cpu_info.cpp | 6 +- be/src/util/cpu_info.h | 1 - be/src/util/date_func.cpp | 6 +- be/src/util/debug_util.h | 1 - be/src/util/disk_info.h | 1 - be/src/util/dynamic_util.cpp | 4 +- be/src/util/dynamic_util.h | 4 +- be/src/util/error_util.h | 1 - be/src/util/internal_queue.h | 108 ++++----- be/src/util/json_util.h | 13 +- be/src/util/mem_info.h | 1 - be/src/util/mutex.cpp | 2 +- be/src/util/mysql_row_buffer.cpp | 17 +- be/src/util/mysql_row_buffer.h | 2 +- be/src/util/network_util.cpp | 6 +- be/src/util/parse_util.h | 4 +- be/src/util/path_builder.cpp | 2 +- be/src/util/perf_counters.cpp | 2 +- be/src/util/pretty_printer.h | 16 +- be/src/util/priority_thread_pool.hpp | 6 +- be/src/util/progress_updater.h | 1 - be/src/util/runtime_profile.cpp | 24 +- be/src/util/runtime_profile.h | 26 +-- be/src/util/stopwatch.hpp | 3 +- be/src/util/streaming_sampler.h | 4 +- be/src/util/symbols_util.cpp | 20 +- be/src/util/template_util.h | 52 ----- be/src/util/thread.cpp | 8 +- be/src/util/thread.h | 6 +- be/src/util/thread_group.h | 116 ++++++++++ be/src/util/thrift_client.cpp | 5 +- be/src/util/thrift_rpc_helper.cpp | 2 +- be/src/util/thrift_server.cpp | 4 +- be/src/util/thrift_server.h | 6 +- be/src/util/thrift_util.h | 2 +- be/src/util/tuple_row_compare.h | 6 +- be/src/util/uid_util.cpp | 8 +- be/src/util/url_coding.cpp | 2 +- be/src/util/url_coding.h | 2 +- be/src/util/url_parser.cpp | 2 +- be/test/agent/cgroups_mgr_test.cpp | 2 +- be/test/exec/csv_scan_node_test.cpp | 3 +- be/test/exec/hash_table_test.cpp | 8 +- be/test/exec/mysql_scan_node_test.cpp | 8 +- be/test/exec/mysql_scanner_test.cpp | 2 +- be/test/exec/olap_scanner_test.cpp | 4 +- be/test/exec/schema_scan_node_test.cpp | 8 +- .../schema_authors_scanner_test.cpp | 10 +- .../schema_charsets_scanner_test.cpp | 10 +- .../schema_collations_scanner_test.cpp | 10 +- .../schema_columns_scanner_test.cpp | 12 +- .../schema_create_table_scanner_test.cpp | 12 +- .../schema_engines_scanner_test.cpp | 10 +- .../schema_open_tables_scanner_test.cpp | 12 +- .../schema_schemata_scanner_test.cpp | 8 +- .../schema_table_names_scanner_test.cpp | 10 +- .../schema_tables_scanner_test.cpp | 12 +- .../schema_variables_scanner_test.cpp | 10 +- be/test/exec/schema_scanner_test.cpp | 10 +- be/test/exprs/binary_predicate_test.cpp | 20 +- be/test/exprs/in_op_test.cpp | 22 +- be/test/exprs/timestamp_functions_test.cpp | 2 - be/test/olap/bit_field_test.cpp | 12 +- .../bloom_filter_column_predicate_test.cpp | 4 +- be/test/olap/bloom_filter_index_test.cpp | 8 +- be/test/olap/bloom_filter_test.cpp | 2 +- be/test/olap/byte_buffer_test.cpp | 19 +- be/test/olap/column_reader_test.cpp | 14 +- be/test/olap/comparison_predicate_test.cpp | 7 +- be/test/olap/delete_handler_test.cpp | 6 +- be/test/olap/in_list_predicate_test.cpp | 4 +- be/test/olap/lru_cache_test.cpp | 6 +- be/test/olap/memory/column_test.cpp | 6 +- be/test/olap/run_length_byte_test.cpp | 86 +++---- be/test/olap/run_length_integer_test.cpp | 24 +- be/test/olap/schema_change_test.cpp | 12 +- be/test/olap/skiplist_test.cpp | 6 +- be/test/olap/stream_index_test.cpp | 2 +- be/test/runtime/buffer_control_block_test.cpp | 22 +- be/test/runtime/buffered_block_mgr2_test.cpp | 219 +++++++++--------- .../runtime/buffered_tuple_stream2_test.cpp | 49 ++-- .../runtime/cache/partition_cache_test.cpp | 17 +- be/test/runtime/data_stream_test.cpp | 43 ++-- be/test/runtime/disk_io_mgr_test.cpp | 75 +++--- be/test/runtime/free_list_test.cpp | 26 +-- be/test/runtime/mem_pool_test.cpp | 12 +- be/test/runtime/result_buffer_mgr_test.cpp | 14 +- be/test/runtime/sorter_test.cpp | 4 +- be/test/runtime/test_env.cc | 2 +- be/test/runtime/thread_resource_mgr_test.cpp | 2 +- be/test/runtime/tmp_file_mgr_test.cpp | 7 +- be/test/udf/uda_test.cpp | 26 +-- be/test/udf/udf_test.cpp | 41 ++-- be/test/util/aes_util_test.cpp | 10 +- be/test/util/arrow/arrow_work_flow_test.cpp | 1 - be/test/util/blocking_queue_test.cpp | 16 +- be/test/util/decompress_test.cpp | 13 +- be/test/util/internal_queue_test.cpp | 34 +-- be/test/util/rle_encoding_test.cpp | 4 +- be/test/util/runtime_profile_test.cpp | 19 +- be/test/util/string_parser_test.cpp | 2 +- be/test/util/thread_group_test.cpp | 81 +++++++ be/test/util/thread_pool_test.cpp | 2 +- .../src/udaf_orthogonal_bitmap/bitmap_value.h | 52 +++-- .../src/udaf_orthogonal_bitmap/string_value.h | 63 ++--- samples/connect/cpp/doris_client.cpp | 19 +- thirdparty/download-thirdparty.sh | 4 +- thirdparty/vars.sh | 1 - 383 files changed, 3443 insertions(+), 3396 deletions(-) delete mode 100644 be/src/util/template_util.h create mode 100644 be/src/util/thread_group.h create mode 100644 be/test/util/thread_group_test.cpp diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index 9c3631c9f6..d88c9b5bf3 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -135,9 +135,9 @@ set(BOOST_ROOT ${THIRDPARTY_DIR}) set(Boost_NO_BOOST_CMAKE OFF) if (NOT APPLE) - find_package(Boost 1.73.0 REQUIRED COMPONENTS regex system filesystem thread date_time program_options) + find_package(Boost 1.73.0 REQUIRED COMPONENTS system date_time) else() - find_package(Boost 1.73.0 COMPONENTS thread regex system filesystem date_time program_options) + find_package(Boost 1.73.0 COMPONENTS system date_time) endif() set(GPERFTOOLS_HOME "${THIRDPARTY_DIR}/gperftools") @@ -482,12 +482,7 @@ set(COMMON_THIRDPARTY librdkafka libs2 snappy - Boost::regex - Boost::system - Boost::filesystem - Boost::thread Boost::date_time - Boost::program_options thrift thriftnb glog diff --git a/be/src/agent/heartbeat_server.cpp b/be/src/agent/heartbeat_server.cpp index 5c178a77ec..764b08af2b 100644 --- a/be/src/agent/heartbeat_server.cpp +++ b/be/src/agent/heartbeat_server.cpp @@ -174,7 +174,7 @@ AgentStatus create_heartbeat_server(ExecEnv* exec_env, uint32_t server_port, ThriftServer** thrift_server, uint32_t worker_thread_num, TMasterInfo* local_master_info) { HeartbeatServer* heartbeat_server = new (nothrow) HeartbeatServer(local_master_info); - if (heartbeat_server == NULL) { + if (heartbeat_server == nullptr) { return DORIS_ERROR; } diff --git a/be/src/agent/pusher.cpp b/be/src/agent/pusher.cpp index 4fba068b56..f122d67ce6 100644 --- a/be/src/agent/pusher.cpp +++ b/be/src/agent/pusher.cpp @@ -53,7 +53,7 @@ AgentStatus Pusher::init() { // Check replica exist OLAPTablePtr olap_table; olap_table = _engine->get_table(_push_req.tablet_id, _push_req.schema_hash); - if (olap_table.get() == NULL) { + if (olap_table.get() == nullptr) { OLAP_LOG_WARNING("get tables failed. tablet_id: %ld, schema_hash: %ld", _push_req.tablet_id, _push_req.schema_hash); return DORIS_PUSH_INVALID_TABLE; @@ -94,10 +94,10 @@ AgentStatus Pusher::_get_tmp_file_dir(const string& root_path, string* download_ if (!std::filesystem::exists(full_path)) { LOG(INFO) << "download dir not exist: " << *download_path; - boost::system::error_code error_code; + std::error_code error_code; std::filesystem::create_directories(*download_path, error_code); - if (0 != error_code) { + if (error_code) { status = DORIS_ERROR; LOG(WARNING) << "create download dir failed.path: " << *download_path << ", error code: " << error_code; @@ -130,7 +130,7 @@ AgentStatus Pusher::process(vector* tablet_infos) { bool is_timeout = false; auto download_cb = [this, estimate_time_out, file_size, &is_timeout](HttpClient* client) { // Check timeout and set timeout - time_t now = time(NULL); + time_t now = time(nullptr); if (_push_req.timeout > 0 && _push_req.timeout < now) { // return status to break this callback VLOG_NOTICE << "check time out. time_out:" << _push_req.timeout << ", now:" << now; @@ -191,9 +191,9 @@ AgentStatus Pusher::process(vector* tablet_infos) { if (status == DORIS_SUCCESS) { // Load delta file - time_t push_begin = time(NULL); + time_t push_begin = time(nullptr); OLAPStatus push_status = _engine->push(_push_req, tablet_infos); - time_t push_finish = time(NULL); + time_t push_finish = time(nullptr); LOG(INFO) << "Push finish, cost time: " << (push_finish - push_begin); if (push_status == OLAPStatus::OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST) { status = DORIS_PUSH_HAD_LOADED; diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 1fb5bb30c4..e02668c6e0 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -74,7 +74,7 @@ DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(agent_task_queue_size, MetricUnit::NOUNIT); const uint32_t TASK_FINISH_MAX_RETRY = 3; const uint32_t PUBLISH_VERSION_MAX_RETRY = 3; -std::atomic_ulong TaskWorkerPool::_s_report_version(time(NULL) * 10000); +std::atomic_ulong TaskWorkerPool::_s_report_version(time(nullptr) * 10000); Mutex TaskWorkerPool::_s_task_signatures_lock; map> TaskWorkerPool::_s_task_signatures; FrontendServiceClientCache TaskWorkerPool::_master_service_client_cache; @@ -1115,7 +1115,8 @@ void TaskWorkerPool::_report_task_worker_thread_callback() { if (_master_info.network_address.port == 0) { // port == 0 means not received heartbeat yet // sleep a short time and try again - LOG(INFO) << "waiting to receive first heartbeat from frontend before doing task report"; + LOG(INFO) + << "waiting to receive first heartbeat from frontend before doing task report"; continue; } @@ -1149,7 +1150,8 @@ void TaskWorkerPool::_report_disk_state_worker_thread_callback() { if (_master_info.network_address.port == 0) { // port == 0 means not received heartbeat yet - LOG(INFO) << "waiting to receive first heartbeat from frontend before doing disk report"; + LOG(INFO) + << "waiting to receive first heartbeat from frontend before doing disk report"; continue; } @@ -1199,7 +1201,8 @@ void TaskWorkerPool::_report_tablet_worker_thread_callback() { if (_master_info.network_address.port == 0) { // port == 0 means not received heartbeat yet - LOG(INFO) << "waiting to receive first heartbeat from frontend before doing tablet report"; + LOG(INFO) << "waiting to receive first heartbeat from frontend before doing tablet " + "report"; continue; } diff --git a/be/src/agent/topic_subscriber.h b/be/src/agent/topic_subscriber.h index 6048c997f3..2f5ed7f5cd 100644 --- a/be/src/agent/topic_subscriber.h +++ b/be/src/agent/topic_subscriber.h @@ -18,9 +18,9 @@ #ifndef DORIS_BE_SRC_AGENT_TOPIC_SUBSCRIBER_H #define DORIS_BE_SRC_AGENT_TOPIC_SUBSCRIBER_H -#include #include #include +#include #include "agent/topic_listener.h" #include "gen_cpp/AgentService_types.h" diff --git a/be/src/agent/utils.cpp b/be/src/agent/utils.cpp index 4ad4de25db..df81d26e68 100644 --- a/be/src/agent/utils.cpp +++ b/be/src/agent/utils.cpp @@ -152,10 +152,10 @@ AgentStatus AgentUtils::rsync_from_remote(const string& remote_host, const strin cmd_stream << " " << remote_host << ":" << remote_file_path << " " << local_file_path; LOG(INFO) << "rsync cmd: " << cmd_stream.str(); - FILE* fp = NULL; + FILE* fp = nullptr; fp = popen(cmd_stream.str().c_str(), "r"); - if (fp == NULL) { + if (fp == nullptr) { return DORIS_ERROR; } @@ -226,7 +226,7 @@ bool AgentUtils::exec_cmd(const string& command, string* errmsg, bool redirect_s // Execute command. FILE* fp = popen(cmd.c_str(), "r"); - if (fp == NULL) { + if (fp == nullptr) { std::stringstream err_stream; err_stream << "popen failed. " << strerror(errno) << ", with errno: " << errno << ".\n"; *errmsg = err_stream.str(); @@ -235,7 +235,7 @@ bool AgentUtils::exec_cmd(const string& command, string* errmsg, bool redirect_s // Get command output. char result[1024] = {'\0'}; - while (fgets(result, sizeof(result), fp) != NULL) { + while (fgets(result, sizeof(result), fp) != nullptr) { *errmsg += result; } diff --git a/be/src/common/logconfig.cpp b/be/src/common/logconfig.cpp index 49684acd69..cb2d748e48 100644 --- a/be/src/common/logconfig.cpp +++ b/be/src/common/logconfig.cpp @@ -107,7 +107,7 @@ bool init_glog(const char* basename, bool install_signal_handler) { FLAGS_log_split_method = "size"; std::string sizestr = rollmode.substr(sizeflag.size(), rollmode.size() - sizeflag.size()); if (sizestr.size() != 0) { - char* end = NULL; + char* end = nullptr; errno = 0; const char* sizecstr = sizestr.c_str(); int64_t ret64 = strtoll(sizecstr, &end, 10); @@ -162,8 +162,8 @@ std::string FormatTimestampForLog(MicrosecondsInt64 micros_since_epoch) { /// Custom your log format here void TaggableLogger::flush() { _stream << _message; - Tags *head = _tags; - Tags *next; + Tags* head = _tags; + Tags* next; while (head) { next = head->next; _stream << "|" << head->key << "=" << head->value; diff --git a/be/src/env/env.h b/be/src/env/env.h index d0215739df..0b8eac7c18 100644 --- a/be/src/env/env.h +++ b/be/src/env/env.h @@ -47,7 +47,7 @@ public: // Create a brand new sequentially-readable file with the specified name. // On success, stores a pointer to the new file in *result and returns OK. - // On failure stores NULL in *result and returns non-OK. If the file does + // On failure stores nullptr in *result and returns non-OK. If the file does // not exist, returns a non-OK status. // // The returned file will only be accessed by one thread at a time. @@ -71,7 +71,7 @@ public: // Create an object that writes to a new file with the specified // name. Deletes any existing file with the same name and creates a // new file. On success, stores a pointer to the new file in - // *result and returns OK. On failure stores NULL in *result and + // *result and returns OK. On failure stores nullptr in *result and // returns non-OK. // // The returned file will only be accessed by one thread at a time. diff --git a/be/src/exec/aggregation_node.cpp b/be/src/exec/aggregation_node.cpp index df3bc5382d..3cadea45f8 100644 --- a/be/src/exec/aggregation_node.cpp +++ b/be/src/exec/aggregation_node.cpp @@ -47,17 +47,17 @@ AggregationNode::AggregationNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : ExecNode(pool, tnode, descs), _intermediate_tuple_id(tnode.agg_node.intermediate_tuple_id), - _intermediate_tuple_desc(NULL), + _intermediate_tuple_desc(nullptr), _output_tuple_id(tnode.agg_node.output_tuple_id), - _output_tuple_desc(NULL), - _singleton_output_tuple(NULL), + _output_tuple_desc(nullptr), + _singleton_output_tuple(nullptr), //_tuple_pool(new MemPool()), // - _process_row_batch_fn(NULL), + _process_row_batch_fn(nullptr), _needs_finalize(tnode.agg_node.need_finalize), - _build_timer(NULL), - _get_results_timer(NULL), - _hash_table_buckets_counter(NULL) {} + _build_timer(nullptr), + _get_results_timer(nullptr), + _hash_table_buckets_counter(nullptr) {} AggregationNode::~AggregationNode() {} @@ -68,7 +68,7 @@ Status AggregationNode::init(const TPlanNode& tnode, RuntimeState* state) { Expr::create_expr_trees(_pool, tnode.agg_node.grouping_exprs, &_probe_expr_ctxs)); for (int i = 0; i < tnode.agg_node.aggregate_functions.size(); ++i) { - AggFnEvaluator* evaluator = NULL; + AggFnEvaluator* evaluator = nullptr; AggFnEvaluator::create(_pool, tnode.agg_node.aggregate_functions[i], &evaluator); _aggregate_evaluators.push_back(evaluator); } @@ -177,9 +177,9 @@ Status AggregationNode::open(RuntimeState* state) { int64_t agg_rows_before = _hash_tbl->size(); - if (_process_row_batch_fn != NULL) { + if (_process_row_batch_fn != nullptr) { _process_row_batch_fn(this, &batch); - } else if (_singleton_output_tuple != NULL) { + } else if (_singleton_output_tuple != nullptr) { SCOPED_TIMER(_build_timer); process_row_batch_no_grouping(&batch, _tuple_pool.get()); } else { @@ -210,7 +210,7 @@ Status AggregationNode::open(RuntimeState* state) { } } - if (_singleton_output_tuple != NULL) { + if (_singleton_output_tuple != nullptr) { _hash_tbl->insert(reinterpret_cast(&_singleton_output_tuple)); ++num_agg_rows; } @@ -225,9 +225,10 @@ Status AggregationNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* // 1. `!need_finalize` means this aggregation node not the level two aggregation node // 2. `_singleton_output_tuple != nullptr` means is not group by // 3. `child(0)->rows_returned() == 0` mean not data from child - // in level two aggregation node should return NULL result + // in level two aggregation node should return nullptr result // level one aggregation node set `eos = true` return directly - if (UNLIKELY(!_needs_finalize && _singleton_output_tuple != nullptr && child(0)->rows_returned() == 0)) { + if (UNLIKELY(!_needs_finalize && _singleton_output_tuple != nullptr && + child(0)->rows_returned() == 0)) { *eos = true; return Status::OK(); } @@ -275,7 +276,7 @@ Status AggregationNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* *eos = _output_iterator.at_end() || reached_limit(); if (*eos) { - if (_hash_tbl.get() != NULL && _hash_table_buckets_counter != NULL) { + if (_hash_tbl.get() != nullptr && _hash_table_buckets_counter != nullptr) { COUNTER_SET(_hash_table_buckets_counter, _hash_tbl->num_buckets()); } } @@ -292,8 +293,8 @@ Status AggregationNode::close(RuntimeState* state) { // them in order to free any memory allocated by UDAs. Finalize() requires a dst tuple // but we don't actually need the result, so allocate a single dummy tuple to avoid // accumulating memory. - Tuple* dummy_dst = NULL; - if (_needs_finalize && _output_tuple_desc != NULL) { + Tuple* dummy_dst = nullptr; + if (_needs_finalize && _output_tuple_desc != nullptr) { dummy_dst = Tuple::create(_output_tuple_desc->byte_size(), _tuple_pool.get()); } while (!_output_iterator.at_end()) { @@ -313,10 +314,10 @@ Status AggregationNode::close(RuntimeState* state) { } } - if (_tuple_pool.get() != NULL) { + if (_tuple_pool.get() != nullptr) { _tuple_pool->free_all(); } - if (_hash_tbl.get() != NULL) { + if (_hash_tbl.get() != nullptr) { _hash_tbl->close(); } @@ -362,17 +363,17 @@ Tuple* AggregationNode::construct_intermediate_tuple() { // This optimization no longer applies with AnyVal if (!(*slot_desc)->type().is_string_type() && !(*slot_desc)->type().is_date_type()) { ExprValue default_value; - void* default_value_ptr = NULL; + void* default_value_ptr = nullptr; switch (evaluator->agg_op()) { case TAggregationOp::MIN: default_value_ptr = default_value.set_to_max((*slot_desc)->type()); - RawValue::write(default_value_ptr, agg_tuple, *slot_desc, NULL); + RawValue::write(default_value_ptr, agg_tuple, *slot_desc, nullptr); break; case TAggregationOp::MAX: default_value_ptr = default_value.set_to_min((*slot_desc)->type()); - RawValue::write(default_value_ptr, agg_tuple, *slot_desc, NULL); + RawValue::write(default_value_ptr, agg_tuple, *slot_desc, nullptr); break; default: @@ -385,7 +386,7 @@ Tuple* AggregationNode::construct_intermediate_tuple() { } void AggregationNode::update_tuple(Tuple* tuple, TupleRow* row) { - DCHECK(tuple != NULL); + DCHECK(tuple != nullptr); AggFnEvaluator::add(_aggregate_evaluators, _agg_fn_ctxs, row, tuple); #if 0 @@ -404,14 +405,15 @@ void AggregationNode::update_tuple(Tuple* tuple, TupleRow* row) { } Tuple* AggregationNode::finalize_tuple(Tuple* tuple, MemPool* pool) { - DCHECK(tuple != NULL); + DCHECK(tuple != nullptr); Tuple* dst = tuple; if (_needs_finalize && _intermediate_tuple_id != _output_tuple_id) { dst = Tuple::create(_output_tuple_desc->byte_size(), pool); } if (_needs_finalize) { - AggFnEvaluator::finalize(_aggregate_evaluators, _agg_fn_ctxs, tuple, dst, + AggFnEvaluator::finalize( + _aggregate_evaluators, _agg_fn_ctxs, tuple, dst, _singleton_output_tuple != nullptr && child(0)->rows_returned() == 0); } else { AggFnEvaluator::serialize(_aggregate_evaluators, _agg_fn_ctxs, tuple); @@ -424,9 +426,9 @@ Tuple* AggregationNode::finalize_tuple(Tuple* tuple, MemPool* pool) { SlotDescriptor* src_slot_desc = _intermediate_tuple_desc->slots()[i]; SlotDescriptor* dst_slot_desc = _output_tuple_desc->slots()[i]; bool src_slot_null = tuple->is_null(src_slot_desc->null_indicator_offset()); - void* src_slot = NULL; + void* src_slot = nullptr; if (!src_slot_null) src_slot = tuple->get_slot(src_slot_desc->tuple_offset()); - RawValue::write(src_slot, dst, dst_slot_desc, NULL); + RawValue::write(src_slot, dst, dst_slot_desc, nullptr); } } return dst; diff --git a/be/src/exec/aggregation_node.h b/be/src/exec/aggregation_node.h index 464d3647d2..a45236ecb9 100644 --- a/be/src/exec/aggregation_node.h +++ b/be/src/exec/aggregation_node.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_AGGREGATION_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_AGGREGATION_NODE_H -#include #include #include "exec/exec_node.h" @@ -64,14 +63,14 @@ public: virtual void push_down_predicate(RuntimeState* state, std::list* expr_ctxs); private: - boost::scoped_ptr _hash_tbl; + std::unique_ptr _hash_tbl; HashTable::Iterator _output_iterator; std::vector _aggregate_evaluators; /// FunctionContext for each agg fn and backing pool. std::vector _agg_fn_ctxs; - boost::scoped_ptr _agg_fn_pool; + std::unique_ptr _agg_fn_pool; // Exprs used to evaluate input rows std::vector _probe_expr_ctxs; @@ -89,7 +88,7 @@ private: TupleDescriptor* _output_tuple_desc; Tuple* _singleton_output_tuple; // result of aggregation w/o GROUP BY - boost::scoped_ptr _tuple_pool; + std::unique_ptr _tuple_pool; typedef void (*ProcessRowBatchFn)(AggregationNode*, RowBatch*); // Jitted ProcessRowBatch function pointer. Null if codegen is disabled. diff --git a/be/src/exec/aggregation_node_ir.cpp b/be/src/exec/aggregation_node_ir.cpp index 0d174411a6..c105760c2e 100644 --- a/be/src/exec/aggregation_node_ir.cpp +++ b/be/src/exec/aggregation_node_ir.cpp @@ -33,7 +33,7 @@ void AggregationNode::process_row_batch_no_grouping(RowBatch* batch, MemPool* po void AggregationNode::process_row_batch_with_grouping(RowBatch* batch, MemPool* pool) { for (int i = 0; i < batch->num_rows(); ++i) { TupleRow* row = batch->get_row(i); - Tuple* agg_tuple = NULL; + Tuple* agg_tuple = nullptr; HashTable::Iterator it = _hash_tbl->find(row); if (it.at_end()) { diff --git a/be/src/exec/analytic_eval_node.cpp b/be/src/exec/analytic_eval_node.cpp index 65867b5ee4..df1b4cea27 100644 --- a/be/src/exec/analytic_eval_node.cpp +++ b/be/src/exec/analytic_eval_node.cpp @@ -35,9 +35,9 @@ AnalyticEvalNode::AnalyticEvalNode(ObjectPool* pool, const TPlanNode& tnode, _intermediate_tuple_desc( descs.get_tuple_descriptor(tnode.analytic_node.intermediate_tuple_id)), _result_tuple_desc(descs.get_tuple_descriptor(tnode.analytic_node.output_tuple_id)), - _buffered_tuple_desc(NULL), - _partition_by_eq_expr_ctx(NULL), - _order_by_eq_expr_ctx(NULL), + _buffered_tuple_desc(nullptr), + _partition_by_eq_expr_ctx(nullptr), + _order_by_eq_expr_ctx(nullptr), _rows_start_offset(0), _rows_end_offset(0), _has_first_val_null_offset(false), @@ -45,13 +45,13 @@ AnalyticEvalNode::AnalyticEvalNode(ObjectPool* pool, const TPlanNode& tnode, _last_result_idx(-1), _prev_pool_last_result_idx(-1), _prev_pool_last_window_idx(-1), - _curr_tuple(NULL), - _dummy_result_tuple(NULL), + _curr_tuple(nullptr), + _dummy_result_tuple(nullptr), _curr_partition_idx(-1), - _prev_input_row(NULL), + _prev_input_row(nullptr), _block_mgr_client(nullptr), _input_eos(false), - _evaluation_timer(NULL) { + _evaluation_timer(nullptr) { if (tnode.analytic_node.__isset.buffered_tuple_id) { _buffered_tuple_desc = descs.get_tuple_descriptor(tnode.analytic_node.buffered_tuple_id); } @@ -108,7 +108,7 @@ Status AnalyticEvalNode::init(const TPlanNode& tnode, RuntimeState* state) { bool has_lead_fn = false; for (int i = 0; i < analytic_node.analytic_functions.size(); ++i) { - AggFnEvaluator* evaluator = NULL; + AggFnEvaluator* evaluator = nullptr; RETURN_IF_ERROR(AggFnEvaluator::create(_pool, analytic_node.analytic_functions[i], true, &evaluator)); _evaluators.push_back(evaluator); @@ -158,20 +158,20 @@ Status AnalyticEvalNode::prepare(RuntimeState* state) { state->obj_pool()->add(ctx); } - if (_partition_by_eq_expr_ctx != NULL || _order_by_eq_expr_ctx != NULL) { - DCHECK(_buffered_tuple_desc != NULL); + if (_partition_by_eq_expr_ctx != nullptr || _order_by_eq_expr_ctx != nullptr) { + DCHECK(_buffered_tuple_desc != nullptr); std::vector tuple_ids; tuple_ids.push_back(child(0)->row_desc().tuple_descriptors()[0]->id()); tuple_ids.push_back(_buffered_tuple_desc->id()); RowDescriptor cmp_row_desc(state->desc_tbl(), tuple_ids, std::vector(2, false)); - if (_partition_by_eq_expr_ctx != NULL) { + if (_partition_by_eq_expr_ctx != nullptr) { RETURN_IF_ERROR( _partition_by_eq_expr_ctx->prepare(state, cmp_row_desc, expr_mem_tracker())); //AddExprCtxToFree(_partition_by_eq_expr_ctx); } - if (_order_by_eq_expr_ctx != NULL) { + if (_order_by_eq_expr_ctx != nullptr) { RETURN_IF_ERROR( _order_by_eq_expr_ctx->prepare(state, cmp_row_desc, expr_mem_tracker())); //AddExprCtxToFree(_order_by_eq_expr_ctx); @@ -219,10 +219,10 @@ Status AnalyticEvalNode::open(RuntimeState* state) { } } - if (_partition_by_eq_expr_ctx != NULL) { + if (_partition_by_eq_expr_ctx != nullptr) { RETURN_IF_ERROR(_partition_by_eq_expr_ctx->open(state)); } - if (_order_by_eq_expr_ctx != NULL) { + if (_order_by_eq_expr_ctx != nullptr) { RETURN_IF_ERROR(_order_by_eq_expr_ctx->open(state)); } @@ -241,7 +241,7 @@ Status AnalyticEvalNode::open(RuntimeState* state) { _curr_child_batch.reset( new RowBatch(child(0)->row_desc(), state->batch_size(), mem_tracker().get())); - while (!_input_eos && _prev_input_row == NULL) { + while (!_input_eos && _prev_input_row == nullptr) { RETURN_IF_ERROR(child(0)->get_next(state, _curr_child_batch.get(), &_input_eos)); if (_curr_child_batch->num_rows() > 0) { _prev_input_row = _curr_child_batch->get_row(0); @@ -252,7 +252,7 @@ Status AnalyticEvalNode::open(RuntimeState* state) { } } - if (_prev_input_row == NULL) { + if (_prev_input_row == nullptr) { DCHECK(_input_eos); // Delete _curr_child_batch to indicate there is no batch to process in GetNext() _curr_child_batch.reset(); @@ -377,7 +377,7 @@ std::string AnalyticEvalNode::debug_state_string(bool detailed) const { void AnalyticEvalNode::add_result_tuple(int64_t stream_idx) { VLOG_ROW << id() << " add_result_tuple idx=" << stream_idx; - DCHECK(_curr_tuple != NULL); + DCHECK(_curr_tuple != nullptr); Tuple* result_tuple = Tuple::create(_result_tuple_desc->byte_size(), _curr_tuple_pool.get()); AggFnEvaluator::get_value(_evaluators, _fn_ctxs, _curr_tuple, result_tuple); @@ -554,7 +554,7 @@ inline void AnalyticEvalNode::init_next_partition(int64_t stream_idx) { _curr_tuple->init(_intermediate_tuple_desc->byte_size()); AggFnEvaluator::init(_evaluators, _fn_ctxs, _curr_tuple); - // Add a result tuple containing values set by Init() (e.g. NULL for sum(), 0 for + // Add a result tuple containing values set by Init() (e.g. nullptr for sum(), 0 for // count()) for output rows that have no input rows in the window. We need to add this // result tuple before any input rows are consumed and the evaluators are updated. if (_fn_scope == ROWS && _window.__isset.window_end && @@ -583,7 +583,7 @@ inline void AnalyticEvalNode::init_next_partition(int64_t stream_idx) { } inline bool AnalyticEvalNode::prev_row_compare(ExprContext* pred_ctx) { - DCHECK(pred_ctx != NULL); + DCHECK(pred_ctx != nullptr); doris_udf::BooleanVal result = pred_ctx->get_boolean_val(_child_tuple_cmp_row); DCHECK(!result.is_null); @@ -594,7 +594,8 @@ Status AnalyticEvalNode::process_child_batches(RuntimeState* state) { // Consume child batches until eos or there are enough rows to return more than an // output batch. Ensuring there is at least one more row left after returning results // allows us to simplify the logic dealing with _last_result_idx and _result_tuples. - while (_curr_child_batch.get() != NULL && num_output_rows_ready() < state->batch_size() + 1) { + while (_curr_child_batch.get() != nullptr && + num_output_rows_ready() < state->batch_size() + 1) { RETURN_IF_CANCELLED(state); //RETURN_IF_ERROR(QueryMaintenance(state)); RETURN_IF_ERROR(process_child_batch(state)); @@ -647,7 +648,7 @@ Status AnalyticEvalNode::process_child_batch(RuntimeState* state) { // row(s) but the incremental state still applies to the current row. bool next_partition = false; - if (_partition_by_eq_expr_ctx != NULL) { + if (_partition_by_eq_expr_ctx != nullptr) { // _partition_by_eq_expr_ctx checks equality over the predicate exprs next_partition = !prev_row_compare(_partition_by_eq_expr_ctx); } @@ -830,7 +831,7 @@ Status AnalyticEvalNode::get_next(RuntimeState* state, RowBatch* row_batch, bool bool output_eos = false; RETURN_IF_ERROR(get_next_output_batch(state, row_batch, &output_eos)); - if (_curr_child_batch.get() == NULL && output_eos) { + if (_curr_child_batch.get() == nullptr && output_eos) { *eos = true; } @@ -857,7 +858,7 @@ Status AnalyticEvalNode::close(RuntimeState* state) { return Status::OK(); } - if (_input_stream.get() != NULL) { + if (_input_stream.get() != nullptr) { _input_stream->close(); } @@ -868,11 +869,11 @@ Status AnalyticEvalNode::close(RuntimeState* state) { // be fewer ctxs than evaluators. We also need to Finalize if _curr_tuple was created // in Open. DCHECK_LE(_fn_ctxs.size(), _evaluators.size()); - DCHECK(_curr_tuple == NULL || _fn_ctxs.size() == _evaluators.size()); + DCHECK(_curr_tuple == nullptr || _fn_ctxs.size() == _evaluators.size()); for (int i = 0; i < _evaluators.size(); ++i) { // Need to make sure finalize is called in case there is any state to clean up. - if (_curr_tuple != NULL) { + if (_curr_tuple != nullptr) { _evaluators[i]->finalize(_fn_ctxs[i], _curr_tuple, _dummy_result_tuple); } @@ -883,27 +884,27 @@ Status AnalyticEvalNode::close(RuntimeState* state) { _fn_ctxs[i]->impl()->close(); } - if (_partition_by_eq_expr_ctx != NULL) { + if (_partition_by_eq_expr_ctx != nullptr) { _partition_by_eq_expr_ctx->close(state); } - if (_order_by_eq_expr_ctx != NULL) { + if (_order_by_eq_expr_ctx != nullptr) { _order_by_eq_expr_ctx->close(state); } - if (_prev_child_batch.get() != NULL) { + if (_prev_child_batch.get() != nullptr) { _prev_child_batch.reset(); } - if (_curr_child_batch.get() != NULL) { + if (_curr_child_batch.get() != nullptr) { _curr_child_batch.reset(); } - if (_curr_tuple_pool.get() != NULL) { + if (_curr_tuple_pool.get() != nullptr) { _curr_tuple_pool->free_all(); } - if (_prev_tuple_pool.get() != NULL) { + if (_prev_tuple_pool.get() != nullptr) { _prev_tuple_pool->free_all(); } - if (_mem_pool.get() != NULL) { + if (_mem_pool.get() != nullptr) { _mem_pool->free_all(); } ExecNode::close(state); @@ -915,11 +916,11 @@ void AnalyticEvalNode::debug_string(int indentation_level, std::stringstream* ou *out << "AnalyticEvalNode(" << " window=" << debug_window_string(); - if (_partition_by_eq_expr_ctx != NULL) { + if (_partition_by_eq_expr_ctx != nullptr) { // *out << " partition_exprs=" << _partition_by_eq_expr_ctx->debug_string(); } - if (_order_by_eq_expr_ctx != NULL) { + if (_order_by_eq_expr_ctx != nullptr) { // *out << " order_by_exprs=" << _order_by_eq_expr_ctx->debug_string(); } diff --git a/be/src/exec/analytic_eval_node.h b/be/src/exec/analytic_eval_node.h index 8c6d7ad2e5..0c3863eba7 100644 --- a/be/src/exec/analytic_eval_node.h +++ b/be/src/exec/analytic_eval_node.h @@ -192,13 +192,13 @@ private: const TupleDescriptor* _result_tuple_desc; // Tuple descriptor of the buffered tuple (identical to the input child tuple, which is - // assumed to come from a single SortNode). NULL if both partition_exprs and + // assumed to come from a single SortNode). nullptr if both partition_exprs and // order_by_exprs are empty. TupleDescriptor* _buffered_tuple_desc; // TupleRow* composed of the first child tuple and the buffered tuple, used by // _partition_by_eq_expr_ctx and _order_by_eq_expr_ctx. Set in prepare() if - // _buffered_tuple_desc is not NULL, allocated from _mem_pool. + // _buffered_tuple_desc is not nullptr, allocated from _mem_pool. TupleRow* _child_tuple_cmp_row; // Expr context for a predicate that checks if child tuple '<' buffered tuple for @@ -264,8 +264,8 @@ private: // MAX_TUPLE_POOL_SIZE bytes. Resources from _prev_tuple_pool are transferred to an // output row batch when all result tuples it contains have been returned and all // window tuples it contains are no longer needed. - boost::scoped_ptr _curr_tuple_pool; - boost::scoped_ptr _prev_tuple_pool; + std::unique_ptr _curr_tuple_pool; + std::unique_ptr _prev_tuple_pool; // The index of the last row from _input_stream associated with output row containing // resources in _prev_tuple_pool. -1 when the pool is empty. Resources from @@ -303,8 +303,8 @@ private: // process_child_batch(). The prev batch is Reset() after calling process_child_batch() // and then swapped with the curr batch so the RowBatch owning _prev_input_row is // stored in _prev_child_batch for the next call to process_child_batch(). - boost::scoped_ptr _prev_child_batch; - boost::scoped_ptr _curr_child_batch; + std::unique_ptr _prev_child_batch; + std::unique_ptr _curr_child_batch; // Block manager client used by _input_stream. Not owned. BufferedBlockMgr2::Client* _block_mgr_client; @@ -317,10 +317,10 @@ private: // buffered data exceeds the available memory in the underlying BufferedBlockMgr, // _input_stream is unpinned (i.e., possibly spilled to disk if necessary). // TODO: Consider re-pinning unpinned streams when possible. - boost::scoped_ptr _input_stream; + std::unique_ptr _input_stream; // Pool used for O(1) allocations that live until close. - boost::scoped_ptr _mem_pool; + std::unique_ptr _mem_pool; // True when there are no more input rows to consume from our child. bool _input_eos; diff --git a/be/src/exec/blocking_join_node.cpp b/be/src/exec/blocking_join_node.cpp index a82e5f7ce4..da977016fb 100644 --- a/be/src/exec/blocking_join_node.cpp +++ b/be/src/exec/blocking_join_node.cpp @@ -38,7 +38,7 @@ Status BlockingJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { BlockingJoinNode::~BlockingJoinNode() { // _left_batch must be cleaned up in close() to ensure proper resource freeing. - DCHECK(_left_batch == NULL); + DCHECK(_left_batch == nullptr); } Status BlockingJoinNode::prepare(RuntimeState* state) { @@ -80,7 +80,7 @@ Status BlockingJoinNode::close(RuntimeState* state) { return Status::OK(); } -void BlockingJoinNode::build_side_thread(RuntimeState* state, boost::promise* status) { +void BlockingJoinNode::build_side_thread(RuntimeState* state, std::promise* status) { status->set_value(construct_build_side(state)); } @@ -99,10 +99,11 @@ Status BlockingJoinNode::open(RuntimeState* state) { // thread, so that the left child can do any initialisation in parallel. // Only do this if we can get a thread token. Otherwise, do this in the // main thread - boost::promise build_side_status; + std::promise build_side_status; add_runtime_exec_option("Join Build-Side Prepared Asynchronously"); - boost::thread(bind(&BlockingJoinNode::build_side_thread, this, state, &build_side_status)); + std::thread(bind(&BlockingJoinNode::build_side_thread, this, state, &build_side_status)) + .detach(); // Open the left child so that it may perform any initialisation in parallel. // Don't exit even if we see an error, we still need to wait for the build thread @@ -126,7 +127,7 @@ Status BlockingJoinNode::open(RuntimeState* state) { if (_left_batch->num_rows() == 0) { if (_left_side_eos) { - init_get_next(NULL /* eos */); + init_get_next(nullptr /* eos */); _eos = true; break; } @@ -166,7 +167,7 @@ std::string BlockingJoinNode::get_left_child_row_string(TupleRow* row) { std::find(_build_tuple_idx_ptr, _build_tuple_idx_ptr + _build_tuple_size, i); if (is_build_tuple != _build_tuple_idx_ptr + _build_tuple_size) { - out << Tuple::to_string(NULL, *row_desc().tuple_descriptors()[i]); + out << Tuple::to_string(nullptr, *row_desc().tuple_descriptors()[i]); } else { out << Tuple::to_string(row->get_tuple(i), *row_desc().tuple_descriptors()[i]); } @@ -179,13 +180,13 @@ std::string BlockingJoinNode::get_left_child_row_string(TupleRow* row) { // This function is replaced by codegen void BlockingJoinNode::create_output_row(TupleRow* out, TupleRow* left, TupleRow* build) { uint8_t* out_ptr = reinterpret_cast(out); - if (left == NULL) { + if (left == nullptr) { memset(out_ptr, 0, _probe_tuple_row_size); } else { memcpy(out_ptr, left, _probe_tuple_row_size); } - if (build == NULL) { + if (build == nullptr) { memset(out_ptr + _probe_tuple_row_size, 0, _build_tuple_row_size); } else { memcpy(out_ptr + _probe_tuple_row_size, build, _build_tuple_row_size); diff --git a/be/src/exec/blocking_join_node.h b/be/src/exec/blocking_join_node.h index f0406d3d8f..f5ae609791 100644 --- a/be/src/exec/blocking_join_node.h +++ b/be/src/exec/blocking_join_node.h @@ -18,9 +18,9 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_BLOCKING_JOIN_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_BLOCKING_JOIN_NODE_H -#include -#include +#include #include +#include #include "exec/exec_node.h" #include "gen_cpp/PlanNodes_types.h" @@ -60,13 +60,13 @@ public: private: const std::string _node_name; TJoinOp::type _join_op; - bool _eos; // if true, nothing left to return in get_next() - boost::scoped_ptr _build_pool; // holds everything referenced from build side + bool _eos; // if true, nothing left to return in get_next() + std::unique_ptr _build_pool; // holds everything referenced from build side // _left_batch must be cleared before calling get_next(). The child node // does not initialize all tuple ptrs in the row, only the ones that it // is responsible for. - boost::scoped_ptr _left_batch; + std::unique_ptr _left_batch; int _left_batch_pos; // current scan pos in _left_batch bool _left_side_eos; // if true, left child has no more rows to process TupleRow* _current_left_child_row; @@ -91,7 +91,7 @@ private: // Init the build-side state for a new left child row (e.g. hash table iterator or list // iterator) given the first row. Used in open() to prepare for get_next(). - // A NULL ptr for first_left_child_row indicates the left child eos. + // A nullptr ptr for first_left_child_row indicates the left child eos. virtual void init_get_next(TupleRow* first_left_child_row) = 0; // We parallelize building the build-side with Opening the @@ -110,7 +110,7 @@ private: // Returns a debug string for the left child's 'row'. They have tuple ptrs that are // uninitialized; the left child only populates the tuple ptrs it is responsible // for. This function outputs just the row values and leaves the build - // side values as NULL. + // side values as nullptr. // This is only used for debugging and outputting the left child rows before // doing the join. std::string get_left_child_row_string(TupleRow* row); @@ -125,7 +125,7 @@ private: private: // Supervises ConstructBuildSide in a separate thread, and returns its status in the // promise parameter. - void build_side_thread(RuntimeState* state, boost::promise* status); + void build_side_thread(RuntimeState* state, std::promise* status); }; } // namespace doris diff --git a/be/src/exec/broker_scan_node.h b/be/src/exec/broker_scan_node.h index 6c4f40e7ca..a05e7c5e6e 100644 --- a/be/src/exec/broker_scan_node.h +++ b/be/src/exec/broker_scan_node.h @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -84,8 +85,7 @@ private: // Scan one range Status scanner_scan(const TBrokerScanRange& scan_range, const std::vector& pre_filter_ctxs, - const std::vector& conjunct_ctxs, - ScannerCounter* counter); + const std::vector& conjunct_ctxs, ScannerCounter* counter); std::unique_ptr create_scanner(const TBrokerScanRange& scan_range, const std::vector& pre_filter_ctxs, diff --git a/be/src/exec/broker_scanner.cpp b/be/src/exec/broker_scanner.cpp index e99bf7e0a2..b265eb0528 100644 --- a/be/src/exec/broker_scanner.cpp +++ b/be/src/exec/broker_scanner.cpp @@ -24,6 +24,7 @@ #include "exec/buffered_reader.h" #include "exec/decompressor.h" #include "exec/exec_node.h" +#include "exec/hdfs_reader_writer.h" #include "exec/local_file_reader.h" #include "exec/plain_binary_line_reader.h" #include "exec/plain_text_line_reader.h" @@ -40,8 +41,6 @@ #include "runtime/tuple.h" #include "util/utf8_check.h" -#include "exec/hdfs_reader_writer.h" - namespace doris { BrokerScanner::BrokerScanner(RuntimeState* state, RuntimeProfile* profile, @@ -167,7 +166,8 @@ Status BrokerScanner::open_file_reader() { } case TFileType::FILE_HDFS: { FileReader* hdfs_file_reader; - RETURN_IF_ERROR(HdfsReaderWriter::create_reader(range.hdfs_params, range.path, start_offset, &hdfs_file_reader)); + RETURN_IF_ERROR(HdfsReaderWriter::create_reader(range.hdfs_params, range.path, start_offset, + &hdfs_file_reader)); BufferedReader* file_reader = new BufferedReader(_profile, hdfs_file_reader); RETURN_IF_ERROR(file_reader->open()); _cur_file_reader = file_reader; @@ -182,8 +182,8 @@ Status BrokerScanner::open_file_reader() { break; } case TFileType::FILE_S3: { - BufferedReader* s3_reader = - new BufferedReader(_profile, new S3Reader(_params.properties, range.path, start_offset)); + BufferedReader* s3_reader = new BufferedReader( + _profile, new S3Reader(_params.properties, range.path, start_offset)); RETURN_IF_ERROR(s3_reader->open()); _cur_file_reader = s3_reader; break; @@ -271,7 +271,7 @@ Status BrokerScanner::open_line_reader() { } // create decompressor. - // _decompressor may be NULL if this is not a compressed file + // _decompressor may be nullptr if this is not a compressed file RETURN_IF_ERROR(create_decompressor(range.format_type)); _file_format_type = range.format_type; @@ -337,7 +337,7 @@ void BrokerScanner::split_line(const Slice& line) { delete row; delete ptr; } else { - const char *value = line.data; + const char* value = line.data; size_t start = 0; // point to the start pos of next col value. size_t curpos = 0; // point to the start pos of separator matching sequence. size_t p1 = 0; // point to the current pos of separator matching sequence. @@ -466,7 +466,8 @@ bool BrokerScanner::convert_one_row(const Slice& line, Tuple* tuple, MemPool* tu // Convert one row to this tuple bool BrokerScanner::line_to_src_tuple(const Slice& line) { - if (_file_format_type != TFileFormatType::FORMAT_PROTO && !validate_utf8(line.data, line.size)) { + if (_file_format_type != TFileFormatType::FORMAT_PROTO && + !validate_utf8(line.data, line.size)) { std::stringstream error_msg; error_msg << "data is not encoded by UTF-8"; _state->append_error_msg_to_file("Unable to display", error_msg.str()); diff --git a/be/src/exec/cross_join_node.h b/be/src/exec/cross_join_node.h index 76bca882c3..6200e05fad 100644 --- a/be/src/exec/cross_join_node.h +++ b/be/src/exec/cross_join_node.h @@ -18,9 +18,8 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_CROSS_JOIN_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_CROSS_JOIN_NODE_H -#include -#include #include +#include #include #include "exec/blocking_join_node.h" @@ -55,7 +54,7 @@ protected: private: // Object pool for build RowBatches, stores all BuildBatches in _build_rows - boost::scoped_ptr _build_batch_pool; + std::unique_ptr _build_batch_pool; // List of build batches, constructed in prepare() RowBatchList _build_batches; RowBatchList::TupleRowIterator _current_build_row; diff --git a/be/src/exec/csv_scan_node.h b/be/src/exec/csv_scan_node.h index 5f830e8302..fe92247cd9 100644 --- a/be/src/exec/csv_scan_node.h +++ b/be/src/exec/csv_scan_node.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_CSV_SCAN_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_CSV_SCAN_NODE_H -#include #include #include @@ -110,11 +109,11 @@ private: int _slot_num; // Pool for allocating tuple data, including all varying-length slots. - boost::scoped_ptr _tuple_pool; + std::unique_ptr _tuple_pool; // Util class for doing real file reading - boost::scoped_ptr _csv_scanner; + std::unique_ptr _csv_scanner; // Helper class for converting text to other types; - boost::scoped_ptr _text_converter; + std::unique_ptr _text_converter; // Current tuple. Tuple* _tuple; // Current RuntimeState diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp index 7b2e3365bb..aefa087f45 100644 --- a/be/src/exec/data_sink.cpp +++ b/be/src/exec/data_sink.cpp @@ -32,8 +32,8 @@ #include "runtime/memory_scratch_sink.h" #include "runtime/mysql_table_sink.h" #include "runtime/odbc_table_sink.h" -#include "runtime/result_sink.h" #include "runtime/result_file_sink.h" +#include "runtime/result_sink.h" #include "runtime/runtime_state.h" #include "util/logging.h" @@ -42,11 +42,9 @@ namespace doris { Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, const std::vector& output_exprs, const TPlanFragmentExecParams& params, - const RowDescriptor& row_desc, - bool is_vec, - boost::scoped_ptr* sink, - DescriptorTbl& desc_tbl) { - DataSink* tmp_sink = NULL; + const RowDescriptor& row_desc, bool is_vec, + std::unique_ptr* sink, DescriptorTbl& desc_tbl) { + DataSink* tmp_sink = nullptr; switch (thrift_sink.type) { case TDataSinkType::DATA_STREAM_SINK: { @@ -60,9 +58,9 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink // TODO: figure out good buffer size based on size of output row if (is_vec) { } else { - tmp_sink = new DataStreamSender(pool, params.sender_id, row_desc, thrift_sink.stream_sink, - params.destinations, 16 * 1024, - send_query_statistics_with_every_batch); + tmp_sink = new DataStreamSender(pool, params.sender_id, row_desc, + thrift_sink.stream_sink, params.destinations, 16 * 1024, + send_query_statistics_with_every_batch); } // RETURN_IF_ERROR(sender->prepare(state->obj_pool(), thrift_sink.stream_sink)); sink->reset(tmp_sink); @@ -171,7 +169,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink } } - if (sink->get() != NULL) { + if (sink->get() != nullptr) { RETURN_IF_ERROR((*sink)->init(thrift_sink)); } diff --git a/be/src/exec/data_sink.h b/be/src/exec/data_sink.h index 8d1bb6497e..30f5580934 100644 --- a/be/src/exec/data_sink.h +++ b/be/src/exec/data_sink.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_DATA_SINK_H #define DORIS_BE_SRC_QUERY_EXEC_DATA_SINK_H -#include #include #include "common/status.h" @@ -74,10 +73,8 @@ public: static Status create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, const std::vector& output_exprs, const TPlanFragmentExecParams& params, - const RowDescriptor& row_desc, - bool is_vec, - boost::scoped_ptr* sink, - DescriptorTbl& desc_tbl); + const RowDescriptor& row_desc, bool is_vec, + std::unique_ptr* sink, DescriptorTbl& desc_tbl); // Returns the runtime profile for the sink. virtual RuntimeProfile* profile() = 0; diff --git a/be/src/exec/decompressor.cpp b/be/src/exec/decompressor.cpp index fe805a9524..2f215f4409 100644 --- a/be/src/exec/decompressor.cpp +++ b/be/src/exec/decompressor.cpp @@ -112,7 +112,7 @@ Status GzipDecompressor::decompress(uint8_t* input, size_t input_len, size_t* in *decompressed_len = output_max_len - _z_strm.avail_out; VLOG_TRACE << "gzip dec ret: " << ret << " input_bytes_read: " << *input_bytes_read - << " decompressed_len: " << *decompressed_len; + << " decompressed_len: " << *decompressed_len; if (ret == Z_BUF_ERROR) { // Z_BUF_ERROR indicates that inflate() could not consume more input or @@ -295,7 +295,7 @@ Status Lz4FrameDecompressor::decompress(uint8_t* input, size_t input_len, size_t // decompress size_t output_len = output_max_len; ret = LZ4F_decompress(_dctx, (void*)output, &output_len, (void*)src, &src_size, - /* LZ4F_decompressOptions_t */ NULL); + /* LZ4F_decompressOptions_t */ nullptr); if (LZ4F_isError(ret)) { std::stringstream ss; ss << "Decompression error: " << std::string(LZ4F_getErrorName(ret)); diff --git a/be/src/exec/es/es_predicate.cpp b/be/src/exec/es/es_predicate.cpp index 170c0e9cc1..63add75e50 100644 --- a/be/src/exec/es/es_predicate.cpp +++ b/be/src/exec/es/es_predicate.cpp @@ -257,7 +257,7 @@ Status EsPredicate::build_disjuncts_list(const Expr* conjunct) { return Status::InternalError("build disjuncts failed: expr is not literal type"); } - ExtLiteral literal(expr->type().type, _context->get_value(expr, NULL)); + ExtLiteral literal(expr->type().type, _context->get_value(expr, nullptr)); std::string col = slot_desc->col_name(); if (_field_context.find(col) != _field_context.end()) { col = _field_context[col]; @@ -276,7 +276,7 @@ Status EsPredicate::build_disjuncts_list(const Expr* conjunct) { return Status::InternalError("build disjuncts failed: number of children is not 2"); } Expr* expr = conjunct->get_child(1); - ExtLiteral literal(expr->type().type, _context->get_value(expr, NULL)); + ExtLiteral literal(expr->type().type, _context->get_value(expr, nullptr)); std::vector query_conditions; query_conditions.emplace_back(literal); std::vector cols; @@ -339,7 +339,7 @@ Status EsPredicate::build_disjuncts_list(const Expr* conjunct) { if (_field_context.find(col) != _field_context.end()) { col = _field_context[col]; } - ExtLiteral literal(type, _context->get_value(expr, NULL)); + ExtLiteral literal(type, _context->get_value(expr, nullptr)); ExtPredicate* predicate = new ExtLikePredicate(TExprNodeType::LIKE_PRED, col, slot_desc->type(), literal); diff --git a/be/src/exec/es/es_scroll_parser.cpp b/be/src/exec/es/es_scroll_parser.cpp index 44ca493e63..396c0869d6 100644 --- a/be/src/exec/es/es_scroll_parser.cpp +++ b/be/src/exec/es/es_scroll_parser.cpp @@ -287,14 +287,14 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, Tuple* tuple, } tuple->set_not_null(slot_desc->null_indicator_offset()); void* slot = tuple->get_slot(slot_desc->tuple_offset()); - // obj[FIELD_ID] must not be NULL + // obj[FIELD_ID] must not be nullptr std::string _id = obj[FIELD_ID].GetString(); size_t len = _id.length(); char* buffer = reinterpret_cast(tuple_pool->try_allocate_unaligned(len)); - if (UNLIKELY(buffer == NULL)) { + if (UNLIKELY(buffer == nullptr)) { std::string details = strings::Substitute(ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow", len, "string slot"); - return tuple_pool->mem_tracker()->MemLimitExceeded(NULL, details, len); + return tuple_pool->mem_tracker()->MemLimitExceeded(nullptr, details, len); } memcpy(buffer, _id.data(), len); reinterpret_cast(slot)->ptr = buffer; @@ -349,10 +349,10 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, Tuple* tuple, } size_t val_size = val.length(); char* buffer = reinterpret_cast(tuple_pool->try_allocate_unaligned(val_size)); - if (UNLIKELY(buffer == NULL)) { + if (UNLIKELY(buffer == nullptr)) { std::string details = strings::Substitute( ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow", val_size, "string slot"); - return tuple_pool->mem_tracker()->MemLimitExceeded(NULL, details, val_size); + return tuple_pool->mem_tracker()->MemLimitExceeded(nullptr, details, val_size); } memcpy(buffer, val.data(), val_size); reinterpret_cast(slot)->ptr = buffer; diff --git a/be/src/exec/es_http_scan_node.cpp b/be/src/exec/es_http_scan_node.cpp index 7a78787721..012110d80e 100644 --- a/be/src/exec/es_http_scan_node.cpp +++ b/be/src/exec/es_http_scan_node.cpp @@ -121,8 +121,8 @@ Status EsHttpScanNode::open(RuntimeState* state) { // if conjunct is constant, compute direct and set eos = true for (int conj_idx = 0; conj_idx < _conjunct_ctxs.size(); ++conj_idx) { if (_conjunct_ctxs[conj_idx]->root()->is_constant()) { - void* value = _conjunct_ctxs[conj_idx]->get_value(NULL); - if (value == NULL || *reinterpret_cast(value) == false) { + void* value = _conjunct_ctxs[conj_idx]->get_value(nullptr); + if (value == nullptr || *reinterpret_cast(value) == false) { _eos = true; } } diff --git a/be/src/exec/es_scan_node.cpp b/be/src/exec/es_scan_node.cpp index b40ad099f6..fad266993b 100644 --- a/be/src/exec/es_scan_node.cpp +++ b/be/src/exec/es_scan_node.cpp @@ -563,7 +563,7 @@ bool EsScanNode::to_ext_literal(ExprContext* context, Expr* expr, TExtLiteral* l case TExprNodeType::DECIMAL_LITERAL: case TExprNodeType::STRING_LITERAL: case TExprNodeType::DATE_LITERAL: - return to_ext_literal(expr->type().type, context->get_value(expr, NULL), literal); + return to_ext_literal(expr->type().type, context->get_value(expr, nullptr), literal); default: return false; } @@ -772,10 +772,10 @@ Status EsScanNode::materialize_row(MemPool* tuple_pool, Tuple* tuple, const string& val = col.string_vals[val_idx]; size_t val_size = val.size(); char* buffer = reinterpret_cast(tuple_pool->try_allocate_unaligned(val_size)); - if (UNLIKELY(buffer == NULL)) { + if (UNLIKELY(buffer == nullptr)) { std::string details = strings::Substitute( ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow", val_size, "string slot"); - return tuple_pool->mem_tracker()->MemLimitExceeded(NULL, details, val_size); + return tuple_pool->mem_tracker()->MemLimitExceeded(nullptr, details, val_size); } memcpy(buffer, val.data(), val_size); reinterpret_cast(slot)->ptr = buffer; diff --git a/be/src/exec/exchange_node.cpp b/be/src/exec/exchange_node.cpp index 14386e8f16..14299c65a6 100644 --- a/be/src/exec/exchange_node.cpp +++ b/be/src/exec/exchange_node.cpp @@ -17,8 +17,6 @@ #include "exec/exchange_node.h" -#include - #include "gen_cpp/PlanNodes_types.h" #include "runtime/data_stream_mgr.h" #include "runtime/data_stream_recvr.h" @@ -32,7 +30,7 @@ namespace doris { ExchangeNode::ExchangeNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : ExecNode(pool, tnode, descs), _num_senders(0), - _stream_recvr(NULL), + _stream_recvr(nullptr), _input_row_desc(descs, tnode.exchange_node.input_row_tuples, std::vector(tnode.nullable_tuples.begin(), tnode.nullable_tuples.begin() + @@ -84,7 +82,8 @@ Status ExchangeNode::open(RuntimeState* state) { // create_merger() will populate its merging heap with batches from the _stream_recvr, // so it is not necessary to call fill_input_row_batch(). if (state->enable_exchange_node_parallel_merge()) { - RETURN_IF_ERROR(_stream_recvr->create_parallel_merger(less_than, state->batch_size(), mem_tracker().get())); + RETURN_IF_ERROR(_stream_recvr->create_parallel_merger(less_than, state->batch_size(), + mem_tracker().get())); } else { RETURN_IF_ERROR(_stream_recvr->create_merger(less_than)); } @@ -107,7 +106,7 @@ Status ExchangeNode::close(RuntimeState* state) { if (_is_merging) { _sort_exec_exprs.close(state); } - if (_stream_recvr != NULL) { + if (_stream_recvr != nullptr) { _stream_recvr->close(); } // _stream_recvr.reset(); @@ -121,8 +120,8 @@ Status ExchangeNode::fill_input_row_batch(RuntimeState* state) { // SCOPED_TIMER(state->total_network_receive_timer()); ret_status = _stream_recvr->get_batch(&_input_batch); } - VLOG_FILE << "exch: has batch=" << (_input_batch == NULL ? "false" : "true") - << " #rows=" << (_input_batch != NULL ? _input_batch->num_rows() : 0) + VLOG_FILE << "exch: has batch=" << (_input_batch == nullptr ? "false" : "true") + << " #rows=" << (_input_batch != nullptr ? _input_batch->num_rows() : 0) << " is_cancelled=" << (ret_status.is_cancelled() ? "true" : "false") << " instance_id=" << state->fragment_instance_id(); return ret_status; @@ -152,7 +151,7 @@ Status ExchangeNode::get_next(RuntimeState* state, RowBatch* output_batch, bool* SCOPED_TIMER(_convert_row_batch_timer); RETURN_IF_CANCELLED(state); // copy rows until we hit the limit/capacity or until we exhaust _input_batch - while (!reached_limit() && !output_batch->at_capacity() && _input_batch != NULL && + while (!reached_limit() && !output_batch->at_capacity() && _input_batch != nullptr && _next_row_idx < _input_batch->num_rows()) { TupleRow* src = _input_batch->get_row(_next_row_idx); @@ -191,12 +190,12 @@ Status ExchangeNode::get_next(RuntimeState* state, RowBatch* output_batch, bool* } // we need more rows - if (_input_batch != NULL) { + if (_input_batch != nullptr) { _input_batch->transfer_resource_ownership(output_batch); } RETURN_IF_ERROR(fill_input_row_batch(state)); - *eos = (_input_batch == NULL); + *eos = (_input_batch == nullptr); if (*eos) { return Status::OK(); } diff --git a/be/src/exec/exchange_node.h b/be/src/exec/exchange_node.h index 7551d88699..6077dd4651 100644 --- a/be/src/exec/exchange_node.h +++ b/be/src/exec/exchange_node.h @@ -18,8 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_EXCHANGE_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_EXCHANGE_NODE_H -#include - #include "exec/exec_node.h" #include "exec/sort_exec_exprs.h" #include "runtime/data_stream_recvr.h" @@ -72,7 +70,7 @@ private: int _num_senders; // needed for _stream_recvr construction // created in prepare() and owned by the RuntimeState - boost::shared_ptr _stream_recvr; + std::shared_ptr _stream_recvr; // our input rows are a prefix of the rows we produce RowDescriptor _input_row_desc; @@ -82,7 +80,7 @@ private: // Current batch of rows from the receiver queue being processed by this node. // Only valid if _is_merging is false. (If _is_merging is true, GetNext() is // delegated to the receiver). Owned by the stream receiver. - // boost::scoped_ptr _input_batch; + // std::unique_ptr _input_batch; RowBatch* _input_batch = nullptr; // Next row to copy from _input_batch. For non-merging exchanges, _input_batch diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index 18b5af196a..2e0249829c 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -89,16 +89,16 @@ bool ExecNode::RowBatchQueue::AddBatchWithTimeout(RowBatch* batch, int64_t timeo } RowBatch* ExecNode::RowBatchQueue::GetBatch() { - RowBatch* result = NULL; + RowBatch* result = nullptr; if (blocking_get(&result)) return result; - return NULL; + return nullptr; } int ExecNode::RowBatchQueue::Cleanup() { int num_io_buffers = 0; - // RowBatch* batch = NULL; - // while ((batch = GetBatch()) != NULL) { + // RowBatch* batch = nullptr; + // while ((batch = GetBatch()) != nullptr) { // num_io_buffers += batch->num_io_buffers(); // delete batch; // } @@ -124,9 +124,9 @@ ExecNode::ExecNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl _debug_action(TDebugAction::WAIT), _limit(tnode.limit), _num_rows_returned(0), - _rows_returned_counter(NULL), - _rows_returned_rate(NULL), - _memory_used_counter(NULL), + _rows_returned_counter(nullptr), + _rows_returned_rate(nullptr), + _memory_used_counter(nullptr), _is_closed(false) {} ExecNode::~ExecNode() {} @@ -174,7 +174,7 @@ Status ExecNode::init(const TPlanNode& tnode, RuntimeState* state) { Status ExecNode::prepare(RuntimeState* state) { RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::PREPARE)); - DCHECK(_runtime_profile.get() != NULL); + DCHECK(_runtime_profile.get() != nullptr); _rows_returned_counter = ADD_COUNTER(_runtime_profile, "RowsReturned", TUnit::UNIT); _rows_returned_rate = runtime_profile()->add_derived_counter( ROW_THROUGHPUT_COUNTER, TUnit::UNIT_PER_SECOND, @@ -227,7 +227,7 @@ Status ExecNode::close(RuntimeState* state) { _is_closed = true; RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::CLOSE)); - if (_rows_returned_counter != NULL) { + if (_rows_returned_counter != nullptr) { COUNTER_SET(_rows_returned_counter, _num_rows_returned); } @@ -271,12 +271,12 @@ void ExecNode::add_runtime_exec_option(const std::string& str) { Status ExecNode::create_tree(RuntimeState* state, ObjectPool* pool, const TPlan& plan, const DescriptorTbl& descs, ExecNode** root) { if (plan.nodes.size() == 0) { - *root = NULL; + *root = nullptr; return Status::OK(); } int node_idx = 0; - RETURN_IF_ERROR(create_tree_helper(state, pool, plan.nodes, descs, NULL, &node_idx, root)); + RETURN_IF_ERROR(create_tree_helper(state, pool, plan.nodes, descs, nullptr, &node_idx, root)); if (node_idx + 1 != plan.nodes.size()) { // TODO: print thrift msg for diagnostic purposes. @@ -299,11 +299,11 @@ Status ExecNode::create_tree_helper(RuntimeState* state, ObjectPool* pool, const TPlanNode& tnode = tnodes[*node_idx]; int num_children = tnodes[*node_idx].num_children; - ExecNode* node = NULL; + ExecNode* node = nullptr; RETURN_IF_ERROR(create_node(state, pool, tnodes[*node_idx], descs, &node)); - // assert(parent != NULL || (node_idx == 0 && root_expr != NULL)); - if (parent != NULL) { + // assert(parent != nullptr || (node_idx == 0 && root_expr != nullptr)); + if (parent != nullptr) { parent->_children.push_back(node); } else { *root = node; @@ -311,7 +311,7 @@ Status ExecNode::create_tree_helper(RuntimeState* state, ObjectPool* pool, for (int i = 0; i < num_children; i++) { ++*node_idx; - RETURN_IF_ERROR(create_tree_helper(state, pool, tnodes, descs, node, node_idx, NULL)); + RETURN_IF_ERROR(create_tree_helper(state, pool, tnodes, descs, node, node_idx, nullptr)); // we are expecting a child, but have used all nodes // this means we have been given a bad tree and must fail @@ -326,11 +326,11 @@ Status ExecNode::create_tree_helper(RuntimeState* state, ObjectPool* pool, // build up tree of profiles; add children >0 first, so that when we print // the profile, child 0 is printed last (makes the output more readable) for (int i = 1; i < node->_children.size(); ++i) { - node->runtime_profile()->add_child(node->_children[i]->runtime_profile(), true, NULL); + node->runtime_profile()->add_child(node->_children[i]->runtime_profile(), true, nullptr); } if (!node->_children.empty()) { - node->runtime_profile()->add_child(node->_children[0]->runtime_profile(), true, NULL); + node->runtime_profile()->add_child(node->_children[0]->runtime_profile(), true, nullptr); } return Status::OK(); diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index fd6a7a1b82..63f048e252 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -248,9 +248,9 @@ protected: /// managed externally. bool AddBatchWithTimeout(RowBatch* batch, int64_t timeout_micros); - /// Gets a row batch from the queue. Returns NULL if there are no more. + /// Gets a row batch from the queue. Returns nullptr if there are no more. /// This function blocks. - /// Returns NULL after Shutdown(). + /// Returns nullptr after Shutdown(). RowBatch* GetBatch(); /// Deletes all row batches in cleanup_queue_. Not valid to call AddBatch() @@ -289,7 +289,7 @@ protected: int64_t _limit; // -1: no limit int64_t _num_rows_returned; - boost::scoped_ptr _runtime_profile; + std::unique_ptr _runtime_profile; /// Account for peak memory used by this node std::shared_ptr _mem_tracker; @@ -299,7 +299,7 @@ protected: /// MemPool for allocating data structures used by expression evaluators in this node. /// Created in Prepare(). - boost::scoped_ptr _expr_mem_pool; + std::unique_ptr _expr_mem_pool; RuntimeProfile::Counter* _rows_returned_counter; RuntimeProfile::Counter* _rows_returned_rate; @@ -323,7 +323,7 @@ protected: bool is_closed() const { return _is_closed; } // TODO(zc) - /// Pointer to the containing SubplanNode or NULL if not inside a subplan. + /// Pointer to the containing SubplanNode or nullptr if not inside a subplan. /// Set by SubplanNode::Init(). Not owned. // SubplanNode* containing_subplan_; diff --git a/be/src/exec/file_reader.h b/be/src/exec/file_reader.h index 447b3f547e..bcb72805c0 100644 --- a/be/src/exec/file_reader.h +++ b/be/src/exec/file_reader.h @@ -39,7 +39,7 @@ public: /** * This interface is used read a whole message, For example: read a message from kafka. * - * if read eof then return Status::OK and length is set 0 and buf is set NULL, + * if read eof then return Status::OK and length is set 0 and buf is set nullptr, * other return readed bytes. */ virtual Status read_one_message(std::unique_ptr* buf, int64_t* length) = 0; diff --git a/be/src/exec/hash_join_node.cpp b/be/src/exec/hash_join_node.cpp index b1b9356e65..551c4b24e8 100644 --- a/be/src/exec/hash_join_node.cpp +++ b/be/src/exec/hash_join_node.cpp @@ -39,8 +39,8 @@ HashJoinNode::HashJoinNode(ObjectPool* pool, const TPlanNode& tnode, const Descr _join_op(tnode.hash_join_node.join_op), _probe_counter(0), _probe_eos(false), - _process_probe_batch_fn(NULL), - _anti_join_last_pos(NULL) { + _process_probe_batch_fn(nullptr), + _anti_join_last_pos(nullptr) { _match_all_probe = (_join_op == TJoinOp::LEFT_OUTER_JOIN || _join_op == TJoinOp::FULL_OUTER_JOIN); _match_one_build = (_join_op == TJoinOp::LEFT_SEMI_JOIN); @@ -53,7 +53,7 @@ HashJoinNode::HashJoinNode(ObjectPool* pool, const TPlanNode& tnode, const Descr HashJoinNode::~HashJoinNode() { // _probe_batch must be cleaned up in close() to ensure proper resource freeing. - DCHECK(_probe_batch == NULL); + DCHECK(_probe_batch == nullptr); } Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { @@ -62,7 +62,7 @@ Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { const std::vector& eq_join_conjuncts = tnode.hash_join_node.eq_join_conjuncts; for (int i = 0; i < eq_join_conjuncts.size(); ++i) { - ExprContext* ctx = NULL; + ExprContext* ctx = nullptr; RETURN_IF_ERROR(Expr::create_expr_tree(_pool, eq_join_conjuncts[i].left, &ctx)); _probe_expr_ctxs.push_back(ctx); RETURN_IF_ERROR(Expr::create_expr_tree(_pool, eq_join_conjuncts[i].right, &ctx)); @@ -158,12 +158,12 @@ Status HashJoinNode::close(RuntimeState* state) { RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::CLOSE)); // Must reset _probe_batch in close() to release resources - _probe_batch.reset(NULL); + _probe_batch.reset(nullptr); - if (_hash_tbl.get() != NULL) { + if (_hash_tbl.get() != nullptr) { _hash_tbl->close(); } - if (_build_pool.get() != NULL) { + if (_build_pool.get() != nullptr) { _build_pool->free_all(); } @@ -179,7 +179,7 @@ Status HashJoinNode::close(RuntimeState* state) { return ExecNode::close(state); } -void HashJoinNode::build_side_thread(RuntimeState* state, boost::promise* status) { +void HashJoinNode::build_side_thread(RuntimeState* state, std::promise* status) { status->set_value(construct_hash_table(state)); } @@ -233,9 +233,9 @@ Status HashJoinNode::open(RuntimeState* state) { // thread, so that the left child can do any initialisation in parallel. // Only do this if we can get a thread token. Otherwise, do this in the // main thread - boost::promise thread_status; + std::promise thread_status; add_runtime_exec_option("Hash Table Built Asynchronously"); - boost::thread(bind(&HashJoinNode::build_side_thread, this, state, &thread_status)); + std::thread(bind(&HashJoinNode::build_side_thread, this, state, &thread_status)).detach(); if (!_runtime_filter_descs.empty()) { RuntimeFilterSlots runtime_filter_slots(_probe_expr_ctxs, _build_expr_ctxs, @@ -421,7 +421,7 @@ Status HashJoinNode::get_next(RuntimeState* state, RowBatch* out_batch, bool* eo if (_match_all_probe && !_matched_probe) { int row_idx = out_batch->add_row(); TupleRow* out_row = out_batch->get_row(row_idx); - create_output_row(out_row, _current_probe_row, NULL); + create_output_row(out_row, _current_probe_row, nullptr); if (eval_conjuncts(conjunct_ctxs, num_conjunct_ctxs, out_row)) { out_batch->commit_last_row(); @@ -497,9 +497,9 @@ Status HashJoinNode::get_next(RuntimeState* state, RowBatch* out_batch, bool* eo *eos = true; if (_match_all_build || _join_op == TJoinOp::RIGHT_ANTI_JOIN) { // output remaining unmatched build rows - TupleRow* build_row = NULL; + TupleRow* build_row = nullptr; if (_join_op == TJoinOp::RIGHT_ANTI_JOIN) { - if (_anti_join_last_pos != NULL) { + if (_anti_join_last_pos != nullptr) { _hash_tbl_iterator = *_anti_join_last_pos; } else { _hash_tbl_iterator = _hash_tbl->begin(); @@ -522,7 +522,7 @@ Status HashJoinNode::get_next(RuntimeState* state, RowBatch* out_batch, bool* eo int row_idx = out_batch->add_row(); TupleRow* out_row = out_batch->get_row(row_idx); - create_output_row(out_row, NULL, build_row); + create_output_row(out_row, nullptr, build_row); if (eval_conjuncts(conjunct_ctxs, num_conjunct_ctxs, out_row)) { out_batch->commit_last_row(); VLOG_ROW << "match row: " << out_row->to_string(row_desc()); @@ -606,7 +606,7 @@ std::string HashJoinNode::get_probe_row_output_string(TupleRow* probe_row) { std::find(_build_tuple_idx_ptr, _build_tuple_idx_ptr + _build_tuple_size, i); if (is_build_tuple != _build_tuple_idx_ptr + _build_tuple_size) { - out << Tuple::to_string(NULL, *row_desc().tuple_descriptors()[i]); + out << Tuple::to_string(nullptr, *row_desc().tuple_descriptors()[i]); } else { out << Tuple::to_string(probe_row->get_tuple(i), *row_desc().tuple_descriptors()[i]); } @@ -632,13 +632,13 @@ void HashJoinNode::debug_string(int indentation_level, std::stringstream* out) c // This function is replaced by codegen void HashJoinNode::create_output_row(TupleRow* out, TupleRow* probe, TupleRow* build) { uint8_t* out_ptr = reinterpret_cast(out); - if (probe == NULL) { + if (probe == nullptr) { memset(out_ptr, 0, _probe_tuple_row_size); } else { memcpy(out_ptr, probe, _probe_tuple_row_size); } - if (build == NULL) { + if (build == nullptr) { memset(out_ptr + _probe_tuple_row_size, 0, _build_tuple_row_size); } else { memcpy(out_ptr + _probe_tuple_row_size, build, _build_tuple_row_size); diff --git a/be/src/exec/hash_join_node.h b/be/src/exec/hash_join_node.h index f097a2c10e..8c914fa447 100644 --- a/be/src/exec/hash_join_node.h +++ b/be/src/exec/hash_join_node.h @@ -18,9 +18,9 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_HASH_JOIN_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_HASH_JOIN_NODE_H -#include -#include +#include #include +#include #include #include "exec/exec_node.h" @@ -65,7 +65,7 @@ protected: private: friend class IRuntimeFilter; - boost::scoped_ptr _hash_tbl; + std::unique_ptr _hash_tbl; HashTable::Iterator _hash_tbl_iterator; // for right outer joins, keep track of what's been joined @@ -92,9 +92,9 @@ private: bool _match_all_build; // output all rows coming from the build input bool _build_unique; // build a hash table without duplicated rows - bool _matched_probe; // if true, we have matched the current probe row - bool _eos; // if true, nothing left to return in get_next() - boost::scoped_ptr _build_pool; // holds everything referenced in _hash_tbl + bool _matched_probe; // if true, we have matched the current probe row + bool _eos; // if true, nothing left to return in get_next() + std::unique_ptr _build_pool; // holds everything referenced in _hash_tbl // Size of the TupleRow (just the Tuple ptrs) from the build (right) and probe (left) // sides. Set to zero if the build/probe tuples are not returned, e.g., for semi joins. @@ -105,7 +105,7 @@ private: // _probe_batch must be cleared before calling get_next(). The child node // does not initialize all tuple ptrs in the row, only the ones that it // is responsible for. - boost::scoped_ptr _probe_batch; + std::unique_ptr _probe_batch; int _probe_batch_pos; // current scan pos in _probe_batch int _probe_counter; bool _probe_eos; // if true, probe child has no more rows to process @@ -140,7 +140,7 @@ private: // Supervises ConstructHashTable in a separate thread, and // returns its status in the promise parameter. - void build_side_thread(RuntimeState* state, boost::promise* status); + void build_side_thread(RuntimeState* state, std::promise* status); // We parallelise building the build-side with Open'ing the // probe-side. If, for example, the probe-side child is another @@ -170,7 +170,7 @@ private: // Returns a debug string for probe_rows. Probe rows have tuple ptrs that are // uninitialized; the left hand child only populates the tuple ptrs it is responsible // for. This function outputs just the probe row values and leaves the build - // side values as NULL. + // side values as nullptr. // This is only used for debugging and outputting the left child rows before // doing the join. std::string get_probe_row_output_string(TupleRow* probe_row); diff --git a/be/src/exec/hash_join_node_ir.cpp b/be/src/exec/hash_join_node_ir.cpp index b512f305ab..b31f7961a9 100644 --- a/be/src/exec/hash_join_node_ir.cpp +++ b/be/src/exec/hash_join_node_ir.cpp @@ -101,7 +101,7 @@ int HashJoinNode::process_probe_batch(RowBatch* out_batch, RowBatch* probe_batch // Handle left outer-join and left semi-join if ((!_matched_probe && _match_all_probe) || ((!_matched_probe && _join_op == TJoinOp::LEFT_ANTI_JOIN))) { - create_output_row(out_row, _current_probe_row, NULL); + create_output_row(out_row, _current_probe_row, nullptr); _matched_probe = true; if (ExecNode::eval_conjuncts(conjunct_ctxs, num_conjunct_ctxs, out_row)) { diff --git a/be/src/exec/hash_table.cpp b/be/src/exec/hash_table.cpp index 867f513316..9275b18b5b 100644 --- a/be/src/exec/hash_table.cpp +++ b/be/src/exec/hash_table.cpp @@ -91,8 +91,8 @@ void HashTable::close() { } bool HashTable::eval_row(TupleRow* row, const std::vector& ctxs) { - // Put a non-zero constant in the result location for NULL. - // We don't want(NULL, 1) to hash to the same as (0, 1). + // Put a non-zero constant in the result location for nullptr. + // We don't want(nullptr, 1) to hash to the same as (0, 1). // This needs to be as big as the biggest primitive type since the bytes // get copied directly. @@ -105,7 +105,7 @@ bool HashTable::eval_row(TupleRow* row, const std::vector& ctxs) { void* loc = _expr_values_buffer + _expr_values_buffer_offsets[i]; void* val = ctxs[i]->get_value(row); - if (val == NULL) { + if (val == nullptr) { // If the table doesn't store nulls, no reason to keep evaluating if (!_stores_nulls) { return true; @@ -118,7 +118,7 @@ bool HashTable::eval_row(TupleRow* row, const std::vector& ctxs) { _expr_value_null_bits[i] = false; } - RawValue::write(val, loc, _build_expr_ctxs[i]->root()->type(), NULL); + RawValue::write(val, loc, _build_expr_ctxs[i]->root()->type(), nullptr); } return has_null; @@ -154,7 +154,7 @@ bool HashTable::equals(TupleRow* build_row) { for (int i = 0; i < _build_expr_ctxs.size(); ++i) { void* val = _build_expr_ctxs[i]->get_value(build_row); - if (val == NULL) { + if (val == nullptr) { if (!(_stores_nulls && _finds_nulls[i])) { return false; } @@ -197,7 +197,7 @@ void HashTable::resize_buckets(int64_t num_buckets) { for (int i = 0; i < _num_buckets; ++i) { Bucket* bucket = &_buckets[i]; Bucket* sister_bucket = &_buckets[i + old_num_buckets]; - Node* last_node = NULL; + Node* last_node = nullptr; Node* node = bucket->_node; while (node != nullptr) { @@ -205,7 +205,7 @@ void HashTable::resize_buckets(int64_t num_buckets) { uint32_t hash = node->_hash; bool node_must_move = true; - Bucket* move_to = NULL; + Bucket* move_to = nullptr; if (doubled_buckets) { node_must_move = ((hash & old_num_buckets) != 0); @@ -251,7 +251,7 @@ void HashTable::grow_node_array() { void HashTable::mem_limit_exceeded(int64_t allocation_size) { _mem_limit_exceeded = true; _exceeded_limit = true; - // if (_state != NULL) { + // if (_state != nullptr) { // _state->set_mem_limit_exceeded(_mem_tracker, allocation_size); // } } @@ -275,7 +275,7 @@ std::string HashTable::debug_string(bool skip_empty, const RowDescriptor* desc) ss << ","; } - if (desc == NULL) { + if (desc == nullptr) { ss << node->_hash << "(" << (void*)node->data() << ")"; } else { ss << (void*)node->data() << " " << node->data()->to_string(*desc); diff --git a/be/src/exec/hash_table.h b/be/src/exec/hash_table.h index ee74d8f66f..d150bb50ff 100644 --- a/be/src/exec/hash_table.h +++ b/be/src/exec/hash_table.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_HASH_TABLE_H #define DORIS_BE_SRC_QUERY_EXEC_HASH_TABLE_H -#include #include #include "codegen/doris_ir.h" @@ -158,14 +157,14 @@ public: // Returns the results of the exprs at 'expr_idx' evaluated over the last row // processed by the HashTable. - // This value is invalid if the expr evaluated to NULL. + // This value is invalid if the expr evaluated to nullptr. // TODO: this is an awkward abstraction but aggregation node can take advantage of // it and save some expr evaluation calls. void* last_expr_value(int expr_idx) const { return _expr_values_buffer + _expr_values_buffer_offsets[expr_idx]; } - // Returns if the expr at 'expr_idx' evaluated to NULL for the last row. + // Returns if the expr at 'expr_idx' evaluated to nullptr for the last row. bool last_expr_value_null(int expr_idx) const { return _expr_value_null_bits[expr_idx]; } // Return beginning of hash table. Advancing this iterator will traverse all @@ -189,7 +188,7 @@ public: // stl-like iterator interface. class Iterator { public: - Iterator() : _table(NULL), _bucket_idx(-1), _node(nullptr) {} + Iterator() : _table(nullptr), _bucket_idx(-1), _node(nullptr) {} // Iterates to the next element. In the case where the iterator was // from a Find, this will lazily evaluate that bucket, only returning @@ -197,10 +196,10 @@ public: template void IR_ALWAYS_INLINE next(); - // Returns the current row or NULL if at end. + // Returns the current row or nullptr if at end. TupleRow* get_row() { if (_node == nullptr) { - return NULL; + return nullptr; } return _node->data(); } @@ -297,7 +296,7 @@ private: }; // Returns the next non-empty bucket and updates idx to be the index of that bucket. - // If there are no more buckets, returns NULL and sets idx to -1 + // If there are no more buckets, returns nullptr and sets idx to -1 Bucket* next_bucket(int64_t* bucket_idx); // Resize the hash table to 'num_buckets' @@ -315,7 +314,7 @@ private: void move_node(Bucket* from_bucket, Bucket* to_bucket, Node* node, Node* previous_node); // Evaluate the exprs over row and cache the results in '_expr_values_buffer'. - // Returns whether any expr evaluated to NULL + // Returns whether any expr evaluated to nullptr // This will be replaced by codegen bool eval_row(TupleRow* row, const std::vector& exprs); diff --git a/be/src/exec/merge_join_node.cpp b/be/src/exec/merge_join_node.cpp index f972ab5f4d..d83e872507 100644 --- a/be/src/exec/merge_join_node.cpp +++ b/be/src/exec/merge_join_node.cpp @@ -47,7 +47,7 @@ int compare_value(const StringValue* left_value, const StringValue* right_value) } MergeJoinNode::MergeJoinNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) - : ExecNode(pool, tnode, descs), _out_batch(NULL) {} + : ExecNode(pool, tnode, descs), _out_batch(nullptr) {} MergeJoinNode::~MergeJoinNode() {} @@ -57,7 +57,7 @@ Status MergeJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { const std::vector& cmp_conjuncts = tnode.merge_join_node.cmp_conjuncts; for (int i = 0; i < cmp_conjuncts.size(); ++i) { - ExprContext* ctx = NULL; + ExprContext* ctx = nullptr; RETURN_IF_ERROR(Expr::create_expr_tree(_pool, cmp_conjuncts[i].left, &ctx)); _left_expr_ctxs.push_back(ctx); RETURN_IF_ERROR(Expr::create_expr_tree(_pool, cmp_conjuncts[i].right, &ctx)); @@ -207,28 +207,28 @@ Status MergeJoinNode::get_next(RuntimeState* state, RowBatch* out_batch, bool* e } void MergeJoinNode::create_output_row(TupleRow* out, TupleRow* left, TupleRow* right) { - if (left == NULL) { + if (left == nullptr) { memset(out, 0, _left_tuple_size); } else { memcpy(out, left, _left_tuple_size); } - if (right != NULL) { + if (right != nullptr) { for (int i = 0; i < _right_tuple_size; ++i) { out->set_tuple(_right_tuple_idx[i], right->get_tuple(i)); } } else { for (int i = 0; i < _right_tuple_size; ++i) { - out->set_tuple(_right_tuple_idx[i], NULL); + out->set_tuple(_right_tuple_idx[i], nullptr); } } } Status MergeJoinNode::compare_row(TupleRow* left_row, TupleRow* right_row, bool* is_lt) { - if (left_row == NULL) { + if (left_row == nullptr) { *is_lt = false; return Status::OK(); - } else if (right_row == NULL) { + } else if (right_row == nullptr) { *is_lt = true; return Status::OK(); } @@ -259,7 +259,7 @@ Status MergeJoinNode::get_next_row(RuntimeState* state, TupleRow* out_row, bool* TupleRow* left_row = _left_child_ctx->current_row; TupleRow* right_row = _right_child_ctx->current_row; - if (left_row == NULL && right_row == NULL) { + if (left_row == nullptr && right_row == nullptr) { *eos = true; return Status::OK(); } @@ -268,10 +268,10 @@ Status MergeJoinNode::get_next_row(RuntimeState* state, TupleRow* out_row, bool* RETURN_IF_ERROR(compare_row(left_row, right_row, &is_lt)); if (is_lt) { - create_output_row(out_row, left_row, NULL); + create_output_row(out_row, left_row, nullptr); RETURN_IF_ERROR(get_input_row(state, 0)); } else { - create_output_row(out_row, NULL, right_row); + create_output_row(out_row, nullptr, right_row); RETURN_IF_ERROR(get_input_row(state, 1)); } @@ -279,7 +279,7 @@ Status MergeJoinNode::get_next_row(RuntimeState* state, TupleRow* out_row, bool* } Status MergeJoinNode::get_input_row(RuntimeState* state, int child_idx) { - ChildReaderContext* ctx = NULL; + ChildReaderContext* ctx = nullptr; if (child_idx == 0) { ctx = _left_child_ctx.get(); @@ -290,7 +290,7 @@ Status MergeJoinNode::get_input_row(RuntimeState* state, int child_idx) { // loop util read a valid data while (!ctx->is_eos && ctx->row_idx >= ctx->batch.num_rows()) { // transfer ownership before get new batch - if (NULL != _out_batch) { + if (nullptr != _out_batch) { ctx->batch.transfer_resource_ownership(_out_batch); } @@ -310,7 +310,7 @@ Status MergeJoinNode::get_input_row(RuntimeState* state, int child_idx) { } if (ctx->row_idx >= ctx->batch.num_rows()) { - ctx->current_row = NULL; + ctx->current_row = nullptr; return Status::OK(); } diff --git a/be/src/exec/merge_join_node.h b/be/src/exec/merge_join_node.h index 5b15eef7cd..d8b294ea18 100644 --- a/be/src/exec/merge_join_node.h +++ b/be/src/exec/merge_join_node.h @@ -18,9 +18,8 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_MERGE_JOIN_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_MERGE_JOIN_NODE_H -#include -#include #include +#include #include #include "exec/exec_node.h" @@ -71,14 +70,14 @@ private: : batch(desc, batch_size, mem_tracker.get()), row_idx(0), is_eos(false), - current_row(NULL) {} + current_row(nullptr) {} }; // _left_batch must be cleared before calling get_next(). used cache child(0)'s data // _right_batch must be cleared before calling get_next(). used cache child(1)'s data // does not initialize all tuple ptrs in the row, only the ones that it // is responsible for. - boost::scoped_ptr _left_child_ctx; - boost::scoped_ptr _right_child_ctx; + std::unique_ptr _left_child_ctx; + std::unique_ptr _right_child_ctx; // _build_tuple_idx[i] is the tuple index of child(1)'s tuple[i] in the output row std::vector _right_tuple_idx; int _right_tuple_size; diff --git a/be/src/exec/merge_node.cpp b/be/src/exec/merge_node.cpp index 13e7d2a7d4..513284e4ca 100644 --- a/be/src/exec/merge_node.cpp +++ b/be/src/exec/merge_node.cpp @@ -33,7 +33,7 @@ MergeNode::MergeNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorT _tuple_id(tnode.merge_node.tuple_id), _const_result_expr_idx(0), _child_idx(INVALID_CHILD_IDX), - _child_row_batch(NULL), + _child_row_batch(nullptr), _child_eos(false), _child_row_idx(0) {} @@ -61,7 +61,7 @@ Status MergeNode::init(const TPlanNode& tnode, RuntimeState* state) { Status MergeNode::prepare(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::prepare(state)); _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id); - DCHECK(_tuple_desc != NULL); + DCHECK(_tuple_desc != nullptr); // Prepare const expr lists. for (int i = 0; i < _const_result_expr_ctx_lists.size(); ++i) { @@ -134,7 +134,7 @@ Status MergeNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) // Fetch from children, evaluate corresponding exprs and materialize. while (_child_idx < _children.size()) { // Row batch was either never set or we're moving on to a different child. - if (_child_row_batch.get() == NULL) { + if (_child_row_batch.get() == nullptr) { RETURN_IF_CANCELLED(state); _child_row_batch.reset(new RowBatch(child(_child_idx)->row_desc(), state->batch_size(), mem_tracker().get())); @@ -173,7 +173,7 @@ Status MergeNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) // Close current child and move on to next one. ++_child_idx; - _child_row_batch.reset(NULL); + _child_row_batch.reset(nullptr); } _child_idx = INVALID_CHILD_IDX; @@ -186,7 +186,7 @@ Status MergeNode::close(RuntimeState* state) { return Status::OK(); } // don't call ExecNode::close(), it always closes all children - _child_row_batch.reset(NULL); + _child_row_batch.reset(nullptr); for (int i = 0; i < _const_result_expr_ctx_lists.size(); ++i) { Expr::close(_const_result_expr_ctx_lists[i], state); } @@ -210,10 +210,10 @@ bool MergeNode::eval_and_materialize_exprs(const std::vector& ctxs int num_conjunct_ctxs = _conjunct_ctxs.size(); do { - TupleRow* child_row = NULL; + TupleRow* child_row = nullptr; if (!const_exprs) { - DCHECK(_child_row_batch != NULL); + DCHECK(_child_row_batch != nullptr); // Non-const expr list. Fetch next row from batch. child_row = _child_row_batch->get_row(_child_row_idx); ++_child_row_idx; diff --git a/be/src/exec/merge_node.h b/be/src/exec/merge_node.h index 7344dabdc7..a81ece1b22 100644 --- a/be/src/exec/merge_node.h +++ b/be/src/exec/merge_node.h @@ -18,8 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_MERGE_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_MERGE_NODE_H -#include - #include "exec/exec_node.h" #include "runtime/mem_pool.h" @@ -70,7 +68,7 @@ private: // Current row batch of current child. We reset the pointer to a new RowBatch // when switching to a different child. - boost::scoped_ptr _child_row_batch; + std::unique_ptr _child_row_batch; // Saved from the last to get_next() on the current child. bool _child_eos; diff --git a/be/src/exec/mysql_scan_node.cpp b/be/src/exec/mysql_scan_node.cpp index 5a78747c37..634f47c784 100644 --- a/be/src/exec/mysql_scan_node.cpp +++ b/be/src/exec/mysql_scan_node.cpp @@ -48,15 +48,15 @@ Status MysqlScanNode::prepare(RuntimeState* state) { return Status::OK(); } - if (NULL == state) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == state) { + return Status::InternalError("input pointer is nullptr."); } RETURN_IF_ERROR(ScanNode::prepare(state)); // get tuple desc _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id); - if (NULL == _tuple_desc) { + if (nullptr == _tuple_desc) { return Status::InternalError("Failed to get tuple descriptor."); } @@ -65,8 +65,8 @@ Status MysqlScanNode::prepare(RuntimeState* state) { const MySQLTableDescriptor* mysql_table = static_cast(_tuple_desc->table_desc()); - if (NULL == mysql_table) { - return Status::InternalError("mysql table pointer is NULL."); + if (nullptr == mysql_table) { + return Status::InternalError("mysql table pointer is nullptr."); } _my_param.host = mysql_table->host(); @@ -77,19 +77,19 @@ Status MysqlScanNode::prepare(RuntimeState* state) { // new one scanner _mysql_scanner.reset(new (std::nothrow) MysqlScanner(_my_param)); - if (_mysql_scanner.get() == NULL) { + if (_mysql_scanner.get() == nullptr) { return Status::InternalError("new a mysql scanner failed."); } _tuple_pool.reset(new (std::nothrow) MemPool(mem_tracker().get())); - if (_tuple_pool.get() == NULL) { + if (_tuple_pool.get() == nullptr) { return Status::InternalError("new a mem pool failed."); } _text_converter.reset(new (std::nothrow) TextConverter('\\')); - if (_text_converter.get() == NULL) { + if (_text_converter.get() == nullptr) { return Status::InternalError("new a text convertor failed."); } @@ -102,8 +102,8 @@ Status MysqlScanNode::open(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::open(state)); VLOG_CRITICAL << "MysqlScanNode::Open"; - if (NULL == state) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == state) { + return Status::InternalError("input pointer is nullptr."); } if (!_is_init) { @@ -148,8 +148,8 @@ Status MysqlScanNode::write_text_slot(char* value, int value_length, SlotDescrip Status MysqlScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) { VLOG_CRITICAL << "MysqlScanNode::GetNext"; - if (NULL == state || NULL == row_batch || NULL == eos) { - return Status::InternalError("input is NULL pointer"); + if (nullptr == state || nullptr == row_batch || nullptr == eos) { + return Status::InternalError("input is nullptr pointer"); } if (!_is_init) { @@ -164,7 +164,7 @@ Status MysqlScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* e int tuple_buffer_size = row_batch->capacity() * _tuple_desc->byte_size(); void* tuple_buffer = _tuple_pool->allocate(tuple_buffer_size); - if (NULL == tuple_buffer) { + if (nullptr == tuple_buffer) { return Status::InternalError("Allocate memory failed."); } @@ -183,8 +183,8 @@ Status MysqlScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* e } // read mysql - char** data = NULL; - unsigned long* length = NULL; + char** data = nullptr; + unsigned long* length = nullptr; RETURN_IF_ERROR(_mysql_scanner->get_next_row(&data, &length, &mysql_eos)); if (mysql_eos) { @@ -212,7 +212,7 @@ Status MysqlScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* e _tuple->set_null(slot_desc->null_indicator_offset()); } else { std::stringstream ss; - ss << "nonnull column contains NULL. table=" << _table_name + ss << "nonnull column contains nullptr. table=" << _table_name << ", column=" << slot_desc->col_name(); return Status::InternalError(ss.str()); } diff --git a/be/src/exec/mysql_scanner.cpp b/be/src/exec/mysql_scanner.cpp index 26718f1ab6..a6be6649fc 100644 --- a/be/src/exec/mysql_scanner.cpp +++ b/be/src/exec/mysql_scanner.cpp @@ -19,26 +19,29 @@ #define __DorisMysql MYSQL #define __DorisMysqlRes MYSQL_RES +#include "common/config.h" #include "common/logging.h" #include "mysql_scanner.h" -#include "common/config.h" - namespace doris { MysqlScanner::MysqlScanner(const MysqlScannerParam& param) - : _my_param(param), _my_conn(NULL), _my_result(NULL), _is_open(false), _field_num(0) {} + : _my_param(param), + _my_conn(nullptr), + _my_result(nullptr), + _is_open(false), + _field_num(0) {} MysqlScanner::~MysqlScanner() { if (_my_result) { mysql_free_result(_my_result); - _my_result = NULL; + _my_result = nullptr; } if (_my_conn) { mysql_close(_my_conn); - _my_conn = NULL; - mysql_library_end(); + _my_conn = nullptr; + mysql_library_end(); } } @@ -48,9 +51,9 @@ Status MysqlScanner::open() { return Status::OK(); } - _my_conn = mysql_init(NULL); + _my_conn = mysql_init(nullptr); - if (NULL == _my_conn) { + if (nullptr == _my_conn) { return Status::InternalError("mysql init failed."); } @@ -59,9 +62,10 @@ Status MysqlScanner::open() { unsigned int mysql_ct = config::external_table_connect_timeout_sec; mysql_options(_my_conn, MYSQL_OPT_CONNECT_TIMEOUT, &mysql_ct); mysql_options(_my_conn, MYSQL_OPT_READ_TIMEOUT, &mysql_ct); - if (NULL == mysql_real_connect(_my_conn, _my_param.host.c_str(), _my_param.user.c_str(), - _my_param.passwd.c_str(), _my_param.db.c_str(), - atoi(_my_param.port.c_str()), NULL, _my_param.client_flag)) { + if (nullptr == mysql_real_connect(_my_conn, _my_param.host.c_str(), _my_param.user.c_str(), + _my_param.passwd.c_str(), _my_param.db.c_str(), + atoi(_my_param.port.c_str()), nullptr, + _my_param.client_flag)) { LOG(WARNING) << "connect Mysql: " << "Host: " << _my_param.host << " user: " << _my_param.user << " passwd: " << _my_param.passwd << " db: " << _my_param.db @@ -101,7 +105,7 @@ Status MysqlScanner::query(const std::string& query) { // use store result because mysql table is small, can load in memory avoid of many RPC _my_result = mysql_store_result(_my_conn); - if (NULL == _my_result) { + if (nullptr == _my_result) { return _error_status("mysql store result failed."); } @@ -152,24 +156,24 @@ Status MysqlScanner::get_next_row(char*** buf, unsigned long** lengths, bool* eo return Status::InternalError("GetNextRow before open."); } - if (NULL == buf || NULL == lengths || NULL == eos) { + if (nullptr == buf || nullptr == lengths || nullptr == eos) { return Status::InternalError("input parameter invalid."); } - if (NULL == _my_result) { + if (nullptr == _my_result) { return Status::InternalError("get next row before query."); } *buf = mysql_fetch_row(_my_result); - if (NULL == *buf) { + if (nullptr == *buf) { *eos = true; return Status::OK(); } *lengths = mysql_fetch_lengths(_my_result); - if (NULL == *lengths) { + if (nullptr == *lengths) { return _error_status("mysql fetch row failed."); } diff --git a/be/src/exec/odbc_connector.cpp b/be/src/exec/odbc_connector.cpp index 7b50a612d1..bc2d787534 100644 --- a/be/src/exec/odbc_connector.cpp +++ b/be/src/exec/odbc_connector.cpp @@ -19,7 +19,6 @@ #include -#include #include #include "common/config.h" @@ -107,8 +106,8 @@ Status ODBCConnector::open() { SQLSetConnectAttr(_dbc, SQL_ATTR_CONNECTION_TIMEOUT, (SQLPOINTER)timeout, 0); // Connect to the Database ODBC_DISPOSE(_dbc, SQL_HANDLE_DBC, - SQLDriverConnect(_dbc, NULL, (SQLCHAR*)_connect_string.c_str(), SQL_NTS, NULL, 0, - NULL, SQL_DRIVER_NOPROMPT), + SQLDriverConnect(_dbc, nullptr, (SQLCHAR*)_connect_string.c_str(), SQL_NTS, + nullptr, 0, nullptr, SQL_DRIVER_NOPROMPT), "driver connect"); LOG(INFO) << "connect success:" << _connect_string.substr(0, _connect_string.find("Pwd=")); @@ -157,15 +156,15 @@ Status ODBCConnector::query() { ? BIG_COLUMN_SIZE_BUFFER : SMALL_COLUMN_SIZE_BUFFER; column_data->target_value_ptr = malloc(sizeof(char) * column_data->buffer_length); - _columns_data.push_back(column_data); + _columns_data.emplace_back(column_data); } // setup the binding for (int i = 0; i < _field_num; i++) { ODBC_DISPOSE(_stmt, SQL_HANDLE_STMT, - SQLBindCol(_stmt, (SQLUSMALLINT)i + 1, _columns_data[i].target_type, - _columns_data[i].target_value_ptr, _columns_data[i].buffer_length, - &(_columns_data[i].strlen_or_ind)), + SQLBindCol(_stmt, (SQLUSMALLINT)i + 1, _columns_data[i]->target_type, + _columns_data[i]->target_value_ptr, _columns_data[i]->buffer_length, + &(_columns_data[i]->strlen_or_ind)), "bind col"); } @@ -264,7 +263,7 @@ Status ODBCConnector::append(const std::string& table_name, RowBatch* batch, case TYPE_STRING: { const auto* string_val = (const StringValue*)(item); - if (string_val->ptr == NULL) { + if (string_val->ptr == nullptr) { if (string_val->len == 0) { fmt::format_to(_insert_stmt_buffer, "{}", "''"); } else { @@ -389,12 +388,10 @@ std::string ODBCConnector::handle_diagnostic_record(SQLHANDLE hHandle, SQLSMALLI while (SQLGetDiagRec(hType, hHandle, ++rec, (SQLCHAR*)(state), &error, reinterpret_cast(message), (SQLSMALLINT)(sizeof(message) / sizeof(WCHAR)), - (SQLSMALLINT*)NULL) == SQL_SUCCESS) { + (SQLSMALLINT*)nullptr) == SQL_SUCCESS) { // Hide data truncated.. if (wcsncmp(reinterpret_cast(state), L"01004", 5)) { - boost::format msg_string("%s %s (%d)"); - msg_string % state % message % error; - diagnostic_msg += msg_string.str(); + diagnostic_msg += fmt::format("{} {} ({})", state, message, error); } } diff --git a/be/src/exec/odbc_connector.h b/be/src/exec/odbc_connector.h index 478c3f04e5..c5d90162f1 100644 --- a/be/src/exec/odbc_connector.h +++ b/be/src/exec/odbc_connector.h @@ -18,20 +18,19 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_ODBC_CONNECTOR_H #define DORIS_BE_SRC_QUERY_EXEC_ODBC_CONNECTOR_H +#include #include #include -#include #include -#include #include #include -#include "exprs/expr_context.h" -#include "runtime/row_batch.h" #include "common/status.h" +#include "exprs/expr_context.h" #include "gen_cpp/Types_types.h" #include "runtime/descriptors.h" +#include "runtime/row_batch.h" namespace doris { @@ -59,7 +58,6 @@ struct DataBinding { ~DataBinding() { free(target_value_ptr); } DataBinding(const DataBinding&) = delete; DataBinding& operator=(const DataBinding&) = delete; - }; // ODBC Connector for scan data from ODBC @@ -75,14 +73,16 @@ public: // write for ODBC table Status init_to_write(RuntimeProfile* profile); - Status append(const std::string& table_name, RowBatch* batch, uint32_t start_send_row, uint32_t* num_row_sent); + Status append(const std::string& table_name, RowBatch* batch, uint32_t start_send_row, + uint32_t* num_row_sent); // use in ODBC transaction Status begin_trans(); // should be call after connect and before query or init_to_write Status abort_trans(); // should be call after transaction abort Status finish_trans(); // should be call after transaction commit - const DataBinding& get_column_data(int i) const { return _columns_data.at(i); } + const DataBinding& get_column_data(int i) const { return *_columns_data.at(i).get(); } + private: void _init_profile(RuntimeProfile*); @@ -111,7 +111,6 @@ private: bool _is_open; bool _is_in_transaction; - SQLSMALLINT _field_num; uint64_t _row_count; @@ -119,7 +118,7 @@ private: SQLHDBC _dbc; SQLHSTMT _stmt; - boost::ptr_vector _columns_data; + std::vector> _columns_data; }; } // namespace doris diff --git a/be/src/exec/odbc_scan_node.cpp b/be/src/exec/odbc_scan_node.cpp index 092113487a..6bdc620993 100644 --- a/be/src/exec/odbc_scan_node.cpp +++ b/be/src/exec/odbc_scan_node.cpp @@ -48,15 +48,15 @@ Status OdbcScanNode::prepare(RuntimeState* state) { return Status::OK(); } - if (NULL == state) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == state) { + return Status::InternalError("input pointer is null."); } RETURN_IF_ERROR(ScanNode::prepare(state)); // get tuple desc _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id); - if (NULL == _tuple_desc) { + if (nullptr == _tuple_desc) { return Status::InternalError("Failed to get tuple descriptor."); } @@ -74,13 +74,13 @@ Status OdbcScanNode::prepare(RuntimeState* state) { _tuple_pool.reset(new (std::nothrow) MemPool(mem_tracker().get())); - if (_tuple_pool.get() == NULL) { + if (_tuple_pool.get() == nullptr) { return Status::InternalError("new a mem pool failed."); } _text_converter.reset(new (std::nothrow) TextConverter('\\')); - if (_text_converter.get() == NULL) { + if (_text_converter.get() == nullptr) { return Status::InternalError("new a text convertor failed."); } @@ -93,8 +93,8 @@ Status OdbcScanNode::open(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::open(state)); VLOG_CRITICAL << "OdbcScanNode::Open"; - if (NULL == state) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == state) { + return Status::InternalError("input pointer is null."); } if (!_is_init) { @@ -127,8 +127,8 @@ Status OdbcScanNode::write_text_slot(char* value, int value_length, SlotDescript Status OdbcScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) { VLOG_CRITICAL << "OdbcScanNode::GetNext"; - if (NULL == state || NULL == row_batch || NULL == eos) { - return Status::InternalError("input is NULL pointer"); + if (nullptr == state || nullptr == row_batch || nullptr == eos) { + return Status::InternalError("input is nullptr pointer"); } if (!_is_init) { @@ -148,7 +148,7 @@ Status OdbcScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo int tuple_buffer_size = row_batch->capacity() * _tuple_desc->byte_size(); void* tuple_buffer = _tuple_pool->allocate(tuple_buffer_size); - if (NULL == tuple_buffer) { + if (nullptr == tuple_buffer) { return Status::InternalError("Allocate memory failed."); } @@ -195,13 +195,13 @@ Status OdbcScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo _tuple->set_null(slot_desc->null_indicator_offset()); } else { std::stringstream ss; - ss << "nonnull column contains NULL. table=" << _table_name + ss << "nonnull column contains nullptr. table=" << _table_name << ", column=" << slot_desc->col_name(); return Status::InternalError(ss.str()); } } else if (column_data.strlen_or_ind > column_data.buffer_length) { std::stringstream ss; - ss << "nonnull column contains NULL. table=" << _table_name + ss << "nonnull column contains nullptr. table=" << _table_name << ", column=" << slot_desc->col_name(); return Status::InternalError(ss.str()); } else { diff --git a/be/src/exec/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp index 9157e1be21..1cb610df34 100644 --- a/be/src/exec/olap_scan_node.cpp +++ b/be/src/exec/olap_scan_node.cpp @@ -46,7 +46,7 @@ OlapScanNode::OlapScanNode(ObjectPool* pool, const TPlanNode& tnode, const Descr : ScanNode(pool, tnode, descs), _tuple_id(tnode.olap_scan_node.tuple_id), _olap_scan_node(tnode.olap_scan_node), - _tuple_desc(NULL), + _tuple_desc(nullptr), _tuple_idx(0), _eos(false), _max_materialized_row_batches(config::doris_scanner_queue_size), @@ -101,10 +101,10 @@ void OlapScanNode::init_scan_profile() { scanner_profile_name = fmt::format("OlapScanner({0})", _olap_scan_node.table_name); } _scanner_profile.reset(new RuntimeProfile(scanner_profile_name)); - runtime_profile()->add_child(_scanner_profile.get(), true, NULL); + runtime_profile()->add_child(_scanner_profile.get(), true, nullptr); _segment_profile.reset(new RuntimeProfile("SegmentIterator")); - _scanner_profile->add_child(_segment_profile.get(), true, NULL); + _scanner_profile->add_child(_segment_profile.get(), true, nullptr); } void OlapScanNode::_init_counter(RuntimeState* state) { @@ -172,7 +172,7 @@ Status OlapScanNode::prepare(RuntimeState* state) { _init_counter(state); _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id); - if (_tuple_desc == NULL) { + if (_tuple_desc == nullptr) { // TODO: make sure we print all available diagnostic output to our error log return Status::InternalError("Failed to get tuple descriptor."); } @@ -283,7 +283,7 @@ Status OlapScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo } // wait for batch from queue - RowBatch* materialized_batch = NULL; + RowBatch* materialized_batch = nullptr; { std::unique_lock l(_row_batches_lock); SCOPED_TIMER(_olap_wait_batch_queue_timer); @@ -298,13 +298,13 @@ Status OlapScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo if (!_materialized_row_batches.empty()) { materialized_batch = _materialized_row_batches.front(); - DCHECK(materialized_batch != NULL); + DCHECK(materialized_batch != nullptr); _materialized_row_batches.pop_front(); } } // return batch - if (NULL != materialized_batch) { + if (nullptr != materialized_batch) { // notify scanner _row_batch_consumed_cv.notify_one(); // get scanner's batch memory @@ -531,8 +531,8 @@ void OlapScanNode::eval_const_conjuncts() { for (int conj_idx = 0; conj_idx < _conjunct_ctxs.size(); ++conj_idx) { // if conjunct is constant, compute direct and set eos = true if (_conjunct_ctxs[conj_idx]->root()->is_constant()) { - void* value = _conjunct_ctxs[conj_idx]->get_value(NULL); - if (value == NULL || *reinterpret_cast(value) == false) { + void* value = _conjunct_ctxs[conj_idx]->get_value(nullptr); + if (value == nullptr || *reinterpret_cast(value) == false) { _eos = true; break; } @@ -666,7 +666,7 @@ Status OlapScanNode::get_hints(const TPaloScanRange& scan_range, int block_row_c std::vector>* sub_scan_range, RuntimeProfile* profile) { auto tablet_id = scan_range.tablet_id; - int32_t schema_hash = strtoul(scan_range.schema_hash.c_str(), NULL, 10); + int32_t schema_hash = strtoul(scan_range.schema_hash.c_str(), nullptr, 10); std::string err; TabletSharedPtr table = StorageEngine::instance()->tablet_manager()->get_tablet( tablet_id, schema_hash, true, &err); @@ -923,7 +923,7 @@ std::pair OlapScanNode::should_push_down_eq_predicate(doris::SlotDe } // get value in result pair - result_pair = std::make_pair(true, _conjunct_ctxs[conj_idx]->get_value(expr, NULL)); + result_pair = std::make_pair(true, _conjunct_ctxs[conj_idx]->get_value(expr, nullptr)); return result_pair; } @@ -990,9 +990,9 @@ Status OlapScanNode::normalize_in_and_eq_predicate(SlotDescriptor* slot, // begin to push InPredicate value into ColumnValueRange HybridSetBase::IteratorBase* iter = pred->hybrid_set()->begin(); while (iter->has_next()) { - // column in (NULL) is always false so continue to + // column in (nullptr) is always false so continue to // dispose next item - if (NULL == iter->get_value()) { + if (nullptr == iter->get_value()) { continue; } auto value = const_cast(iter->get_value()); @@ -1023,7 +1023,7 @@ Status OlapScanNode::normalize_in_and_eq_predicate(SlotDescriptor* slot, } auto value = result_pair.second; - // where A = NULL should return empty result set + // where A = nullptr should return empty result set if (value != nullptr) { RETURN_IF_ERROR( change_fixed_value_range(temp_range, slot->type().type, value, @@ -1073,8 +1073,8 @@ Status OlapScanNode::normalize_not_in_and_not_eq_predicate(SlotDescriptor* slot, // begin to push InPredicate value into ColumnValueRange auto iter = pred->hybrid_set()->begin(); while (iter->has_next()) { - // column not in (NULL) is always true - if (NULL == iter->get_value()) { + // column not in (nullptr) is always true + if (nullptr == iter->get_value()) { continue; } auto value = const_cast(iter->get_value()); @@ -1090,7 +1090,7 @@ Status OlapScanNode::normalize_not_in_and_not_eq_predicate(SlotDescriptor* slot, iter->next(); } - // only where a in ('a', 'b', NULL) contain NULL will + // only where a in ('a', 'b', nullptr) contain nullptr will // clear temp_range to whole range, no need do intersection if (is_key_column(slot->col_name())) { filter_conjuncts_index.emplace_back(conj_idx); @@ -1219,9 +1219,9 @@ Status OlapScanNode::normalize_noneq_binary_predicate(SlotDescriptor* slot, continue; } - void* value = _conjunct_ctxs[conj_idx]->get_value(expr, NULL); + void* value = _conjunct_ctxs[conj_idx]->get_value(expr, nullptr); // for case: where col > null - if (value == NULL) { + if (value == nullptr) { continue; } @@ -1439,7 +1439,7 @@ void OlapScanNode::transfer_thread(RuntimeState* state) { } } - RowBatch* scan_batch = NULL; + RowBatch* scan_batch = nullptr; { // 1 scanner idle task not empty, assign new scanner task std::unique_lock l(_scan_batches_lock); @@ -1467,7 +1467,7 @@ void OlapScanNode::transfer_thread(RuntimeState* state) { // because scan_batch wouldn't be useful anymore if (UNLIKELY(_transfer_done)) { delete scan_batch; - scan_batch = NULL; + scan_batch = nullptr; } } else { if (_scanner_done) { @@ -1476,7 +1476,7 @@ void OlapScanNode::transfer_thread(RuntimeState* state) { } } - if (NULL != scan_batch) { + if (nullptr != scan_batch) { add_one_batch(scan_batch); } } // end of transfer while @@ -1513,7 +1513,7 @@ void OlapScanNode::scanner_thread(OlapScanner* scanner) { Status status = Status::OK(); bool eos = false; RuntimeState* state = scanner->runtime_state(); - DCHECK(NULL != state); + DCHECK(nullptr != state); if (!scanner->is_open()) { status = scanner->open(); if (!status.ok()) { @@ -1586,7 +1586,7 @@ void OlapScanNode::scanner_thread(OlapScanner* scanner) { if (UNLIKELY(row_batch->num_rows() == 0)) { // may be failed, push already, scan node delete this batch. delete row_batch; - row_batch = NULL; + row_batch = nullptr; } else { row_batchs.push_back(row_batch); __sync_fetch_and_add(&_buffered_bytes, diff --git a/be/src/exec/olap_utils.h b/be/src/exec/olap_utils.h index a510a172e0..129bd2c6eb 100644 --- a/be/src/exec/olap_utils.h +++ b/be/src/exec/olap_utils.h @@ -112,7 +112,7 @@ static const char base64_pad = '='; inline size_t base64_encode(const char* data, size_t length, char* encoded_data) { size_t output_length = (size_t)(4.0 * ceil((double)length / 3.0)); - if (encoded_data == NULL) { + if (encoded_data == nullptr) { return 0; } diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp index 0715d02d85..689fe0aaf5 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -167,7 +167,7 @@ inline Status ParquetReaderWrap::set_field_null(Tuple* tuple, const SlotDescript if (!slot_desc->is_nullable()) { std::stringstream str_error; str_error << "The field name(" << slot_desc->col_name() - << ") is not allowed null, but Parquet field is NULL."; + << ") is not allowed null, but Parquet field is null."; LOG(WARNING) << str_error.str(); return Status::RuntimeError(str_error.str()); } diff --git a/be/src/exec/partitioned_aggregation_node.cc b/be/src/exec/partitioned_aggregation_node.cc index b2d9db1017..54081e7ab2 100644 --- a/be/src/exec/partitioned_aggregation_node.cc +++ b/be/src/exec/partitioned_aggregation_node.cc @@ -100,33 +100,33 @@ PartitionedAggregationNode::PartitionedAggregationNode(ObjectPool* pool, const T output_tuple_desc_(descs.get_tuple_descriptor(output_tuple_id_)), needs_finalize_(tnode.agg_node.need_finalize), needs_serialize_(false), - output_partition_(NULL), - process_batch_no_grouping_fn_(NULL), - process_batch_fn_(NULL), - process_batch_streaming_fn_(NULL), - build_timer_(NULL), - ht_resize_timer_(NULL), - ht_resize_counter_(NULL), - get_results_timer_(NULL), - num_hash_buckets_(NULL), - num_hash_filled_buckets_(NULL), - num_hash_probe_(NULL), - num_hash_failed_probe_(NULL), - num_hash_travel_length_(NULL), - num_hash_collisions_(NULL), - partitions_created_(NULL), - max_partition_level_(NULL), - num_row_repartitioned_(NULL), - num_repartitions_(NULL), - num_spilled_partitions_(NULL), - largest_partition_percent_(NULL), - streaming_timer_(NULL), - num_processed_rows_(NULL), - num_passthrough_rows_(NULL), - preagg_estimated_reduction_(NULL), - preagg_streaming_ht_min_reduction_(NULL), + output_partition_(nullptr), + process_batch_no_grouping_fn_(nullptr), + process_batch_fn_(nullptr), + process_batch_streaming_fn_(nullptr), + build_timer_(nullptr), + ht_resize_timer_(nullptr), + ht_resize_counter_(nullptr), + get_results_timer_(nullptr), + num_hash_buckets_(nullptr), + num_hash_filled_buckets_(nullptr), + num_hash_probe_(nullptr), + num_hash_failed_probe_(nullptr), + num_hash_travel_length_(nullptr), + num_hash_collisions_(nullptr), + partitions_created_(nullptr), + max_partition_level_(nullptr), + num_row_repartitioned_(nullptr), + num_repartitions_(nullptr), + num_spilled_partitions_(nullptr), + largest_partition_percent_(nullptr), + streaming_timer_(nullptr), + num_processed_rows_(nullptr), + num_passthrough_rows_(nullptr), + preagg_estimated_reduction_(nullptr), + preagg_streaming_ht_min_reduction_(nullptr), // estimated_input_cardinality_(tnode.agg_node.estimated_input_cardinality), - singleton_output_tuple_(NULL), + singleton_output_tuple_(nullptr), singleton_output_tuple_returned_(true), partition_eos_(false), child_eos_(false), @@ -310,14 +310,14 @@ Status PartitionedAggregationNode::open(RuntimeState* state) { SCOPED_TIMER(build_timer_); if (grouping_exprs_.empty()) { - if (process_batch_no_grouping_fn_ != NULL) { + if (process_batch_no_grouping_fn_ != nullptr) { RETURN_IF_ERROR(process_batch_no_grouping_fn_(this, &batch)); } else { RETURN_IF_ERROR(ProcessBatchNoGrouping(&batch)); } } else { // There is grouping, so we will do partitioned aggregation. - if (process_batch_fn_ != NULL) { + if (process_batch_fn_ != nullptr) { RETURN_IF_ERROR(process_batch_fn_(this, &batch, ht_ctx_.get())); } else { RETURN_IF_ERROR(ProcessBatch(&batch, ht_ctx_.get())); @@ -346,7 +346,7 @@ Status PartitionedAggregationNode::get_next(RuntimeState* state, RowBatch* row_b // 1. `!need_finalize` means this aggregation node not the level two aggregation node // 2. `grouping_exprs_.size() == 0 ` means is not group by // 3. `child(0)->rows_returned() == 0` mean not data from child - // in level two aggregation node should return NULL result + // in level two aggregation node should return nullptr result // level one aggregation node set `eos = true` return directly if (UNLIKELY(grouping_exprs_.size() == 0 && !needs_finalize_ && child(0)->rows_returned() == 0)) { @@ -402,9 +402,9 @@ Status PartitionedAggregationNode::CopyStringData(const SlotDescriptor& slot_des FOREACH_ROW(row_batch, first_row_idx, batch_iter) { Tuple* tuple = batch_iter.get()->get_tuple(0); StringValue* sv = reinterpret_cast(tuple->get_slot(slot_desc.tuple_offset())); - if (sv == NULL || sv->len == 0) continue; + if (sv == nullptr || sv->len == 0) continue; char* new_ptr = reinterpret_cast(pool->try_allocate(sv->len)); - if (UNLIKELY(new_ptr == NULL)) { + if (UNLIKELY(new_ptr == nullptr)) { string details = Substitute( "Cannot perform aggregation at node with id $0." " Failed to allocate $1 output bytes.", @@ -469,16 +469,16 @@ void PartitionedAggregationNode::GetSingletonOutput(RowBatch* row_batch) { // of Reset()/Open()/GetNext()* calls. row_batch->tuple_data_pool()->acquire_data(mem_pool_.get(), true); // This node no longer owns the memory for singleton_output_tuple_. - singleton_output_tuple_ = NULL; + singleton_output_tuple_ = nullptr; } Status PartitionedAggregationNode::GetRowsFromPartition(RuntimeState* state, RowBatch* row_batch) { DCHECK(!row_batch->at_capacity()); if (output_iterator_.AtEnd()) { // Done with this partition, move onto the next one. - if (output_partition_ != NULL) { + if (output_partition_ != nullptr) { output_partition_->Close(false); - output_partition_ = NULL; + output_partition_ = nullptr; } if (aggregated_partitions_.empty() && spilled_partitions_.empty()) { // No more partitions, all done. @@ -487,7 +487,7 @@ Status PartitionedAggregationNode::GetRowsFromPartition(RuntimeState* state, Row } // Process next partition. RETURN_IF_ERROR(NextPartition()); - DCHECK(output_partition_ != NULL); + DCHECK(output_partition_ != nullptr); } SCOPED_TIMER(get_results_timer_); @@ -533,7 +533,7 @@ Status PartitionedAggregationNode::GetRowsStreaming(RuntimeState* state, RowBatc DCHECK(!child_eos_); DCHECK(is_streaming_preagg_); - if (child_batch_ == NULL) { + if (child_batch_ == nullptr) { child_batch_.reset( new RowBatch(child(0)->row_desc(), state->batch_size(), mem_tracker().get())); } @@ -575,7 +575,7 @@ Status PartitionedAggregationNode::GetRowsStreaming(RuntimeState* state, RowBatc } } - if (process_batch_streaming_fn_ != NULL) { + if (process_batch_streaming_fn_ != nullptr) { RETURN_IF_ERROR(process_batch_streaming_fn_(this, needs_serialize_, child_batch_.get(), out_batch, ht_ctx_.get(), remaining_capacity)); @@ -656,7 +656,7 @@ void PartitionedAggregationNode::CleanupHashTbl(const vector if (needs_finalize_) { // Finalize() requires a dst tuple but we don't actually need the result, // so allocate a single dummy tuple to avoid accumulating memory. - Tuple* dummy_dst = NULL; + Tuple* dummy_dst = nullptr; dummy_dst = Tuple::create(output_tuple_desc_->byte_size(), mem_pool_.get()); while (!it.AtEnd()) { Tuple* tuple = it.GetTuple(); @@ -693,7 +693,7 @@ Status PartitionedAggregationNode::close(RuntimeState* state) { // Iterate through the remaining rows in the hash table and call Serialize/Finalize on // them in order to free any memory allocated by UDAs - if (output_partition_ != NULL) { + if (output_partition_ != nullptr) { CleanupHashTbl(output_partition_->agg_fn_evals, output_iterator_); output_partition_->Close(false); } @@ -793,7 +793,7 @@ Status PartitionedAggregationNode::Partition::SerializeStreamForSpilling() { // TODO: if it happens to not be a string, we could serialize in place. This is // a future optimization since it is very unlikely to have a serialize phase // for those UDAs. - DCHECK(parent->serialize_stream_.get() != NULL); + DCHECK(parent->serialize_stream_.get() != nullptr); DCHECK(!parent->serialize_stream_->is_pinned()); // Serialize and copy the spilled partition's stream into the new stream. @@ -812,12 +812,12 @@ Status PartitionedAggregationNode::Partition::SerializeStreamForSpilling() { parent->CleanupHashTbl(agg_fn_evals, it); hash_tbl->Close(); hash_tbl.reset(); - aggregated_row_stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES); + aggregated_row_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES); return status; } } - aggregated_row_stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES); + aggregated_row_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES); aggregated_row_stream.swap(parent->serialize_stream_); // Recreate the serialize_stream (and reserve 1 buffer) now in preparation for // when we need to spill again. We need to have this available before we need @@ -858,7 +858,7 @@ Status PartitionedAggregationNode::Partition::Spill(bool more_aggregate_rows) { NewAggFnEvaluator::Close(agg_fn_evals, parent->state_); agg_fn_evals.clear(); - if (agg_fn_pool.get() != NULL) { + if (agg_fn_pool.get() != nullptr) { agg_fn_pool->free_all(); agg_fn_pool.reset(); } @@ -890,22 +890,22 @@ Status PartitionedAggregationNode::Partition::Spill(bool more_aggregate_rows) { void PartitionedAggregationNode::Partition::Close(bool finalize_rows) { if (is_closed) return; is_closed = true; - if (aggregated_row_stream.get() != NULL) { - if (finalize_rows && hash_tbl.get() != NULL) { + if (aggregated_row_stream.get() != nullptr) { + if (finalize_rows && hash_tbl.get() != nullptr) { // We need to walk all the rows and Finalize them here so the UDA gets a chance // to cleanup. If the hash table is gone (meaning this was spilled), the rows // should have been finalized/serialized in Spill(). parent->CleanupHashTbl(agg_fn_evals, hash_tbl->Begin(parent->ht_ctx_.get())); } - aggregated_row_stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES); + aggregated_row_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES); } - if (hash_tbl.get() != NULL) hash_tbl->Close(); - if (unaggregated_row_stream.get() != NULL) { - unaggregated_row_stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES); + if (hash_tbl.get() != nullptr) hash_tbl->Close(); + if (unaggregated_row_stream.get() != nullptr) { + unaggregated_row_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES); } for (NewAggFnEvaluator* eval : agg_fn_evals) eval->Close(parent->state_); - if (agg_fn_pool.get() != NULL) agg_fn_pool->free_all(); + if (agg_fn_pool.get() != nullptr) agg_fn_pool->free_all(); } Tuple* PartitionedAggregationNode::ConstructSingletonOutputTuple( @@ -922,7 +922,7 @@ Tuple* PartitionedAggregationNode::ConstructIntermediateTuple( const int varlen_size = GroupingExprsVarlenSize(); const int tuple_data_size = fixed_size + varlen_size; uint8_t* tuple_data = pool->try_allocate(tuple_data_size); - if (UNLIKELY(tuple_data == NULL)) { + if (UNLIKELY(tuple_data == nullptr)) { stringstream str; str << "Memory exceed limit. Cannot perform aggregation at node with id $0. Failed " << "to allocate $1 bytes for intermediate tuple. " @@ -933,7 +933,7 @@ Tuple* PartitionedAggregationNode::ConstructIntermediateTuple( << "You can change the limit by session variable exec_mem_limit."; string details = Substitute(str.str(), _id, tuple_data_size); *status = pool->mem_tracker()->MemLimitExceeded(state_, details, tuple_data_size); - return NULL; + return nullptr; } memset(tuple_data, 0, fixed_size); Tuple* intermediate_tuple = reinterpret_cast(tuple_data); @@ -946,7 +946,7 @@ Tuple* PartitionedAggregationNode::ConstructIntermediateTuple( Tuple* PartitionedAggregationNode::ConstructIntermediateTuple( const vector& agg_fn_evals, BufferedTupleStream3* stream, Status* status) { - DCHECK(stream != NULL && status != NULL); + DCHECK(stream != nullptr && status != nullptr); // Allocate space for the entire tuple in the stream. const int fixed_size = intermediate_tuple_desc_->byte_size(); const int varlen_size = GroupingExprsVarlenSize(); @@ -1013,10 +1013,10 @@ void PartitionedAggregationNode::InitAggSlots(const vector& intermediate_tuple_desc_->slots().begin() + grouping_exprs_.size(); for (int i = 0; i < agg_fn_evals.size(); ++i, ++slot_desc) { // To minimize branching on the UpdateTuple path, initialize the result value so that - // the Add() UDA function can ignore the NULL bit of its destination value. E.g. for - // SUM(), if we initialize the destination value to 0 (with the NULL bit set), we can + // the Add() UDA function can ignore the nullptr bit of its destination value. E.g. for + // SUM(), if we initialize the destination value to 0 (with the nullptr bit set), we can // just start adding to the destination value (rather than repeatedly checking the - // destination NULL bit. The codegen'd version of UpdateSlot() exploits this to + // destination nullptr bit. The codegen'd version of UpdateSlot() exploits this to // eliminate a branch per value. // // For boolean and numeric types, the default values are false/0, so the nullable @@ -1029,7 +1029,7 @@ void PartitionedAggregationNode::InitAggSlots(const vector& void PartitionedAggregationNode::UpdateTuple(NewAggFnEvaluator** agg_fn_evals, Tuple* tuple, TupleRow* row, bool is_merge) { - DCHECK(tuple != NULL || agg_fns_.empty()); + DCHECK(tuple != nullptr || agg_fns_.empty()); for (int i = 0; i < agg_fns_.size(); ++i) { if (is_merge) { agg_fn_evals[i]->Merge(row->get_tuple(0), tuple); @@ -1041,7 +1041,7 @@ void PartitionedAggregationNode::UpdateTuple(NewAggFnEvaluator** agg_fn_evals, T Tuple* PartitionedAggregationNode::GetOutputTuple(const vector& agg_fn_evals, Tuple* tuple, MemPool* pool) { - DCHECK(tuple != NULL || agg_fn_evals.empty()) << tuple; + DCHECK(tuple != nullptr || agg_fn_evals.empty()) << tuple; Tuple* dst = tuple; if (needs_finalize_ && intermediate_tuple_id_ != output_tuple_id_) { dst = Tuple::create(output_tuple_desc_->byte_size(), pool); @@ -1060,9 +1060,9 @@ Tuple* PartitionedAggregationNode::GetOutputTuple(const vectorslots()[i]; SlotDescriptor* dst_slot_desc = output_tuple_desc_->slots()[i]; bool src_slot_null = tuple->is_null(src_slot_desc->null_indicator_offset()); - void* src_slot = NULL; + void* src_slot = nullptr; if (!src_slot_null) src_slot = tuple->get_slot(src_slot_desc->tuple_offset()); - RawValue::write(src_slot, dst, dst_slot_desc, NULL); + RawValue::write(src_slot, dst, dst_slot_desc, nullptr); } } return dst; @@ -1139,7 +1139,7 @@ Status PartitionedAggregationNode::CreateHashPartitions(int level, int single_pa if (partition == nullptr) continue; if (partition->aggregated_row_stream == nullptr) { // Failed to create the aggregated row stream - cannot create a hash table. - // Just continue with a NULL hash table so rows will be passed through. + // Just continue with a nullptr hash table so rows will be passed through. DCHECK(is_streaming_preagg_); } else { bool got_memory; @@ -1356,7 +1356,7 @@ Status PartitionedAggregationNode::ProcessStream(BufferedTupleStream3* input_str batch.reset(); } while (!eos); } - input_stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES); + input_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES); return Status::OK(); } diff --git a/be/src/exec/partitioned_aggregation_node.h b/be/src/exec/partitioned_aggregation_node.h index eb8358050b..ff0d823d5a 100644 --- a/be/src/exec/partitioned_aggregation_node.h +++ b/be/src/exec/partitioned_aggregation_node.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_EXEC_NEW_PARTITIONED_AGGREGATION_NODE_H #define DORIS_BE_SRC_EXEC_NEW_PARTITIONED_AGGREGATION_NODE_H -#include #include #include "exec/exec_node.h" @@ -55,7 +54,7 @@ class SlotDescriptor; // /// Each partition contains these structures: /// 1) Hash Table for aggregated rows. This contains just the hash table directory -/// structure but not the rows themselves. This is NULL for spilled partitions when +/// structure but not the rows themselves. This is nullptr for spilled partitions when /// we stop maintaining the hash table. /// 2) MemPool for var-len result data for rows in the hash table. If the aggregate /// function returns a string, we cannot append it to the tuple stream as that @@ -207,7 +206,7 @@ private: /// Permanent and result allocations for these allocators are allocated from /// 'expr_perm_pool_' and 'expr_results_pool_' respectively. std::vector agg_fn_evals_; - boost::scoped_ptr agg_fn_pool_; + std::unique_ptr agg_fn_pool_; /// Exprs used to evaluate input rows std::vector grouping_exprs_; @@ -227,16 +226,16 @@ private: RuntimeState* state_; /// Allocator for hash table memory. - boost::scoped_ptr ht_allocator_; + std::unique_ptr ht_allocator_; /// MemPool used to allocate memory for when we don't have grouping and don't initialize /// the partitioning structures, or during Close() when creating new output tuples. /// For non-grouping aggregations, the ownership of the pool's memory is transferred /// to the output batch on eos. The pool should not be Reset() to allow amortizing /// memory allocation over a series of Reset()/Open()/GetNext()* calls. - boost::scoped_ptr mem_pool_; + std::unique_ptr mem_pool_; // MemPool for allocations made by copying expr results - boost::scoped_ptr expr_results_pool_; + std::unique_ptr expr_results_pool_; /// The current partition and iterator to the next row in its hash table that we need /// to return in GetNext() @@ -331,14 +330,14 @@ private: /// BEGIN: Members that must be Reset() /// Result of aggregation w/o GROUP BY. - /// Note: can be NULL even if there is no grouping if the result tuple is 0 width + /// Note: can be nullptr even if there is no grouping if the result tuple is 0 width /// e.g. select 1 from table group by col. Tuple* singleton_output_tuple_; bool singleton_output_tuple_returned_; /// Row batch used as argument to GetNext() for the child node preaggregations. Store /// in node to avoid reallocating for every GetNext() call when streaming. - boost::scoped_ptr child_batch_; + std::unique_ptr child_batch_; /// If true, no more rows to output from partitions. bool partition_eos_; @@ -349,10 +348,10 @@ private: /// Used for hash-related functionality, such as evaluating rows and calculating hashes. /// It also owns the evaluators for the grouping and build expressions used during hash /// table insertion and probing. - boost::scoped_ptr ht_ctx_; + std::unique_ptr ht_ctx_; /// Object pool that holds the Partition objects in hash_partitions_. - boost::scoped_ptr partition_pool_; + std::unique_ptr partition_pool_; /// Current partitions we are partitioning into. IMPALA-5788: For the case where we /// rebuild a spilled partition that fits in memory, all pointers in this vector will @@ -393,7 +392,7 @@ private: /// created and an OK status is returned. Status InitStreams(); - /// Initializes the hash table. 'aggregated_row_stream' must be non-NULL. + /// Initializes the hash table. 'aggregated_row_stream' must be non-nullptr. /// Sets 'got_memory' to true if the hash table was initialised or false on OOM. Status InitHashTable(bool* got_memory); @@ -412,7 +411,7 @@ private: /// if 'more_aggregate_rows' is true or the unaggregated stream otherwise. Status Spill(bool more_aggregate_rows); - bool is_spilled() const { return hash_tbl.get() == NULL; } + bool is_spilled() const { return hash_tbl.get() == nullptr; } PartitionedAggregationNode* parent; @@ -428,36 +427,36 @@ private: const int idx; /// Hash table for this partition. - /// Can be NULL if this partition is no longer maintaining a hash table (i.e. + /// Can be nullptr if this partition is no longer maintaining a hash table (i.e. /// is spilled or we are passing through all rows for this partition). - boost::scoped_ptr hash_tbl; + std::unique_ptr hash_tbl; /// Clone of parent's agg_fn_evals_. Permanent allocations come from /// 'agg_fn_perm_pool' and result allocations come from the ExecNode's /// 'expr_results_pool_'. std::vector agg_fn_evals; - boost::scoped_ptr agg_fn_pool; + std::unique_ptr agg_fn_pool; /// Tuple stream used to store aggregated rows. When the partition is not spilled, /// (meaning the hash table is maintained), this stream is pinned and contains the /// memory referenced by the hash table. When it is spilled, this consumes reservation /// for a write buffer only during repartitioning of aggregated rows. /// - /// For streaming preaggs, this may be NULL if sufficient memory is not available. - /// In that case hash_tbl is also NULL and all rows for the partition will be passed + /// For streaming preaggs, this may be nullptr if sufficient memory is not available. + /// In that case hash_tbl is also nullptr and all rows for the partition will be passed /// through. - boost::scoped_ptr aggregated_row_stream; + std::unique_ptr aggregated_row_stream; - /// Unaggregated rows that are spilled. Always NULL for streaming pre-aggregations. + /// Unaggregated rows that are spilled. Always nullptr for streaming pre-aggregations. /// Always unpinned. Has a write buffer allocated when the partition is spilled and /// unaggregated rows are being processed. - boost::scoped_ptr unaggregated_row_stream; + std::unique_ptr unaggregated_row_stream; }; /// Stream used to store serialized spilled rows. Only used if needs_serialize_ /// is set. This stream is never pinned and only used in Partition::Spill as a /// a temporary buffer. - boost::scoped_ptr serialize_stream_; + std::unique_ptr serialize_stream_; /// Accessor for 'hash_tbls_' that verifies consistency with the partitions. PartitionedHashTable* ALWAYS_INLINE GetHashTable(int partition_idx) { @@ -486,7 +485,7 @@ private: /// Copies grouping values stored in 'ht_ctx_' that were computed over 'current_row_' /// using 'grouping_expr_evals_'. Aggregation expr slots are set to their initial - /// values. Returns NULL if there was not enough memory to allocate the tuple or errors + /// values. Returns nullptr if there was not enough memory to allocate the tuple or errors /// occurred. In which case, 'status' is set. Allocates tuple and var-len data for /// grouping exprs from stream. Var-len data for aggregate exprs is allocated from the /// FunctionContexts, so is stored outside the stream. If stream's small buffers get @@ -495,7 +494,7 @@ private: BufferedTupleStream3* stream, Status* status); /// Constructs intermediate tuple, allocating memory from pool instead of the stream. - /// Returns NULL and sets status if there is not enough memory to allocate the tuple. + /// Returns nullptr and sets status if there is not enough memory to allocate the tuple. Tuple* ConstructIntermediateTuple(const std::vector& agg_fn_evals, MemPool* pool, Status* status); @@ -657,7 +656,7 @@ private: /// Tries to build the first partition in 'spilled_partitions_'. /// If successful, set *built_partition to the partition. The caller owns the partition /// and is responsible for closing it. If unsuccessful because the partition could not - /// fit in memory, set *built_partition to NULL and append the spilled partition to the + /// fit in memory, set *built_partition to nullptr and append the spilled partition to the /// head of 'spilled_partitions_' so it can be processed by /// RepartitionSpilledPartition(). Status BuildSpilledPartition(Partition** built_partition); diff --git a/be/src/exec/partitioned_aggregation_node_ir.cc b/be/src/exec/partitioned_aggregation_node_ir.cc index 6d45a2cf79..ae62c64b6d 100644 --- a/be/src/exec/partitioned_aggregation_node_ir.cc +++ b/be/src/exec/partitioned_aggregation_node_ir.cc @@ -83,7 +83,7 @@ void IR_ALWAYS_INLINE PartitionedAggregationNode::EvalAndHashPrefetchGroup( if (is_null) { expr_vals_cache->SetRowNull(); } else if (config::enable_prefetch) { - if (LIKELY(hash_tbl != NULL)) hash_tbl->PrefetchBucket(hash); + if (LIKELY(hash_tbl != nullptr)) hash_tbl->PrefetchBucket(hash); } expr_vals_cache->NextRow(); } @@ -105,8 +105,8 @@ Status PartitionedAggregationNode::ProcessRow(TupleRow* row, PartitionedHashTabl PartitionedHashTable* hash_tbl = GetHashTable(partition_idx); Partition* dst_partition = hash_partitions_[partition_idx]; DCHECK(dst_partition != nullptr); - DCHECK_EQ(dst_partition->is_spilled(), hash_tbl == NULL); - if (hash_tbl == NULL) { + DCHECK_EQ(dst_partition->is_spilled(), hash_tbl == nullptr); + if (hash_tbl == nullptr) { // This partition is already spilled, just append the row. return AppendSpilledRow(dst_partition, row); } @@ -143,7 +143,7 @@ Status PartitionedAggregationNode::AddIntermediateTuple(Partition* partition, Tu partition->agg_fn_evals, partition->aggregated_row_stream.get(), &process_batch_status_); - if (LIKELY(intermediate_tuple != NULL)) { + if (LIKELY(intermediate_tuple != nullptr)) { UpdateTuple(partition->agg_fn_evals.data(), intermediate_tuple, row, AGGREGATED_ROWS); // After copying and initializing the tuple, insert it into the hash table. insert_it.SetTuple(intermediate_tuple, hash); @@ -188,7 +188,7 @@ Status PartitionedAggregationNode::ProcessBatchStreaming(bool needs_serialize, R // Tuple is not going into hash table, add it to the output batch. Tuple* intermediate_tuple = ConstructIntermediateTuple( agg_fn_evals_, out_batch->tuple_data_pool(), &process_batch_status_); - if (UNLIKELY(intermediate_tuple == NULL)) { + if (UNLIKELY(intermediate_tuple == nullptr)) { DCHECK(!process_batch_status_.ok()); return std::move(process_batch_status_); } @@ -217,7 +217,7 @@ bool PartitionedAggregationNode::TryAddToHashTable(PartitionedHashTableCtx* ht_c PartitionedHashTable* hash_tbl, TupleRow* in_row, uint32_t hash, int* remaining_capacity, Status* status) { - DCHECK(remaining_capacity != NULL); + DCHECK(remaining_capacity != nullptr); DCHECK_EQ(hash_tbl, partition->hash_tbl.get()); DCHECK_GE(*remaining_capacity, 0); bool found; @@ -231,7 +231,7 @@ bool PartitionedAggregationNode::TryAddToHashTable(PartitionedHashTableCtx* ht_c } else { intermediate_tuple = ConstructIntermediateTuple( partition->agg_fn_evals, partition->aggregated_row_stream.get(), status); - if (LIKELY(intermediate_tuple != NULL)) { + if (LIKELY(intermediate_tuple != nullptr)) { it.SetTuple(intermediate_tuple, hash); --(*remaining_capacity); } else { diff --git a/be/src/exec/partitioned_hash_table.cc b/be/src/exec/partitioned_hash_table.cc index b6801a44ee..0f6092e05b 100644 --- a/be/src/exec/partitioned_hash_table.cc +++ b/be/src/exec/partitioned_hash_table.cc @@ -43,8 +43,8 @@ static uint32_t SEED_PRIMES[] = { 1431655781, 1183186591, 622729787, 472882027, 338294347, 275604541, 41161739, 29999999, 27475109, 611603, 16313357, 11380003, 21261403, 33393119, 101, 71043403}; -// Put a non-zero constant in the result location for NULL. -// We don't want(NULL, 1) to hash to the same as (0, 1). +// Put a non-zero constant in the result location for nullptr. +// We don't want(nullptr, 1) to hash to the same as (0, 1). // This needs to be as big as the biggest primitive type since the bytes // get copied directly. // TODO find a better approach, since primitives like CHAR(N) can be up @@ -74,7 +74,7 @@ PartitionedHashTableCtx::PartitionedHashTableCtx(const std::vector& build finds_some_nulls_(std::accumulate(finds_nulls_.begin(), finds_nulls_.end(), false, std::logical_or())), level_(0), - scratch_row_(NULL), + scratch_row_(nullptr), mem_pool_(mem_pool), expr_results_pool_(expr_results_pool) { DCHECK(tracker_ != nullptr); @@ -100,7 +100,7 @@ Status PartitionedHashTableCtx::Init(ObjectPool* pool, RuntimeState* state, int const RowDescriptor& row_desc_probe) { int scratch_row_size = sizeof(Tuple*) * num_build_tuples; scratch_row_ = reinterpret_cast(malloc(scratch_row_size)); - if (UNLIKELY(scratch_row_ == NULL)) { + if (UNLIKELY(scratch_row_ == nullptr)) { return Status::InternalError( Substitute("Failed to allocate $0 bytes for scratch row of " "PartitionedHashTableCtx.", @@ -130,7 +130,7 @@ Status PartitionedHashTableCtx::Create( const std::vector& finds_nulls, int32_t initial_seed, int max_levels, int num_build_tuples, MemPool* mem_pool, MemPool* expr_results_pool, const std::shared_ptr& tracker, const RowDescriptor& row_desc, - const RowDescriptor& row_desc_probe, boost::scoped_ptr* ht_ctx) { + const RowDescriptor& row_desc_probe, std::unique_ptr* ht_ctx) { ht_ctx->reset(new PartitionedHashTableCtx(build_exprs, probe_exprs, stores_nulls, finds_nulls, initial_seed, max_levels, mem_pool, expr_results_pool, tracker)); @@ -150,7 +150,7 @@ Status PartitionedHashTableCtx::Open(RuntimeState* state) { void PartitionedHashTableCtx::Close(RuntimeState* state) { free(scratch_row_); - scratch_row_ = NULL; + scratch_row_ = nullptr; expr_values_cache_.Close(tracker_); for (int i = 0; i < build_expr_evals_.size(); i++) { build_expr_evals_[i]->close(state); @@ -204,14 +204,14 @@ bool PartitionedHashTableCtx::EvalRow(TupleRow* row, const vector& for (int i = 0; i < ctxs.size(); ++i) { void* loc = expr_values_cache_.ExprValuePtr(expr_values, i); void* val = ctxs[i]->get_value(row); - if (val == NULL) { + if (val == nullptr) { // If the table doesn't store nulls, no reason to keep evaluating if (!stores_nulls_) return true; expr_values_null[i] = true; val = reinterpret_cast(&NULL_VALUE); has_null = true; DCHECK_LE(build_exprs_[i]->type().get_slot_size(), sizeof(NULL_VALUE)); - RawValue::write(val, loc, build_exprs_[i]->type(), NULL); + RawValue::write(val, loc, build_exprs_[i]->type(), nullptr); } else { expr_values_null[i] = false; DCHECK_LE(build_exprs_[i]->type().get_slot_size(), sizeof(NULL_VALUE)); @@ -257,7 +257,7 @@ bool PartitionedHashTableCtx::Equals(TupleRow* build_row, const uint8_t* expr_va const uint8_t* expr_values_null) const noexcept { for (int i = 0; i < build_expr_evals_.size(); ++i) { void* val = build_expr_evals_[i]->get_value(build_row); - if (val == NULL) { + if (val == nullptr) { if (!(FORCE_NULL_EQUALITY || finds_nulls_[i])) return false; if (!expr_values_null[i]) return false; continue; @@ -281,13 +281,13 @@ template bool PartitionedHashTableCtx::Equals(TupleRow* build_row, PartitionedHashTableCtx::ExprValuesCache::ExprValuesCache() : capacity_(0), - cur_expr_values_(NULL), - cur_expr_values_null_(NULL), - cur_expr_values_hash_(NULL), - cur_expr_values_hash_end_(NULL), - expr_values_array_(NULL), - expr_values_null_array_(NULL), - expr_values_hash_array_(NULL), + cur_expr_values_(nullptr), + cur_expr_values_null_(nullptr), + cur_expr_values_hash_(nullptr), + cur_expr_values_hash_end_(nullptr), + expr_values_array_(nullptr), + expr_values_null_array_(nullptr), + expr_values_hash_array_(nullptr), null_bitmap_(0) {} Status PartitionedHashTableCtx::ExprValuesCache::Init(RuntimeState* state, @@ -338,10 +338,10 @@ Status PartitionedHashTableCtx::ExprValuesCache::Init(RuntimeState* state, void PartitionedHashTableCtx::ExprValuesCache::Close(const std::shared_ptr& tracker) { if (capacity_ == 0) return; - cur_expr_values_ = NULL; - cur_expr_values_null_ = NULL; - cur_expr_values_hash_ = NULL; - cur_expr_values_hash_end_ = NULL; + cur_expr_values_ = nullptr; + cur_expr_values_null_ = nullptr; + cur_expr_values_hash_ = nullptr; + cur_expr_values_hash_end_ = nullptr; expr_values_array_.reset(); expr_values_null_array_.reset(); expr_values_hash_array_.reset(); @@ -402,11 +402,11 @@ PartitionedHashTable::PartitionedHashTable(bool quadratic_probing, Suballocator* stores_duplicates_(stores_duplicates), quadratic_probing_(quadratic_probing), total_data_page_size_(0), - next_node_(NULL), + next_node_(nullptr), node_remaining_current_page_(0), num_duplicate_nodes_(0), max_num_buckets_(max_num_buckets), - buckets_(NULL), + buckets_(nullptr), num_buckets_(num_buckets), num_filled_buckets_(0), num_buckets_with_duplicates_(0), @@ -419,7 +419,7 @@ PartitionedHashTable::PartitionedHashTable(bool quadratic_probing, Suballocator* num_resizes_(0) { DCHECK_EQ((num_buckets & (num_buckets - 1)), 0) << "num_buckets must be a power of 2"; DCHECK_GT(num_buckets, 0) << "num_buckets must be larger than 0"; - DCHECK(stores_tuples_ || stream != NULL); + DCHECK(stores_tuples_ || stream != nullptr); } Status PartitionedHashTable::Init(bool* got_memory) { @@ -486,7 +486,7 @@ Status PartitionedHashTable::ResizeBuckets(int64_t num_buckets, std::unique_ptr new_allocation; RETURN_IF_ERROR(allocator_->Allocate(new_size, &new_allocation)); - if (new_allocation == NULL) { + if (new_allocation == nullptr) { *got_memory = false; return Status::OK(); } @@ -501,7 +501,7 @@ Status PartitionedHashTable::ResizeBuckets(int64_t num_buckets, Bucket* bucket_to_copy = &buckets_[iter.bucket_idx_]; bool found = false; int64_t bucket_idx = - Probe(new_buckets, num_buckets, NULL, bucket_to_copy->hash, &found); + Probe(new_buckets, num_buckets, nullptr, bucket_to_copy->hash, &found); DCHECK(!found); DCHECK_NE(bucket_idx, Iterator::BUCKET_NOT_FOUND) << " Probe failed even though " @@ -537,7 +537,7 @@ void PartitionedHashTable::DebugStringTuple(std::stringstream& ss, HtData& htdat } else { ss << "(" << htdata.flat_row << ")"; } - if (desc != NULL) { + if (desc != nullptr) { Tuple* row[num_build_tuples_]; ss << " " << GetRow(htdata, reinterpret_cast(row))->to_string(*desc); } @@ -561,7 +561,7 @@ string PartitionedHashTable::DebugString(bool skip_empty, bool show_match, DuplicateNode* node = buckets_[i].bucketData.duplicates; bool first = true; ss << " [D] "; - while (node != NULL) { + while (node != nullptr) { if (!first) ss << ","; DebugStringTuple(ss, node->htdata, desc); node = node->next; diff --git a/be/src/exec/partitioned_hash_table.h b/be/src/exec/partitioned_hash_table.h index 8cfa4a560d..7dd4dc4c21 100644 --- a/be/src/exec/partitioned_hash_table.h +++ b/be/src/exec/partitioned_hash_table.h @@ -18,8 +18,6 @@ #ifndef DORIS_BE_SRC_EXEC_NEW_PARTITIONED_HASH_TABLE_H #define DORIS_BE_SRC_EXEC_NEW_PARTITIONED_HASH_TABLE_H -#include -#include #include #include @@ -117,7 +115,7 @@ public: int num_build_tuples, MemPool* mem_pool, MemPool* expr_results_pool, const std::shared_ptr& tracker, const RowDescriptor& row_desc, const RowDescriptor& row_desc_probe, - boost::scoped_ptr* ht_ctx); + std::unique_ptr* ht_ctx); /// Initialize the build and probe expression evaluators. Status Open(RuntimeState* state); @@ -141,14 +139,14 @@ public: TupleRow* ALWAYS_INLINE scratch_row() const { return scratch_row_; } /// Returns the results of the expression at 'expr_idx' evaluated at the current row. - /// This value is invalid if the expr evaluated to NULL. + /// This value is invalid if the expr evaluated to nullptr. /// TODO: this is an awkward abstraction but aggregation node can take advantage of /// it and save some expr evaluation calls. void* ALWAYS_INLINE ExprValue(int expr_idx) const { return expr_values_cache_.ExprValuePtr(expr_values_cache_.cur_expr_values(), expr_idx); } - /// Returns if the expression at 'expr_idx' is evaluated to NULL for the current row. + /// Returns if the expression at 'expr_idx' is evaluated to nullptr for the current row. bool ALWAYS_INLINE ExprValueNull(int expr_idx) const { return static_cast(*(expr_values_cache_.cur_expr_values_null() + expr_idx)); } @@ -158,7 +156,7 @@ public: /// 'cur_expr_values_', the nullness of expressions values in 'cur_expr_values_null_', /// and the hashed expression values in 'cur_expr_values_hash_'. Returns false if this /// row should be rejected (doesn't need to be processed further) because it contains - /// NULL. These need to be inlined in the IR module so we can find and replace the + /// nullptr. These need to be inlined in the IR module so we can find and replace the /// calls to EvalBuildRow()/EvalProbeRow(). bool IR_ALWAYS_INLINE EvalAndHashBuild(TupleRow* row); bool IR_ALWAYS_INLINE EvalAndHashProbe(TupleRow* row); @@ -188,7 +186,7 @@ public: /// expression in each row. 'cur_expr_values_null_' is a pointer into this array. /// - 'expr_values_hash_array_' is an array of cached hash values of the rows. /// 'cur_expr_values_hash_' is a pointer into this array. - /// - 'null_bitmap_' is a bitmap which indicates rows evaluated to NULL. + /// - 'null_bitmap_' is a bitmap which indicates rows evaluated to nullptr. /// /// ExprValuesCache provides an iterator like interface for performing a write pass /// followed by a read pass. We refrain from providing an interface for random accesses @@ -273,7 +271,7 @@ public: uint8_t* ALWAYS_INLINE cur_expr_values() const { return cur_expr_values_; } /// Returns null indicator bytes for the current row, one per expression. Non-zero - /// bytes mean NULL, zero bytes mean non-NULL. Indexed by the expression index. + /// bytes mean nullptr, zero bytes mean non-nullptr. Indexed by the expression index. /// These are uint8_t instead of bool to simplify codegen with IRBuilder. /// TODO: is there actually a valid reason why this is necessary for codegen? uint8_t* ALWAYS_INLINE cur_expr_values_null() const { return cur_expr_values_null_; } @@ -325,20 +323,20 @@ public: /// Array for caching up to 'capacity_' number of rows worth of evaluated expression /// values. Each row consumes 'expr_values_bytes_per_row_' number of bytes. - boost::scoped_array expr_values_array_; + std::unique_ptr expr_values_array_; /// Array for caching up to 'capacity_' number of rows worth of null booleans. /// Each row contains 'num_exprs_' booleans to indicate nullness of expression values. - /// Used when the hash table supports NULL. Use 'uint8_t' to guarantee each entry is 1 + /// Used when the hash table supports nullptr. Use 'uint8_t' to guarantee each entry is 1 /// byte as sizeof(bool) is implementation dependent. The IR depends on this /// assumption. - boost::scoped_array expr_values_null_array_; + std::unique_ptr expr_values_null_array_; /// Array for caching up to 'capacity_' number of rows worth of hashed values. - boost::scoped_array expr_values_hash_array_; + std::unique_ptr expr_values_hash_array_; /// One bit for each row. A bit is set if that row is not hashed as it's evaluated - /// to NULL but the hash table doesn't support NULL. Such rows may still be included + /// to nullptr but the hash table doesn't support nullptr. Such rows may still be included /// in outputs for certain join types (e.g. left anti joins). Bitmap null_bitmap_; @@ -421,7 +419,7 @@ private: uint32_t HashVariableLenRow(const uint8_t* expr_values, const uint8_t* expr_values_null) const; /// Evaluate the exprs over row, storing the values into 'expr_values' and nullness into - /// 'expr_values_null'. Returns whether any expr evaluated to NULL. This will be + /// 'expr_values_null'. Returns whether any expr evaluated to nullptr. This will be /// replaced by codegen. bool EvalRow(TupleRow* row, const std::vector& ctxs, uint8_t* expr_values, uint8_t* expr_values_null) noexcept; @@ -515,7 +513,7 @@ private: /// TODO: Fold this flag in the next pointer below. bool matched; - /// Chain to next duplicate node, NULL when end of list. + /// Chain to next duplicate node, nullptr when end of list. DuplicateNode* next; HtData htdata; }; @@ -555,7 +553,7 @@ public: /// hash table. /// - num_build_tuples: number of Tuples in the build tuple row. /// - tuple_stream: the tuple stream which contains the tuple rows index by the - /// hash table. Can be NULL if the rows contain only a single tuple, in which + /// hash table. Can be nullptr if the rows contain only a single tuple, in which /// case the 'tuple_stream' is unused. /// - max_num_buckets: the maximum number of buckets that can be stored. If we /// try to grow the number of buckets to a larger number, the inserts will fail. @@ -717,7 +715,10 @@ public: public: IR_ALWAYS_INLINE Iterator() - : table_(NULL), scratch_row_(NULL), bucket_idx_(BUCKET_NOT_FOUND), node_(NULL) {} + : table_(nullptr), + scratch_row_(nullptr), + bucket_idx_(BUCKET_NOT_FOUND), + node_(nullptr) {} /// Iterates to the next element. It should be called only if !AtEnd(). void IR_ALWAYS_INLINE Next(); @@ -797,10 +798,10 @@ private: /// Performs the probing operation according to the probing algorithm (linear or /// quadratic. Returns one of the following: /// (a) the index of the bucket that contains the entry that matches with the last row - /// evaluated in 'ht_ctx'. If 'ht_ctx' is NULL then it does not check for row + /// evaluated in 'ht_ctx'. If 'ht_ctx' is nullptr then it does not check for row /// equality and returns the index of the first empty bucket. /// (b) the index of the first empty bucket according to the probing algorithm (linear - /// or quadratic), if the entry is not in the hash table or 'ht_ctx' is NULL. + /// or quadratic), if the entry is not in the hash table or 'ht_ctx' is nullptr. /// (c) Iterator::BUCKET_NOT_FOUND if the probe was not successful, i.e. the maximum /// distance was traveled without finding either an empty or a matching bucket. /// Using the returned index value, the caller can create an iterator that can be @@ -822,7 +823,7 @@ private: PartitionedHashTableCtx* ht_ctx, uint32_t hash, bool* found); /// Performs the insert logic. Returns the HtData* of the bucket or duplicate node - /// where the data should be inserted. Returns NULL if the insert was not successful + /// where the data should be inserted. Returns nullptr if the insert was not successful /// and either sets 'status' to OK if it failed because not enough reservation was /// available or the error if an error was encountered. HtData* IR_ALWAYS_INLINE InsertInternal(PartitionedHashTableCtx* ht_ctx, Status* status); @@ -848,8 +849,8 @@ private: /// the bucket is converted to a DuplicateNode. That is, the contents of 'data' of the /// bucket are copied to a DuplicateNode and 'data' is updated to pointing to a /// DuplicateNode. - /// Returns NULL and sets 'status' to OK if the node array could not grow, i.e. there - /// was not enough memory to allocate a new DuplicateNode. Returns NULL and sets + /// Returns nullptr and sets 'status' to OK if the node array could not grow, i.e. there + /// was not enough memory to allocate a new DuplicateNode. Returns nullptr and sets /// 'status' to an error if another error was encountered. DuplicateNode* IR_ALWAYS_INLINE InsertDuplicateNode(int64_t bucket_idx, Status* status); @@ -888,7 +889,7 @@ private: /// Suballocator to allocate data pages and hash table buckets with. Suballocator* allocator_; - /// Stream contains the rows referenced by the hash table. Can be NULL if the + /// Stream contains the rows referenced by the hash table. Can be nullptr if the /// row only contains a single tuple, in which case the TupleRow indirection /// is removed by the hash table. BufferedTupleStream3* tuple_stream_; diff --git a/be/src/exec/partitioned_hash_table.inline.h b/be/src/exec/partitioned_hash_table.inline.h index 0432ccbcb7..32a556231f 100644 --- a/be/src/exec/partitioned_hash_table.inline.h +++ b/be/src/exec/partitioned_hash_table.inline.h @@ -53,7 +53,7 @@ template inline int64_t PartitionedHashTable::Probe(Bucket* buckets, int64_t num_buckets, PartitionedHashTableCtx* ht_ctx, uint32_t hash, bool* found) { - DCHECK(buckets != NULL); + DCHECK(buckets != nullptr); DCHECK_GT(num_buckets, 0); *found = false; int64_t bucket_idx = hash & (num_buckets - 1); @@ -66,7 +66,7 @@ inline int64_t PartitionedHashTable::Probe(Bucket* buckets, int64_t num_buckets, Bucket* bucket = &buckets[bucket_idx]; if (LIKELY(!bucket->filled)) return bucket_idx; if (hash == bucket->hash) { - if (ht_ctx != NULL && + if (ht_ctx != nullptr && ht_ctx->Equals(GetRow(bucket, ht_ctx->scratch_row_))) { *found = true; return bucket_idx; @@ -102,7 +102,7 @@ inline PartitionedHashTable::HtData* PartitionedHashTable::InsertInternal( if (found) { // We need to insert a duplicate node, note that this may fail to allocate memory. DuplicateNode* new_node = InsertDuplicateNode(bucket_idx, status); - if (UNLIKELY(new_node == NULL)) return NULL; + if (UNLIKELY(new_node == nullptr)) return nullptr; return &new_node->htdata; } else { PrepareBucketForInsert(bucket_idx, hash); @@ -115,7 +115,7 @@ inline bool PartitionedHashTable::Insert(PartitionedHashTableCtx* ht_ctx, Status* status) { HtData* htdata = InsertInternal(ht_ctx, status); // If successful insert, update the contents of the newly inserted entry with 'idx'. - if (LIKELY(htdata != NULL)) { + if (LIKELY(htdata != nullptr)) { if (stores_tuples()) { htdata->tuple = row->get_tuple(0); } else { @@ -145,7 +145,7 @@ inline PartitionedHashTable::Iterator PartitionedHashTable::FindProbeRow( int64_t bucket_idx = Probe(buckets_, num_buckets_, ht_ctx, hash, &found); if (found) { return Iterator(this, ht_ctx->scratch_row(), bucket_idx, - stores_duplicates() ? buckets_[bucket_idx].bucketData.duplicates : NULL); + stores_duplicates() ? buckets_[bucket_idx].bucketData.duplicates : nullptr); } return End(); } @@ -156,7 +156,7 @@ inline PartitionedHashTable::Iterator PartitionedHashTable::FindBuildRowBucket( ++num_probes_; uint32_t hash = ht_ctx->expr_values_cache()->CurExprValuesHash(); int64_t bucket_idx = Probe(buckets_, num_buckets_, ht_ctx, hash, found); - DuplicateNode* duplicates = NULL; + DuplicateNode* duplicates = nullptr; if (stores_duplicates() && LIKELY(bucket_idx != Iterator::BUCKET_NOT_FOUND)) { duplicates = buckets_[bucket_idx].bucketData.duplicates; } @@ -166,7 +166,7 @@ inline PartitionedHashTable::Iterator PartitionedHashTable::FindBuildRowBucket( inline PartitionedHashTable::Iterator PartitionedHashTable::Begin( const PartitionedHashTableCtx* ctx) { int64_t bucket_idx = Iterator::BUCKET_NOT_FOUND; - DuplicateNode* node = NULL; + DuplicateNode* node = nullptr; NextFilledBucket(&bucket_idx, &node); return Iterator(this, ctx->scratch_row(), bucket_idx, node); } @@ -174,7 +174,7 @@ inline PartitionedHashTable::Iterator PartitionedHashTable::Begin( inline PartitionedHashTable::Iterator PartitionedHashTable::FirstUnmatched( PartitionedHashTableCtx* ctx) { int64_t bucket_idx = Iterator::BUCKET_NOT_FOUND; - DuplicateNode* node = NULL; + DuplicateNode* node = nullptr; NextFilledBucket(&bucket_idx, &node); Iterator it(this, ctx->scratch_row(), bucket_idx, node); // Check whether the bucket, or its first duplicate node, is matched. If it is not @@ -191,13 +191,13 @@ inline void PartitionedHashTable::NextFilledBucket(int64_t* bucket_idx, Duplicat ++*bucket_idx; for (; *bucket_idx < num_buckets_; ++*bucket_idx) { if (buckets_[*bucket_idx].filled) { - *node = stores_duplicates() ? buckets_[*bucket_idx].bucketData.duplicates : NULL; + *node = stores_duplicates() ? buckets_[*bucket_idx].bucketData.duplicates : nullptr; return; } } // Reached the end of the hash table. *bucket_idx = Iterator::BUCKET_NOT_FOUND; - *node = NULL; + *node = nullptr; } inline void PartitionedHashTable::PrepareBucketForInsert(int64_t bucket_idx, uint32_t hash) { @@ -230,7 +230,7 @@ inline PartitionedHashTable::DuplicateNode* PartitionedHashTable::InsertDuplicat // Allocate one duplicate node for the new data and one for the preexisting data, // if needed. while (node_remaining_current_page_ < 1 + !bucket->hasDuplicates) { - if (UNLIKELY(!GrowNodeArray(status))) return NULL; + if (UNLIKELY(!GrowNodeArray(status))) return nullptr; } if (!bucket->hasDuplicates) { // This is the first duplicate in this bucket. It means that we need to convert @@ -238,7 +238,7 @@ inline PartitionedHashTable::DuplicateNode* PartitionedHashTable::InsertDuplicat next_node_->htdata.flat_row = bucket->bucketData.htdata.flat_row; DCHECK(!bucket->matched); next_node_->matched = false; - next_node_->next = NULL; + next_node_->next = nullptr; AppendNextNode(bucket); bucket->hasDuplicates = true; ++num_buckets_with_duplicates_; @@ -262,10 +262,10 @@ inline TupleRow* IR_ALWAYS_INLINE PartitionedHashTable::GetRow(HtData& htdata, inline TupleRow* IR_ALWAYS_INLINE PartitionedHashTable::GetRow(Bucket* bucket, TupleRow* row) const { - DCHECK(bucket != NULL); + DCHECK(bucket != nullptr); if (UNLIKELY(stores_duplicates() && bucket->hasDuplicates)) { DuplicateNode* duplicate = bucket->bucketData.duplicates; - DCHECK(duplicate != NULL); + DCHECK(duplicate != nullptr); return GetRow(duplicate->htdata, row); } else { return GetRow(bucket->bucketData.htdata, row); @@ -274,11 +274,11 @@ inline TupleRow* IR_ALWAYS_INLINE PartitionedHashTable::GetRow(Bucket* bucket, inline TupleRow* IR_ALWAYS_INLINE PartitionedHashTable::Iterator::GetRow() const { DCHECK(!AtEnd()); - DCHECK(table_ != NULL); - DCHECK(scratch_row_ != NULL); + DCHECK(table_ != nullptr); + DCHECK(scratch_row_ != nullptr); Bucket* bucket = &table_->buckets_[bucket_idx_]; if (UNLIKELY(table_->stores_duplicates() && bucket->hasDuplicates)) { - DCHECK(node_ != NULL); + DCHECK(node_ != nullptr); return table_->GetRow(node_->htdata, scratch_row_); } else { return table_->GetRow(bucket->bucketData.htdata, scratch_row_); @@ -291,7 +291,7 @@ inline Tuple* IR_ALWAYS_INLINE PartitionedHashTable::Iterator::GetTuple() const Bucket* bucket = &table_->buckets_[bucket_idx_]; // TODO: To avoid the hasDuplicates check, store the HtData* in the Iterator. if (UNLIKELY(table_->stores_duplicates() && bucket->hasDuplicates)) { - DCHECK(node_ != NULL); + DCHECK(node_ != nullptr); return node_->htdata.tuple; } else { return bucket->bucketData.htdata.tuple; @@ -329,7 +329,7 @@ inline bool PartitionedHashTable::Iterator::IsMatched() const { inline void PartitionedHashTable::Iterator::SetAtEnd() { bucket_idx_ = BUCKET_NOT_FOUND; - node_ = NULL; + node_ = nullptr; } template @@ -345,7 +345,7 @@ inline void PartitionedHashTable::Iterator::PrefetchBucket() { inline void PartitionedHashTable::Iterator::Next() { DCHECK(!AtEnd()); if (table_->stores_duplicates() && table_->buckets_[bucket_idx_].hasDuplicates && - node_->next != NULL) { + node_->next != nullptr) { node_ = node_->next; } else { table_->NextFilledBucket(&bucket_idx_, &node_); @@ -355,11 +355,11 @@ inline void PartitionedHashTable::Iterator::Next() { inline void PartitionedHashTable::Iterator::NextDuplicate() { DCHECK(!AtEnd()); if (table_->stores_duplicates() && table_->buckets_[bucket_idx_].hasDuplicates && - node_->next != NULL) { + node_->next != nullptr) { node_ = node_->next; } else { bucket_idx_ = BUCKET_NOT_FOUND; - node_ = NULL; + node_ = nullptr; } } @@ -368,7 +368,7 @@ inline void PartitionedHashTable::Iterator::NextUnmatched() { Bucket* bucket = &table_->buckets_[bucket_idx_]; // Check if there is any remaining unmatched duplicate node in the current bucket. if (table_->stores_duplicates() && bucket->hasDuplicates) { - while (node_->next != NULL) { + while (node_->next != nullptr) { node_ = node_->next; if (!node_->matched) return; } @@ -381,7 +381,7 @@ inline void PartitionedHashTable::Iterator::NextUnmatched() { if (!table_->stores_duplicates() || !bucket->hasDuplicates) { if (!bucket->matched) return; } else { - while (node_->matched && node_->next != NULL) { + while (node_->matched && node_->next != nullptr) { node_ = node_->next; } if (!node_->matched) return; diff --git a/be/src/exec/pl_task_root.cpp b/be/src/exec/pl_task_root.cpp index fef20d5a91..4b1b0aecb9 100644 --- a/be/src/exec/pl_task_root.cpp +++ b/be/src/exec/pl_task_root.cpp @@ -20,7 +20,7 @@ namespace doris { ExchangeNode::ExchangeNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) - : ExecNode(pool, tnode, descs), _num_senders(0), _stream_recvr(NULL), _next_row_idx(0) {} + : ExecNode(pool, tnode, descs), _num_senders(0), _stream_recvr(nullptr), _next_row_idx(0) {} ExchangeNode::~ExchangeNode() {} @@ -70,7 +70,7 @@ Status ExchangeNode::get_next(RuntimeState* state, RowBatch* output_batch, bool* SCOPED_TIMER(_convert_row_batch_timer); // copy rows until we hit the limit/capacity or until we exhaust _input_batch - while (!reached_limit() && !output_batch->is_full() && _input_batch.get() != NULL && + while (!reached_limit() && !output_batch->is_full() && _input_batch.get() != nullptr && _next_row_idx < _input_batch->capacity()) { TupleRow* src = _input_batch->get_row(_next_row_idx); @@ -104,14 +104,14 @@ Status ExchangeNode::get_next(RuntimeState* state, RowBatch* output_batch, bool* } // we need more rows - if (_input_batch.get() != NULL) { + if (_input_batch.get() != nullptr) { _input_batch->transfer_resource_ownership(output_batch); } bool is_cancelled = true; _input_batch.reset(_stream_recvr->get_batch(&is_cancelled)); - VLOG_FILE << "exch: has batch=" << (_input_batch.get() == NULL ? "false" : "true") - << " #rows=" << (_input_batch.get() != NULL ? _input_batch->num_rows() : 0) + VLOG_FILE << "exch: has batch=" << (_input_batch.get() == nullptr ? "false" : "true") + << " #rows=" << (_input_batch.get() != nullptr ? _input_batch->num_rows() : 0) << " is_cancelled=" << (is_cancelled ? "true" : "false") << " instance_id=" << state->fragment_instance_id(); @@ -119,7 +119,7 @@ Status ExchangeNode::get_next(RuntimeState* state, RowBatch* output_batch, bool* return Status::Cancelled("Cancelled"); } - *eos = (_input_batch.get() == NULL); + *eos = (_input_batch.get() == nullptr); if (*eos) { return Status::OK(); diff --git a/be/src/exec/read_write_util.h b/be/src/exec/read_write_util.h index c13092f995..c52fedecb8 100644 --- a/be/src/exec/read_write_util.h +++ b/be/src/exec/read_write_util.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_READ_WRITE_UTIL_H #define DORIS_BE_SRC_QUERY_EXEC_READ_WRITE_UTIL_H -#include #include #include "common/logging.h" diff --git a/be/src/exec/repeat_node.cpp b/be/src/exec/repeat_node.cpp index 9b07340015..78d937edd2 100644 --- a/be/src/exec/repeat_node.cpp +++ b/be/src/exec/repeat_node.cpp @@ -46,7 +46,7 @@ Status RepeatNode::prepare(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::prepare(state)); _runtime_state = state; _tuple_desc = state->desc_tbl().get_tuple_descriptor(_output_tuple_id); - if (_tuple_desc == NULL) { + if (_tuple_desc == nullptr) { return Status::InternalError("Failed to get tuple descriptor."); } @@ -89,8 +89,8 @@ Status RepeatNode::get_repeated_batch(RowBatch* child_row_batch, int repeat_id_i for (int j = 0; src_it != src_tuple_descs.end() && dst_it != dst_tuple_descs.end(); ++src_it, ++dst_it, ++j) { Tuple* src_tuple = src_row->get_tuple(j); - if (src_tuple == NULL) { - dst_row->set_tuple(j, NULL); + if (src_tuple == nullptr) { + dst_row->set_tuple(j, nullptr); continue; } diff --git a/be/src/exec/row_batch_list.h b/be/src/exec/row_batch_list.h index 706d813897..c5503c444b 100644 --- a/be/src/exec/row_batch_list.h +++ b/be/src/exec/row_batch_list.h @@ -42,7 +42,7 @@ public: class TupleRowIterator { public: // Dummy constructor - TupleRowIterator() : _list(NULL), _row_idx(0) {} + TupleRowIterator() : _list(nullptr), _row_idx(0) {} virtual ~TupleRowIterator() {} // Returns true if this iterator is at the end, i.e. get_row() cannot be called. diff --git a/be/src/exec/s3_reader.h b/be/src/exec/s3_reader.h index 1676ba5b51..0de0b0944e 100644 --- a/be/src/exec/s3_reader.h +++ b/be/src/exec/s3_reader.h @@ -47,7 +47,7 @@ public: /** * This interface is used read a whole message, For example: read a message from kafka. * - * if read eof then return Status::OK and length is set 0 and buf is set NULL, + * if read eof then return Status::OK and length is set 0 and buf is set nullptr, * other return readed bytes. */ virtual Status read_one_message(std::unique_ptr* buf, int64_t* length) override; diff --git a/be/src/exec/scanner_ir.cpp b/be/src/exec/scanner_ir.cpp index 6aea5f6bfe..e14e57458e 100644 --- a/be/src/exec/scanner_ir.cpp +++ b/be/src/exec/scanner_ir.cpp @@ -45,11 +45,11 @@ extern "C" double ir_string_to_double(const char* s, int len, StringParser::Pars } extern "C" bool ir_is_null_string(const char* data, int len) { - return data == NULL || (len == 2 && data[0] == '\\' && data[1] == 'N'); + return data == nullptr || (len == 2 && data[0] == '\\' && data[1] == 'N'); } extern "C" bool ir_generic_is_null_string(const char* s, int slen, const char* n, int nlen) { - return s == NULL || (slen == nlen && StringCompare(s, slen, n, nlen, slen) == 0); + return s == nullptr || (slen == nlen && StringCompare(s, slen, n, nlen, slen) == 0); } #endif } diff --git a/be/src/exec/schema_scan_node.cpp b/be/src/exec/schema_scan_node.cpp index 1a6352cd0d..4a85c0f5fb 100644 --- a/be/src/exec/schema_scan_node.cpp +++ b/be/src/exec/schema_scan_node.cpp @@ -36,18 +36,18 @@ SchemaScanNode::SchemaScanNode(ObjectPool* pool, const TPlanNode& tnode, const D _is_init(false), _table_name(tnode.schema_scan_node.table_name), _tuple_id(tnode.schema_scan_node.tuple_id), - _src_tuple_desc(NULL), - _dest_tuple_desc(NULL), + _src_tuple_desc(nullptr), + _dest_tuple_desc(nullptr), _tuple_idx(0), _slot_num(0), - _tuple_pool(NULL), - _schema_scanner(NULL), - _src_tuple(NULL), - _dest_tuple(NULL) {} + _tuple_pool(nullptr), + _schema_scanner(nullptr), + _src_tuple(nullptr), + _dest_tuple(nullptr) {} SchemaScanNode::~SchemaScanNode() { delete[] reinterpret_cast(_src_tuple); - _src_tuple = NULL; + _src_tuple = nullptr; } Status SchemaScanNode::init(const TPlanNode& tnode, RuntimeState* state) { @@ -94,8 +94,8 @@ Status SchemaScanNode::prepare(RuntimeState* state) { return Status::OK(); } - if (NULL == state) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == state) { + return Status::InternalError("input pointer is nullptr."); } RETURN_IF_ERROR(ScanNode::prepare(state)); @@ -103,14 +103,14 @@ Status SchemaScanNode::prepare(RuntimeState* state) { // new one mem pool _tuple_pool.reset(new (std::nothrow) MemPool(mem_tracker().get())); - if (NULL == _tuple_pool.get()) { + if (nullptr == _tuple_pool.get()) { return Status::InternalError("Allocate MemPool failed."); } // get dest tuple desc _dest_tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id); - if (NULL == _dest_tuple_desc) { + if (nullptr == _dest_tuple_desc) { return Status::InternalError("Failed to get tuple descriptor."); } @@ -119,28 +119,28 @@ Status SchemaScanNode::prepare(RuntimeState* state) { const SchemaTableDescriptor* schema_table = static_cast(_dest_tuple_desc->table_desc()); - if (NULL == schema_table) { + if (nullptr == schema_table) { return Status::InternalError("Failed to get schema table descriptor."); } // new one scanner _schema_scanner.reset(SchemaScanner::create(schema_table->schema_table_type())); - if (NULL == _schema_scanner.get()) { - return Status::InternalError("schema scanner get NULL pointer."); + if (nullptr == _schema_scanner.get()) { + return Status::InternalError("schema scanner get nullptr pointer."); } RETURN_IF_ERROR(_schema_scanner->init(&_scanner_param, _pool)); // get column info from scanner _src_tuple_desc = _schema_scanner->tuple_desc(); - if (NULL == _src_tuple_desc) { + if (nullptr == _src_tuple_desc) { return Status::InternalError("failed to get src schema tuple desc."); } _src_tuple = reinterpret_cast(new (std::nothrow) char[_src_tuple_desc->byte_size()]); - if (NULL == _src_tuple) { + if (nullptr == _src_tuple) { return Status::InternalError("new src tuple failed."); } @@ -189,8 +189,8 @@ Status SchemaScanNode::open(RuntimeState* state) { return Status::InternalError("Open before Init."); } - if (NULL == state) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == state) { + return Status::InternalError("input pointer is nullptr."); } SCOPED_TIMER(_runtime_profile->total_time_counter()); @@ -234,8 +234,8 @@ Status SchemaScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* return Status::InternalError("GetNext before Init."); } - if (NULL == state || NULL == row_batch || NULL == eos) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == state || nullptr == row_batch || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); } RETURN_IF_CANCELLED(state); @@ -250,7 +250,7 @@ Status SchemaScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* int tuple_buffer_size = row_batch->capacity() * _dest_tuple_desc->byte_size(); void* tuple_buffer = _tuple_pool->allocate(tuple_buffer_size); - if (NULL == tuple_buffer) { + if (nullptr == tuple_buffer) { return Status::InternalError("Allocate tuple buffer failed."); } diff --git a/be/src/exec/schema_scan_node.h b/be/src/exec/schema_scan_node.h index 3bbc8043ce..2052de05ca 100644 --- a/be/src/exec/schema_scan_node.h +++ b/be/src/exec/schema_scan_node.h @@ -18,8 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCAN_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCAN_NODE_H -#include - #include "exec/scan_node.h" #include "exec/schema_scanner.h" #include "gen_cpp/Descriptors_types.h" @@ -80,9 +78,9 @@ private: // slot num need to fill in and return int _slot_num; // Pool for allocating tuple data, including all varying-length slots. - boost::scoped_ptr _tuple_pool; + std::unique_ptr _tuple_pool; // Jni helper for scanning an schema table. - boost::scoped_ptr _schema_scanner; + std::unique_ptr _schema_scanner; // Current tuple. Tuple* _src_tuple; Tuple* _dest_tuple; diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp index ef9ded8866..20d796fd7f 100644 --- a/be/src/exec/schema_scanner.cpp +++ b/be/src/exec/schema_scanner.cpp @@ -23,12 +23,12 @@ #include "exec/schema_scanner/schema_dummy_scanner.h" #include "exec/schema_scanner/schema_schema_privileges_scanner.h" #include "exec/schema_scanner/schema_schemata_scanner.h" +#include "exec/schema_scanner/schema_statistics_scanner.h" #include "exec/schema_scanner/schema_table_privileges_scanner.h" #include "exec/schema_scanner/schema_tables_scanner.h" #include "exec/schema_scanner/schema_user_privileges_scanner.h" #include "exec/schema_scanner/schema_variables_scanner.h" #include "exec/schema_scanner/schema_views_scanner.h" -#include "exec/schema_scanner/schema_statistics_scanner.h" namespace doris { @@ -36,10 +36,10 @@ DorisServer* SchemaScanner::_s_doris_server; SchemaScanner::SchemaScanner(ColumnDesc* columns, int column_num) : _is_init(false), - _param(NULL), + _param(nullptr), _columns(columns), _column_num(column_num), - _tuple_desc(NULL) {} + _tuple_desc(nullptr) {} SchemaScanner::~SchemaScanner() {} @@ -56,8 +56,8 @@ Status SchemaScanner::get_next_row(Tuple* tuple, MemPool* pool, bool* eos) { return Status::InternalError("used before initialized."); } - if (NULL == tuple || NULL == pool || NULL == eos) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == tuple || nullptr == pool || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); } *eos = true; @@ -69,7 +69,7 @@ Status SchemaScanner::init(SchemaScannerParam* param, ObjectPool* pool) { return Status::OK(); } - if (NULL == param || NULL == pool || NULL == _columns) { + if (nullptr == param || nullptr == pool || nullptr == _columns) { return Status::InternalError("invalid parameter"); } @@ -154,7 +154,7 @@ Status SchemaScanner::create_tuple_desc(ObjectPool* pool) { SlotDescriptor* slot = pool->add(new (std::nothrow) SlotDescriptor(t_slot_desc)); - if (NULL == slot) { + if (nullptr == slot) { return Status::InternalError("no memory for _tuple_desc."); } @@ -167,7 +167,7 @@ Status SchemaScanner::create_tuple_desc(ObjectPool* pool) { t_tuple_desc.__set_numNullBytes((null_byte * 8 + null_bit + 7) / 8); _tuple_desc = pool->add(new (std::nothrow) TupleDescriptor(t_tuple_desc)); - if (NULL == _tuple_desc) { + if (nullptr == _tuple_desc) { return Status::InternalError("no memory for _tuple_desc."); } diff --git a/be/src/exec/schema_scanner.h b/be/src/exec/schema_scanner.h index 74d0a92b69..c6311dd99d 100644 --- a/be/src/exec/schema_scanner.h +++ b/be/src/exec/schema_scanner.h @@ -46,13 +46,13 @@ struct SchemaScannerParam { int64_t thread_id; SchemaScannerParam() - : db(NULL), - table(NULL), - wild(NULL), - user(NULL), - user_ip(NULL), - current_user_ident(NULL), - ip(NULL), + : db(nullptr), + table(nullptr), + wild(nullptr), + user(nullptr), + user_ip(nullptr), + current_user_ident(nullptr), + ip(nullptr), port(0) {} }; diff --git a/be/src/exec/schema_scanner/schema_columns_scanner.cpp b/be/src/exec/schema_scanner/schema_columns_scanner.cpp index 2b8ce49d8d..390a0af5bc 100644 --- a/be/src/exec/schema_scanner/schema_columns_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_columns_scanner.cpp @@ -68,21 +68,21 @@ Status SchemaColumnsScanner::start(RuntimeState* state) { } // get all database TGetDbsParams db_params; - if (NULL != _param->db) { + if (nullptr != _param->db) { db_params.__set_pattern(*(_param->db)); } - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { db_params.__set_current_user_ident(*_param->current_user_ident); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { db_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { db_params.__set_user_ip(*(_param->user_ip)); } } - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR( SchemaHelper::get_db_names(*(_param->ip), _param->port, db_params, &_db_result)); } else { @@ -393,18 +393,18 @@ Status SchemaColumnsScanner::get_new_desc() { TDescribeTableParams desc_params; desc_params.__set_db(_db_result.dbs[_db_index - 1]); desc_params.__set_table_name(_table_result.tables[_table_index++]); - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { desc_params.__set_current_user_ident(*(_param->current_user_ident)); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { desc_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { desc_params.__set_user_ip(*(_param->user_ip)); } } - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR(SchemaHelper::describe_table(*(_param->ip), _param->port, desc_params, &_desc_result)); } else { @@ -418,21 +418,21 @@ Status SchemaColumnsScanner::get_new_desc() { Status SchemaColumnsScanner::get_new_table() { TGetTablesParams table_params; table_params.__set_db(_db_result.dbs[_db_index++]); - if (NULL != _param->table) { + if (nullptr != _param->table) { table_params.__set_pattern(*(_param->table)); } - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { table_params.__set_current_user_ident(*(_param->current_user_ident)); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { table_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { table_params.__set_user_ip(*(_param->user_ip)); } } - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR(SchemaHelper::get_table_names(*(_param->ip), _param->port, table_params, &_table_result)); } else { @@ -446,8 +446,8 @@ Status SchemaColumnsScanner::get_next_row(Tuple* tuple, MemPool* pool, bool* eos if (!_is_init) { return Status::InternalError("use this class before inited."); } - if (NULL == tuple || NULL == pool || NULL == eos) { - return Status::InternalError("input parameter is NULL."); + if (nullptr == tuple || nullptr == pool || nullptr == eos) { + return Status::InternalError("input parameter is nullptr."); } while (_column_index >= _desc_result.columns.size()) { if (_table_index >= _table_result.tables.size()) { diff --git a/be/src/exec/schema_scanner/schema_helper.cpp b/be/src/exec/schema_scanner/schema_helper.cpp index b0f9c08cfd..f59cc00eda 100644 --- a/be/src/exec/schema_scanner/schema_helper.cpp +++ b/be/src/exec/schema_scanner/schema_helper.cpp @@ -17,8 +17,8 @@ #include "exec/schema_scanner/schema_helper.h" -#include #include +#include #include "exec/text_converter.hpp" #include "gen_cpp/FrontendService.h" diff --git a/be/src/exec/schema_scanner/schema_schema_privileges_scanner.cpp b/be/src/exec/schema_scanner/schema_schema_privileges_scanner.cpp index d0e665eefd..f6756912e6 100644 --- a/be/src/exec/schema_scanner/schema_schema_privileges_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_schema_privileges_scanner.cpp @@ -100,13 +100,13 @@ Status SchemaSchemaPrivilegesScanner::fill_one_row(Tuple* tuple, MemPool* pool) Status SchemaSchemaPrivilegesScanner::fill_one_col(const std::string* src, MemPool* pool, void* slot) { - if (NULL == slot || NULL == pool || NULL == src) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == slot || nullptr == pool || nullptr == src) { + return Status::InternalError("input pointer is nullptr."); } StringValue* str_slot = reinterpret_cast(slot); str_slot->len = src->length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, src->c_str(), str_slot->len); @@ -115,21 +115,21 @@ Status SchemaSchemaPrivilegesScanner::fill_one_col(const std::string* src, MemPo Status SchemaSchemaPrivilegesScanner::get_new_table() { TGetTablesParams table_params; - if (NULL != _param->wild) { + if (nullptr != _param->wild) { table_params.__set_pattern(*(_param->wild)); } - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { table_params.__set_current_user_ident(*(_param->current_user_ident)); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { table_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { table_params.__set_user_ip(*(_param->user_ip)); } } - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR(SchemaHelper::list_schema_privilege_status(*(_param->ip), _param->port, table_params, &_priv_result)); } else { @@ -143,8 +143,8 @@ Status SchemaSchemaPrivilegesScanner::get_next_row(Tuple* tuple, MemPool* pool, if (!_is_init) { return Status::InternalError("Used before initialized."); } - if (NULL == tuple || NULL == pool || NULL == eos) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == tuple || nullptr == pool || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); } if (_priv_index >= _priv_result.privileges.size()) { *eos = true; diff --git a/be/src/exec/schema_scanner/schema_schemata_scanner.cpp b/be/src/exec/schema_scanner/schema_schemata_scanner.cpp index 5f9ae4ebc3..d4a69b37b4 100644 --- a/be/src/exec/schema_scanner/schema_schemata_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_schemata_scanner.cpp @@ -43,21 +43,21 @@ Status SchemaSchemataScanner::start(RuntimeState* state) { return Status::InternalError("used before initial."); } TGetDbsParams db_params; - if (NULL != _param->wild) { + if (nullptr != _param->wild) { db_params.__set_pattern(*(_param->wild)); } - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { db_params.__set_current_user_ident(*(_param->current_user_ident)); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { db_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { db_params.__set_user_ip(*(_param->user_ip)); } } - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR( SchemaHelper::get_db_names(*(_param->ip), _param->port, db_params, &_db_result)); } else { @@ -88,7 +88,7 @@ Status SchemaSchemataScanner::fill_one_row(Tuple* tuple, MemPool* pool) { StringValue* str_slot = reinterpret_cast(slot); str_slot->len = strlen("utf8") + 1; str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memory failed."); } memcpy(str_slot->ptr, "utf8", str_slot->len); @@ -99,7 +99,7 @@ Status SchemaSchemataScanner::fill_one_row(Tuple* tuple, MemPool* pool) { StringValue* str_slot = reinterpret_cast(slot); str_slot->len = strlen("utf8_general_ci") + 1; str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memory failed."); } memcpy(str_slot->ptr, "utf8_general_ci", str_slot->len); @@ -114,8 +114,8 @@ Status SchemaSchemataScanner::get_next_row(Tuple* tuple, MemPool* pool, bool* eo if (!_is_init) { return Status::InternalError("Used before Initialized."); } - if (NULL == tuple || NULL == pool || NULL == eos) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == tuple || nullptr == pool || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); } if (_db_index >= _db_result.dbs.size()) { *eos = true; diff --git a/be/src/exec/schema_scanner/schema_table_privileges_scanner.cpp b/be/src/exec/schema_scanner/schema_table_privileges_scanner.cpp index 993c1bb319..132114bd2c 100644 --- a/be/src/exec/schema_scanner/schema_table_privileges_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_table_privileges_scanner.cpp @@ -109,13 +109,13 @@ Status SchemaTablePrivilegesScanner::fill_one_row(Tuple* tuple, MemPool* pool) { Status SchemaTablePrivilegesScanner::fill_one_col(const std::string* src, MemPool* pool, void* slot) { - if (NULL == slot || NULL == pool || NULL == src) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == slot || nullptr == pool || nullptr == src) { + return Status::InternalError("input pointer is nullptr."); } StringValue* str_slot = reinterpret_cast(slot); str_slot->len = src->length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, src->c_str(), str_slot->len); @@ -124,21 +124,21 @@ Status SchemaTablePrivilegesScanner::fill_one_col(const std::string* src, MemPoo Status SchemaTablePrivilegesScanner::get_new_table() { TGetTablesParams table_params; - if (NULL != _param->wild) { + if (nullptr != _param->wild) { table_params.__set_pattern(*(_param->wild)); } - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { table_params.__set_current_user_ident(*(_param->current_user_ident)); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { table_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { table_params.__set_user_ip(*(_param->user_ip)); } } - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR(SchemaHelper::list_table_privilege_status(*(_param->ip), _param->port, table_params, &_priv_result)); } else { @@ -152,8 +152,8 @@ Status SchemaTablePrivilegesScanner::get_next_row(Tuple* tuple, MemPool* pool, b if (!_is_init) { return Status::InternalError("Used before initialized."); } - if (NULL == tuple || NULL == pool || NULL == eos) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == tuple || nullptr == pool || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); } if (_priv_index >= _priv_result.privileges.size()) { *eos = true; diff --git a/be/src/exec/schema_scanner/schema_tables_scanner.cpp b/be/src/exec/schema_scanner/schema_tables_scanner.cpp index 96e95a5b8f..078ac3f135 100644 --- a/be/src/exec/schema_scanner/schema_tables_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_tables_scanner.cpp @@ -62,21 +62,21 @@ Status SchemaTablesScanner::start(RuntimeState* state) { return Status::InternalError("used before initialized."); } TGetDbsParams db_params; - if (NULL != _param->db) { + if (nullptr != _param->db) { db_params.__set_pattern(*(_param->db)); } - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { db_params.__set_current_user_ident(*(_param->current_user_ident)); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { db_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { db_params.__set_user_ip(*(_param->user_ip)); } } - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR( SchemaHelper::get_db_names(*(_param->ip), _param->port, db_params, &_db_result)); } else { @@ -107,7 +107,7 @@ Status SchemaTablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string* src = &tbl_status.name; str_slot->len = src->length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, src->c_str(), str_slot->len); @@ -119,7 +119,7 @@ Status SchemaTablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string* src = &tbl_status.type; str_slot->len = src->length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, src->c_str(), str_slot->len); @@ -131,7 +131,7 @@ Status SchemaTablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string* src = &tbl_status.engine; str_slot->len = src->length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, src->c_str(), str_slot->len); @@ -213,7 +213,7 @@ Status SchemaTablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string* src = &tbl_status.collation; str_slot->len = src->length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, src->c_str(), str_slot->len); @@ -234,7 +234,7 @@ Status SchemaTablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) { str_slot->ptr = nullptr; } else { str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, src->c_str(), str_slot->len); @@ -247,21 +247,21 @@ Status SchemaTablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) { Status SchemaTablesScanner::get_new_table() { TGetTablesParams table_params; table_params.__set_db(_db_result.dbs[_db_index++]); - if (NULL != _param->wild) { + if (nullptr != _param->wild) { table_params.__set_pattern(*(_param->wild)); } - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { table_params.__set_current_user_ident(*(_param->current_user_ident)); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { table_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { table_params.__set_user_ip(*(_param->user_ip)); } } - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR(SchemaHelper::list_table_status(*(_param->ip), _param->port, table_params, &_table_result)); } else { @@ -275,8 +275,8 @@ Status SchemaTablesScanner::get_next_row(Tuple* tuple, MemPool* pool, bool* eos) if (!_is_init) { return Status::InternalError("Used before initialized."); } - if (NULL == tuple || NULL == pool || NULL == eos) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == tuple || nullptr == pool || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); } while (_table_index >= _table_result.tables.size()) { if (_db_index < _db_result.dbs.size()) { diff --git a/be/src/exec/schema_scanner/schema_user_privileges_scanner.cpp b/be/src/exec/schema_scanner/schema_user_privileges_scanner.cpp index 9436bc90e5..1d931c8bab 100644 --- a/be/src/exec/schema_scanner/schema_user_privileges_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_user_privileges_scanner.cpp @@ -91,13 +91,13 @@ Status SchemaUserPrivilegesScanner::fill_one_row(Tuple* tuple, MemPool* pool) { Status SchemaUserPrivilegesScanner::fill_one_col(const std::string* src, MemPool* pool, void* slot) { - if (NULL == slot || NULL == pool || NULL == src) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == slot || nullptr == pool || nullptr == src) { + return Status::InternalError("input pointer is nullptr."); } StringValue* str_slot = reinterpret_cast(slot); str_slot->len = src->length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, src->c_str(), str_slot->len); @@ -106,21 +106,21 @@ Status SchemaUserPrivilegesScanner::fill_one_col(const std::string* src, MemPool Status SchemaUserPrivilegesScanner::get_new_table() { TGetTablesParams table_params; - if (NULL != _param->wild) { + if (nullptr != _param->wild) { table_params.__set_pattern(*(_param->wild)); } - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { table_params.__set_current_user_ident(*(_param->current_user_ident)); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { table_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { table_params.__set_user_ip(*(_param->user_ip)); } } - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR(SchemaHelper::list_user_privilege_status(*(_param->ip), _param->port, table_params, &_priv_result)); } else { @@ -134,8 +134,8 @@ Status SchemaUserPrivilegesScanner::get_next_row(Tuple* tuple, MemPool* pool, bo if (!_is_init) { return Status::InternalError("Used before initialized."); } - if (NULL == tuple || NULL == pool || NULL == eos) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == tuple || nullptr == pool || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); } if (_priv_index >= _priv_result.privileges.size()) { *eos = true; diff --git a/be/src/exec/schema_scanner/schema_views_scanner.cpp b/be/src/exec/schema_scanner/schema_views_scanner.cpp index f626584148..52d7148e06 100644 --- a/be/src/exec/schema_scanner/schema_views_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_views_scanner.cpp @@ -51,21 +51,21 @@ Status SchemaViewsScanner::start(RuntimeState* state) { return Status::InternalError("used before initialized."); } TGetDbsParams db_params; - if (NULL != _param->db) { + if (nullptr != _param->db) { db_params.__set_pattern(*(_param->db)); } - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { db_params.__set_current_user_ident(*(_param->current_user_ident)); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { db_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { db_params.__set_user_ip(*(_param->user_ip)); } } - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR( SchemaHelper::get_db_names(*(_param->ip), _param->port, db_params, &_db_result)); } else { @@ -96,7 +96,7 @@ Status SchemaViewsScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string* src = &tbl_status.name; str_slot->len = src->length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, src->c_str(), str_slot->len); @@ -108,7 +108,7 @@ Status SchemaViewsScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string* ddl_sql = &tbl_status.ddl_sql; str_slot->len = ddl_sql->length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, ddl_sql->c_str(), str_slot->len); @@ -121,7 +121,7 @@ Status SchemaViewsScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string check_option = "NONE"; str_slot->len = check_option.length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, check_option.c_str(), str_slot->len); @@ -134,7 +134,7 @@ Status SchemaViewsScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string is_updatable = "NO"; str_slot->len = is_updatable.length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, is_updatable.c_str(), str_slot->len); @@ -147,7 +147,7 @@ Status SchemaViewsScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string definer = "root@%"; str_slot->len = definer.length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, definer.c_str(), str_slot->len); @@ -160,7 +160,7 @@ Status SchemaViewsScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string security_type = "DEFINER"; str_slot->len = security_type.length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, security_type.c_str(), str_slot->len); @@ -173,7 +173,7 @@ Status SchemaViewsScanner::fill_one_row(Tuple* tuple, MemPool* pool) { const std::string encoding = "utf8"; str_slot->len = encoding.length(); str_slot->ptr = (char*)pool->allocate(str_slot->len); - if (NULL == str_slot->ptr) { + if (nullptr == str_slot->ptr) { return Status::InternalError("Allocate memcpy failed."); } memcpy(str_slot->ptr, encoding.c_str(), str_slot->len); @@ -187,22 +187,22 @@ Status SchemaViewsScanner::fill_one_row(Tuple* tuple, MemPool* pool) { Status SchemaViewsScanner::get_new_table() { TGetTablesParams table_params; table_params.__set_db(_db_result.dbs[_db_index++]); - if (NULL != _param->wild) { + if (nullptr != _param->wild) { table_params.__set_pattern(*(_param->wild)); } - if (NULL != _param->current_user_ident) { + if (nullptr != _param->current_user_ident) { table_params.__set_current_user_ident(*(_param->current_user_ident)); } else { - if (NULL != _param->user) { + if (nullptr != _param->user) { table_params.__set_user(*(_param->user)); } - if (NULL != _param->user_ip) { + if (nullptr != _param->user_ip) { table_params.__set_user_ip(*(_param->user_ip)); } } table_params.__set_type("VIEW"); - if (NULL != _param->ip && 0 != _param->port) { + if (nullptr != _param->ip && 0 != _param->port) { RETURN_IF_ERROR(SchemaHelper::list_table_status(*(_param->ip), _param->port, table_params, &_table_result)); } else { @@ -216,8 +216,8 @@ Status SchemaViewsScanner::get_next_row(Tuple* tuple, MemPool* pool, bool* eos) if (!_is_init) { return Status::InternalError("Used before initialized."); } - if (NULL == tuple || NULL == pool || NULL == eos) { - return Status::InternalError("input pointer is NULL."); + if (nullptr == tuple || nullptr == pool || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); } while (_table_index >= _table_result.tables.size()) { if (_db_index < _db_result.dbs.size()) { diff --git a/be/src/exec/select_node.cpp b/be/src/exec/select_node.cpp index 1c8585d4c3..25057686c4 100644 --- a/be/src/exec/select_node.cpp +++ b/be/src/exec/select_node.cpp @@ -27,7 +27,7 @@ namespace doris { SelectNode::SelectNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : ExecNode(pool, tnode, descs), - _child_row_batch(NULL), + _child_row_batch(nullptr), _child_row_idx(0), _child_eos(false) {} diff --git a/be/src/exec/select_node.h b/be/src/exec/select_node.h index 1d6c468253..91c723ae88 100644 --- a/be/src/exec/select_node.h +++ b/be/src/exec/select_node.h @@ -18,8 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_SELECT_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_SELECT_NODE_H -#include - #include "exec/exec_node.h" #include "runtime/mem_pool.h" @@ -41,7 +39,7 @@ public: private: // current row batch of child - boost::scoped_ptr _child_row_batch; + std::unique_ptr _child_row_batch; // index of current row in _child_row_batch int _child_row_idx; diff --git a/be/src/exec/set_operation_node.cpp b/be/src/exec/set_operation_node.cpp index 523efe0c86..0ca6dd57c1 100644 --- a/be/src/exec/set_operation_node.cpp +++ b/be/src/exec/set_operation_node.cpp @@ -74,12 +74,12 @@ Status SetOperationNode::close(RuntimeState* state) { RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::CLOSE)); // Must reset _probe_batch in close() to release resources - _probe_batch.reset(NULL); + _probe_batch.reset(nullptr); - if (_hash_tbl.get() != NULL) { + if (_hash_tbl.get() != nullptr) { _hash_tbl->close(); } - if (_build_pool.get() != NULL) { + if (_build_pool.get() != nullptr) { _build_pool->free_all(); } diff --git a/be/src/exec/sort_exec_exprs.cpp b/be/src/exec/sort_exec_exprs.cpp index 35f8e63685..7eceb1894d 100644 --- a/be/src/exec/sort_exec_exprs.cpp +++ b/be/src/exec/sort_exec_exprs.cpp @@ -20,15 +20,16 @@ namespace doris { Status SortExecExprs::init(const TSortInfo& sort_info, ObjectPool* pool) { - return init(sort_info.ordering_exprs, - sort_info.__isset.sort_tuple_slot_exprs ? &sort_info.sort_tuple_slot_exprs : NULL, - pool); + return init( + sort_info.ordering_exprs, + sort_info.__isset.sort_tuple_slot_exprs ? &sort_info.sort_tuple_slot_exprs : nullptr, + pool); } Status SortExecExprs::init(const std::vector& ordering_exprs, const std::vector* sort_tuple_slot_exprs, ObjectPool* pool) { RETURN_IF_ERROR(Expr::create_expr_trees(pool, ordering_exprs, &_lhs_ordering_expr_ctxs)); - if (sort_tuple_slot_exprs != NULL) { + if (sort_tuple_slot_exprs != nullptr) { _materialize_tuple = true; RETURN_IF_ERROR( Expr::create_expr_trees(pool, *sort_tuple_slot_exprs, &_sort_tuple_slot_expr_ctxs)); diff --git a/be/src/exec/sort_exec_exprs.h b/be/src/exec/sort_exec_exprs.h index 03a6ea188c..898fd0bf08 100644 --- a/be/src/exec/sort_exec_exprs.h +++ b/be/src/exec/sort_exec_exprs.h @@ -37,7 +37,7 @@ public: Status init(const TSortInfo& sort_info, ObjectPool* pool); // Initialize the ordering and (optionally) materialization expressions from the thrift - // TExprs into the specified pool. sort_tuple_slot_exprs is NULL if the tuple is not + // TExprs into the specified pool. sort_tuple_slot_exprs is nullptr if the tuple is not // materialized. Status init(const std::vector& ordering_exprs, const std::vector* sort_tuple_slot_exprs, ObjectPool* pool); diff --git a/be/src/exec/spill_sort_node.cc b/be/src/exec/spill_sort_node.cc index 3003058d61..ef527a18ed 100644 --- a/be/src/exec/spill_sort_node.cc +++ b/be/src/exec/spill_sort_node.cc @@ -28,7 +28,7 @@ namespace doris { SpillSortNode::SpillSortNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : ExecNode(pool, tnode, descs), _offset(tnode.sort_node.__isset.offset ? tnode.sort_node.offset : 0), - _sorter(NULL), + _sorter(nullptr), _num_rows_skipped(0) {} SpillSortNode::~SpillSortNode() {} @@ -60,7 +60,7 @@ Status SpillSortNode::open(RuntimeState* state) { // These objects must be created after opening the _sort_exec_exprs. Avoid creating // them after every reset()/open(). - if (_sorter.get() == NULL) { + if (_sorter.get() == nullptr) { TupleRowComparator less_than(_sort_exec_exprs, _is_asc_order, _nulls_first); // Create and initialize the external sort impl object _sorter.reset(new SpillSorter(less_than, _sort_exec_exprs.sort_tuple_slot_expr_ctxs(), @@ -124,7 +124,7 @@ Status SpillSortNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* e Status SpillSortNode::reset(RuntimeState* state) { _num_rows_skipped = 0; - if (_sorter.get() != NULL) { + if (_sorter.get() != nullptr) { _sorter->reset(); } // return ExecNode::reset(state); diff --git a/be/src/exec/spill_sort_node.h b/be/src/exec/spill_sort_node.h index c33c70e8ec..954df0925a 100644 --- a/be/src/exec/spill_sort_node.h +++ b/be/src/exec/spill_sort_node.h @@ -65,7 +65,7 @@ private: // BEGIN: Members that must be reset() // Object used for external sorting. - boost::scoped_ptr _sorter; + std::unique_ptr _sorter; // Keeps track of the number of rows skipped for handling _offset. int64_t _num_rows_skipped; diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp index 9b04f061ce..7d470bf33c 100644 --- a/be/src/exec/tablet_info.cpp +++ b/be/src/exec/tablet_info.cpp @@ -398,7 +398,7 @@ uint32_t OlapTablePartitionParam::_compute_dist_hash(Tuple* key) const { if (slot != nullptr) { hash_val = RawValue::zlib_crc32(slot, slot_desc->type(), hash_val); } else { - //NULL is treat as 0 when hash + //nullptr is treat as 0 when hash static const int INT_VALUE = 0; static const TypeDescriptor INT_TYPE(TYPE_INT); hash_val = RawValue::zlib_crc32(&INT_VALUE, INT_TYPE, hash_val); diff --git a/be/src/exec/topn_node.cpp b/be/src/exec/topn_node.cpp index b47ab63c8f..7e98e1d329 100644 --- a/be/src/exec/topn_node.cpp +++ b/be/src/exec/topn_node.cpp @@ -37,11 +37,11 @@ namespace doris { TopNNode::TopNNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : ExecNode(pool, tnode, descs), _offset(tnode.sort_node.__isset.offset ? tnode.sort_node.offset : 0), - _materialized_tuple_desc(NULL), - _tuple_row_less_than(NULL), - _tuple_pool(NULL), + _materialized_tuple_desc(nullptr), + _tuple_row_less_than(nullptr), + _tuple_pool(nullptr), _num_rows_skipped(0), - _priority_queue() {} + _priority_queue(nullptr) {} TopNNode::~TopNNode() {} @@ -167,7 +167,7 @@ Status TopNNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK(); } - if (_tuple_pool.get() != NULL) { + if (_tuple_pool.get() != nullptr) { _tuple_pool->free_all(); } _sort_exec_exprs.close(state); @@ -182,14 +182,14 @@ void TopNNode::insert_tuple_row(TupleRow* input_row) { _tuple_pool->allocate(_materialized_tuple_desc->byte_size())); insert_tuple->materialize_exprs(input_row, *_materialized_tuple_desc, _sort_exec_exprs.sort_tuple_slot_expr_ctxs(), - _tuple_pool.get(), NULL, NULL); + _tuple_pool.get(), nullptr, nullptr); _priority_queue->push(insert_tuple); } else { DCHECK(!_priority_queue->empty()); Tuple* top_tuple = _priority_queue->top(); _tmp_tuple->materialize_exprs(input_row, *_materialized_tuple_desc, - _sort_exec_exprs.sort_tuple_slot_expr_ctxs(), NULL, - NULL, NULL); + _sort_exec_exprs.sort_tuple_slot_expr_ctxs(), nullptr, + nullptr, nullptr); if ((*_tuple_row_less_than)(_tmp_tuple, top_tuple)) { // TODO: DeepCopy will allocate new buffers for the string data. This needs diff --git a/be/src/exec/topn_node.h b/be/src/exec/topn_node.h index 74a7dd02dc..5c291471dd 100644 --- a/be/src/exec/topn_node.h +++ b/be/src/exec/topn_node.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_TOPN_NODE_H #define DORIS_BE_SRC_QUERY_EXEC_TOPN_NODE_H -#include #include #include "exec/exec_node.h" @@ -75,7 +74,7 @@ private: TupleDescriptor* _materialized_tuple_desc; // Comparator for _priority_queue. - boost::scoped_ptr _tuple_row_less_than; + std::unique_ptr _tuple_row_less_than; // After computing the TopN in the priority_queue, pop them and put them in this vector std::vector _sorted_top_n; @@ -86,7 +85,7 @@ private: Tuple* _tmp_tuple; // Stores everything referenced in _priority_queue - boost::scoped_ptr _tuple_pool; + std::unique_ptr _tuple_pool; // Iterator over elements in _sorted_top_n. std::vector::iterator _get_next_iter; @@ -102,7 +101,7 @@ private: // Number of rows skipped. Used for adhering to _offset. int64_t _num_rows_skipped; - // The priority queue will never have more elements in it than the LIMIT. + // The priority queue will never have more elements in it than the LIMIT. std::unique_ptr, TupleRowComparator>> _priority_queue; // END: Members that must be Reset() diff --git a/be/src/exec/union_node.h b/be/src/exec/union_node.h index 756ff87c19..5b49f716e4 100644 --- a/be/src/exec/union_node.h +++ b/be/src/exec/union_node.h @@ -17,8 +17,6 @@ #pragma once -#include - #include "codegen/doris_ir.h" #include "exec/exec_node.h" #include "runtime/row_batch.h" @@ -80,7 +78,7 @@ private: /// Current row batch of current child. We reset the pointer to a new RowBatch /// when switching to a different child. - boost::scoped_ptr _child_batch; + std::unique_ptr _child_batch; /// Index of current row in child_row_batch_. int _child_row_idx; @@ -88,8 +86,8 @@ private: typedef void (*UnionMaterializeBatchFn)(UnionNode*, RowBatch*, uint8_t**); /// Vector of pointers to codegen'ed materialize_batch functions. The vector contains one /// function for each child. The size of the vector should be equal to the number of - /// children. If a child is passthrough, there should be a NULL for that child. If - /// Codegen is disabled, there should be a NULL for every child. + /// children. If a child is passthrough, there should be a nullptr for that child. If + /// Codegen is disabled, there should be a nullptr for every child. std::vector _codegend_union_materialize_batch_fns; /// Saved from the last to GetNext() on the current child. diff --git a/be/src/exprs/agg_fn.h b/be/src/exprs/agg_fn.h index 4c2a9c4c29..aa15a67c89 100644 --- a/be/src/exprs/agg_fn.h +++ b/be/src/exprs/agg_fn.h @@ -96,7 +96,7 @@ public: /// 'agg_fn'. The returned AggFn lives in the ObjectPool of 'state'. 'row_desc' is /// the row descriptor of the input tuple row; 'intermediate_slot_desc' is the slot /// descriptor of the intermediate value; 'output_slot_desc' is the slot descriptor - /// of the output value. On failure, returns error status and sets 'agg_fn' to NULL. + /// of the output value. On failure, returns error status and sets 'agg_fn' to nullptr. static Status Create(const TExpr& texpr, const RowDescriptor& row_desc, const SlotDescriptor& intermediate_slot_desc, const SlotDescriptor& output_slot_desc, RuntimeState* state, diff --git a/be/src/exprs/agg_fn_evaluator.cpp b/be/src/exprs/agg_fn_evaluator.cpp index f8a3d60db1..bcb7e35af1 100644 --- a/be/src/exprs/agg_fn_evaluator.cpp +++ b/be/src/exprs/agg_fn_evaluator.cpp @@ -84,10 +84,10 @@ Status AggFnEvaluator::create(ObjectPool* pool, const TExpr& desc, bool is_analy int node_idx = 0; for (int i = 0; i < desc.nodes[0].num_children; ++i) { ++node_idx; - Expr* expr = NULL; - ExprContext* ctx = NULL; + Expr* expr = nullptr; + ExprContext* ctx = nullptr; RETURN_IF_ERROR( - Expr::create_tree_from_thrift(pool, desc.nodes, NULL, &node_idx, &expr, &ctx)); + Expr::create_tree_from_thrift(pool, desc.nodes, nullptr, &node_idx, &expr, &ctx)); (*result)->_input_exprs_ctxs.push_back(ctx); } return Status::OK(); @@ -102,15 +102,15 @@ AggFnEvaluator::AggFnEvaluator(const TExprNode& desc, bool is_analytic_fn) _function_type(desc.fn.binary_type), _total_mem_consumption(0), _accumulated_mem_consumption(0), - _intermediate_slot_desc(NULL), - _output_slot_desc(NULL), - _init_fn(NULL), - _update_fn(NULL), - _remove_fn(NULL), - _merge_fn(NULL), - _serialize_fn(NULL), - _get_value_fn(NULL), - _finalize_fn(NULL) { + _intermediate_slot_desc(nullptr), + _output_slot_desc(nullptr), + _init_fn(nullptr), + _update_fn(nullptr), + _remove_fn(nullptr), + _merge_fn(nullptr), + _serialize_fn(nullptr), + _get_value_fn(nullptr), + _finalize_fn(nullptr) { if (_fn.name.function_name == "count") { _agg_op = COUNT; } else if (_fn.name.function_name == "min") { @@ -141,11 +141,11 @@ Status AggFnEvaluator::prepare(RuntimeState* state, const RowDescriptor& desc, M const SlotDescriptor* output_slot_desc, const std::shared_ptr& mem_tracker, FunctionContext** agg_fn_ctx) { - DCHECK(pool != NULL); - DCHECK(intermediate_slot_desc != NULL); - DCHECK(_intermediate_slot_desc == NULL); + DCHECK(pool != nullptr); + DCHECK(intermediate_slot_desc != nullptr); + DCHECK(_intermediate_slot_desc == nullptr); _output_slot_desc = output_slot_desc; - //DCHECK(_intermediate_slot_desc == NULL); + //DCHECK(_intermediate_slot_desc == nullptr); _intermediate_slot_desc = intermediate_slot_desc; _string_buffer_len = 0; @@ -162,7 +162,7 @@ Status AggFnEvaluator::prepare(RuntimeState* state, const RowDescriptor& desc, M } // window has intermediate_slot_type - if (_intermediate_slot_desc != NULL) { + if (_intermediate_slot_desc != nullptr) { _staging_intermediate_val = create_any_val(obj_pool, _intermediate_slot_desc->type()); _staging_merge_input_val = create_any_val(obj_pool, _intermediate_slot_desc->type()); } @@ -196,40 +196,40 @@ Status AggFnEvaluator::prepare(RuntimeState* state, const RowDescriptor& desc, M // Load the function pointers. RETURN_IF_ERROR(UserFunctionCache::instance()->get_function_ptr( _fn.id, _fn.aggregate_fn.init_fn_symbol, _fn.hdfs_location, _fn.checksum, &_init_fn, - NULL)); + nullptr)); RETURN_IF_ERROR(UserFunctionCache::instance()->get_function_ptr( _fn.id, _fn.aggregate_fn.update_fn_symbol, _fn.hdfs_location, _fn.checksum, &_update_fn, - NULL)); + nullptr)); // Merge() is not loaded if evaluating the agg fn as an analytic function. if (!_is_analytic_fn) { RETURN_IF_ERROR(UserFunctionCache::instance()->get_function_ptr( _fn.id, _fn.aggregate_fn.merge_fn_symbol, _fn.hdfs_location, _fn.checksum, - &_merge_fn, NULL)); + &_merge_fn, nullptr)); } // Serialize and Finalize are optional if (!_fn.aggregate_fn.serialize_fn_symbol.empty()) { RETURN_IF_ERROR(UserFunctionCache::instance()->get_function_ptr( _fn.id, _fn.aggregate_fn.serialize_fn_symbol, _fn.hdfs_location, _fn.checksum, - &_serialize_fn, NULL)); + &_serialize_fn, nullptr)); } if (!_fn.aggregate_fn.finalize_fn_symbol.empty()) { RETURN_IF_ERROR(UserFunctionCache::instance()->get_function_ptr( _fn.id, _fn.aggregate_fn.finalize_fn_symbol, _fn.hdfs_location, _fn.checksum, - &_finalize_fn, NULL)); + &_finalize_fn, nullptr)); } if (!_fn.aggregate_fn.get_value_fn_symbol.empty()) { RETURN_IF_ERROR(UserFunctionCache::instance()->get_function_ptr( _fn.id, _fn.aggregate_fn.get_value_fn_symbol, _fn.hdfs_location, _fn.checksum, - &_get_value_fn, NULL)); + &_get_value_fn, nullptr)); } if (!_fn.aggregate_fn.remove_fn_symbol.empty()) { RETURN_IF_ERROR(UserFunctionCache::instance()->get_function_ptr( _fn.id, _fn.aggregate_fn.remove_fn_symbol, _fn.hdfs_location, _fn.checksum, - &_remove_fn, NULL)); + &_remove_fn, nullptr)); } std::vector arg_types; @@ -270,7 +270,7 @@ void AggFnEvaluator::close(RuntimeState* state) { // Utility to put val into an AnyVal struct inline void AggFnEvaluator::set_any_val(const void* slot, const TypeDescriptor& type, AnyVal* dst) { - if (slot == NULL) { + if (slot == nullptr) { dst->is_null = true; return; } @@ -411,7 +411,7 @@ inline void AggFnEvaluator::set_output_slot(const AnyVal* src, const SlotDescrip bool AggFnEvaluator::is_in_hybridmap(void* input_val, Tuple* dst, bool* is_add_buckets) { bool is_in_hashset = false; - HybridSetBase* _set_ptr = NULL; + HybridSetBase* _set_ptr = nullptr; _set_ptr = _hybrid_map->find_or_insert_set(reinterpret_cast(dst), is_add_buckets); is_in_hashset = _set_ptr->find(input_val); @@ -424,7 +424,7 @@ bool AggFnEvaluator::is_in_hybridmap(void* input_val, Tuple* dst, bool* is_add_b // This function would be replaced in codegen. void AggFnEvaluator::init(FunctionContext* agg_fn_ctx, Tuple* dst) { - DCHECK(_init_fn != NULL); + DCHECK(_init_fn != nullptr); reinterpret_cast(_init_fn)(agg_fn_ctx, _staging_intermediate_val); set_output_slot(_staging_intermediate_val, _intermediate_slot_desc, dst); agg_fn_ctx->impl()->set_num_updates(0); @@ -659,12 +659,12 @@ bool AggFnEvaluator::sum_distinct_data_filter(TupleRow* row, Tuple* dst) { void AggFnEvaluator::update_or_merge(FunctionContext* agg_fn_ctx, TupleRow* row, Tuple* dst, void* fn) { - if (fn == NULL) { + if (fn == nullptr) { return; } bool dst_null = dst->is_null(_intermediate_slot_desc->null_indicator_offset()); - void* dst_slot = NULL; + void* dst_slot = nullptr; if (!dst_null) { dst_slot = dst->get_slot(_intermediate_slot_desc->tuple_offset()); @@ -780,7 +780,7 @@ void AggFnEvaluator::merge(FunctionContext* agg_fn_ctx, TupleRow* row, Tuple* ds static void set_any_val2(const SlotDescriptor* desc, Tuple* tuple, AnyVal* dst) { bool is_null = tuple->is_null(desc->null_indicator_offset()); - void* slot = NULL; + void* slot = nullptr; if (!is_null) { slot = tuple->get_slot(desc->tuple_offset()); } @@ -788,7 +788,7 @@ static void set_any_val2(const SlotDescriptor* desc, Tuple* tuple, AnyVal* dst) } void AggFnEvaluator::merge(FunctionContext* agg_fn_ctx, Tuple* src, Tuple* dst) { - DCHECK(_merge_fn != NULL); + DCHECK(_merge_fn != nullptr); set_any_val2(_intermediate_slot_desc, dst, _staging_intermediate_val); set_any_val2(_intermediate_slot_desc, src, _staging_merge_input_val); @@ -813,16 +813,16 @@ void AggFnEvaluator::serialize_or_finalize(FunctionContext* agg_fn_ctx, Tuple* s const SlotDescriptor* dst_slot_desc, Tuple* dst, void* fn, bool add_null) { // DCHECK_EQ(dst_slot_desc->type().type, _return_type.type); - if (src == NULL) { + if (src == nullptr) { src = dst; } - if (fn == NULL && src == dst) { + if (fn == nullptr && src == dst) { return; } // same bool src_slot_null = add_null || src->is_null(_intermediate_slot_desc->null_indicator_offset()); - void* src_slot = NULL; + void* src_slot = nullptr; if (!src_slot_null) { src_slot = src->get_slot(_intermediate_slot_desc->tuple_offset()); @@ -832,9 +832,9 @@ void AggFnEvaluator::serialize_or_finalize(FunctionContext* agg_fn_ctx, Tuple* s // if (_is_analytic_fn) { // No fn was given but the src and dst tuples are different (doing a finalize()). // Just copy the src slot into the dst tuple. - if (fn == NULL) { + if (fn == nullptr) { DCHECK_EQ(_intermediate_slot_desc->type(), dst_slot_desc->type()); - RawValue::write(src_slot, dst, dst_slot_desc, NULL); + RawValue::write(src_slot, dst, dst_slot_desc, nullptr); return; } // } @@ -922,11 +922,11 @@ void AggFnEvaluator::serialize_or_finalize(FunctionContext* agg_fn_ctx, Tuple* s } void AggFnEvaluator::serialize(FunctionContext* agg_fn_ctx, Tuple* tuple) { - serialize_or_finalize(agg_fn_ctx, NULL, _intermediate_slot_desc, tuple, _serialize_fn); + serialize_or_finalize(agg_fn_ctx, nullptr, _intermediate_slot_desc, tuple, _serialize_fn); } //void AggFnEvaluator::finalize(FunctionContext* agg_fn_ctx, Tuple* tuple) { -// serialize_or_finalize(agg_fn_ctx, NULL, _output_slot_desc, tuple, _finalize_fn); +// serialize_or_finalize(agg_fn_ctx, nullptr, _output_slot_desc, tuple, _finalize_fn); //} std::string AggFnEvaluator::debug_string(const std::vector& exprs) { diff --git a/be/src/exprs/agg_fn_evaluator.h b/be/src/exprs/agg_fn_evaluator.h index 2ed950df3e..e1af1585e3 100644 --- a/be/src/exprs/agg_fn_evaluator.h +++ b/be/src/exprs/agg_fn_evaluator.h @@ -18,8 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXPRS_AGG_FN_EVALUATOR_H #define DORIS_BE_SRC_QUERY_EXPRS_AGG_FN_EVALUATOR_H -#include -#include #include #include @@ -98,7 +96,7 @@ public: return _agg_op == AggregationOp::COUNT && _input_exprs_ctxs.empty(); } bool is_builtin() const { return _function_type == TFunctionBinaryType::BUILTIN; } - bool supports_serialize() const { return _serialize_fn != NULL; } + bool supports_serialize() const { return _serialize_fn != nullptr; } static std::string debug_string(const std::vector& exprs); std::string debug_string() const; @@ -184,10 +182,10 @@ private: const bool _is_merge; /// Indicates which functions must be loaded. const bool _is_analytic_fn; - boost::scoped_ptr _hybrid_map; + std::unique_ptr _hybrid_map; bool _is_multi_distinct; std::vector _input_exprs_ctxs; - boost::scoped_array _string_buffer; //for count distinct + std::unique_ptr _string_buffer; //for count distinct int _string_buffer_len; //for count distinct std::shared_ptr _mem_tracker; // saved c'tor param @@ -210,7 +208,7 @@ private: // Context to run the aggregate functions. // TODO: this and _pool make this not thread safe but they are easy to duplicate // per thread. - // boost::scoped_ptr _ctx; + // std::unique_ptr _ctx; // Created to a subclass of AnyVal for type(). We use this to convert values // from the UDA interface to the Expr interface. @@ -260,7 +258,8 @@ private: // taking TupleRow to the UDA signature taking AnvVals. // void serialize_or_finalize(FunctionContext* agg_fn_ctx, const SlotDescriptor* dst_slot_desc, Tuple* dst, void* fn); void serialize_or_finalize(FunctionContext* agg_fn_ctx, Tuple* src, - const SlotDescriptor* dst_slot_desc, Tuple* dst, void* fn, bool add_null = false); + const SlotDescriptor* dst_slot_desc, Tuple* dst, void* fn, + bool add_null = false); // Writes the result in src into dst pointed to by _output_slot_desc void set_output_slot(const doris_udf::AnyVal* src, const SlotDescriptor* dst_slot_desc, @@ -271,16 +270,16 @@ private: inline void AggFnEvaluator::add(doris_udf::FunctionContext* agg_fn_ctx, TupleRow* row, Tuple* dst) { agg_fn_ctx->impl()->increment_num_updates(); - update(agg_fn_ctx, row, dst, _is_merge ? _merge_fn : _update_fn, NULL); + update(agg_fn_ctx, row, dst, _is_merge ? _merge_fn : _update_fn, nullptr); } inline void AggFnEvaluator::remove(doris_udf::FunctionContext* agg_fn_ctx, TupleRow* row, Tuple* dst) { agg_fn_ctx->impl()->increment_num_removes(); - update(agg_fn_ctx, row, dst, _remove_fn, NULL); + update(agg_fn_ctx, row, dst, _remove_fn, nullptr); } -inline void AggFnEvaluator::finalize(doris_udf::FunctionContext* agg_fn_ctx, Tuple* src, - Tuple* dst, bool add_null) { +inline void AggFnEvaluator::finalize(doris_udf::FunctionContext* agg_fn_ctx, Tuple* src, Tuple* dst, + bool add_null) { serialize_or_finalize(agg_fn_ctx, src, _output_slot_desc, dst, _finalize_fn, add_null); } inline void AggFnEvaluator::get_value(doris_udf::FunctionContext* agg_fn_ctx, Tuple* src, diff --git a/be/src/exprs/aggregate_functions.cpp b/be/src/exprs/aggregate_functions.cpp index 09295a07bd..231a0fdc66 100644 --- a/be/src/exprs/aggregate_functions.cpp +++ b/be/src/exprs/aggregate_functions.cpp @@ -29,9 +29,9 @@ #include "runtime/datetime_value.h" #include "runtime/runtime_state.h" #include "runtime/string_value.h" +#include "util/counts.h" #include "util/debug_util.h" #include "util/tdigest.h" -#include "util/counts.h" // TODO: this file should be cross compiled and then all of the builtin // aggregate functions will have a codegen enabled path. Then we can remove @@ -51,7 +51,7 @@ using doris_udf::DateTimeVal; using doris_udf::StringVal; using doris_udf::AnyVal; -// Delimiter to use if the separator is NULL. +// Delimiter to use if the separator is nullptr. static const StringVal DEFAULT_STRING_CONCAT_DELIM((uint8_t*)", ", 2); void AggregateFunctions::init_null(FunctionContext*, AnyVal* dst) { @@ -174,19 +174,19 @@ void AggregateFunctions::count_remove(FunctionContext*, const AnyVal& src, BigIn } struct PercentileState { - Counts counts; + Counts counts; double quantile = -1.0; }; void AggregateFunctions::percentile_init(FunctionContext* ctx, StringVal* dst) { dst->is_null = false; dst->len = sizeof(PercentileState); - dst->ptr = (uint8_t*) new PercentileState(); + dst->ptr = (uint8_t*)new PercentileState(); } template void AggregateFunctions::percentile_update(FunctionContext* ctx, const T& src, - const DoubleVal& quantile, StringVal* dst) { + const DoubleVal& quantile, StringVal* dst) { if (src.is_null) { return; } @@ -199,7 +199,8 @@ void AggregateFunctions::percentile_update(FunctionContext* ctx, const T& src, percentile->quantile = quantile.val; } -void AggregateFunctions::percentile_merge(FunctionContext* ctx, const StringVal& src, StringVal* dst) { +void AggregateFunctions::percentile_merge(FunctionContext* ctx, const StringVal& src, + StringVal* dst) { DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(PercentileState), dst->len); @@ -227,7 +228,7 @@ StringVal AggregateFunctions::percentile_serialize(FunctionContext* ctx, const S StringVal result(ctx, sizeof(double) + serialize_size); memcpy(result.ptr, &percentile->quantile, sizeof(double)); percentile->counts.serialize(result.ptr + sizeof(double)); - + delete percentile; return result; } @@ -275,7 +276,7 @@ void AggregateFunctions::percentile_approx_update(FunctionContext* ctx, const T& if (src.is_null) { return; } - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(PercentileApproxState), dst->len); PercentileApproxState* percentile = reinterpret_cast(dst->ptr); @@ -291,7 +292,7 @@ void AggregateFunctions::percentile_approx_update(FunctionContext* ctx, const T& if (src.is_null) { return; } - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(PercentileApproxState), dst->len); PercentileApproxState* percentile = reinterpret_cast(dst->ptr); @@ -315,7 +316,7 @@ StringVal AggregateFunctions::percentile_approx_serialize(FunctionContext* ctx, void AggregateFunctions::percentile_approx_merge(FunctionContext* ctx, const StringVal& src, StringVal* dst) { - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(PercentileApproxState), dst->len); double quantile; @@ -380,7 +381,7 @@ void AggregateFunctions::avg_update(FunctionContext* ctx, const T& src, StringVa if (src.is_null) { return; } - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(AvgState), dst->len); AvgState* avg = reinterpret_cast(dst->ptr); avg->sum += src.val; @@ -392,7 +393,7 @@ void AggregateFunctions::decimalv2_avg_update(FunctionContext* ctx, const Decima if (src.is_null) { return; } - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(DecimalV2AvgState), dst->len); DecimalV2AvgState* avg = reinterpret_cast(dst->ptr); @@ -415,11 +416,11 @@ StringVal AggregateFunctions::decimalv2_avg_serialize(FunctionContext* ctx, cons template void AggregateFunctions::avg_remove(FunctionContext* ctx, const T& src, StringVal* dst) { // Remove doesn't need to explicitly check the number of calls to Update() or Remove() - // because Finalize() returns NULL if count is 0. + // because Finalize() returns nullptr if count is 0. if (src.is_null) { return; } - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(AvgState), dst->len); AvgState* avg = reinterpret_cast(dst->ptr); avg->sum -= src.val; @@ -430,11 +431,11 @@ void AggregateFunctions::avg_remove(FunctionContext* ctx, const T& src, StringVa void AggregateFunctions::decimalv2_avg_remove(doris_udf::FunctionContext* ctx, const DecimalV2Val& src, StringVal* dst) { // Remove doesn't need to explicitly check the number of calls to Update() or Remove() - // because Finalize() returns NULL if count is 0. + // because Finalize() returns nullptr if count is 0. if (src.is_null) { return; } - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(DecimalV2AvgState), dst->len); DecimalV2AvgState* avg = reinterpret_cast(dst->ptr); @@ -449,7 +450,7 @@ void AggregateFunctions::decimalv2_avg_remove(doris_udf::FunctionContext* ctx, void AggregateFunctions::avg_merge(FunctionContext* ctx, const StringVal& src, StringVal* dst) { const AvgState* src_struct = reinterpret_cast(src.ptr); - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(AvgState), dst->len); AvgState* dst_struct = reinterpret_cast(dst->ptr); dst_struct->sum += src_struct->sum; @@ -460,7 +461,7 @@ void AggregateFunctions::decimalv2_avg_merge(FunctionContext* ctx, const StringV StringVal* dst) { DecimalV2AvgState src_struct; memcpy(&src_struct, src.ptr, sizeof(DecimalV2AvgState)); - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(DecimalV2AvgState), dst->len); DecimalV2AvgState* dst_struct = reinterpret_cast(dst->ptr); @@ -514,7 +515,7 @@ void AggregateFunctions::timestamp_avg_update(FunctionContext* ctx, const DateTi if (src.is_null) { return; } - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(AvgState), dst->len); AvgState* avg = reinterpret_cast(dst->ptr); double val = DateTimeValue::from_datetime_val(src); @@ -527,7 +528,7 @@ void AggregateFunctions::timestamp_avg_remove(FunctionContext* ctx, const DateTi if (src.is_null) { return; } - DCHECK(dst->ptr != NULL); + DCHECK(dst->ptr != nullptr); DCHECK_EQ(sizeof(AvgState), dst->len); AvgState* avg = reinterpret_cast(dst->ptr); double val = DateTimeValue::from_datetime_val(src); @@ -719,7 +720,7 @@ void AggregateFunctions::max(FunctionContext*, const LargeIntVal& src, LargeIntV void AggregateFunctions::init_null_string(FunctionContext* c, StringVal* dst) { dst->is_null = true; - dst->ptr = NULL; + dst->ptr = nullptr; dst->len = 0; } @@ -821,7 +822,7 @@ void AggregateFunctions::string_concat(FunctionContext* ctx, const StringVal& sr // string starts with the separator of the first value that arrived in // StringConcatUpdate(). using StringConcatHeader = int64_t; -// Delimiter to use if the separator is NULL. +// Delimiter to use if the separator is nullptr. void AggregateFunctions::string_concat_update(FunctionContext* ctx, const StringVal& src, StringVal* result) { @@ -2013,7 +2014,7 @@ DoubleVal AggregateFunctions::knuth_var_finalize(FunctionContext* ctx, const Str } DecimalV2Val AggregateFunctions::decimalv2_knuth_var_finalize(FunctionContext* ctx, - const StringVal& state_sv) { + const StringVal& state_sv) { DCHECK_EQ(state_sv.len, sizeof(DecimalV2KnuthVarianceState)); DecimalV2KnuthVarianceState* state = reinterpret_cast(state_sv.ptr); @@ -2026,7 +2027,7 @@ DecimalV2Val AggregateFunctions::decimalv2_knuth_var_finalize(FunctionContext* c } DoubleVal AggregateFunctions::knuth_var_pop_finalize(FunctionContext* ctx, - const StringVal& state_sv) { + const StringVal& state_sv) { DCHECK_EQ(state_sv.len, sizeof(KnuthVarianceState)); KnuthVarianceState* state = reinterpret_cast(state_sv.ptr); if (state->count == 0) return DoubleVal::null(); @@ -2036,7 +2037,7 @@ DoubleVal AggregateFunctions::knuth_var_pop_finalize(FunctionContext* ctx, } DecimalV2Val AggregateFunctions::decimalv2_knuth_var_pop_finalize(FunctionContext* ctx, - const StringVal& state_sv) { + const StringVal& state_sv) { DCHECK_EQ(state_sv.len, sizeof(DecimalV2KnuthVarianceState)); DecimalV2KnuthVarianceState* state = reinterpret_cast(state_sv.ptr); @@ -2059,7 +2060,7 @@ DoubleVal AggregateFunctions::knuth_stddev_finalize(FunctionContext* ctx, } DecimalV2Val AggregateFunctions::decimalv2_knuth_stddev_finalize(FunctionContext* ctx, - const StringVal& state_sv) { + const StringVal& state_sv) { DCHECK_EQ(state_sv.len, sizeof(DecimalV2KnuthVarianceState)); DecimalV2KnuthVarianceState* state = reinterpret_cast(state_sv.ptr); @@ -2083,7 +2084,7 @@ DoubleVal AggregateFunctions::knuth_stddev_pop_finalize(FunctionContext* ctx, } DecimalV2Val AggregateFunctions::decimalv2_knuth_stddev_pop_finalize(FunctionContext* ctx, - const StringVal& state_sv) { + const StringVal& state_sv) { DCHECK_EQ(state_sv.len, sizeof(DecimalV2KnuthVarianceState)); DecimalV2KnuthVarianceState* state = reinterpret_cast(state_sv.ptr); @@ -2395,17 +2396,23 @@ template void AggregateFunctions::sum(FunctionContext*, con template void AggregateFunctions::sum(FunctionContext*, const DoubleVal& src, DoubleVal* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, BooleanVal* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, TinyIntVal* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, SmallIntVal* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, IntVal* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, BigIntVal* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, LargeIntVal* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, FloatVal* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, DoubleVal* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, DateTimeVal* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, DecimalV2Val* dst); -template void AggregateFunctions::min_init(doris_udf::FunctionContext *, StringVal* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, + BooleanVal* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, + TinyIntVal* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, + SmallIntVal* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, IntVal* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, BigIntVal* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, + LargeIntVal* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, FloatVal* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, DoubleVal* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, + DateTimeVal* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, + DecimalV2Val* dst); +template void AggregateFunctions::min_init(doris_udf::FunctionContext*, StringVal* dst); template void AggregateFunctions::min(FunctionContext*, const BooleanVal& src, BooleanVal* dst); @@ -2439,17 +2446,23 @@ template void AggregateFunctions::avg_remove(doris_udf:: doris_udf::SmallIntVal const&, doris_udf::StringVal*); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, BooleanVal* dst); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, TinyIntVal* dst); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, SmallIntVal* dst); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, IntVal* dst); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, BigIntVal* dst); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, LargeIntVal* dst); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, FloatVal* dst); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, DoubleVal* dst); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, DateTimeVal* dst); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, DecimalV2Val* dst); -template void AggregateFunctions::max_init(doris_udf::FunctionContext *, StringVal* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, + BooleanVal* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, + TinyIntVal* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, + SmallIntVal* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, IntVal* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, BigIntVal* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, + LargeIntVal* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, FloatVal* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, DoubleVal* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, + DateTimeVal* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, + DecimalV2Val* dst); +template void AggregateFunctions::max_init(doris_udf::FunctionContext*, StringVal* dst); template void AggregateFunctions::max(FunctionContext*, const BooleanVal& src, BooleanVal* dst); @@ -2769,8 +2782,9 @@ template void AggregateFunctions::offset_fn_update(FunctionContext const DecimalV2Val&, DecimalV2Val* dst); -template void AggregateFunctions::percentile_update( - FunctionContext* ctx, const BigIntVal&, const DoubleVal&, StringVal*); +template void AggregateFunctions::percentile_update(FunctionContext* ctx, + const BigIntVal&, const DoubleVal&, + StringVal*); template void AggregateFunctions::percentile_approx_update( FunctionContext* ctx, const doris_udf::DoubleVal&, const doris_udf::DoubleVal&, diff --git a/be/src/exprs/aggregate_functions.h b/be/src/exprs/aggregate_functions.h index 1668305f42..2f1e67bcf3 100644 --- a/be/src/exprs/aggregate_functions.h +++ b/be/src/exprs/aggregate_functions.h @@ -36,9 +36,9 @@ class HybridSetBase; class AggregateFunctions { public: - // Initializes dst to NULL. + // Initializes dst to nullptr. static void init_null(doris_udf::FunctionContext*, doris_udf::AnyVal* dst); - // Initializes dst to NULL and sets dst->ptr to NULL. + // Initializes dst to nullptr and sets dst->ptr to nullptr. static void init_null_string(doris_udf::FunctionContext* c, doris_udf::StringVal* dst); // Initializes dst to 0 and is_null = true. @@ -53,7 +53,6 @@ public: template static void init_zero_not_null(doris_udf::FunctionContext*, T* dst); - template static void sum_remove(doris_udf::FunctionContext* ctx, const SRC_VAL& src, DST_VAL* dst); @@ -78,8 +77,8 @@ public: static void percentile_init(FunctionContext* ctx, StringVal* dst); template - static void percentile_update(FunctionContext* ctx, const T& src, - const DoubleVal& quantile, StringVal* dst); + static void percentile_update(FunctionContext* ctx, const T& src, const DoubleVal& quantile, + StringVal* dst); static void percentile_merge(FunctionContext* ctx, const StringVal& src, StringVal* dst); diff --git a/be/src/exprs/anyval_util.cpp b/be/src/exprs/anyval_util.cpp index 18cecad096..c5fd416d4e 100644 --- a/be/src/exprs/anyval_util.cpp +++ b/be/src/exprs/anyval_util.cpp @@ -39,7 +39,7 @@ Status allocate_any_val(RuntimeState* state, MemPool* pool, const TypeDescriptor const int anyval_size = AnyValUtil::any_val_size(type); const int anyval_alignment = AnyValUtil::any_val_alignment(type); *result = reinterpret_cast(pool->try_allocate_aligned(anyval_size, anyval_alignment)); - if (*result == NULL) { + if (*result == nullptr) { return pool->mem_tracker()->MemLimitExceeded(state, mem_limit_exceeded_msg, anyval_size); } memset(static_cast(*result), 0, anyval_size); @@ -97,7 +97,7 @@ AnyVal* create_any_val(ObjectPool* pool, const TypeDescriptor& type) { default: DCHECK(false) << "Unsupported type: " << type.type; - return NULL; + return nullptr; } } diff --git a/be/src/exprs/anyval_util.h b/be/src/exprs/anyval_util.h index 3eb6cf641d..90d2f59deb 100644 --- a/be/src/exprs/anyval_util.h +++ b/be/src/exprs/anyval_util.h @@ -333,7 +333,7 @@ public: // Utility to put val into an AnyVal struct static void set_any_val(const void* slot, const TypeDescriptor& type, doris_udf::AnyVal* dst) { - if (slot == NULL) { + if (slot == nullptr) { dst->is_null = true; return; } @@ -403,13 +403,13 @@ public: } } - /// Templated equality functions. These assume the input values are not NULL. + /// Templated equality functions. These assume the input values are not nullptr. template static inline bool equals(const PrimitiveType& type, const T& x, const T& y) { return equals_internal(x, y); } - /// Templated equality functions. These assume the input values are not NULL. + /// Templated equality functions. These assume the input values are not nullptr. template static inline bool equals(const T& x, const T& y) { return equals_internal(x, y); diff --git a/be/src/exprs/arithmetic_expr.cpp b/be/src/exprs/arithmetic_expr.cpp index 90cce5a364..06546c2c95 100644 --- a/be/src/exprs/arithmetic_expr.cpp +++ b/be/src/exprs/arithmetic_expr.cpp @@ -43,9 +43,9 @@ Expr* ArithmeticExpr::from_thrift(const TExprNode& node) { case TExprOpcode::BITNOT: return new BitNotExpr(node); default: - return NULL; + return nullptr; } - return NULL; + return nullptr; } #define BINARY_OP_CHECK_ZERO_FN(TYPE, CLASS, FN, OP) \ diff --git a/be/src/exprs/binary_predicate.cpp b/be/src/exprs/binary_predicate.cpp index 6bc7cd65fe..ef772cbc98 100644 --- a/be/src/exprs/binary_predicate.cpp +++ b/be/src/exprs/binary_predicate.cpp @@ -58,7 +58,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) { case TPrimitiveType::DECIMALV2: return new EqDecimalV2ValPred(node); default: - return NULL; + return nullptr; } } case TExprOpcode::NE: { @@ -89,7 +89,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) { case TPrimitiveType::DECIMALV2: return new NeDecimalV2ValPred(node); default: - return NULL; + return nullptr; } } case TExprOpcode::LT: { @@ -120,7 +120,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) { case TPrimitiveType::DECIMALV2: return new LtDecimalV2ValPred(node); default: - return NULL; + return nullptr; } } case TExprOpcode::LE: { @@ -151,7 +151,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) { case TPrimitiveType::DECIMALV2: return new LeDecimalV2ValPred(node); default: - return NULL; + return nullptr; } } case TExprOpcode::GT: { @@ -182,7 +182,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) { case TPrimitiveType::DECIMALV2: return new GtDecimalV2ValPred(node); default: - return NULL; + return nullptr; } } case TExprOpcode::GE: { @@ -213,7 +213,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) { case TPrimitiveType::DECIMALV2: return new GeDecimalV2ValPred(node); default: - return NULL; + return nullptr; } } case TExprOpcode::EQ_FOR_NULL: { @@ -244,13 +244,13 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) { case TPrimitiveType::DECIMALV2: return new EqForNullDecimalV2ValPred(node); default: - return NULL; + return nullptr; } } default: - return NULL; + return nullptr; } - return NULL; + return nullptr; } std::string BinaryPredicate::debug_string() const { diff --git a/be/src/exprs/bitmap_function.h b/be/src/exprs/bitmap_function.h index 2cba6854c7..36e24d51a1 100644 --- a/be/src/exprs/bitmap_function.h +++ b/be/src/exprs/bitmap_function.h @@ -61,8 +61,10 @@ public: const StringVal& dst); static BigIntVal bitmap_min(FunctionContext* ctx, const StringVal& str); - static BigIntVal bitmap_and_count(FunctionContext* ctx, const StringVal& lhs, const StringVal& rhs); - static BigIntVal bitmap_or_count(FunctionContext* ctx, const StringVal& lhs, const StringVal& rhs); + static BigIntVal bitmap_and_count(FunctionContext* ctx, const StringVal& lhs, + const StringVal& rhs); + static BigIntVal bitmap_or_count(FunctionContext* ctx, const StringVal& lhs, + const StringVal& rhs); static StringVal bitmap_serialize(FunctionContext* ctx, const StringVal& src); static StringVal to_bitmap(FunctionContext* ctx, const StringVal& src); @@ -79,7 +81,7 @@ public: // Example: // "" will be converted to an empty Bitmap // "1,2,3" will be converted to Bitmap with its Bit 1, 2, 3 set. - // "-1, 1" will get NULL, because -1 is not a valid bit for Bitmap + // "-1, 1" will get nullptr, because -1 is not a valid bit for Bitmap static StringVal bitmap_from_string(FunctionContext* ctx, const StringVal& input); static BooleanVal bitmap_contains(FunctionContext* ctx, const StringVal& src, const BigIntVal& input); @@ -104,13 +106,14 @@ public: template static BigIntVal bitmap_intersect_finalize(FunctionContext* ctx, const StringVal& src); static BigIntVal bitmap_max(FunctionContext* ctx, const StringVal& str); - static StringVal bitmap_subset_in_range(FunctionContext* ctx, const StringVal& src, - const BigIntVal& range_start, const BigIntVal& range_end); + static StringVal bitmap_subset_in_range(FunctionContext* ctx, const StringVal& src, + const BigIntVal& range_start, + const BigIntVal& range_end); static StringVal bitmap_subset_limit(FunctionContext* ctx, const StringVal& src, - const BigIntVal& range_start, const BigIntVal& cardinality_limit); - static StringVal sub_bitmap(FunctionContext* ctx, const StringVal& src, - const BigIntVal& offset, const BigIntVal& cardinality_limit); - + const BigIntVal& range_start, + const BigIntVal& cardinality_limit); + static StringVal sub_bitmap(FunctionContext* ctx, const StringVal& src, const BigIntVal& offset, + const BigIntVal& cardinality_limit); }; } // namespace doris #endif //DORIS_BE_SRC_QUERY_EXPRS_BITMAP_FUNCTION_H diff --git a/be/src/exprs/block_bloom_filter_impl.cc b/be/src/exprs/block_bloom_filter_impl.cc index bd1ca439ed..4eda3947e8 100644 --- a/be/src/exprs/block_bloom_filter_impl.cc +++ b/be/src/exprs/block_bloom_filter_impl.cc @@ -221,7 +221,7 @@ Status BlockBloomFilter::merge(const BlockBloomFilter& other) { // Moreover for a reference "other" to be an AlwaysTrueFilter the reference needs // to be created from a nullptr and so we get into undefined behavior territory. // Comparing AlwaysTrueFilter with "&other" results in a compiler warning for - // comparing a non-null argument "other" with NULL [-Wnonnull-compare]. + // comparing a non-null argument "other" with nullptr [-Wnonnull-compare]. // For above reasons, guard against it. CHECK_NE(kAlwaysTrueFilter, &other); diff --git a/be/src/exprs/bloomfilter_predicate.cpp b/be/src/exprs/bloomfilter_predicate.cpp index 4bc7584867..5d52d855d8 100644 --- a/be/src/exprs/bloomfilter_predicate.cpp +++ b/be/src/exprs/bloomfilter_predicate.cpp @@ -96,7 +96,7 @@ Status BloomFilterPredicate::prepare(RuntimeState* state, IBloomFilterFuncBase* return Status::OK(); } _filter.reset(filter); - if (NULL == _filter.get()) { + if (nullptr == _filter.get()) { return Status::InternalError("Unknown column type."); } _is_prepare = true; @@ -114,7 +114,7 @@ BooleanVal BloomFilterPredicate::get_boolean_val(ExprContext* ctx, TupleRow* row return BooleanVal(true); } const void* lhs_slot = ctx->get_value(_children[0], row); - if (lhs_slot == NULL) { + if (lhs_slot == nullptr) { return BooleanVal::null(); } _scan_rows++; diff --git a/be/src/exprs/case_expr.cpp b/be/src/exprs/case_expr.cpp index 79fa34c67a..e1290fc7b0 100644 --- a/be/src/exprs/case_expr.cpp +++ b/be/src/exprs/case_expr.cpp @@ -175,8 +175,8 @@ bool CaseExpr::any_val_eq(const TypeDescriptor& type, const AnyVal* v1, const An FunctionContext* fn_ctx = ctx->fn_context(_fn_context_index); \ CaseExprState* state = reinterpret_cast( \ fn_ctx->get_function_state(FunctionContext::THREAD_LOCAL)); \ - DCHECK(state->case_val != NULL); \ - DCHECK(state->when_val != NULL); \ + DCHECK(state->case_val != nullptr); \ + DCHECK(state->when_val != nullptr); \ int num_children = _children.size(); \ if (has_case_expr()) { \ /* All case and when exprs return the same type */ \ diff --git a/be/src/exprs/cast_expr.cpp b/be/src/exprs/cast_expr.cpp index 3e907bddbf..e22a3cf639 100644 --- a/be/src/exprs/cast_expr.cpp +++ b/be/src/exprs/cast_expr.cpp @@ -40,9 +40,9 @@ Expr* CastExpr::from_thrift(const TExprNode& node) { case TPrimitiveType::DOUBLE: return new CastDoubleExpr(node); default: - return NULL; + return nullptr; } - return NULL; + return nullptr; } #define CAST_SAME(CLASS, TYPE, FN) \ diff --git a/be/src/exprs/conditional_functions.h b/be/src/exprs/conditional_functions.h index c579fb98c3..42312a3ae0 100644 --- a/be/src/exprs/conditional_functions.h +++ b/be/src/exprs/conditional_functions.h @@ -108,7 +108,7 @@ protected: IfExpr(const TExprNode& node); }; -// Returns the first non-NULL value in the list, or NULL if there are no non-NULL values. +// Returns the first non-nullptr value in the list, or nullptr if there are no non-nullptr values. class CoalesceExpr : public Expr { public: virtual ~CoalesceExpr(); diff --git a/be/src/exprs/conditional_functions_ir.cpp b/be/src/exprs/conditional_functions_ir.cpp index 14412e0465..f36ac2d3e3 100644 --- a/be/src/exprs/conditional_functions_ir.cpp +++ b/be/src/exprs/conditional_functions_ir.cpp @@ -45,9 +45,9 @@ IF_NULL_COMPUTE_FUNCTION(LargeIntVal, large_int_val); TYPE NullIfExpr::get_##type_name(ExprContext* ctx, TupleRow* row) { \ DCHECK_EQ(_children.size(), 2); \ TYPE lhs_val = _children[0]->get_##type_name(ctx, row); \ - /* Short-circuit in case lhs_val is NULL. Can never be equal to RHS. */ \ + /* Short-circuit in case lhs_val is nullptr. Can never be equal to RHS. */ \ if (lhs_val.is_null) return TYPE::null(); \ - /* Get rhs and return NULL if lhs == rhs, lhs otherwise */ \ + /* Get rhs and return nullptr if lhs == rhs, lhs otherwise */ \ TYPE rhs_val = _children[1]->get_##type_name(ctx, row); \ if (!rhs_val.is_null && AnyValUtil::equals(_children[0]->type(), lhs_val, rhs_val)) { \ return TYPE::null(); \ diff --git a/be/src/exprs/encryption_functions.cpp b/be/src/exprs/encryption_functions.cpp index 6aa3140c7f..8ba3709b0a 100644 --- a/be/src/exprs/encryption_functions.cpp +++ b/be/src/exprs/encryption_functions.cpp @@ -17,8 +17,6 @@ #include "exprs/encryption_functions.h" -#include - #include "exprs/anyval_util.h" #include "exprs/expr.h" #include "runtime/string_value.h" @@ -39,12 +37,12 @@ StringVal EncryptionFunctions::aes_encrypt(FunctionContext* ctx, const StringVal // cipher_len = (clearLen/16 + 1) * 16; int cipher_len = src.len + 16; - boost::scoped_array p; + std::unique_ptr p; p.reset(new char[cipher_len]); int ret_code = AesUtil::encrypt(AES_128_ECB, (unsigned char*)src.ptr, src.len, (unsigned char*)key.ptr, - key.len, NULL, true, (unsigned char*)p.get()); + key.len, nullptr, true, (unsigned char*)p.get()); if (ret_code < 0) { return StringVal::null(); } @@ -58,12 +56,12 @@ StringVal EncryptionFunctions::aes_decrypt(FunctionContext* ctx, const StringVal } int cipher_len = src.len; - boost::scoped_array p; + std::unique_ptr p; p.reset(new char[cipher_len]); int ret_code = AesUtil::decrypt(AES_128_ECB, (unsigned char*)src.ptr, src.len, (unsigned char*)key.ptr, - key.len, NULL, true, (unsigned char*)p.get()); + key.len, nullptr, true, (unsigned char*)p.get()); if (ret_code < 0) { return StringVal::null(); } @@ -76,7 +74,7 @@ StringVal EncryptionFunctions::from_base64(FunctionContext* ctx, const StringVal } int cipher_len = src.len; - boost::scoped_array p; + std::unique_ptr p; p.reset(new char[cipher_len]); int ret_code = base64_decode((const char*)src.ptr, src.len, p.get()); @@ -92,7 +90,7 @@ StringVal EncryptionFunctions::to_base64(FunctionContext* ctx, const StringVal& } int cipher_len = (size_t)(4.0 * ceil((double)src.len / 3.0)); - boost::scoped_array p; + std::unique_ptr p; p.reset(new char[cipher_len]); int ret_code = base64_encode((unsigned char*)src.ptr, src.len, (unsigned char*)p.get()); diff --git a/be/src/exprs/expr.cpp b/be/src/exprs/expr.cpp index 544af354e9..97352dade5 100644 --- a/be/src/exprs/expr.cpp +++ b/be/src/exprs/expr.cpp @@ -65,7 +65,7 @@ void init_builtins_dummy() { // Call one function from each of the classes to pull all the symbols // from that class in. // TODO: is there a better way to do this? - AggregateFunctions::init_null(NULL, NULL); + AggregateFunctions::init_null(nullptr, nullptr); } FunctionContext* Expr::register_function_context(ExprContext* ctx, RuntimeState* state, @@ -243,12 +243,12 @@ Expr::~Expr() {} Status Expr::create_expr_tree(ObjectPool* pool, const TExpr& texpr, ExprContext** ctx) { // input is empty if (texpr.nodes.size() == 0) { - *ctx = NULL; + *ctx = nullptr; return Status::OK(); } int node_idx = 0; - Expr* e = NULL; - Status status = create_tree_from_thrift(pool, texpr.nodes, NULL, &node_idx, &e, ctx); + Expr* e = nullptr; + Status status = create_tree_from_thrift(pool, texpr.nodes, nullptr, &node_idx, &e, ctx); if (status.ok() && node_idx + 1 != texpr.nodes.size()) { status = Status::InternalError( "Expression tree only partially reconstructed. Not all thrift nodes were used."); @@ -280,20 +280,20 @@ Status Expr::create_tree_from_thrift(ObjectPool* pool, const std::vectoradd_child(expr); } else { - DCHECK(root_expr != NULL); - DCHECK(ctx != NULL); + DCHECK(root_expr != nullptr); + DCHECK(ctx != nullptr); *root_expr = expr; *ctx = pool->add(new ExprContext(expr)); } for (int i = 0; i < num_children; i++) { *node_idx += 1; - RETURN_IF_ERROR(create_tree_from_thrift(pool, nodes, expr, node_idx, NULL, NULL)); + RETURN_IF_ERROR(create_tree_from_thrift(pool, nodes, expr, node_idx, nullptr, nullptr)); // we are expecting a child, but have used all nodes // this means we have been given a bad tree and must fail if (*node_idx >= nodes.size()) { @@ -459,8 +459,8 @@ int Expr::compute_results_layout(const std::vector& exprs, std::vectortype().type == TYPE_CHAR || exprs[i]->type().type == TYPE_VARCHAR - || exprs[i]->type().type == TYPE_STRING) { + if (exprs[i]->type().type == TYPE_CHAR || exprs[i]->type().type == TYPE_VARCHAR || + exprs[i]->type().type == TYPE_STRING) { data[i].byte_size = 16; data[i].variable_length = true; } else { @@ -564,9 +564,9 @@ void Expr::close(RuntimeState* state, ExprContext* context, #if 0 if (scope == FunctionContext::FRAGMENT_LOCAL) { // This is the final, non-cloned context to close. Clean up the whole Expr. - if (cache_entry_ != NULL) { + if (cache_entry_ != nullptr) { LibCache::instance()->DecrementUseCount(cache_entry_); - cache_entry_ = NULL; + cache_entry_ = nullptr; } } #endif @@ -574,7 +574,7 @@ void Expr::close(RuntimeState* state, ExprContext* context, Status Expr::clone_if_not_exists(const std::vector& ctxs, RuntimeState* state, std::vector* new_ctxs) { - DCHECK(new_ctxs != NULL); + DCHECK(new_ctxs != nullptr); if (!new_ctxs->empty()) { // 'ctxs' was already cloned into '*new_ctxs', nothing to do. DCHECK_EQ(new_ctxs->size(), ctxs.size()); @@ -665,43 +665,43 @@ const Expr* Expr::expr_without_cast(const Expr* expr) { doris_udf::AnyVal* Expr::get_const_val(ExprContext* context) { if (!is_constant()) { - return NULL; + return nullptr; } - if (_constant_val.get() != NULL) { + if (_constant_val.get() != nullptr) { return _constant_val.get(); } switch (_type.type) { case TYPE_BOOLEAN: { - _constant_val.reset(new BooleanVal(get_boolean_val(context, NULL))); + _constant_val.reset(new BooleanVal(get_boolean_val(context, nullptr))); break; } case TYPE_TINYINT: { - _constant_val.reset(new TinyIntVal(get_tiny_int_val(context, NULL))); + _constant_val.reset(new TinyIntVal(get_tiny_int_val(context, nullptr))); break; } case TYPE_SMALLINT: { - _constant_val.reset(new SmallIntVal(get_small_int_val(context, NULL))); + _constant_val.reset(new SmallIntVal(get_small_int_val(context, nullptr))); break; } case TYPE_INT: { - _constant_val.reset(new IntVal(get_int_val(context, NULL))); + _constant_val.reset(new IntVal(get_int_val(context, nullptr))); break; } case TYPE_BIGINT: { - _constant_val.reset(new BigIntVal(get_big_int_val(context, NULL))); + _constant_val.reset(new BigIntVal(get_big_int_val(context, nullptr))); break; } case TYPE_LARGEINT: { - _constant_val.reset(new LargeIntVal(get_large_int_val(context, NULL))); + _constant_val.reset(new LargeIntVal(get_large_int_val(context, nullptr))); break; } case TYPE_FLOAT: { - _constant_val.reset(new FloatVal(get_float_val(context, NULL))); + _constant_val.reset(new FloatVal(get_float_val(context, nullptr))); break; } case TYPE_DOUBLE: case TYPE_TIME: { - _constant_val.reset(new DoubleVal(get_double_val(context, NULL))); + _constant_val.reset(new DoubleVal(get_double_val(context, nullptr))); break; } case TYPE_CHAR: @@ -709,17 +709,17 @@ doris_udf::AnyVal* Expr::get_const_val(ExprContext* context) { case TYPE_HLL: case TYPE_OBJECT: case TYPE_STRING: { - _constant_val.reset(new StringVal(get_string_val(context, NULL))); + _constant_val.reset(new StringVal(get_string_val(context, nullptr))); break; } case TYPE_DATE: case TYPE_DATETIME: { - _constant_val.reset(new DateTimeVal(get_datetime_val(context, NULL))); + _constant_val.reset(new DateTimeVal(get_datetime_val(context, nullptr))); break; } case TYPE_DECIMALV2: { - _constant_val.reset(new DecimalV2Val(get_decimalv2_val(context, NULL))); + _constant_val.reset(new DecimalV2Val(get_decimalv2_val(context, nullptr))); break; } case TYPE_NULL: { @@ -727,13 +727,13 @@ doris_udf::AnyVal* Expr::get_const_val(ExprContext* context) { break; } case TYPE_ARRAY: { - _constant_val.reset(new CollectionVal(get_array_val(context, NULL))); + _constant_val.reset(new CollectionVal(get_array_val(context, nullptr))); break; } default: DCHECK(false) << "Type not implemented: " << type(); } - DCHECK(_constant_val.get() != NULL); + DCHECK(_constant_val.get() != nullptr); return _constant_val.get(); } diff --git a/be/src/exprs/expr.h b/be/src/exprs/expr.h index 5bc3c50a6b..795dca6bc0 100644 --- a/be/src/exprs/expr.h +++ b/be/src/exprs/expr.h @@ -34,8 +34,7 @@ #include "runtime/tuple_row.h" #include "runtime/types.h" #include "udf/udf.h" -//#include -// + #undef USING_DORIS_UDF #define USING_DORIS_UDF using namespace doris_udf @@ -77,7 +76,7 @@ public: // evaluate expr and return pointer to result. The result is // valid as long as 'row' doesn't change. // TODO: stop having the result cached in this Expr object - void* get_value(TupleRow* row) { return NULL; } + void* get_value(TupleRow* row) { return nullptr; } // Vectorize Evalute expr and return result column index. // Result cached in batch and valid as long as batch. @@ -149,7 +148,7 @@ public: static const Expr* expr_without_cast(const Expr* expr); // Returns true if expr doesn't contain slotrefs, ie, can be evaluated - // with get_value(NULL). The default implementation returns true if all of + // with get_value(nullptr). The default implementation returns true if all of // the children are constant. virtual bool is_constant() const; @@ -177,7 +176,7 @@ public: /// Create a new ScalarExpr based on thrift Expr 'texpr'. The newly created ScalarExpr /// is stored in ObjectPool 'pool' and returned in 'expr' on success. 'row_desc' is the - /// tuple row descriptor of the input tuple row. On failure, 'expr' is set to NULL and + /// tuple row descriptor of the input tuple row. On failure, 'expr' is set to nullptr and /// the expr tree (if created) will be closed. Error status will be returned too. static Status create(const TExpr& texpr, const RowDescriptor& row_desc, RuntimeState* state, ObjectPool* pool, Expr** expr, const std::shared_ptr& tracker); @@ -207,7 +206,7 @@ public: /// Convenience function for opening multiple expr trees. static Status open(const std::vector& ctxs, RuntimeState* state); - /// Clones each ExprContext for multiple expr trees. 'new_ctxs' must be non-NULL. + /// Clones each ExprContext for multiple expr trees. 'new_ctxs' must be non-nullptr. /// Idempotent: if '*new_ctxs' is empty, a clone of each context in 'ctxs' will be added /// to it, and if non-empty, it is assumed CloneIfNotExists() was already called and the /// call is a no-op. The new ExprContexts are created in state->obj_pool(). @@ -232,7 +231,7 @@ public: std::vector* offsets, int* var_result_begin); /// If this expr is constant, evaluates the expr with no input row argument and returns - /// the output. Returns NULL if the argument is not constant. The returned AnyVal* is + /// the output. Returns nullptr if the argument is not constant. The returned AnyVal* is /// owned by this expr. This should only be called after Open() has been called on this /// expr. virtual AnyVal* get_const_val(ExprContext* context); @@ -401,7 +400,7 @@ private: /// Creates an expr tree for the node rooted at 'node_idx' via depth-first traversal. /// parameters /// nodes: vector of thrift expression nodes to be translated - /// parent: parent of node at node_idx (or NULL for node_idx == 0) + /// parent: parent of node at node_idx (or nullptr for node_idx == 0) /// node_idx: /// in: root of TExprNode tree /// out: next node in 'nodes' that isn't part of tree diff --git a/be/src/exprs/expr_context.cpp b/be/src/exprs/expr_context.cpp index fcc67c1b5d..ac2694ab94 100644 --- a/be/src/exprs/expr_context.cpp +++ b/be/src/exprs/expr_context.cpp @@ -35,7 +35,7 @@ namespace doris { ExprContext::ExprContext(Expr* root) - : _fn_contexts_ptr(NULL), + : _fn_contexts_ptr(nullptr), _root(root), _is_clone(false), _prepared(false), @@ -53,7 +53,7 @@ ExprContext::~ExprContext() { Status ExprContext::prepare(RuntimeState* state, const RowDescriptor& row_desc, const std::shared_ptr& tracker) { DCHECK(tracker != nullptr) << std::endl << get_stack_trace(); - DCHECK(_pool.get() == NULL); + DCHECK(_pool.get() == nullptr); _prepared = true; // TODO: use param tracker to replace instance_mem_tracker, be careful about tracker's life cycle // _pool.reset(new MemPool(new MemTracker(-1))); @@ -91,8 +91,8 @@ void ExprContext::close(RuntimeState* state) { for (int i = 0; i < _fn_contexts.size(); ++i) { _fn_contexts[i]->impl()->close(); } - // _pool can be NULL if Prepare() was never called - if (_pool != NULL) { + // _pool can be nullptr if Prepare() was never called + if (_pool != nullptr) { _pool->free_all(); } _closed = true; @@ -111,7 +111,7 @@ int ExprContext::register_func(RuntimeState* state, Status ExprContext::clone(RuntimeState* state, ExprContext** new_ctx) { DCHECK(_prepared); DCHECK(_opened); - DCHECK(*new_ctx == NULL); + DCHECK(*new_ctx == nullptr); *new_ctx = state->obj_pool()->add(new ExprContext(_root)); (*new_ctx)->_pool.reset(new MemPool(_pool->mem_tracker())); @@ -130,7 +130,7 @@ Status ExprContext::clone(RuntimeState* state, ExprContext** new_ctx) { Status ExprContext::clone(RuntimeState* state, ExprContext** new_ctx, Expr* root) { DCHECK(_prepared); DCHECK(_opened); - DCHECK(*new_ctx == NULL); + DCHECK(*new_ctx == nullptr); *new_ctx = state->obj_pool()->add(new ExprContext(root)); (*new_ctx)->_pool.reset(new MemPool(_pool->mem_tracker())); @@ -175,12 +175,12 @@ bool ExprContext::is_nullable() { void* ExprContext::get_value(Expr* e, TupleRow* row) { switch (e->_type.type) { case TYPE_NULL: { - return NULL; + return nullptr; } case TYPE_BOOLEAN: { doris_udf::BooleanVal v = e->get_boolean_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.bool_val = v.val; return &_result.bool_val; @@ -188,7 +188,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_TINYINT: { doris_udf::TinyIntVal v = e->get_tiny_int_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.tinyint_val = v.val; return &_result.tinyint_val; @@ -196,7 +196,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_SMALLINT: { doris_udf::SmallIntVal v = e->get_small_int_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.smallint_val = v.val; return &_result.smallint_val; @@ -204,7 +204,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_INT: { doris_udf::IntVal v = e->get_int_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.int_val = v.val; return &_result.int_val; @@ -212,7 +212,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_BIGINT: { doris_udf::BigIntVal v = e->get_big_int_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.bigint_val = v.val; return &_result.bigint_val; @@ -220,7 +220,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_LARGEINT: { doris_udf::LargeIntVal v = e->get_large_int_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.large_int_val = v.val; return &_result.large_int_val; @@ -228,7 +228,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_FLOAT: { doris_udf::FloatVal v = e->get_float_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.float_val = v.val; return &_result.float_val; @@ -237,7 +237,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_DOUBLE: { doris_udf::DoubleVal v = e->get_double_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.double_val = v.val; return &_result.double_val; @@ -259,7 +259,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_CHAR: { doris_udf::StringVal v = e->get_string_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.string_val.ptr = reinterpret_cast(v.ptr); _result.string_val.len = v.len; @@ -274,7 +274,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_DATETIME: { doris_udf::DateTimeVal v = e->get_datetime_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.datetime_val = DateTimeValue::from_datetime_val(v); return &_result.datetime_val; @@ -282,7 +282,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_DECIMALV2: { DecimalV2Val v = e->get_decimalv2_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.decimalv2_val = DecimalV2Value::from_decimal_val(v); return &_result.decimalv2_val; @@ -290,7 +290,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { case TYPE_ARRAY: { doris_udf::CollectionVal v = e->get_array_val(this, row); if (v.is_null) { - return NULL; + return nullptr; } _result.array_val = CollectionValue::from_collection_val(v); @@ -298,7 +298,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { } default: DCHECK(false) << "Type not implemented: " << e->_type; - return NULL; + return nullptr; } } @@ -376,7 +376,7 @@ Status ExprContext::get_const_value(RuntimeState* state, Expr& expr, AnyVal** co const TypeDescriptor& result_type = expr.type(); ObjectPool* obj_pool = state->obj_pool(); *const_val = create_any_val(obj_pool, result_type); - if (*const_val == NULL) { + if (*const_val == nullptr) { return Status::InternalError("Could not create any val"); } diff --git a/be/src/exprs/expr_context.h b/be/src/exprs/expr_context.h index 5c7f4ecb2d..45896a2636 100644 --- a/be/src/exprs/expr_context.h +++ b/be/src/exprs/expr_context.h @@ -69,7 +69,7 @@ public: /// originals but have their own MemPool and thread-local state. Clone() should be used /// to create an ExprContext for each execution thread that needs to evaluate /// 'root'. Note that clones are already opened. '*new_context' must be initialized by - /// the caller to NULL. + /// the caller to nullptr. Status clone(RuntimeState* state, ExprContext** new_context); Status clone(RuntimeState* state, ExprContext** new_ctx, Expr* root); @@ -81,7 +81,7 @@ public: /// result in result_. void* get_value(TupleRow* row); - /// Convenience functions: print value into 'str' or 'stream'. NULL turns into "NULL". + /// Convenience functions: print value into 'str' or 'stream'. nullptr turns into "NULL". void print_value(TupleRow* row, std::string* str); void print_value(void* value, std::string* str); void print_value(void* value, std::stringstream* stream); @@ -132,7 +132,7 @@ public: bool opened() { return _opened; } /// If 'expr' is constant, evaluates it with no input row argument and returns the - /// result in 'const_val'. Sets 'const_val' to NULL if the argument is not constant. + /// result in 'const_val'. Sets 'const_val' to nullptr if the argument is not constant. /// The returned AnyVal and associated varlen data is owned by this evaluator. This /// should only be called after Open() has been called on this expr. Returns an error /// if there was an error evaluating the expression or if memory could not be allocated diff --git a/be/src/exprs/expr_value.h b/be/src/exprs/expr_value.h index f55a6d6501..62c6be8a1a 100644 --- a/be/src/exprs/expr_value.h +++ b/be/src/exprs/expr_value.h @@ -57,7 +57,7 @@ struct ExprValue { float_val(0.0), double_val(0.0), string_data(), - string_val(NULL, 0), + string_val(nullptr, 0), datetime_val(), decimalv2_val(0), array_val() {} @@ -100,7 +100,7 @@ struct ExprValue { void* set_to_zero(const TypeDescriptor& type) { switch (type.type) { case TYPE_NULL: - return NULL; + return nullptr; case TYPE_BOOLEAN: bool_val = false; @@ -140,7 +140,7 @@ struct ExprValue { default: DCHECK(false); - return NULL; + return nullptr; } } @@ -148,7 +148,7 @@ struct ExprValue { void* set_to_min(const TypeDescriptor& type) { switch (type.type) { case TYPE_NULL: - return NULL; + return nullptr; case TYPE_BOOLEAN: bool_val = false; @@ -188,7 +188,7 @@ struct ExprValue { default: DCHECK(false); - return NULL; + return nullptr; } } @@ -196,7 +196,7 @@ struct ExprValue { void* set_to_max(const TypeDescriptor& type) { switch (type.type) { case TYPE_NULL: - return NULL; + return nullptr; case TYPE_BOOLEAN: bool_val = true; @@ -236,7 +236,7 @@ struct ExprValue { default: DCHECK(false); - return NULL; + return nullptr; } } }; diff --git a/be/src/exprs/in_predicate.cpp b/be/src/exprs/in_predicate.cpp index 9db94cea2c..53143d7d46 100644 --- a/be/src/exprs/in_predicate.cpp +++ b/be/src/exprs/in_predicate.cpp @@ -41,7 +41,7 @@ Status InPredicate::prepare(RuntimeState* state, HybridSetBase* hset) { return Status::OK(); } _hybrid_set.reset(hset); - if (NULL == _hybrid_set) { + if (nullptr == _hybrid_set) { return Status::InternalError("Unknown column type."); } _is_prepare = true; @@ -64,8 +64,8 @@ Status InPredicate::open(RuntimeState* state, ExprContext* context, } } - void* value = context->get_value(_children[i], NULL); - if (value == NULL) { + void* value = context->get_value(_children[i], nullptr); + if (value == nullptr) { _null_in_set = true; continue; } @@ -86,7 +86,7 @@ Status InPredicate::prepare(RuntimeState* state, const RowDescriptor& row_desc, return Status::InternalError("no Function operator in."); } _hybrid_set.reset(HybridSetBase::create_set(_children[0]->type().type)); - if (NULL == _hybrid_set.get()) { + if (nullptr == _hybrid_set.get()) { return Status::InternalError("Unknown column type."); } @@ -96,7 +96,7 @@ Status InPredicate::prepare(RuntimeState* state, const RowDescriptor& row_desc, } void InPredicate::insert(void* value) { - if (NULL == value) { + if (nullptr == value) { _null_in_set = true; } else { _hybrid_set->insert(value); @@ -121,7 +121,7 @@ std::string InPredicate::debug_string() const { // a, b is a column or a expr that contain slot BooleanVal InPredicate::get_boolean_val(ExprContext* ctx, TupleRow* row) { void* lhs_slot = ctx->get_value(_children[0], row); - if (lhs_slot == NULL) { + if (lhs_slot == nullptr) { return BooleanVal::null(); } // if find in const set, return true diff --git a/be/src/exprs/in_predicate.h b/be/src/exprs/in_predicate.h index b90c1b9711..35a8f0b553 100644 --- a/be/src/exprs/in_predicate.h +++ b/be/src/exprs/in_predicate.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_QUERY_EXPRS_IN_PREDICATE_H #define DORIS_BE_SRC_QUERY_EXPRS_IN_PREDICATE_H -#include #include #include @@ -68,7 +67,7 @@ private: const bool _is_not_in; bool _is_prepare; bool _null_in_set; - boost::shared_ptr _hybrid_set; + std::shared_ptr _hybrid_set; }; } // namespace doris diff --git a/be/src/exprs/info_func.cpp b/be/src/exprs/info_func.cpp index 2aab808e17..5f4d1ae29b 100644 --- a/be/src/exprs/info_func.cpp +++ b/be/src/exprs/info_func.cpp @@ -56,7 +56,7 @@ void* InfoFunc::compute_fn(Expr* e, TupleRow* row) { } #endif - return NULL; + return nullptr; } } // namespace doris diff --git a/be/src/exprs/json_functions.cpp b/be/src/exprs/json_functions.cpp index ed04f00bbf..e85f214508 100644 --- a/be/src/exprs/json_functions.cpp +++ b/be/src/exprs/json_functions.cpp @@ -262,14 +262,14 @@ rapidjson::Value* JsonFunctions::match_value(const std::vector& parsed root = &((*root)[col.c_str()]); } } else { - // root is not a nested type, return NULL + // root is not a nested type, return nullptr return nullptr; } } if (UNLIKELY(index != -1)) { // judge the rapidjson:Value, which base the top's result, - // if not array return NULL;else get the index value from the array + // if not array return nullptr;else get the index value from the array if (root->IsArray()) { if (root->IsNull()) { return nullptr; diff --git a/be/src/exprs/like_predicate.cpp b/be/src/exprs/like_predicate.cpp index 933d9e0285..b76d46d79c 100644 --- a/be/src/exprs/like_predicate.cpp +++ b/be/src/exprs/like_predicate.cpp @@ -179,7 +179,7 @@ void LikePredicate::regexp_like_prepare(FunctionContext* context, // If both the pattern and the match parameter are constant, we pre-compile the // regular expression once here. Otherwise, the RE is compiled per row in RegexpLike() if (context->is_arg_constant(1) && context->is_arg_constant(2)) { - StringVal* pattern = NULL; + StringVal* pattern = nullptr; pattern = reinterpret_cast(context->get_constant_arg(1)); if (pattern->is_null) { return; @@ -187,7 +187,7 @@ void LikePredicate::regexp_like_prepare(FunctionContext* context, StringVal* match_parameter = reinterpret_cast(context->get_constant_arg(2)); std::stringstream error; if (match_parameter->is_null) { - error << "NULL match parameter"; + error << "match parameter is null"; context->set_error(error.str().c_str()); return; } diff --git a/be/src/exprs/math_functions.cpp b/be/src/exprs/math_functions.cpp index 53ac852a8b..f2f5aad1c1 100644 --- a/be/src/exprs/math_functions.cpp +++ b/be/src/exprs/math_functions.cpp @@ -250,7 +250,7 @@ DoubleVal MathFunctions::pow(FunctionContext* ctx, const DoubleVal& base, const void MathFunctions::rand_prepare(FunctionContext* ctx, FunctionContext::FunctionStateScope scope) { std::mt19937* generator = reinterpret_cast(ctx->allocate(sizeof(std::mt19937))); - if (UNLIKELY(generator == NULL)) { + if (UNLIKELY(generator == nullptr)) { LOG(ERROR) << "allocate random seed generator failed."; return; } @@ -429,7 +429,7 @@ StringVal MathFunctions::conv_int(FunctionContext* ctx, const BigIntVal& num, // If a negative target base is given, num should be interpreted in 2's complement. if (std::abs(src_base.val) < MIN_BASE || std::abs(src_base.val) > MAX_BASE || std::abs(dest_base.val) < MIN_BASE || std::abs(dest_base.val) > MAX_BASE) { - // Return NULL like Hive does. + // Return nullptr like Hive does. return StringVal::null(); } // Invalid input. @@ -458,7 +458,7 @@ StringVal MathFunctions::conv_string(FunctionContext* ctx, const StringVal& num_ // If a negative target base is given, num should be interpreted in 2's complement. if (std::abs(src_base.val) < MIN_BASE || std::abs(src_base.val) > MAX_BASE || std::abs(dest_base.val) < MIN_BASE || std::abs(dest_base.val) > MAX_BASE) { - // Return NULL like Hive does. + // Return nullptr like Hive does. return StringVal::null(); } // Convert digits in num_str in src_base to decimal. @@ -703,11 +703,11 @@ void* MathFunctions::greatest_bigint(Expr* e, TupleRow* row) { int32_t num_args = e->get_num_children(); int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is NULL, we can return early.. + // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. for (int i = 0; i < num_args; ++i) { int64_t* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == NULL) { - return NULL; + if (arg == nullptr) { + return nullptr; } if (*arg > *reinterpret_cast(e->children()[result_idx]->get_value(row))) { @@ -721,11 +721,11 @@ void* MathFunctions::greatest_double(Expr* e, TupleRow* row) { DCHECK_GE(e->get_num_children(), 1); int32_t num_args = e->get_num_children(); int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is NULL, we can return early.. + // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. for (int i = 0; i < num_args; ++i) { double* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == NULL) { - return NULL; + if (arg == nullptr) { + return nullptr; } if (*arg > *reinterpret_cast(e->children()[result_idx]->get_value(row))) { @@ -739,11 +739,11 @@ void* MathFunctions::greatest_string(Expr* e, TupleRow* row) { DCHECK_GE(e->get_num_children(), 1); int32_t num_args = e->get_num_children(); int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is NULL, we can return early.. + // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. for (int i = 0; i < num_args; ++i) { StringValue* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == NULL) { - return NULL; + if (arg == nullptr) { + return nullptr; } if (*arg > *reinterpret_cast(e->children()[result_idx]->get_value(row))) { result_idx = i; @@ -756,11 +756,11 @@ void* MathFunctions::greatest_timestamp(Expr* e, TupleRow* row) { DCHECK_GE(e->get_num_children(), 1); int32_t num_args = e->get_num_children(); int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is NULL, we can return early.. + // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. for (int i = 0; i < num_args; ++i) { DateTimeValue* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == NULL) { - return NULL; + if (arg == nullptr) { + return nullptr; } if (*arg > *reinterpret_cast(e->children()[result_idx]->get_value(row))) { result_idx = i; @@ -773,11 +773,11 @@ void* MathFunctions::least_bigint(Expr* e, TupleRow* row) { int32_t num_args = e->get_num_children(); int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is NULL, we can return early.. + // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. for (int i = 0; i < num_args; ++i) { int64_t* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == NULL) { - return NULL; + if (arg == nullptr) { + return nullptr; } if (*arg < *reinterpret_cast(e->children()[result_idx]->get_value(row))) { @@ -792,11 +792,11 @@ void* MathFunctions::least_double(Expr* e, TupleRow* row) { DCHECK_GE(e->get_num_children(), 1); int32_t num_args = e->get_num_children(); int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is NULL, we can return early.. + // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. for (int i = 0; i < num_args; ++i) { double* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == NULL) { - return NULL; + if (arg == nullptr) { + return nullptr; } if (*arg < *reinterpret_cast(e->children()[result_idx]->get_value(row))) { @@ -810,11 +810,11 @@ void* MathFunctions::least_decimalv2(Expr* e, TupleRow* row) { DCHECK_GE(e->get_num_children(), 1); int32_t num_args = e->get_num_children(); int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is NULL, we can return early.. + // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. for (int i = 0; i < num_args; ++i) { DecimalV2Value* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == NULL) { - return NULL; + if (arg == nullptr) { + return nullptr; } if (*arg < *reinterpret_cast(e->children()[result_idx]->get_value(row))) { result_idx = i; @@ -828,11 +828,11 @@ void* MathFunctions::least_string(Expr* e, TupleRow* row) { DCHECK_GE(e->get_num_children(), 1); int32_t num_args = e->get_num_children(); int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is NULL, we can return early.. + // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. for (int i = 0; i < num_args; ++i) { StringValue* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == NULL) { - return NULL; + if (arg == nullptr) { + return nullptr; } if (*arg < *reinterpret_cast(e->children()[result_idx]->get_value(row))) { result_idx = i; @@ -845,11 +845,11 @@ void* MathFunctions::least_timestamp(Expr* e, TupleRow* row) { DCHECK_GE(e->get_num_children(), 1); int32_t num_args = e->get_num_children(); int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is NULL, we can return early.. + // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. for (int i = 0; i < num_args; ++i) { DateTimeValue* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == NULL) { - return NULL; + if (arg == nullptr) { + return nullptr; } if (*arg < *reinterpret_cast(e->children()[result_idx]->get_value(row))) { result_idx = i; diff --git a/be/src/exprs/new_agg_fn_evaluator.cc b/be/src/exprs/new_agg_fn_evaluator.cc index b7f92d54aa..7a2209ba7f 100644 --- a/be/src/exprs/new_agg_fn_evaluator.cc +++ b/be/src/exprs/new_agg_fn_evaluator.cc @@ -321,7 +321,7 @@ static void SetAnyVal(const SlotDescriptor& desc, Tuple* tuple, AnyVal* dst) { // Utility to put val into an AnyVal struct inline void NewAggFnEvaluator::set_any_val(const void* slot, const TypeDescriptor& type, AnyVal* dst) { - if (slot == NULL) { + if (slot == nullptr) { dst->is_null = true; return; } diff --git a/be/src/exprs/new_agg_fn_evaluator.h b/be/src/exprs/new_agg_fn_evaluator.h index a03626ecf1..822e8fc77d 100644 --- a/be/src/exprs/new_agg_fn_evaluator.h +++ b/be/src/exprs/new_agg_fn_evaluator.h @@ -18,8 +18,6 @@ #ifndef IMPALA_EXPRS_AGG_FN_EVALUATOR_H #define IMPALA_EXPRS_AGG_FN_EVALUATOR_H -#include -#include #include #include "codegen/doris_ir.h" @@ -180,7 +178,8 @@ public: Tuple* dst); static void Serialize(const std::vector& evals, Tuple* dst); static void GetValue(const std::vector& evals, Tuple* src, Tuple* dst); - static void Finalize(const std::vector& evals, Tuple* src, Tuple* dst, bool add_null = false); + static void Finalize(const std::vector& evals, Tuple* src, Tuple* dst, + bool add_null = false); /// Free local allocations made in UDA functions and input arguments' evals. //void FreeLocalAllocations(); @@ -216,7 +215,7 @@ private: /// This contains runtime state such as constant input arguments to the aggregate /// functions and a FreePool from which the intermediate values are allocated. /// Owned by this evaluator. - boost::scoped_ptr agg_fn_ctx_; + std::unique_ptr agg_fn_ctx_; /// Evaluators for input expressions for this aggregate function. /// Empty if there is no input expression (e.g. count(*)). @@ -256,12 +255,13 @@ private: /// Sets up the arguments to call 'fn'. This converts from the agg-expr signature, /// taking TupleRow to the UDA signature taking AnyVals. Writes the serialize/finalize - /// result to the given destination slot/tuple. 'fn' can be NULL to indicate the src + /// result to the given destination slot/tuple. 'fn' can be nullptr to indicate the src /// value should simply be written into the destination. Note that StringVal result is /// from local allocation (which will be freed in the next QueryMaintenance()) so it /// needs to be copied out if it needs to survive beyond QueryMaintenance() (e.g. if /// 'dst' lives in a row batch). - void SerializeOrFinalize(Tuple* src, const SlotDescriptor& dst_slot_desc, Tuple* dst, void* fn, bool add_null = false); + void SerializeOrFinalize(Tuple* src, const SlotDescriptor& dst_slot_desc, Tuple* dst, void* fn, + bool add_null = false); // Sets 'dst' to the value from 'slot'. void set_any_val(const void* slot, const TypeDescriptor& type, doris_udf::AnyVal* dst); @@ -282,7 +282,8 @@ inline void NewAggFnEvaluator::Serialize(Tuple* tuple) { } inline void NewAggFnEvaluator::Finalize(Tuple* agg_val, Tuple* output_val, bool add_null) { - SerializeOrFinalize(agg_val, agg_fn_.output_slot_desc(), output_val, agg_fn_.finalize_fn(), add_null); + SerializeOrFinalize(agg_val, agg_fn_.output_slot_desc(), output_val, agg_fn_.finalize_fn(), + add_null); } inline void NewAggFnEvaluator::GetValue(Tuple* src, Tuple* dst) { diff --git a/be/src/exprs/new_in_predicate.cpp b/be/src/exprs/new_in_predicate.cpp index c713b82f8f..b0a6390373 100644 --- a/be/src/exprs/new_in_predicate.cpp +++ b/be/src/exprs/new_in_predicate.cpp @@ -93,7 +93,7 @@ BooleanVal InPredicate::templated_in(FunctionContext* ctx, const T& val, int num if (strategy == SET_LOOKUP) { SetLookupState* state = reinterpret_cast*>( ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL)); - DCHECK(state != NULL); + DCHECK(state != nullptr); found = set_lookup(state, val); } else { DCHECK_EQ(strategy, ITERATE); @@ -107,7 +107,7 @@ BooleanVal InPredicate::templated_in(FunctionContext* ctx, const T& val, int num template BooleanVal InPredicate::set_lookup(SetLookupState* state, const T& v) { - DCHECK(state != NULL); + DCHECK(state != nullptr); SetType val = get_val(state->type, v); bool found = state->val_set.find(val) != state->val_set.end(); if (found) { diff --git a/be/src/exprs/new_in_predicate.h b/be/src/exprs/new_in_predicate.h index e71917d942..6238894a8d 100644 --- a/be/src/exprs/new_in_predicate.h +++ b/be/src/exprs/new_in_predicate.h @@ -308,10 +308,10 @@ private: template struct SetLookupState { - /// If true, there is at least one NULL constant in the IN list. + /// If true, there is at least one nullptr constant in the IN list. bool contains_null; - /// The set of all non-NULL constant values in the IN list. + /// The set of all non-nullptr constant values in the IN list. /// Note: std::unordered_set and std::binary_search performed worse based on the /// in-predicate-benchmark std::set val_set; diff --git a/be/src/exprs/operators.cpp b/be/src/exprs/operators.cpp index 2380bbc8e0..aaffc0b576 100644 --- a/be/src/exprs/operators.cpp +++ b/be/src/exprs/operators.cpp @@ -17,8 +17,6 @@ #include "exprs/operators.h" -#include - #include "exprs/anyval_util.h" #include "runtime/datetime_value.h" #include "runtime/string_value.h" diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 17e7f3b3c1..407e50b70e 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -420,7 +420,7 @@ Expr* create_literal(ObjectPool* pool, PrimitiveType type, const void* data) { } default: DCHECK(false); - return NULL; + return nullptr; } node.__set_node_type(get_expr_node_type(type)); node.__set_type(create_type_desc(type)); diff --git a/be/src/exprs/scalar_fn_call.cpp b/be/src/exprs/scalar_fn_call.cpp index c887073973..2a7fc824dd 100644 --- a/be/src/exprs/scalar_fn_call.cpp +++ b/be/src/exprs/scalar_fn_call.cpp @@ -32,10 +32,10 @@ namespace doris { ScalarFnCall::ScalarFnCall(const TExprNode& node) : Expr(node), _vararg_start_idx(node.__isset.vararg_start_idx ? node.vararg_start_idx : -1), - _scalar_fn_wrapper(NULL), - _prepare_fn(NULL), - _close_fn(NULL), - _scalar_fn(NULL) { + _scalar_fn_wrapper(nullptr), + _prepare_fn(nullptr), + _close_fn(nullptr), + _scalar_fn(nullptr) { DCHECK_NE(_fn.binary_type, TFunctionBinaryType::HIVE); } @@ -74,7 +74,7 @@ Status ScalarFnCall::prepare(RuntimeState* state, const RowDescriptor& desc, Exp _fn_context_index = context->register_func(state, return_type, arg_types, varargs_buffer_size); // _scalar_fn = OpcodeRegistry::instance()->get_function_ptr(_opcode); Status status = Status::OK(); - if (_scalar_fn == NULL) { + if (_scalar_fn == nullptr) { if (SymbolsUtil::is_mangled(_fn.scalar_fn.symbol)) { status = UserFunctionCache::instance()->get_function_ptr( _fn.id, _fn.scalar_fn.symbol, _fn.hdfs_location, _fn.checksum, &_scalar_fn, @@ -87,7 +87,7 @@ Status ScalarFnCall::prepare(RuntimeState* state, const RowDescriptor& desc, Exp // ColumnType ret_type(INVALID_TYPE); // ret_type = ColumnType(thrift_to_type(_fn.ret_type)); std::string symbol = SymbolsUtil::mangle_user_function(_fn.scalar_fn.symbol, arg_types, - _fn.has_var_args, NULL); + _fn.has_var_args, nullptr); status = UserFunctionCache::instance()->get_function_ptr( _fn.id, symbol, _fn.hdfs_location, _fn.checksum, &_scalar_fn, &_cache_entry); } @@ -160,10 +160,10 @@ Status ScalarFnCall::open(RuntimeState* state, ExprContext* ctx, // Opens and inits children RETURN_IF_ERROR(Expr::open(state, ctx, scope)); FunctionContext* fn_ctx = ctx->fn_context(_fn_context_index); - if (_scalar_fn != NULL) { + if (_scalar_fn != nullptr) { // We're in the interpreted path (i.e. no JIT). Populate our FunctionContext's // staging_input_vals, which will be reused across calls to _scalar_fn. - DCHECK(_scalar_fn_wrapper == NULL); + DCHECK(_scalar_fn_wrapper == nullptr); ObjectPool* obj_pool = state->obj_pool(); std::vector* input_vals = fn_ctx->impl()->staging_input_vals(); for (int i = 0; i < num_fixed_args(); ++i) { @@ -185,7 +185,7 @@ Status ScalarFnCall::open(RuntimeState* state, ExprContext* ctx, fn_ctx->impl()->set_constant_args(constant_args); } - if (_prepare_fn != NULL) { + if (_prepare_fn != nullptr) { if (scope == FunctionContext::FRAGMENT_LOCAL) { _prepare_fn(fn_ctx, FunctionContext::FRAGMENT_LOCAL); if (fn_ctx->has_error()) { @@ -205,7 +205,7 @@ Status ScalarFnCall::open(RuntimeState* state, ExprContext* ctx, if (_fn.name.function_name == "round" && _type.type == TYPE_DOUBLE) { DCHECK_EQ(_children.size(), 2); if (_children[1]->is_constant()) { - IntVal scale_arg = _children[1]->get_int_val(ctx, NULL); + IntVal scale_arg = _children[1]->get_int_val(ctx, nullptr); _output_scale = scale_arg.val; } } @@ -215,7 +215,7 @@ Status ScalarFnCall::open(RuntimeState* state, ExprContext* ctx, void ScalarFnCall::close(RuntimeState* state, ExprContext* context, FunctionContext::FunctionStateScope scope) { - if (_fn_context_index != -1 && _close_fn != NULL) { + if (_fn_context_index != -1 && _close_fn != nullptr) { FunctionContext* fn_ctx = context->fn_context(_fn_context_index); _close_fn(fn_ctx, FunctionContext::THREAD_LOCAL); if (scope == FunctionContext::FRAGMENT_LOCAL) { @@ -244,7 +244,7 @@ Status ScalarFnCall::get_function(RuntimeState* state, const std::string& symbol LlvmCodeGen* codegen; RETURN_IF_ERROR(state->GetCodegen(&codegen)); Function* ir_fn = codegen->module()->getFunction(symbol); - if (ir_fn == NULL) { + if (ir_fn == nullptr) { std::stringstream ss; ss << "Unable to locate function " << symbol << " from LLVM module " << _fn.hdfs_location; @@ -264,7 +264,7 @@ void ScalarFnCall::evaluate_children(ExprContext* context, TupleRow* row, uint8_t* varargs_buffer = fn_ctx->impl()->varargs_buffer(); for (int i = 0; i < _children.size(); ++i) { void* src_slot = context->get_value(_children[i], row); - AnyVal* dst_val = NULL; + AnyVal* dst_val = nullptr; if (_vararg_start_idx == -1 || i < _vararg_start_idx) { dst_val = (*input_vals)[i]; } else { @@ -277,7 +277,7 @@ void ScalarFnCall::evaluate_children(ExprContext* context, TupleRow* row, template RETURN_TYPE ScalarFnCall::interpret_eval(ExprContext* context, TupleRow* row) { - DCHECK(_scalar_fn != NULL); + DCHECK(_scalar_fn != nullptr); FunctionContext* fn_ctx = context->fn_context(_fn_context_index); std::vector* input_vals = fn_ctx->impl()->staging_input_vals(); @@ -427,8 +427,8 @@ typedef CollectionVal (*ArrayWrapper)(ExprContext*, TupleRow*); // TODO: macroify this? BooleanVal ScalarFnCall::get_boolean_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_BOOLEAN); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } BooleanWrapper fn = reinterpret_cast(_scalar_fn_wrapper); @@ -437,8 +437,8 @@ BooleanVal ScalarFnCall::get_boolean_val(ExprContext* context, TupleRow* row) { TinyIntVal ScalarFnCall::get_tiny_int_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_TINYINT); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } TinyIntWrapper fn = reinterpret_cast(_scalar_fn_wrapper); @@ -447,8 +447,8 @@ TinyIntVal ScalarFnCall::get_tiny_int_val(ExprContext* context, TupleRow* row) { SmallIntVal ScalarFnCall::get_small_int_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_SMALLINT); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } SmallIntWrapper fn = reinterpret_cast(_scalar_fn_wrapper); @@ -457,8 +457,8 @@ SmallIntVal ScalarFnCall::get_small_int_val(ExprContext* context, TupleRow* row) IntVal ScalarFnCall::get_int_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_INT); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } IntWrapper fn = reinterpret_cast(_scalar_fn_wrapper); @@ -467,8 +467,8 @@ IntVal ScalarFnCall::get_int_val(ExprContext* context, TupleRow* row) { BigIntVal ScalarFnCall::get_big_int_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_BIGINT); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } BigIntWrapper fn = reinterpret_cast(_scalar_fn_wrapper); @@ -477,8 +477,8 @@ BigIntVal ScalarFnCall::get_big_int_val(ExprContext* context, TupleRow* row) { LargeIntVal ScalarFnCall::get_large_int_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_LARGEINT); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } LargeIntWrapper fn = reinterpret_cast(_scalar_fn_wrapper); @@ -487,8 +487,8 @@ LargeIntVal ScalarFnCall::get_large_int_val(ExprContext* context, TupleRow* row) FloatVal ScalarFnCall::get_float_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_FLOAT); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } FloatWrapper fn = reinterpret_cast(_scalar_fn_wrapper); @@ -497,8 +497,8 @@ FloatVal ScalarFnCall::get_float_val(ExprContext* context, TupleRow* row) { DoubleVal ScalarFnCall::get_double_val(ExprContext* context, TupleRow* row) { DCHECK(_type.type == TYPE_DOUBLE || _type.type == TYPE_TIME); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } @@ -508,8 +508,8 @@ DoubleVal ScalarFnCall::get_double_val(ExprContext* context, TupleRow* row) { StringVal ScalarFnCall::get_string_val(ExprContext* context, TupleRow* row) { DCHECK(_type.is_string_type()); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } StringWrapper fn = reinterpret_cast(_scalar_fn_wrapper); @@ -518,8 +518,8 @@ StringVal ScalarFnCall::get_string_val(ExprContext* context, TupleRow* row) { DateTimeVal ScalarFnCall::get_datetime_val(ExprContext* context, TupleRow* row) { DCHECK(_type.is_date_type()); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } DatetimeWrapper fn = reinterpret_cast(_scalar_fn_wrapper); @@ -528,8 +528,8 @@ DateTimeVal ScalarFnCall::get_datetime_val(ExprContext* context, TupleRow* row) DecimalV2Val ScalarFnCall::get_decimalv2_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_DECIMALV2); - DCHECK(context != NULL); - if (_scalar_fn_wrapper == NULL) { + DCHECK(context != nullptr); + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } DecimalV2Wrapper fn = reinterpret_cast(_scalar_fn_wrapper); @@ -538,9 +538,9 @@ DecimalV2Val ScalarFnCall::get_decimalv2_val(ExprContext* context, TupleRow* row CollectionVal ScalarFnCall::get_array_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_ARRAY); - DCHECK(context != NULL); + DCHECK(context != nullptr); - if (_scalar_fn_wrapper == NULL) { + if (_scalar_fn_wrapper == nullptr) { return interpret_eval(context, row); } diff --git a/be/src/exprs/scalar_fn_call.h b/be/src/exprs/scalar_fn_call.h index 33ae72ffd5..e94e9d04bc 100644 --- a/be/src/exprs/scalar_fn_call.h +++ b/be/src/exprs/scalar_fn_call.h @@ -45,7 +45,7 @@ class TExprNode; /// - Testing /// - Test cancellation /// - Type descs in UDA test harness -/// - Allow more functions to be NULL in UDA test harness +/// - Allow more functions to be nullptr in UDA test harness class ScalarFnCall : public Expr { public: virtual std::string debug_string() const; diff --git a/be/src/exprs/slot_ref.cpp b/be/src/exprs/slot_ref.cpp index bc106d2516..0068c03760 100644 --- a/be/src/exprs/slot_ref.cpp +++ b/be/src/exprs/slot_ref.cpp @@ -78,7 +78,7 @@ Status SlotRef::prepare(RuntimeState* state, const RowDescriptor& row_desc, Expr } const SlotDescriptor* slot_desc = state->desc_tbl().get_slot_descriptor(_slot_id); - if (slot_desc == NULL) { + if (slot_desc == nullptr) { // TODO: create macro MAKE_ERROR() that returns a stream std::stringstream error; error << "couldn't resolve slot descriptor " << _slot_id; @@ -130,7 +130,7 @@ std::string SlotRef::debug_string() const { BooleanVal SlotRef::get_boolean_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_BOOLEAN); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return BooleanVal::null(); } return BooleanVal(*reinterpret_cast(t->get_slot(_slot_offset))); @@ -139,7 +139,7 @@ BooleanVal SlotRef::get_boolean_val(ExprContext* context, TupleRow* row) { TinyIntVal SlotRef::get_tiny_int_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_TINYINT); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return TinyIntVal::null(); } @@ -149,7 +149,7 @@ TinyIntVal SlotRef::get_tiny_int_val(ExprContext* context, TupleRow* row) { SmallIntVal SlotRef::get_small_int_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_SMALLINT); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return SmallIntVal::null(); } return SmallIntVal(*reinterpret_cast(t->get_slot(_slot_offset))); @@ -158,7 +158,7 @@ SmallIntVal SlotRef::get_small_int_val(ExprContext* context, TupleRow* row) { IntVal SlotRef::get_int_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_INT); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return IntVal::null(); } return IntVal(*reinterpret_cast(t->get_slot(_slot_offset))); @@ -167,7 +167,7 @@ IntVal SlotRef::get_int_val(ExprContext* context, TupleRow* row) { BigIntVal SlotRef::get_big_int_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_BIGINT); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return BigIntVal::null(); } return BigIntVal(*reinterpret_cast(t->get_slot(_slot_offset))); @@ -176,7 +176,7 @@ BigIntVal SlotRef::get_big_int_val(ExprContext* context, TupleRow* row) { LargeIntVal SlotRef::get_large_int_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_LARGEINT); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return LargeIntVal::null(); } return LargeIntVal(reinterpret_cast(t->get_slot(_slot_offset))->value); @@ -185,7 +185,7 @@ LargeIntVal SlotRef::get_large_int_val(ExprContext* context, TupleRow* row) { FloatVal SlotRef::get_float_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_FLOAT); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return FloatVal::null(); } return FloatVal(*reinterpret_cast(t->get_slot(_slot_offset))); @@ -194,7 +194,7 @@ FloatVal SlotRef::get_float_val(ExprContext* context, TupleRow* row) { DoubleVal SlotRef::get_double_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_DOUBLE); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return DoubleVal::null(); } return DoubleVal(*reinterpret_cast(t->get_slot(_slot_offset))); @@ -203,7 +203,7 @@ DoubleVal SlotRef::get_double_val(ExprContext* context, TupleRow* row) { StringVal SlotRef::get_string_val(ExprContext* context, TupleRow* row) { DCHECK(_type.is_string_type()); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return StringVal::null(); } StringVal result; @@ -215,7 +215,7 @@ StringVal SlotRef::get_string_val(ExprContext* context, TupleRow* row) { DateTimeVal SlotRef::get_datetime_val(ExprContext* context, TupleRow* row) { DCHECK(_type.is_date_type()); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return DateTimeVal::null(); } DateTimeValue* tv = reinterpret_cast(t->get_slot(_slot_offset)); @@ -227,7 +227,7 @@ DateTimeVal SlotRef::get_datetime_val(ExprContext* context, TupleRow* row) { DecimalV2Val SlotRef::get_decimalv2_val(ExprContext* context, TupleRow* row) { DCHECK_EQ(_type.type, TYPE_DECIMALV2); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return DecimalV2Val::null(); } @@ -238,7 +238,7 @@ doris_udf::CollectionVal SlotRef::get_array_val(ExprContext* context, TupleRow* DCHECK_EQ(_type.type, TYPE_ARRAY); Tuple* t = row->get_tuple(_tuple_idx); - if (t == NULL || t->is_null(_null_indicator_offset)) { + if (t == nullptr || t->is_null(_null_indicator_offset)) { return CollectionVal::null(); } diff --git a/be/src/exprs/slot_ref.h b/be/src/exprs/slot_ref.h index cbeb4b6cb0..110140e470 100644 --- a/be/src/exprs/slot_ref.h +++ b/be/src/exprs/slot_ref.h @@ -33,7 +33,7 @@ public: SlotRef(const SlotDescriptor* desc); virtual 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 + // TODO: this is a hack to allow aggregation nodes to work around nullptr slot // descriptors. Ideally the FE would dictate the type of the intermediate SlotRefs. SlotRef(const SlotDescriptor* desc, const TypeDescriptor& type); @@ -87,8 +87,8 @@ inline bool SlotRef::vector_compute_fn(Expr* expr, VectorizedRowBatch* /* batch inline void* SlotRef::get_value(Expr* expr, TupleRow* row) { SlotRef* ref = (SlotRef*)expr; Tuple* t = row->get_tuple(ref->_tuple_idx); - if (t == NULL || t->is_null(ref->_null_indicator_offset)) { - return NULL; + if (t == nullptr || t->is_null(ref->_null_indicator_offset)) { + return nullptr; } return t->get_slot(ref->_slot_offset); } diff --git a/be/src/exprs/string_functions.cpp b/be/src/exprs/string_functions.cpp index 1c95f2f57a..aedf6331a3 100644 --- a/be/src/exprs/string_functions.cpp +++ b/be/src/exprs/string_functions.cpp @@ -16,8 +16,6 @@ // under the License. #include "exprs/string_functions.h" -#include "util/vectorized-tool/lower.h" -#include "util/vectorized-tool/upper.h" #include @@ -30,6 +28,8 @@ #include "runtime/string_value.hpp" #include "runtime/tuple_row.h" #include "util/url_parser.h" +#include "util/vectorized-tool/lower.h" +#include "util/vectorized-tool/upper.h" // NOTE: be careful not to use string::append. It is not performant. namespace doris { @@ -120,7 +120,7 @@ StringVal StringFunctions::substring(FunctionContext* context, const StringVal& // string left(string input, int len) // This behaves identically to the mysql implementation. StringVal StringFunctions::left(FunctionContext* context, const StringVal& str, const IntVal& len) { - if (len.val >= str.len) return str; + if (len.val >= str.len) return str; return substring(context, str, 1, len); } @@ -320,8 +320,8 @@ StringVal StringFunctions::append_trailing_char_if_absent( // Implementation of LENGTH // int length(string input) -// Returns the length in bytes of input. If input == NULL, returns -// NULL per MySQL +// Returns the length in bytes of input. If input == nullptr, returns +// nullptr per MySQL IntVal StringFunctions::length(FunctionContext* context, const StringVal& str) { if (str.is_null) { return IntVal::null(); @@ -331,8 +331,8 @@ IntVal StringFunctions::length(FunctionContext* context, const StringVal& str) { // Implementation of CHAR_LENGTH // int char_utf8_length(string input) -// Returns the length of characters of input. If input == NULL, returns -// NULL per MySQL +// Returns the length of characters of input. If input == nullptr, returns +// nullptr per MySQL IntVal StringFunctions::char_utf8_length(FunctionContext* context, const StringVal& str) { if (str.is_null) { return IntVal::null(); @@ -544,7 +544,7 @@ bool StringFunctions::set_re2_options(const StringVal& match_parameter, std::str return true; } -// The caller owns the returned regex. Returns NULL if the pattern could not be compiled. +// The caller owns the returned regex. Returns nullptr if the pattern could not be compiled. static re2::RE2* compile_regex(const StringVal& pattern, std::string* error_str, const StringVal& match_parameter) { re2::StringPiece pattern_sp(reinterpret_cast(pattern.ptr), pattern.len); @@ -557,7 +557,7 @@ static re2::RE2* compile_regex(const StringVal& pattern, std::string* error_str, options.set_dot_nl(true); if (!match_parameter.is_null && !StringFunctions::set_re2_options(match_parameter, error_str, &options)) { - return NULL; + return nullptr; } re2::RE2* re = new re2::RE2(pattern_sp, options); if (!re->ok()) { @@ -566,7 +566,7 @@ static re2::RE2* compile_regex(const StringVal& pattern, std::string* error_str, << "Error: " << re->error(); *error_str = ss.str(); delete re; - return NULL; + return nullptr; } return re; } @@ -586,7 +586,7 @@ void StringFunctions::regexp_prepare(FunctionContext* context, } std::string error_str; re2::RE2* re = compile_regex(*pattern, &error_str, StringVal::null()); - if (re == NULL) { + if (re == nullptr) { context->set_error(error_str.c_str()); return; } @@ -614,11 +614,11 @@ StringVal StringFunctions::regexp_extract(FunctionContext* context, const String re2::RE2* re = reinterpret_cast( context->get_function_state(FunctionContext::FRAGMENT_LOCAL)); std::unique_ptr scoped_re; // destroys re if we have to locally compile it - if (re == NULL) { + if (re == nullptr) { DCHECK(!context->is_arg_constant(1)); std::string error_str; re = compile_regex(pattern, &error_str, StringVal::null()); - if (re == NULL) { + if (re == nullptr) { context->add_warning(error_str.c_str()); return StringVal::null(); } @@ -650,12 +650,12 @@ StringVal StringFunctions::regexp_replace(FunctionContext* context, const String re2::RE2* re = reinterpret_cast( context->get_function_state(FunctionContext::FRAGMENT_LOCAL)); - std::unique_ptr scoped_re; // destroys re if state->re is NULL - if (re == NULL) { + std::unique_ptr scoped_re; // destroys re if state->re is nullptr + if (re == nullptr) { DCHECK(!context->is_arg_constant(1)); std::string error_str; re = compile_regex(pattern, &error_str, StringVal::null()); - if (re == NULL) { + if (re == nullptr) { context->add_warning(error_str.c_str()); return StringVal::null(); } @@ -807,7 +807,7 @@ StringVal StringFunctions::parse_url(FunctionContext* ctx, const StringVal& url, StringVal newPart = AnyValUtil::from_string_temp(ctx, part_str); void* state = ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL); UrlParser::UrlPart url_part; - if (state != NULL) { + if (state != nullptr) { url_part = *reinterpret_cast(state); } else { DCHECK(!ctx->is_arg_constant(1)); @@ -850,7 +850,7 @@ StringVal StringFunctions::parse_url_key(FunctionContext* ctx, const StringVal& } void* state = ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL); UrlParser::UrlPart url_part; - if (state != NULL) { + if (state != nullptr) { url_part = *reinterpret_cast(state); } else { DCHECK(!ctx->is_arg_constant(1)); @@ -892,7 +892,8 @@ StringVal StringFunctions::money_format(FunctionContext* context, const DecimalV DecimalV2Value rounded(0); DecimalV2Value::from_decimal_val(v).round(&rounded, 2, HALF_UP); - return do_money_format(context, rounded.int_value(), abs(rounded.frac_value() / 10000000)); + return do_money_format(context, rounded.int_value(), + abs(rounded.frac_value() / 10000000)); } StringVal StringFunctions::money_format(FunctionContext* context, const BigIntVal& v) { @@ -973,9 +974,9 @@ StringVal StringFunctions::replace(FunctionContext* context, const StringVal& or if (origStr.is_null || oldStr.is_null || newStr.is_null) { return StringVal::null(); } - // Empty string is a substring of all strings. + // Empty string is a substring of all strings. // If old str is an empty string, the std::string.find(oldStr) is always return 0. - // With an empty old str, there is no need to do replace. + // With an empty old str, there is no need to do replace. if (oldStr.len == 0) { return origStr; } @@ -993,8 +994,8 @@ StringVal StringFunctions::replace(FunctionContext* context, const StringVal& or } // Implementation of BIT_LENGTH // int bit_length(string input) -// Returns the length in bits of input. If input == NULL, returns -// NULL per MySQL +// Returns the length in bits of input. If input == nullptr, returns +// nullptr per MySQL IntVal StringFunctions::bit_length(FunctionContext* context, const StringVal& str) { if (str.is_null) { return IntVal::null(); diff --git a/be/src/exprs/timestamp_functions.cpp b/be/src/exprs/timestamp_functions.cpp index 610187a43b..34eeecfb2a 100644 --- a/be/src/exprs/timestamp_functions.cpp +++ b/be/src/exprs/timestamp_functions.cpp @@ -182,30 +182,32 @@ IntVal TimestampFunctions::week_of_year(FunctionContext* context, const DateTime return IntVal::null(); } -IntVal TimestampFunctions::year_week(FunctionContext *context, const DateTimeVal &ts_val) { - return year_week(context, ts_val, doris_udf::IntVal{0}); +IntVal TimestampFunctions::year_week(FunctionContext* context, const DateTimeVal& ts_val) { + return year_week(context, ts_val, doris_udf::IntVal {0}); } -IntVal TimestampFunctions::year_week(FunctionContext *context, const DateTimeVal &ts_val, const doris_udf::IntVal &mode) { +IntVal TimestampFunctions::year_week(FunctionContext* context, const DateTimeVal& ts_val, + const doris_udf::IntVal& mode) { if (ts_val.is_null) { return IntVal::null(); } - const DateTimeValue &ts_value = DateTimeValue::from_datetime_val(ts_val); + const DateTimeValue& ts_value = DateTimeValue::from_datetime_val(ts_val); if (ts_value.is_valid_date()) { return ts_value.year_week(mysql_week_mode(mode.val)); } return IntVal::null(); } -IntVal TimestampFunctions::week(FunctionContext *context, const DateTimeVal &ts_val) { - return week(context, ts_val, doris_udf::IntVal{0}); +IntVal TimestampFunctions::week(FunctionContext* context, const DateTimeVal& ts_val) { + return week(context, ts_val, doris_udf::IntVal {0}); } -IntVal TimestampFunctions::week(FunctionContext *context, const DateTimeVal &ts_val, const doris_udf::IntVal& mode) { +IntVal TimestampFunctions::week(FunctionContext* context, const DateTimeVal& ts_val, + const doris_udf::IntVal& mode) { if (ts_val.is_null) { return IntVal::null(); } - const DateTimeValue &ts_value = DateTimeValue::from_datetime_val(ts_val); + const DateTimeValue& ts_value = DateTimeValue::from_datetime_val(ts_val); if (ts_value.is_valid_date()) { return {ts_value.week(mysql_week_mode(mode.val))}; } @@ -236,10 +238,11 @@ IntVal TimestampFunctions::second(FunctionContext* context, const DateTimeVal& t return IntVal(ts_value.second()); } -DateTimeVal TimestampFunctions::make_date(FunctionContext *ctx, const IntVal &year, const IntVal &count) { +DateTimeVal TimestampFunctions::make_date(FunctionContext* ctx, const IntVal& year, + const IntVal& count) { if (count.val > 0) { // year-1-1 - DateTimeValue ts_value{year.val * 10000000000 + 101000000}; + DateTimeValue ts_value {year.val * 10000000000 + 101000000}; ts_value.set_type(TIME_DATE); DateTimeVal ts_val; ts_value.to_datetime_val(&ts_val); @@ -300,7 +303,7 @@ StringVal TimestampFunctions::month_name(FunctionContext* ctx, const DateTimeVal } const DateTimeValue& ts_value = DateTimeValue::from_datetime_val(ts_val); const char* name = ts_value.month_name(); - if (name == NULL) { + if (name == nullptr) { return StringVal::null(); } return AnyValUtil::from_string_temp(ctx, name); @@ -312,7 +315,7 @@ StringVal TimestampFunctions::day_name(FunctionContext* ctx, const DateTimeVal& } const DateTimeValue& ts_value = DateTimeValue::from_datetime_val(ts_val); const char* name = ts_value.day_name(); - if (name == NULL) { + if (name == nullptr) { return StringVal::null(); } return AnyValUtil::from_string_temp(ctx, name); diff --git a/be/src/exprs/timestamp_functions.h b/be/src/exprs/timestamp_functions.h index 155837c606..ff1112be65 100644 --- a/be/src/exprs/timestamp_functions.h +++ b/be/src/exprs/timestamp_functions.h @@ -18,11 +18,7 @@ #ifndef DORIS_BE_SRC_QUERY_EXPRS_TIMESTAMP_FUNCTIONS_H #define DORIS_BE_SRC_QUERY_EXPRS_TIMESTAMP_FUNCTIONS_H -#include -#include -#include -#include -#include +#include #include "runtime/datetime_value.h" #include "runtime/string_value.h" @@ -69,16 +65,16 @@ public: const doris_udf::DateTimeVal& ts_val); static doris_udf::IntVal week_of_year(doris_udf::FunctionContext* context, const doris_udf::DateTimeVal& ts_val); - static doris_udf::IntVal year_week(doris_udf::FunctionContext *context, - const doris_udf::DateTimeVal &ts_val); - static doris_udf::IntVal year_week(doris_udf::FunctionContext *context, - const doris_udf::DateTimeVal &ts_val, - const doris_udf::IntVal ¶); - static doris_udf::IntVal week(doris_udf::FunctionContext *context, - const doris_udf::DateTimeVal &ts_val); - static doris_udf::IntVal week(doris_udf::FunctionContext *context, - const doris_udf::DateTimeVal &ts_val, - const doris_udf::IntVal &mode); + static doris_udf::IntVal year_week(doris_udf::FunctionContext* context, + const doris_udf::DateTimeVal& ts_val); + static doris_udf::IntVal year_week(doris_udf::FunctionContext* context, + const doris_udf::DateTimeVal& ts_val, + const doris_udf::IntVal& para); + static doris_udf::IntVal week(doris_udf::FunctionContext* context, + const doris_udf::DateTimeVal& ts_val); + static doris_udf::IntVal week(doris_udf::FunctionContext* context, + const doris_udf::DateTimeVal& ts_val, + const doris_udf::IntVal& mode); static doris_udf::IntVal hour(doris_udf::FunctionContext* context, const doris_udf::DateTimeVal& ts_val); static doris_udf::IntVal minute(doris_udf::FunctionContext* context, @@ -88,8 +84,8 @@ public: // Date/time functions. static doris_udf::DateTimeVal make_date(doris_udf::FunctionContext* ctx, - const doris_udf::IntVal& year, - const doris_udf::IntVal& count); + const doris_udf::IntVal& year, + const doris_udf::IntVal& count); static doris_udf::DateTimeVal to_date(doris_udf::FunctionContext* ctx, const doris_udf::DateTimeVal& ts_val); static doris_udf::IntVal date_diff(doris_udf::FunctionContext* ctx, diff --git a/be/src/exprs/tuple_is_null_predicate.cpp b/be/src/exprs/tuple_is_null_predicate.cpp index 31ca79ad84..c0b0f2fe8c 100644 --- a/be/src/exprs/tuple_is_null_predicate.cpp +++ b/be/src/exprs/tuple_is_null_predicate.cpp @@ -47,7 +47,7 @@ Status TupleIsNullPredicate::prepare(RuntimeState* state, const RowDescriptor& r BooleanVal TupleIsNullPredicate::get_boolean_val(ExprContext* ctx, TupleRow* row) { int count = 0; for (int i = 0; i < _tuple_idxs.size(); ++i) { - count += row->get_tuple(_tuple_idxs[i]) == NULL; + count += row->get_tuple(_tuple_idxs[i]) == nullptr; } return BooleanVal(!_tuple_idxs.empty() && count == _tuple_idxs.size()); } diff --git a/be/src/gutil/gscoped_ptr.h b/be/src/gutil/gscoped_ptr.h index f6630c223a..87b9c966a0 100644 --- a/be/src/gutil/gscoped_ptr.h +++ b/be/src/gutil/gscoped_ptr.h @@ -85,7 +85,7 @@ // gscoped_array. This is because casting array pointers may not be safe. // // ------------------------------------------------------------------------- -// Cloudera notes: this should be used in preference to boost::scoped_ptr since +// Cloudera notes: this should be used in preference to std::unique_ptr since // it offers a ::release() method like unique_ptr. We unfortunately cannot // just use unique_ptr because it has an inconsistent implementation in // some of the older compilers we have to support. diff --git a/be/src/http/action/checksum_action.h b/be/src/http/action/checksum_action.h index 8e342e1206..302a83bbe6 100644 --- a/be/src/http/action/checksum_action.h +++ b/be/src/http/action/checksum_action.h @@ -18,7 +18,7 @@ #ifndef DORIS_BE_SRC_HTTP_CHECKSUM_ACTION_H #define DORIS_BE_SRC_HTTP_CHECKSUM_ACTION_H -#include +#include #include "http/http_handler.h" diff --git a/be/src/http/action/reload_tablet_action.h b/be/src/http/action/reload_tablet_action.h index da3a775208..cf518d3059 100644 --- a/be/src/http/action/reload_tablet_action.h +++ b/be/src/http/action/reload_tablet_action.h @@ -18,8 +18,6 @@ #ifndef DORIS_BE_SRC_HTTP_RELOAD_TABLET_ACTION_H #define DORIS_BE_SRC_HTTP_RELOAD_TABLET_ACTION_H -#include - #include "gen_cpp/AgentService_types.h" #include "http/http_handler.h" diff --git a/be/src/http/action/restore_tablet_action.h b/be/src/http/action/restore_tablet_action.h index f96bfb4d32..182531f3ad 100644 --- a/be/src/http/action/restore_tablet_action.h +++ b/be/src/http/action/restore_tablet_action.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_HTTP_RESTORE_TABLET_ACTION_H #define DORIS_BE_SRC_HTTP_RESTORE_TABLET_ACTION_H -#include #include #include diff --git a/be/src/http/action/snapshot_action.h b/be/src/http/action/snapshot_action.h index c606c39762..665a5db4a3 100644 --- a/be/src/http/action/snapshot_action.h +++ b/be/src/http/action/snapshot_action.h @@ -18,7 +18,8 @@ #ifndef DORIS_BE_SRC_HTTP_SNAPSHOT_ACTION_H #define DORIS_BE_SRC_HTTP_SNAPSHOT_ACTION_H -#include +#include +#include #include "http/http_handler.h" diff --git a/be/src/olap/aggregate_func.h b/be/src/olap/aggregate_func.h index 621441c7e0..a64ed14836 100644 --- a/be/src/olap/aggregate_func.h +++ b/be/src/olap/aggregate_func.h @@ -425,7 +425,7 @@ struct AggregateFuncTraitscreation_time(); int64_t interval_threshold = config::base_compaction_interval_seconds_since_last_operation; - int64_t interval_since_last_base_compaction = time(NULL) - base_creation_time; + int64_t interval_since_last_base_compaction = time(nullptr) - base_creation_time; if (interval_since_last_base_compaction > interval_threshold) { LOG(INFO) << "satisfy the base compaction policy. tablet=" << _tablet->full_name() << ", interval_since_last_base_compaction=" << interval_since_last_base_compaction diff --git a/be/src/olap/bloom_filter_writer.cpp b/be/src/olap/bloom_filter_writer.cpp index 105d6692f4..b69c9d08f7 100644 --- a/be/src/olap/bloom_filter_writer.cpp +++ b/be/src/olap/bloom_filter_writer.cpp @@ -50,8 +50,8 @@ uint64_t BloomFilterIndexWriter::estimate_buffered_memory() { OLAPStatus BloomFilterIndexWriter::write_to_buffer(OutStream* out_stream) { OLAPStatus res = OLAP_SUCCESS; - if (NULL == out_stream) { - OLAP_LOG_WARNING("out stream is NULL"); + if (nullptr == out_stream) { + OLAP_LOG_WARNING("out stream is null"); return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -79,8 +79,8 @@ OLAPStatus BloomFilterIndexWriter::write_to_buffer(OutStream* out_stream) { OLAPStatus BloomFilterIndexWriter::write_to_buffer(char* buffer, size_t buffer_size) { OLAPStatus res = OLAP_SUCCESS; - if (NULL == buffer) { - OLAP_LOG_WARNING("out stream is NULL."); + if (nullptr == buffer) { + OLAP_LOG_WARNING("out stream is nullptr."); return OLAP_ERR_INPUT_PARAMETER_ERROR; } diff --git a/be/src/olap/byte_buffer.cpp b/be/src/olap/byte_buffer.cpp index f2e99bfd43..0cc877ee51 100644 --- a/be/src/olap/byte_buffer.cpp +++ b/be/src/olap/byte_buffer.cpp @@ -24,7 +24,7 @@ namespace doris { StorageByteBuffer::StorageByteBuffer() - : _array(NULL), _capacity(0), _limit(0), _position(0), _is_mmap(false) {} + : _array(nullptr), _capacity(0), _limit(0), _position(0), _is_mmap(false) {} StorageByteBuffer::BufDeleter::BufDeleter() : _is_mmap(false), _mmap_length(0) {} @@ -34,7 +34,7 @@ void StorageByteBuffer::BufDeleter::set_mmap(size_t mmap_length) { } void StorageByteBuffer::BufDeleter::operator()(char* p) { - if (NULL == p) { + if (nullptr == p) { return; } @@ -53,8 +53,8 @@ StorageByteBuffer* StorageByteBuffer::create(uint64_t capacity) { char* memory = new (std::nothrow) char[capacity]; StorageByteBuffer* buf = new (std::nothrow) StorageByteBuffer; - if (buf != NULL && memory != NULL) { - buf->_buf = boost::shared_ptr(memory, BufDeleter()); + if (buf != nullptr && memory != nullptr) { + buf->_buf = std::shared_ptr(memory, BufDeleter()); buf->_array = buf->_buf.get(); buf->_capacity = capacity; buf->_limit = capacity; @@ -63,23 +63,23 @@ StorageByteBuffer* StorageByteBuffer::create(uint64_t capacity) { SAFE_DELETE(buf); SAFE_DELETE(memory); - return NULL; + return nullptr; } StorageByteBuffer* StorageByteBuffer::reference_buffer(StorageByteBuffer* reference, uint64_t offset, uint64_t length) { - if (NULL == reference || 0 == length) { - return NULL; + if (nullptr == reference || 0 == length) { + return nullptr; } if (offset + length > reference->capacity()) { - return NULL; + return nullptr; } StorageByteBuffer* buf = new (std::nothrow) StorageByteBuffer(); - if (NULL == buf) { - return NULL; + if (nullptr == buf) { + return nullptr; } buf->_buf = reference->_buf; @@ -97,7 +97,7 @@ StorageByteBuffer* StorageByteBuffer::mmap(void* start, uint64_t length, int pro if (MAP_FAILED == memory) { OLAP_LOG_WARNING("fail to mmap. [errno='%d' errno_str='%s']", Errno::no(), Errno::str()); - return NULL; + return nullptr; } BufDeleter deleter; @@ -105,13 +105,13 @@ StorageByteBuffer* StorageByteBuffer::mmap(void* start, uint64_t length, int pro StorageByteBuffer* buf = new (std::nothrow) StorageByteBuffer(); - if (NULL == buf) { + if (nullptr == buf) { deleter(memory); OLAP_LOG_WARNING("fail to allocate StorageByteBuffer."); - return NULL; + return nullptr; } - buf->_buf = boost::shared_ptr(memory, deleter); + buf->_buf = std::shared_ptr(memory, deleter); buf->_array = buf->_buf.get(); buf->_capacity = length; buf->_limit = length; @@ -121,18 +121,18 @@ StorageByteBuffer* StorageByteBuffer::mmap(void* start, uint64_t length, int pro StorageByteBuffer* StorageByteBuffer::mmap(FileHandler* handler, uint64_t offset, int prot, int flags) { - if (NULL == handler) { + if (nullptr == handler) { OLAP_LOG_WARNING("invalid file handler"); - return NULL; + return nullptr; } size_t length = handler->length(); int fd = handler->fd(); - char* memory = (char*)::mmap(NULL, length, prot, flags, fd, offset); + char* memory = (char*)::mmap(nullptr, length, prot, flags, fd, offset); if (MAP_FAILED == memory) { OLAP_LOG_WARNING("fail to mmap. [errno='%d' errno_str='%s']", Errno::no(), Errno::str()); - return NULL; + return nullptr; } BufDeleter deleter; @@ -140,13 +140,13 @@ StorageByteBuffer* StorageByteBuffer::mmap(FileHandler* handler, uint64_t offset StorageByteBuffer* buf = new (std::nothrow) StorageByteBuffer(); - if (NULL == buf) { + if (nullptr == buf) { deleter(memory); OLAP_LOG_WARNING("fail to allocate StorageByteBuffer."); - return NULL; + return nullptr; } - buf->_buf = boost::shared_ptr(memory, deleter); + buf->_buf = std::shared_ptr(memory, deleter); buf->_array = buf->_buf.get(); buf->_capacity = length; buf->_limit = length; diff --git a/be/src/olap/byte_buffer.h b/be/src/olap/byte_buffer.h index 02574ff34b..0ac4e41a24 100644 --- a/be/src/olap/byte_buffer.h +++ b/be/src/olap/byte_buffer.h @@ -18,8 +18,6 @@ #ifndef DORIS_BE_SRC_OLAP_COLUMN_FILE_BYTE_BUFFER_H #define DORIS_BE_SRC_OLAP_COLUMN_FILE_BYTE_BUFFER_H -#include - #include "olap/file_helper.h" #include "olap/olap_define.h" #include "util/mem_util.hpp" @@ -178,7 +176,7 @@ public: // 返回ByteBuffer内部的char数组 const char* array() const { return _array; } const char* array(size_t position) const { - return position >= _limit ? NULL : &_array[position]; + return position >= _limit ? nullptr : &_array[position]; } char* array() { return _array; } @@ -202,7 +200,7 @@ private: StorageByteBuffer(); private: - boost::shared_ptr _buf; // 托管的内存 + std::shared_ptr _buf; // 托管的内存 char* _array; uint64_t _capacity; uint64_t _limit; diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index e48392ae16..9000ce2acf 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -28,8 +28,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -133,9 +133,9 @@ Status DataDir::_init_cluster_id() { } // obtain lock of all cluster id paths - FILE* fp = NULL; + FILE* fp = nullptr; fp = fopen(cluster_id_path.c_str(), "r+b"); - if (fp == NULL) { + if (fp == nullptr) { RETURN_NOT_OK_STATUS_WITH_WARN( Status::IOError( strings::Substitute("failed to open cluster id file $0", cluster_id_path)), @@ -145,7 +145,7 @@ Status DataDir::_init_cluster_id() { int lock_res = flock(fp->_fileno, LOCK_EX | LOCK_NB); if (lock_res < 0) { fclose(fp); - fp = NULL; + fp = nullptr; RETURN_NOT_OK_STATUS_WITH_WARN( Status::IOError( strings::Substitute("failed to flock cluster id file $0", cluster_id_path)), @@ -227,7 +227,7 @@ Status DataDir::_init_file_system() { } FILE* mount_tablet = nullptr; - if ((mount_tablet = setmntent(kMtabPath, "r")) == NULL) { + if ((mount_tablet = setmntent(kMtabPath, "r")) == nullptr) { RETURN_NOT_OK_STATUS_WITH_WARN( Status::IOError(strings::Substitute("setmntent file $0 failed, err=$1", _path, errno_to_string(errno))), @@ -235,10 +235,10 @@ Status DataDir::_init_file_system() { } bool is_find = false; - struct mntent* mount_entry = NULL; + struct mntent* mount_entry = nullptr; struct mntent ent; char buf[1024]; - while ((mount_entry = getmntent_r(mount_tablet, &ent, buf, sizeof(buf))) != NULL) { + while ((mount_entry = getmntent_r(mount_tablet, &ent, buf, sizeof(buf))) != nullptr) { if (strcmp(_path.c_str(), mount_entry->mnt_dir) == 0 || strcmp(_path.c_str(), mount_entry->mnt_fsname) == 0) { is_find = true; @@ -531,10 +531,10 @@ OLAPStatus DataDir::load() { auto load_tablet_func = [this, &tablet_ids, &failed_tablet_ids]( int64_t tablet_id, int32_t schema_hash, const std::string& value) -> bool { - OLAPStatus status = _tablet_manager->load_tablet_from_meta(this, tablet_id, schema_hash, - value, false, false, false, false); - if (status != OLAP_SUCCESS && status != OLAP_ERR_TABLE_ALREADY_DELETED_ERROR - && status != OLAP_ERR_ENGINE_INSERT_OLD_TABLET) { + OLAPStatus status = _tablet_manager->load_tablet_from_meta( + this, tablet_id, schema_hash, value, false, false, false, false); + if (status != OLAP_SUCCESS && status != OLAP_ERR_TABLE_ALREADY_DELETED_ERROR && + status != OLAP_ERR_ENGINE_INSERT_OLD_TABLET) { // load_tablet_from_meta() may return OLAP_ERR_TABLE_ALREADY_DELETED_ERROR // which means the tablet status is DELETED // This may happen when the tablet was just deleted before the BE restarted, @@ -544,9 +544,9 @@ OLAPStatus DataDir::load() { // Therefore, we believe that this situation is not a failure. // Besides, load_tablet_from_meta() may return OLAP_ERR_ENGINE_INSERT_OLD_TABLET - // when BE is restarting and the older tablet have been added to the + // when BE is restarting and the older tablet have been added to the // garbage collection queue but not deleted yet. - // In this case, since the data_dirs are parallel loaded, a later loaded tablet + // In this case, since the data_dirs are parallel loaded, a later loaded tablet // may be older than previously loaded one, which should not be acknowledged as a // failure. LOG(WARNING) << "load tablet from header failed. status:" << status @@ -653,7 +653,8 @@ void DataDir::remove_pending_ids(const std::string& id) { // gc unused tablet schemahash dir void DataDir::perform_path_gc_by_tablet() { std::unique_lock lck(_check_path_mutex); - _check_path_cv.wait(lck, [this] { return _stop_bg_worker || !_all_tablet_schemahash_paths.empty(); }); + _check_path_cv.wait( + lck, [this] { return _stop_bg_worker || !_all_tablet_schemahash_paths.empty(); }); if (_stop_bg_worker) { return; } @@ -785,8 +786,8 @@ void DataDir::perform_path_scan() { _all_tablet_schemahash_paths.insert(tablet_schema_hash_path); std::set rowset_files; - ret = FileUtils::list_dirs_files(tablet_schema_hash_path, nullptr, - &rowset_files, Env::Default()); + ret = FileUtils::list_dirs_files(tablet_schema_hash_path, nullptr, &rowset_files, + Env::Default()); if (!ret.ok()) { LOG(WARNING) << "fail to walk dir. [path=" << tablet_schema_hash_path << "] error[" << ret.to_string() << "]"; @@ -799,8 +800,8 @@ void DataDir::perform_path_scan() { } } } - LOG(INFO) << "scan data dir path: " << _path - << " finished. path size: " << _all_check_paths.size() + _all_tablet_schemahash_paths.size(); + LOG(INFO) << "scan data dir path: " << _path << " finished. path size: " + << _all_check_paths.size() + _all_tablet_schemahash_paths.size(); _check_path_cv.notify_one(); } diff --git a/be/src/olap/decimal12.h b/be/src/olap/decimal12.h index dcd0fab971..bcab67bae3 100644 --- a/be/src/olap/decimal12.h +++ b/be/src/olap/decimal12.h @@ -27,7 +27,6 @@ namespace doris { // the sign of integer must be same as fraction struct decimal12_t { - decimal12_t& operator+=(const decimal12_t& value) { fraction += value.fraction; integer += value.integer; @@ -96,7 +95,7 @@ struct decimal12_t { const char* value_string = str.c_str(); const char* sign = strchr(value_string, '-'); - if (sign != NULL) { + if (sign != nullptr) { if (sign != value_string) { return OLAP_ERR_INPUT_PARAMETER_ERROR; } else { @@ -105,8 +104,8 @@ struct decimal12_t { } const char* sepr = strchr(value_string, '.'); - if ((sepr != NULL && sepr - value_string > MAX_INT_DIGITS_NUM) || - (sepr == NULL && strlen(value_string) > MAX_INT_DIGITS_NUM)) { + if ((sepr != nullptr && sepr - value_string > MAX_INT_DIGITS_NUM) || + (sepr == nullptr && strlen(value_string) > MAX_INT_DIGITS_NUM)) { integer = 999999999999999999; fraction = 999999999; } else { @@ -117,13 +116,13 @@ struct decimal12_t { sscanf(value_string, "%18ld.%9d", &integer, &fraction); } - int32_t frac_len = - (NULL != sepr) ? MAX_FRAC_DIGITS_NUM - strlen(sepr + 1) : MAX_FRAC_DIGITS_NUM; + int32_t frac_len = (nullptr != sepr) ? MAX_FRAC_DIGITS_NUM - strlen(sepr + 1) + : MAX_FRAC_DIGITS_NUM; frac_len = frac_len > 0 ? frac_len : 0; fraction *= g_power_table[frac_len]; } - if (sign != NULL) { + if (sign != nullptr) { fraction = -fraction; integer = -integer; } diff --git a/be/src/olap/field.h b/be/src/olap/field.h index 63ef199985..f90eae43ce 100644 --- a/be/src/olap/field.h +++ b/be/src/olap/field.h @@ -143,7 +143,7 @@ public: } } - // Only compare column content, without considering NULL condition. + // Only compare column content, without considering nullptr condition. // RETURNS: // 0 means equal, // -1 means left less than right, @@ -152,9 +152,9 @@ public: // Compare two types of cell. // This function differs compare in that this function compare cell which - // will consider the condition which cell may be NULL. While compare only - // compare column content without considering NULL condition. - // Only compare column content, without considering NULL condition. + // will consider the condition which cell may be nullptr. While compare only + // compare column content without considering nullptr condition. + // Only compare column content, without considering nullptr condition. // RETURNS: // 0 means equal, // -1 means left less than right, diff --git a/be/src/olap/file_helper.cpp b/be/src/olap/file_helper.cpp index 2ffd8f186f..be1b43ad4b 100644 --- a/be/src/olap/file_helper.cpp +++ b/be/src/olap/file_helper.cpp @@ -38,7 +38,7 @@ namespace doris { Cache* FileHandler::_s_fd_cache = nullptr; FileHandler::FileHandler() - : _fd(-1), _wr_length(0), _file_name(""), _is_using_cache(false), _cache_handle(NULL) { + : _fd(-1), _wr_length(0), _file_name(""), _is_using_cache(false), _cache_handle(nullptr) { static std::once_flag once_flag; #ifdef BE_TEST std::call_once(once_flag, [] { @@ -81,7 +81,7 @@ OLAPStatus FileHandler::open(const string& file_name, int flag) { } VLOG_NOTICE << "success to open file. file_name=" << file_name << ", mode=" << flag - << " fd=" << _fd; + << " fd=" << _fd; _is_using_cache = false; _file_name = file_name; return OLAP_SUCCESS; @@ -102,12 +102,12 @@ OLAPStatus FileHandler::open_with_cache(const string& file_name, int flag) { CacheKey key(file_name.c_str(), file_name.size()); _cache_handle = _s_fd_cache->lookup(key); - if (NULL != _cache_handle) { + if (nullptr != _cache_handle) { FileDescriptor* file_desc = reinterpret_cast(_s_fd_cache->value(_cache_handle)); _fd = file_desc->fd; - VLOG_NOTICE << "success to open file with cache. file_name=" << file_name << ", mode=" << flag - << " fd=" << _fd; + VLOG_NOTICE << "success to open file with cache. file_name=" << file_name + << ", mode=" << flag << " fd=" << _fd; } else { _fd = ::open(file_name.c_str(), flag); if (_fd < 0) { @@ -122,7 +122,7 @@ OLAPStatus FileHandler::open_with_cache(const string& file_name, int flag) { FileDescriptor* file_desc = new FileDescriptor(_fd); _cache_handle = _s_fd_cache->insert(key, file_desc, 1, &_delete_cache_file_descriptor); VLOG_NOTICE << "success to open file with cache. " - << "file_name=" << file_name << ", mode=" << flag << ", fd=" << _fd; + << "file_name=" << file_name << ", mode=" << flag << ", fd=" << _fd; } _is_using_cache = true; _file_name = file_name; @@ -151,14 +151,14 @@ OLAPStatus FileHandler::open_with_mode(const string& file_name, int flag, int mo } VLOG_NOTICE << "success to open file. file_name=" << file_name << ", mode=" << mode - << ", fd=" << _fd; + << ", fd=" << _fd; _file_name = file_name; return OLAP_SUCCESS; } OLAPStatus FileHandler::_release() { _s_fd_cache->release(_cache_handle); - _cache_handle = NULL; + _cache_handle = nullptr; _is_using_cache = false; return OLAP_SUCCESS; } @@ -189,7 +189,7 @@ OLAPStatus FileHandler::close() { } VLOG_NOTICE << "finished to close file. " - << "file_name=" << _file_name << ", fd=" << _fd; + << "file_name=" << _file_name << ", fd=" << _fd; _fd = -1; _file_name = ""; _wr_length = 0; @@ -299,14 +299,14 @@ off_t FileHandler::length() const { return stat_data.st_size; } -FileHandlerWithBuf::FileHandlerWithBuf() : _fp(NULL), _file_name("") {} +FileHandlerWithBuf::FileHandlerWithBuf() : _fp(nullptr), _file_name("") {} FileHandlerWithBuf::~FileHandlerWithBuf() { this->close(); } OLAPStatus FileHandlerWithBuf::open(const string& file_name, const char* mode) { - if (_fp != NULL && _file_name == file_name) { + if (_fp != nullptr && _file_name == file_name) { return OLAP_SUCCESS; } @@ -316,7 +316,7 @@ OLAPStatus FileHandlerWithBuf::open(const string& file_name, const char* mode) { _fp = ::fopen(file_name.c_str(), mode); - if (NULL == _fp) { + if (nullptr == _fp) { char errmsg[64]; LOG(WARNING) << "failed to open file. [err= " << strerror_r(errno, errmsg, 64) << " file_name='" << file_name << "' flag='" << mode << "']"; @@ -327,7 +327,7 @@ OLAPStatus FileHandlerWithBuf::open(const string& file_name, const char* mode) { } VLOG_NOTICE << "success to open file. " - << "file_name=" << file_name << ", mode=" << mode; + << "file_name=" << file_name << ", mode=" << mode; _file_name = file_name; return OLAP_SUCCESS; } @@ -337,7 +337,7 @@ OLAPStatus FileHandlerWithBuf::open_with_mode(const string& file_name, const cha } OLAPStatus FileHandlerWithBuf::close() { - if (NULL == _fp) { + if (nullptr == _fp) { return OLAP_SUCCESS; } @@ -349,14 +349,14 @@ OLAPStatus FileHandlerWithBuf::close() { return OLAP_ERR_IO_ERROR; } - _fp = NULL; + _fp = nullptr; _file_name = ""; return OLAP_SUCCESS; } OLAPStatus FileHandlerWithBuf::read(void* buf, size_t size) { - if (OLAP_UNLIKELY(NULL == _fp)) { - OLAP_LOG_WARNING("Fail to write, fp is NULL!"); + if (OLAP_UNLIKELY(nullptr == _fp)) { + OLAP_LOG_WARNING("Fail to write, fp is nullptr!"); return OLAP_ERR_NOT_INITED; } @@ -379,8 +379,8 @@ OLAPStatus FileHandlerWithBuf::read(void* buf, size_t size) { } OLAPStatus FileHandlerWithBuf::pread(void* buf, size_t size, size_t offset) { - if (OLAP_UNLIKELY(NULL == _fp)) { - OLAP_LOG_WARNING("Fail to write, fp is NULL!"); + if (OLAP_UNLIKELY(nullptr == _fp)) { + OLAP_LOG_WARNING("Fail to write, fp is nullptr!"); return OLAP_ERR_NOT_INITED; } @@ -396,8 +396,8 @@ OLAPStatus FileHandlerWithBuf::pread(void* buf, size_t size, size_t offset) { } OLAPStatus FileHandlerWithBuf::write(const void* buf, size_t buf_size) { - if (OLAP_UNLIKELY(NULL == _fp)) { - OLAP_LOG_WARNING("Fail to write, fp is NULL!"); + if (OLAP_UNLIKELY(nullptr == _fp)) { + OLAP_LOG_WARNING("Fail to write, fp is nullptr!"); return OLAP_ERR_NOT_INITED; } @@ -414,8 +414,8 @@ OLAPStatus FileHandlerWithBuf::write(const void* buf, size_t buf_size) { } OLAPStatus FileHandlerWithBuf::pwrite(const void* buf, size_t buf_size, size_t offset) { - if (OLAP_UNLIKELY(NULL == _fp)) { - OLAP_LOG_WARNING("Fail to write, fp is NULL!"); + if (OLAP_UNLIKELY(nullptr == _fp)) { + OLAP_LOG_WARNING("Fail to write, fp is nullptr!"); return OLAP_ERR_NOT_INITED; } diff --git a/be/src/olap/file_helper.h b/be/src/olap/file_helper.h index e89cbd37a5..d74892fe20 100644 --- a/be/src/olap/file_helper.h +++ b/be/src/olap/file_helper.h @@ -240,7 +240,7 @@ private: template OLAPStatus FileHeader::prepare( FileHandlerType* file_handler) { - if (NULL == file_handler) { + if (nullptr == file_handler) { return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -274,7 +274,7 @@ OLAPStatus FileHeader::prepare( template OLAPStatus FileHeader::serialize( FileHandlerType* file_handler) { - if (NULL == file_handler) { + if (nullptr == file_handler) { return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -310,7 +310,7 @@ OLAPStatus FileHeader::serialize( template OLAPStatus FileHeader::unserialize( FileHandlerType* file_handler) { - if (NULL == file_handler) { + if (nullptr == file_handler) { return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -347,10 +347,10 @@ OLAPStatus FileHeader::unserialize( } VLOG_NOTICE << "fix head loaded. file_length=" << _fixed_file_header.file_length - << ", checksum=" << _fixed_file_header.checksum - << ", protobuf_length=" << _fixed_file_header.protobuf_length - << ", magic_number=" << _fixed_file_header.magic_number - << ", version=" << _fixed_file_header.version; + << ", checksum=" << _fixed_file_header.checksum + << ", protobuf_length=" << _fixed_file_header.protobuf_length + << ", magic_number=" << _fixed_file_header.magic_number + << ", version=" << _fixed_file_header.version; if (OLAP_SUCCESS != file_handler->pread(&_extra_fixed_header, sizeof(_extra_fixed_header), _fixed_file_header_size)) { @@ -362,7 +362,7 @@ OLAPStatus FileHeader::unserialize( std::unique_ptr buf(new (std::nothrow) char[_fixed_file_header.protobuf_length]); - if (NULL == buf.get()) { + if (nullptr == buf.get()) { char errmsg[64]; LOG(WARNING) << "malloc protobuf buf error. file=" << file_handler->file_name() << ", error=" << strerror_r(errno, errmsg, 64); diff --git a/be/src/olap/file_stream.cpp b/be/src/olap/file_stream.cpp index 1108e79af8..ad1614f720 100644 --- a/be/src/olap/file_stream.cpp +++ b/be/src/olap/file_stream.cpp @@ -26,8 +26,8 @@ ReadOnlyFileStream::ReadOnlyFileStream(FileHandler* handler, StorageByteBuffer** Decompressor decompressor, uint32_t compress_buffer_size, OlapReaderStatistics* stats) : _file_cursor(handler, 0, 0), - _compressed_helper(NULL), - _uncompressed(NULL), + _compressed_helper(nullptr), + _uncompressed(nullptr), _shared_buffer(shared_buffer), _decompressor(decompressor), _compress_buffer_size(compress_buffer_size + sizeof(StreamHead)), @@ -38,8 +38,8 @@ ReadOnlyFileStream::ReadOnlyFileStream(FileHandler* handler, StorageByteBuffer** uint64_t offset, uint64_t length, Decompressor decompressor, uint32_t compress_buffer_size, OlapReaderStatistics* stats) : _file_cursor(handler, offset, length), - _compressed_helper(NULL), - _uncompressed(NULL), + _compressed_helper(nullptr), + _uncompressed(nullptr), _shared_buffer(shared_buffer), _decompressor(decompressor), _compress_buffer_size(compress_buffer_size + sizeof(StreamHead)), @@ -48,11 +48,11 @@ ReadOnlyFileStream::ReadOnlyFileStream(FileHandler* handler, StorageByteBuffer** OLAPStatus ReadOnlyFileStream::_assure_data() { // if still has data in uncompressed - if (OLAP_LIKELY(_uncompressed != NULL && _uncompressed->remaining() > 0)) { + if (OLAP_LIKELY(_uncompressed != nullptr && _uncompressed->remaining() > 0)) { return OLAP_SUCCESS; } else if (_file_cursor.eof()) { VLOG_TRACE << "STREAM EOF. length=" << _file_cursor.length() - << ", used=" << _file_cursor.position(); + << ", used=" << _file_cursor.position(); return OLAP_ERR_COLUMN_STREAM_EOF; } @@ -103,17 +103,17 @@ OLAPStatus ReadOnlyFileStream::seek(PositionProvider* position) { // 先seek到解压前的位置,也就是writer中写入的spilled byte int64_t compressed_position = position->get_next(); int64_t uncompressed_bytes = position->get_next(); - if (_current_compress_position == compressed_position && NULL != _uncompressed) { + if (_current_compress_position == compressed_position && nullptr != _uncompressed) { /* * 多数情况下不会出现_uncompressed为NULL的情况, - * 但varchar类型的数据可能会导致查询中出现_uncompressed == NULL 。 + * 但varchar类型的数据可能会导致查询中出现_uncompressed == nullptr 。 * 假设查询恰好命中A压缩块的最后一行, 而相临下一个 - * 中压缩块varchar全是空串,会导致_uncompressed == NULL。 + * 中压缩块varchar全是空串,会导致_uncompressed == nullptr。 * 如果后面的segmentreader中还需要再次遍历A压缩块,会出现空指针。 */ } else { _file_cursor.seek(compressed_position); - _uncompressed = NULL; + _uncompressed = nullptr; res = _assure_data(); if (OLAP_LIKELY(OLAP_SUCCESS == res)) { diff --git a/be/src/olap/file_stream.h b/be/src/olap/file_stream.h index 323e9a8ea0..d33454a77e 100644 --- a/be/src/olap/file_stream.h +++ b/be/src/olap/file_stream.h @@ -64,12 +64,12 @@ public: inline OLAPStatus init() { _compressed_helper = StorageByteBuffer::create(_compress_buffer_size); - if (NULL == _compressed_helper) { + if (nullptr == _compressed_helper) { OLAP_LOG_WARNING("fail to create compressed buffer"); return OLAP_ERR_MALLOC_ERROR; } - _uncompressed = NULL; + _uncompressed = nullptr; return OLAP_SUCCESS; } @@ -97,7 +97,7 @@ public: uint64_t stream_length() { return _file_cursor.length(); } bool eof() { - if (_uncompressed == NULL) { + if (_uncompressed == nullptr) { return _file_cursor.eof(); } else { return _file_cursor.eof() && _uncompressed->remaining() == 0; @@ -110,8 +110,8 @@ public: size_t get_buffer_size() { return _compress_buffer_size; } inline void get_buf(char** buf, uint32_t* remaining_bytes) { - if (UNLIKELY(_uncompressed == NULL)) { - *buf = NULL; + if (UNLIKELY(_uncompressed == nullptr)) { + *buf = nullptr; *remaining_bytes = 0; } else { *buf = _uncompressed->array(); @@ -124,7 +124,7 @@ public: inline void set_position(uint32_t pos) { _uncompressed->set_position(pos); } inline int remaining() { - if (_uncompressed == NULL) { + if (_uncompressed == nullptr) { return 0; } return _uncompressed->remaining(); diff --git a/be/src/olap/fs/block_manager.h b/be/src/olap/fs/block_manager.h index 55d80de2ce..75ab135cb2 100644 --- a/be/src/olap/fs/block_manager.h +++ b/be/src/olap/fs/block_manager.h @@ -179,7 +179,7 @@ struct BlockManagerOptions { BlockManagerOptions() = default; // The memory tracker under which all new memory trackers will be parented. - // If NULL, new memory trackers will be parented to the root tracker. + // If nullptr, new memory trackers will be parented to the root tracker. std::shared_ptr parent_mem_tracker; // If false, metrics will not be produced. diff --git a/be/src/olap/hll.cpp b/be/src/olap/hll.cpp index a50517ca43..12fb912b3a 100644 --- a/be/src/olap/hll.cpp +++ b/be/src/olap/hll.cpp @@ -15,11 +15,12 @@ // specific language governing permissions and limitations // under the License. +#include "olap/hll.h" + #include #include #include "common/logging.h" -#include "olap/hll.h" #include "runtime/string_value.h" #include "util/coding.h" @@ -450,7 +451,7 @@ void HllSetResolver::parse() { // skip LengthValueType char* pdata = _buf_ref; _set_type = (HllDataType)pdata[0]; - char* sparse_data = NULL; + char* sparse_data = nullptr; switch (_set_type) { case HLL_DATA_EXPLICIT: // first byte : type diff --git a/be/src/olap/in_stream.cpp b/be/src/olap/in_stream.cpp index b597523569..0809b274c7 100644 --- a/be/src/olap/in_stream.cpp +++ b/be/src/olap/in_stream.cpp @@ -31,8 +31,8 @@ InStream::InStream(std::vector* inputs, const std::vectorremaining() >= len)) { slice = StorageByteBuffer::reference_buffer(_compressed, _compressed->position(), len); - if (OLAP_UNLIKELY(NULL == slice)) { + if (OLAP_UNLIKELY(nullptr == slice)) { return OLAP_ERR_MALLOC_ERROR; } @@ -66,7 +66,7 @@ OLAPStatus InStream::_slice(uint64_t chunk_size, StorageByteBuffer** out_slice) // 这里并不分配chuck_size, 而是分配一个最大值, 这样利于减少内存碎片 slice = StorageByteBuffer::create(_compress_buffer_size); - if (OLAP_UNLIKELY(NULL == slice)) { + if (OLAP_UNLIKELY(nullptr == slice)) { return OLAP_ERR_MALLOC_ERROR; } @@ -86,7 +86,7 @@ OLAPStatus InStream::_slice(uint64_t chunk_size, StorageByteBuffer** out_slice) _inputs[_current_range]->position(), _inputs[_current_range]->remaining()); - if (OLAP_UNLIKELY(NULL == _compressed)) { + if (OLAP_UNLIKELY(nullptr == _compressed)) { SAFE_DELETE(slice); return OLAP_ERR_MALLOC_ERROR; } @@ -119,9 +119,9 @@ OLAPStatus InStream::_slice(uint64_t chunk_size, StorageByteBuffer** out_slice) OLAPStatus InStream::_assure_data() { OLAPStatus res = OLAP_SUCCESS; - if (OLAP_LIKELY(_uncompressed != NULL && _uncompressed->remaining() > 0)) { + if (OLAP_LIKELY(_uncompressed != nullptr && _uncompressed->remaining() > 0)) { return OLAP_SUCCESS; - } else if (OLAP_UNLIKELY((_uncompressed == NULL || _uncompressed->remaining() == 0) && + } else if (OLAP_UNLIKELY((_uncompressed == nullptr || _uncompressed->remaining() == 0) && (_current_offset == _length))) { return OLAP_ERR_COLUMN_STREAM_EOF; } @@ -131,7 +131,7 @@ OLAPStatus InStream::_assure_data() { // 到这里说明当前uncompress没有什么可以读了,input拿数据 // 如果没有compress。或者compress耗尽,用_seek向后一个buff移动 - if (_compressed == NULL || _compressed->remaining() == 0) { + if (_compressed == nullptr || _compressed->remaining() == 0) { res = _seek(_current_offset); if (OLAP_SUCCESS != res) { return res; @@ -153,7 +153,7 @@ OLAPStatus InStream::_assure_data() { // 向后移动整体偏移 _current_offset += sizeof(StreamHead); - StorageByteBuffer* slice = NULL; + StorageByteBuffer* slice = nullptr; // 根据head取一块buf,这里应该要调整_current_offset res = _slice(head.length, &slice); @@ -169,7 +169,7 @@ OLAPStatus InStream::_assure_data() { } else { _uncompressed = StorageByteBuffer::create(_compress_buffer_size); - if (OLAP_UNLIKELY(NULL == _uncompressed)) { + if (OLAP_UNLIKELY(nullptr == _uncompressed)) { res = OLAP_ERR_MALLOC_ERROR; } else { res = _decompressor(slice, _uncompressed); @@ -205,7 +205,7 @@ OLAPStatus InStream::_seek(uint64_t position) { for (uint32_t i = 0; i < _inputs.size(); i++) { if (_offsets[i] <= position && position - _offsets[i] < _inputs[i]->remaining()) { // don't need to malloc _compressed if current range don't be changed. - if (!(_current_range == i && NULL != _compressed)) { + if (!(_current_range == i && nullptr != _compressed)) { _current_range = i; SAFE_DELETE(_compressed); _compressed = @@ -261,7 +261,7 @@ OLAPStatus InStream::seek(PositionProvider* position) { _uncompressed->position() + uncompressed_bytes); return res; } - } else if (_uncompressed != NULL) { + } else if (_uncompressed != nullptr) { // mark the uncompressed buffer as done res = _uncompressed->set_position(_uncompressed->limit()); diff --git a/be/src/olap/in_stream.h b/be/src/olap/in_stream.h index b2f602ea63..492097c5ab 100644 --- a/be/src/olap/in_stream.h +++ b/be/src/olap/in_stream.h @@ -104,7 +104,7 @@ public: return _uncompressed->array(offset); } - return NULL; + return nullptr; } private: @@ -132,11 +132,11 @@ public: InStreamBufferWrapper(InStream* input) : std::streambuf(), _stream(input), _skip_size(0) {} virtual ~InStreamBufferWrapper() {} virtual int_type underflow() { - if (NULL != _stream) { + if (nullptr != _stream) { if (OLAP_SUCCESS == _stream->skip(_skip_size)) { char* buf = const_cast(_stream->available_buffer()); - if (NULL != buf) { + if (nullptr != buf) { size_t read_length = _stream->available(); setg(buf, buf, buf + read_length); _skip_size = read_length; diff --git a/be/src/olap/lru_cache.cpp b/be/src/olap/lru_cache.cpp index 0a2ce0fe30..3ae16db15c 100644 --- a/be/src/olap/lru_cache.cpp +++ b/be/src/olap/lru_cache.cpp @@ -90,7 +90,7 @@ LRUHandle* HandleTable::insert(LRUHandle* h) { _head_insert(head, h); ++_elems; - if (old == NULL) { + if (old == nullptr) { if (_elems > _length) { // Since each cache entry is fairly large, we aim for a small // average linked list length (<= 1). @@ -123,7 +123,7 @@ void HandleTable::remove(const LRUHandle* h) { LRUHandle** HandleTable::_find_pointer(const CacheKey& key, uint32_t hash) { LRUHandle** ptr = &(_list[hash & (_length - 1)]->next_hash); - while (*ptr != NULL && ((*ptr)->hash != hash || key != (*ptr)->key())) { + while (*ptr != nullptr && ((*ptr)->hash != hash || key != (*ptr)->key())) { ptr = &(*ptr)->next_hash; } @@ -156,7 +156,7 @@ void HandleTable::_resize() { uint32_t count = 0; for (uint32_t i = 0; i < _length; i++) { LRUHandle* h = _list[i]->next_hash; - while (h != NULL) { + while (h != nullptr) { LRUHandle* next = h->next_hash; uint32_t hash = h->hash; LRUHandle* head = new_list[hash & (new_length - 1)]; @@ -452,10 +452,12 @@ uint32_t ShardedLRUCache::_shard(uint32_t hash) { return hash >> (32 - kNumShardBits); } -ShardedLRUCache::ShardedLRUCache(const std::string& name, size_t total_capacity, - LRUCacheType type, std::shared_ptr parent) - : _name(name), _last_id(1), - _mem_tracker(MemTracker::CreateTracker(-1, name, parent, true, false, MemTrackerLevel::OVERVIEW)) { +ShardedLRUCache::ShardedLRUCache(const std::string& name, size_t total_capacity, LRUCacheType type, + std::shared_ptr parent) + : _name(name), + _last_id(1), + _mem_tracker(MemTracker::CreateTracker(-1, name, parent, true, false, + MemTrackerLevel::OVERVIEW)) { const size_t per_shard = (total_capacity + (kNumShards - 1)) / kNumShards; for (int s = 0; s < kNumShards; s++) { _shards[s] = new LRUCache(type); @@ -549,17 +551,20 @@ void ShardedLRUCache::update_cache_metrics() const { usage->set_value(total_usage); lookup_count->set_value(total_lookup_count); hit_count->set_value(total_hit_count); - usage_ratio->set_value(total_capacity == 0 ? 0 : ((double) total_usage / total_capacity)); - hit_ratio->set_value(total_lookup_count == 0 ? 0 : ((double) total_hit_count / total_lookup_count)); - + usage_ratio->set_value(total_capacity == 0 ? 0 : ((double)total_usage / total_capacity)); + hit_ratio->set_value(total_lookup_count == 0 ? 0 + : ((double)total_hit_count / total_lookup_count)); + _mem_tracker->Consume(total_usage - _mem_tracker->consumption()); } -Cache* new_lru_cache(const std::string& name, size_t capacity, std::shared_ptr parent_tracker) { +Cache* new_lru_cache(const std::string& name, size_t capacity, + std::shared_ptr parent_tracker) { return new ShardedLRUCache(name, capacity, LRUCacheType::SIZE, parent_tracker); } -Cache* new_typed_lru_cache(const std::string& name, size_t capacity, LRUCacheType type, std::shared_ptr parent_tracker) { +Cache* new_typed_lru_cache(const std::string& name, size_t capacity, LRUCacheType type, + std::shared_ptr parent_tracker) { return new ShardedLRUCache(name, capacity, type, parent_tracker); } diff --git a/be/src/olap/lru_cache.h b/be/src/olap/lru_cache.h index 94e262df30..b9f71f1a2e 100644 --- a/be/src/olap/lru_cache.h +++ b/be/src/olap/lru_cache.h @@ -29,7 +29,7 @@ namespace doris { cur += str.size(); \ } else { \ OLAP_LOG_WARNING("construct cache key buf not enough."); \ - return CacheKey(NULL, 0); \ + return CacheKey(nullptr, 0); \ } \ } while (0) @@ -41,7 +41,7 @@ namespace doris { cur += sizeof(numeric); \ } else { \ OLAP_LOG_WARNING("construct cache key buf not enough."); \ - return CacheKey(NULL, 0); \ + return CacheKey(nullptr, 0); \ } \ } while (0) @@ -49,21 +49,21 @@ class Cache; class CacheKey; enum LRUCacheType { - SIZE, // The capacity of cache is based on the size of cache entry. - NUMBER // The capacity of cache is based on the number of cache entry. + SIZE, // The capacity of cache is based on the size of cache entry. + NUMBER // The capacity of cache is based on the number of cache entry. }; // Create a new cache with a specified name and a fixed SIZE capacity. // This implementation of Cache uses a least-recently-used eviction policy. extern Cache* new_lru_cache(const std::string& name, size_t capacity, - std::shared_ptr parent_tracekr = nullptr); + std::shared_ptr parent_tracekr = nullptr); extern Cache* new_typed_lru_cache(const std::string& name, size_t capacity, LRUCacheType type, - std::shared_ptr parent_tracekr = nullptr); + std::shared_ptr parent_tracekr = nullptr); class CacheKey { public: - CacheKey() : _data(NULL), _size(0) {} + CacheKey() : _data(nullptr), _size(0) {} // Create a slice that refers to d[0,n-1]. CacheKey(const char* d, size_t n) : _data(d), _size(n) {} @@ -93,7 +93,7 @@ public: // Change this slice to refer to an empty array void clear() { - _data = NULL; + _data = nullptr; _size = 0; } @@ -172,7 +172,7 @@ public: void (*deleter)(const CacheKey& key, void* value), CachePriority priority = CachePriority::NORMAL) = 0; - // If the cache has no mapping for "key", returns NULL. + // If the cache has no mapping for "key", returns nullptr. // // Else return a handle that corresponds to the mapping. The caller // must call this->release(handle) when the returned mapping is no @@ -234,7 +234,7 @@ typedef struct LRUHandle { size_t charge; size_t key_length; size_t total_size; // including key length - bool in_cache; // Whether entry is in the cache. + bool in_cache; // Whether entry is in the cache. uint32_t refs; uint32_t hash; // Hash of key(); used for fast sharding and comparisons CachePriority priority = CachePriority::NORMAL; @@ -265,7 +265,7 @@ typedef struct LRUHandle { class HandleTable { public: - HandleTable() : _length(0), _elems(0), _list(NULL) { _resize(); } + HandleTable() : _length(0), _elems(0), _list(nullptr) { _resize(); } ~HandleTable(); @@ -360,8 +360,8 @@ static const int kNumShards = 1 << kNumShardBits; class ShardedLRUCache : public Cache { public: - explicit ShardedLRUCache(const std::string& name, size_t total_capacity, - LRUCacheType type, std::shared_ptr parent); + explicit ShardedLRUCache(const std::string& name, size_t total_capacity, LRUCacheType type, + std::shared_ptr parent); // TODO(fdy): 析构时清除所有cache元素 virtual ~ShardedLRUCache(); virtual Handle* insert(const CacheKey& key, void* value, size_t charge, @@ -378,6 +378,7 @@ public: private: void update_cache_metrics() const; + private: static inline uint32_t _hash_slice(const CacheKey& s); static uint32_t _shard(uint32_t hash); diff --git a/be/src/olap/memory/hash_index.cpp b/be/src/olap/memory/hash_index.cpp index 16f3bf4e9f..9503f12604 100644 --- a/be/src/olap/memory/hash_index.cpp +++ b/be/src/olap/memory/hash_index.cpp @@ -50,7 +50,7 @@ struct alignas(64) HashChunk { const uint64_t HashIndex::npos; HashIndex::HashIndex(size_t capacity) - : _size(0), _max_size(0), _num_chunks(0), _chunk_mask(0), _chunks(NULL) { + : _size(0), _max_size(0), _num_chunks(0), _chunk_mask(0), _chunks(nullptr) { size_t min_chunk = (capacity * 14 / 12 + HashChunk::CAPACITY - 1) / HashChunk::CAPACITY; if (min_chunk == 0) { return; diff --git a/be/src/olap/olap_cond.cpp b/be/src/olap/olap_cond.cpp index 4bc6b51768..fec9c8a513 100644 --- a/be/src/olap/olap_cond.cpp +++ b/be/src/olap/olap_cond.cpp @@ -386,20 +386,22 @@ int Cond::del_eval(const std::pair& stat) const { ret = DEL_SATISFIED; } else if (stat.first->is_null() && !stat.second->is_null()) { ret = DEL_PARTIAL_SATISFIED; - } else if (!stat.first->is_null() && !stat.second->is_null()){ + } else if (!stat.first->is_null() && !stat.second->is_null()) { ret = DEL_NOT_SATISFIED; } else { - CHECK(false) << "It will not happen when the stat's min is not null and max is null"; + CHECK(false) + << "It will not happen when the stat's min is not null and max is null"; } } else { if (stat.first->is_null() && stat.second->is_null()) { ret = DEL_NOT_SATISFIED; } else if (stat.first->is_null() && !stat.second->is_null()) { ret = DEL_PARTIAL_SATISFIED; - } else if (!stat.first->is_null() && !stat.second->is_null()){ + } else if (!stat.first->is_null() && !stat.second->is_null()) { ret = DEL_SATISFIED; } else { - CHECK(false) << "It will not happen when the stat's min is not null and max is null"; + CHECK(false) + << "It will not happen when the stat's min is not null and max is null"; } } return ret; @@ -440,7 +442,7 @@ bool Cond::eval(const BloomFilter& bf) const { return false; } case OP_IS: { - // IS [NOT] NULL can only used in to filter IS NULL predicate. + // IS [NOT] nullptr can only used in to filter IS nullptr predicate. if (operand_field->is_null()) { return bf.test_bytes(nullptr, 0); } @@ -481,7 +483,7 @@ bool Cond::eval(const segment_v2::BloomFilter* bf) const { return false; } case OP_IS: { - // IS [NOT] NULL can only used in to filter IS NULL predicate. + // IS [NOT] nullptr can only used in to filter IS nullptr predicate. return operand_field->is_null() == bf->test_bytes(nullptr, 0); } default: @@ -520,7 +522,7 @@ bool CondColumn::eval(const RowCursor& row) const { return true; } -bool CondColumn::eval(const std::pair &statistic) const { +bool CondColumn::eval(const std::pair& statistic) const { for (auto& each_cond : _conds) { // As long as there is one condition not satisfied, we can return false if (!each_cond->eval(statistic)) { @@ -623,7 +625,7 @@ bool Conditions::delete_conditions_eval(const RowCursor& row) const { } VLOG_NOTICE << "Row meets the delete conditions. " - << "condition_count=" << _columns.size() << ", row=" << row.to_string(); + << "condition_count=" << _columns.size() << ", row=" << row.to_string(); return true; } diff --git a/be/src/olap/olap_define.h b/be/src/olap/olap_define.h index fd3a7943a2..ea34452a99 100644 --- a/be/src/olap/olap_define.h +++ b/be/src/olap/olap_define.h @@ -451,12 +451,12 @@ const std::string ROWSET_ID_PREFIX = "s_"; #define DECLARE_SINGLETON(classname) \ public: \ static classname* instance() { \ - classname* p_instance = NULL; \ + classname* p_instance = nullptr; \ try { \ static classname s_instance; \ p_instance = &s_instance; \ } catch (...) { \ - p_instance = NULL; \ + p_instance = nullptr; \ } \ return p_instance; \ } \ @@ -467,19 +467,19 @@ protected: \ private: \ ~classname(); -#define SAFE_DELETE(ptr) \ - do { \ - if (NULL != ptr) { \ - delete ptr; \ - ptr = NULL; \ - } \ +#define SAFE_DELETE(ptr) \ + do { \ + if (nullptr != ptr) { \ + delete ptr; \ + ptr = nullptr; \ + } \ } while (0) #define SAFE_DELETE_ARRAY(ptr) \ do { \ - if (NULL != ptr) { \ + if (nullptr != ptr) { \ delete[] ptr; \ - ptr = NULL; \ + ptr = nullptr; \ } \ } while (0) diff --git a/be/src/olap/olap_index.cpp b/be/src/olap/olap_index.cpp index de71f518bd..14ae7e4434 100644 --- a/be/src/olap/olap_index.cpp +++ b/be/src/olap/olap_index.cpp @@ -47,7 +47,7 @@ MemIndex::~MemIndex() { _num_entries = 0; for (vector::iterator it = _meta.begin(); it != _meta.end(); ++it) { free(it->buffer.data); - it->buffer.data = NULL; + it->buffer.data = nullptr; it->buffer.length = 0; } } @@ -60,7 +60,7 @@ OLAPStatus MemIndex::load_segment(const char* file, size_t* current_num_rows_per uint32_t adler_checksum = 0; uint32_t num_entries = 0; - if (file == NULL) { + if (file == nullptr) { res = OLAP_ERR_INPUT_PARAMETER_ERROR; LOG(WARNING) << "load index error. file=" << file << ", res=" << res; return res; @@ -127,7 +127,7 @@ OLAPStatus MemIndex::load_segment(const char* file, size_t* current_num_rows_per _num_entries = meta.range.last; _meta.push_back(meta); - (current_num_rows_per_row_block == NULL || + (current_num_rows_per_row_block == nullptr || (*current_num_rows_per_row_block = meta.file_header.message().num_rows_per_block())); if (OLAP_UNLIKELY(num_entries == 0)) { @@ -318,7 +318,7 @@ OLAPStatus MemIndex::load_segment(const char* file, size_t* current_num_rows_per OLAPStatus MemIndex::init(size_t short_key_len, size_t new_short_key_len, size_t short_key_num, std::vector* short_key_columns) { if (short_key_columns == nullptr) { - LOG(WARNING) << "fail to init MemIndex, NULL short key columns."; + LOG(WARNING) << "fail to init MemIndex, nullptr short key columns."; return OLAP_ERR_INDEX_LOAD_ERROR; } diff --git a/be/src/olap/olap_index.h b/be/src/olap/olap_index.h index 08f6f6cdca..1b9c704c41 100644 --- a/be/src/olap/olap_index.h +++ b/be/src/olap/olap_index.h @@ -138,7 +138,7 @@ struct SegmentMetaInfo { SegmentMetaInfo() { range.first = range.last = 0; buffer.length = 0; - buffer.data = NULL; + buffer.data = nullptr; } const size_t count() const { return range.last - range.first; } diff --git a/be/src/olap/options.cpp b/be/src/olap/options.cpp index 4a64568106..d9ce6c4264 100644 --- a/be/src/olap/options.cpp +++ b/be/src/olap/options.cpp @@ -109,11 +109,11 @@ OLAPStatus parse_root_path(const string& root_path, StorePath* path) { path->capacity_bytes = -1; if (!capacity_str.empty()) { if (!valid_signed_number(capacity_str) || - strtol(capacity_str.c_str(), NULL, 10) < 0) { + strtol(capacity_str.c_str(), nullptr, 10) < 0) { LOG(WARNING) << "invalid capacity of store path, capacity=" << capacity_str; return OLAP_ERR_INPUT_PARAMETER_ERROR; } - path->capacity_bytes = strtol(capacity_str.c_str(), NULL, 10) * GB_EXCHANGE_BYTE; + path->capacity_bytes = strtol(capacity_str.c_str(), nullptr, 10) * GB_EXCHANGE_BYTE; } path->storage_medium = TStorageMedium::HDD; diff --git a/be/src/olap/out_stream.cpp b/be/src/olap/out_stream.cpp index b6900c64ee..2b7000ec23 100644 --- a/be/src/olap/out_stream.cpp +++ b/be/src/olap/out_stream.cpp @@ -29,7 +29,7 @@ OutStreamFactory::OutStreamFactory(CompressKind compress_kind, uint32_t stream_b : _compress_kind(compress_kind), _stream_buffer_size(stream_buffer_size) { switch (compress_kind) { case COMPRESS_NONE: - _compressor = NULL; + _compressor = nullptr; break; #ifdef DORIS_WITH_LZO @@ -56,17 +56,17 @@ OutStreamFactory::~OutStreamFactory() { OutStream* OutStreamFactory::create_stream(uint32_t column_unique_id, StreamInfoMessage::Kind kind) { - OutStream* stream = NULL; + OutStream* stream = nullptr; if (StreamInfoMessage::ROW_INDEX == kind || StreamInfoMessage::BLOOM_FILTER == kind) { - stream = new (std::nothrow) OutStream(_stream_buffer_size, NULL); + stream = new (std::nothrow) OutStream(_stream_buffer_size, nullptr); } else { stream = new (std::nothrow) OutStream(_stream_buffer_size, _compressor); } - if (NULL == stream) { + if (nullptr == stream) { OLAP_LOG_WARNING("fail to allocate OutStream."); - return NULL; + return nullptr; } StreamName stream_name(column_unique_id, kind); @@ -78,9 +78,9 @@ OutStream::OutStream(uint32_t buffer_size, Compressor compressor) : _buffer_size(buffer_size), _compressor(compressor), _is_suppressed(false), - _current(NULL), - _compressed(NULL), - _overflow(NULL), + _current(nullptr), + _compressed(nullptr), + _overflow(nullptr), _spilled_bytes(0) {} OutStream::~OutStream() { @@ -98,7 +98,7 @@ OLAPStatus OutStream::_create_new_input_buffer() { SAFE_DELETE(_current); _current = StorageByteBuffer::create(_buffer_size + sizeof(StreamHead)); - if (NULL != _current) { + if (nullptr != _current) { _current->set_position(sizeof(StreamHead)); return OLAP_SUCCESS; } else { @@ -149,29 +149,29 @@ void OutStream::_output_uncompress() { _spilled_bytes += _current->limit(); _write_head(_current, 0, StreamHead::UNCOMPRESSED, _current->limit() - sizeof(StreamHead)); _output_buffers.push_back(_current); - _current = NULL; + _current = nullptr; } void OutStream::_output_compressed() { _compressed->flip(); _output_buffers.push_back(_compressed); _compressed = _overflow; - _overflow = NULL; + _overflow = nullptr; } OLAPStatus OutStream::_make_sure_output_buffer() { - if (NULL == _compressed) { + if (nullptr == _compressed) { _compressed = StorageByteBuffer::create(_buffer_size + sizeof(StreamHead)); - if (NULL == _compressed) { + if (nullptr == _compressed) { return OLAP_ERR_MALLOC_ERROR; } } - if (NULL == _overflow) { + if (nullptr == _overflow) { _overflow = StorageByteBuffer::create(_buffer_size + sizeof(StreamHead)); - if (NULL == _overflow) { + if (nullptr == _overflow) { return OLAP_ERR_MALLOC_ERROR; } } @@ -182,12 +182,12 @@ OLAPStatus OutStream::_make_sure_output_buffer() { OLAPStatus OutStream::_spill() { OLAPStatus res = OLAP_SUCCESS; - if (_current == NULL || _current->position() == sizeof(StreamHead)) { + if (_current == nullptr || _current->position() == sizeof(StreamHead)) { return OLAP_SUCCESS; } // If it is not compressed, read current directly. Note that current will be cleared and set to NULL after output - if (_compressor == NULL) { + if (_compressor == nullptr) { _current->flip(); _output_uncompress(); } else { @@ -235,7 +235,7 @@ OLAPStatus OutStream::_spill() { if (head_pos != 0) { // There was data in _compressed before, in this case, output compressed first, - // At this time _overflow must be empty + // At this time _overflow must be empty _output_compressed(); } @@ -257,7 +257,7 @@ OLAPStatus OutStream::write(const char* buffer, uint64_t length) { // In the case of uncompressed, current will be put into the list and cannot be reused. The reason is // If it is reused, the previous content will be modified, so it needs to be redistributed. // Only allocate once and the second block will hang up - if (NULL == _current) { + if (nullptr == _current) { res = _create_new_input_buffer(); if (OLAP_SUCCESS != res) { return res; @@ -292,7 +292,7 @@ OLAPStatus OutStream::write(const char* buffer, uint64_t length) { void OutStream::get_position(PositionEntryWriter* index_entry) const { index_entry->add_position(_spilled_bytes); - if (NULL != _current) { + if (nullptr != _current) { index_entry->add_position(_current->position() - sizeof(StreamHead)); } else { index_entry->add_position(0); @@ -359,7 +359,7 @@ OLAPStatus OutStream::write_to_file(FileHandler* file_handle, uint32_t write_mby int64_t sleep_time = total_stream_len / write_mbytes_per_sec - delta_time_us; if (sleep_time > 0) { VLOG_TRACE << "sleep to limit merge speed. time=" << sleep_time - << ", bytes=" << total_stream_len; + << ", bytes=" << total_stream_len; SleepFor(MonoDelta::FromMicroseconds(sleep_time)); } } @@ -377,7 +377,7 @@ OLAPStatus OutStream::flush() { return res; } - if (NULL != _compressed && 0 != _compressed->position()) { + if (nullptr != _compressed && 0 != _compressed->position()) { _output_compressed(); SAFE_DELETE(_compressed); } diff --git a/be/src/olap/out_stream.h b/be/src/olap/out_stream.h index 2376758593..533f6a1fdb 100644 --- a/be/src/olap/out_stream.h +++ b/be/src/olap/out_stream.h @@ -97,7 +97,7 @@ public: void print_position_debug_info() { VLOG_TRACE << "compress: " << _spilled_bytes; - if (_current != NULL) { + if (_current != nullptr) { VLOG_TRACE << "uncompress=" << (_current->position() - sizeof(StreamHead)); } else { VLOG_TRACE << "uncompress 0"; diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp index d907aeaceb..747568c749 100644 --- a/be/src/olap/push_handler.cpp +++ b/be/src/olap/push_handler.cpp @@ -66,7 +66,7 @@ OLAPStatus PushHandler::process_streaming_ingestion(TabletSharedPtr tablet, cons res = _do_streaming_ingestion(tablet, request, push_type, &tablet_vars, tablet_info_vec); if (res == OLAP_SUCCESS) { - if (tablet_info_vec != NULL) { + if (tablet_info_vec != nullptr) { _get_tablet_infos(tablet_vars, tablet_info_vec); } LOG(INFO) << "process realtime push successfully. " @@ -192,7 +192,7 @@ OLAPStatus PushHandler::_do_streaming_ingestion(TabletSharedPtr tablet, const TP void PushHandler::_get_tablet_infos(const std::vector& tablet_vars, std::vector* tablet_info_vec) { for (const TabletVars& tablet_var : tablet_vars) { - if (tablet_var.tablet.get() == NULL) { + if (tablet_var.tablet.get() == nullptr) { continue; } @@ -346,7 +346,7 @@ OLAPStatus PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new OLAPStatus res = OLAP_SUCCESS; RowCursor row; BinaryFile raw_file; - IBinaryReader* reader = NULL; + IBinaryReader* reader = nullptr; uint32_t num_rows = 0; PUniqueId load_id; load_id.set_hi(0); @@ -521,7 +521,7 @@ OLAPStatus BinaryFile::init(const char* path) { } IBinaryReader* IBinaryReader::create(bool need_decompress) { - IBinaryReader* reader = NULL; + IBinaryReader* reader = nullptr; if (need_decompress) { #ifdef DORIS_WITH_LZO reader = new (std::nothrow) LzoBinaryReader(); @@ -532,7 +532,7 @@ IBinaryReader* IBinaryReader::create(bool need_decompress) { return reader; } -BinaryReader::BinaryReader() : IBinaryReader(), _row_buf(NULL), _row_buf_size(0) {} +BinaryReader::BinaryReader() : IBinaryReader(), _row_buf(nullptr), _row_buf_size(0) {} OLAPStatus BinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) { OLAPStatus res = OLAP_SUCCESS; @@ -574,7 +574,7 @@ OLAPStatus BinaryReader::finalize() { OLAPStatus BinaryReader::next(RowCursor* row) { OLAPStatus res = OLAP_SUCCESS; - if (!_ready || NULL == row) { + if (!_ready || nullptr == row) { // Here i assume _ready means all states were set up correctly return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -668,9 +668,9 @@ OLAPStatus BinaryReader::next(RowCursor* row) { LzoBinaryReader::LzoBinaryReader() : IBinaryReader(), - _row_buf(NULL), - _row_compressed_buf(NULL), - _row_info_buf(NULL), + _row_buf(nullptr), + _row_compressed_buf(nullptr), + _row_info_buf(nullptr), _max_row_num(0), _max_row_buf_size(0), _max_compressed_buf_size(0), @@ -719,7 +719,7 @@ OLAPStatus LzoBinaryReader::finalize() { OLAPStatus LzoBinaryReader::next(RowCursor* row) { OLAPStatus res = OLAP_SUCCESS; - if (!_ready || NULL == row) { + if (!_ready || nullptr == row) { // Here i assume _ready means all states were set up correctly return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -885,7 +885,7 @@ OLAPStatus PushBrokerReader::init(const Schema* schema, const TBrokerScanRange& TQueryGlobals query_globals; _runtime_state.reset( new RuntimeState(params, query_options, query_globals, ExecEnv::GetInstance())); - DescriptorTbl* desc_tbl = NULL; + DescriptorTbl* desc_tbl = nullptr; Status status = DescriptorTbl::create(_runtime_state->obj_pool(), t_desc_tbl, &desc_tbl); if (UNLIKELY(!status.ok())) { LOG(WARNING) << "Failed to create descriptor table, msg: " << status.get_error_msg(); diff --git a/be/src/olap/push_handler.h b/be/src/olap/push_handler.h index e1fb787fd3..d782ee4d10 100644 --- a/be/src/olap/push_handler.h +++ b/be/src/olap/push_handler.h @@ -122,7 +122,7 @@ public: protected: IBinaryReader() - : _file(NULL), + : _file(nullptr), _content_len(0), _curr(0), _adler_checksum(ADLER32_INIT), @@ -200,7 +200,8 @@ public: MemPool* mem_pool() { return _mem_pool.get(); } private: - OLAPStatus fill_field_row(RowCursorCell* dst, const char* src,bool src_null, MemPool* mem_pool, FieldType type); + OLAPStatus fill_field_row(RowCursorCell* dst, const char* src, bool src_null, MemPool* mem_pool, + FieldType type); bool _ready; bool _eof; TupleDescriptor* _tuple_desc; diff --git a/be/src/olap/row_block2.cpp b/be/src/olap/row_block2.cpp index 97cbdd1530..20ed4ae029 100644 --- a/be/src/olap/row_block2.cpp +++ b/be/src/olap/row_block2.cpp @@ -114,7 +114,7 @@ std::string RowBlockV2::debug_string() { ss << row(i).debug_string(); if (i != num_rows() - 1) { ss << "\n"; - } + } } return ss.str(); } diff --git a/be/src/olap/row_cursor.cpp b/be/src/olap/row_cursor.cpp index e2f8b5a2f4..b97dc0a2c2 100644 --- a/be/src/olap/row_cursor.cpp +++ b/be/src/olap/row_cursor.cpp @@ -28,7 +28,8 @@ using std::string; using std::vector; namespace doris { -RowCursor::RowCursor() : _fixed_len(0), _variable_len(0), _string_field_count(0), _long_text_buf(nullptr) {} +RowCursor::RowCursor() + : _fixed_len(0), _variable_len(0), _string_field_count(0), _long_text_buf(nullptr) {} RowCursor::~RowCursor() { delete[] _owned_fixed_buf; @@ -78,7 +79,8 @@ OLAPStatus RowCursor::_init(const std::vector& schema, return _init(columns); } -OLAPStatus RowCursor::_init_scan_key(const TabletSchema& schema, const std::vector& scan_keys) { +OLAPStatus RowCursor::_init_scan_key(const TabletSchema& schema, + const std::vector& scan_keys) { // NOTE: cid equal with column index // Hyperloglog cannot be key, no need to handle it _variable_len = 0; @@ -226,7 +228,7 @@ OLAPStatus RowCursor::allocate_memory_for_string_type(const TabletSchema& schema slice->data = *(long_text_ptr); slice->size = DEFAULT_TEXT_LENGTH; ++long_text_ptr; - } else if (_variable_len > 0){ + } else if (_variable_len > 0) { variable_ptr = column_schema(cid)->allocate_memory(fixed_ptr + 1, variable_ptr); } } diff --git a/be/src/olap/rowset/bit_field_reader.cpp b/be/src/olap/rowset/bit_field_reader.cpp index 60f5224552..f809914bb6 100644 --- a/be/src/olap/rowset/bit_field_reader.cpp +++ b/be/src/olap/rowset/bit_field_reader.cpp @@ -24,17 +24,17 @@ namespace doris { BitFieldReader::BitFieldReader(ReadOnlyFileStream* input) - : _input(input), _byte_reader(NULL), _current('\0'), _bits_left(0) {} + : _input(input), _byte_reader(nullptr), _current('\0'), _bits_left(0) {} BitFieldReader::~BitFieldReader() { SAFE_DELETE(_byte_reader); } OLAPStatus BitFieldReader::init() { - if (NULL == _byte_reader) { + if (nullptr == _byte_reader) { _byte_reader = new (std::nothrow) RunLengthByteReader(_input); - if (NULL == _byte_reader) { + if (nullptr == _byte_reader) { OLAP_LOG_WARNING("fail to create RunLengthByteReader"); return OLAP_ERR_MALLOC_ERROR; } diff --git a/be/src/olap/rowset/bit_field_writer.cpp b/be/src/olap/rowset/bit_field_writer.cpp index 3e01f8e89b..99701372fa 100644 --- a/be/src/olap/rowset/bit_field_writer.cpp +++ b/be/src/olap/rowset/bit_field_writer.cpp @@ -24,7 +24,7 @@ namespace doris { BitFieldWriter::BitFieldWriter(OutStream* output) - : _output(output), _byte_writer(NULL), _current(0), _bits_left(8) {} + : _output(output), _byte_writer(nullptr), _current(0), _bits_left(8) {} BitFieldWriter::~BitFieldWriter() { SAFE_DELETE(_byte_writer); @@ -33,7 +33,7 @@ BitFieldWriter::~BitFieldWriter() { OLAPStatus BitFieldWriter::init() { _byte_writer = new (std::nothrow) RunLengthByteWriter(_output); - if (NULL == _byte_writer) { + if (nullptr == _byte_writer) { OLAP_LOG_WARNING("fail to create RunLengthByteWriter"); return OLAP_ERR_MALLOC_ERROR; } @@ -83,7 +83,7 @@ OLAPStatus BitFieldWriter::flush() { } void BitFieldWriter::get_position(PositionEntryWriter* index_entry) const { - if (NULL != _byte_writer) { + if (nullptr != _byte_writer) { _byte_writer->get_position(index_entry); } else { // for stream diff --git a/be/src/olap/rowset/column_data_writer.cpp b/be/src/olap/rowset/column_data_writer.cpp index 33452a256c..d2b44c863b 100644 --- a/be/src/olap/rowset/column_data_writer.cpp +++ b/be/src/olap/rowset/column_data_writer.cpp @@ -39,10 +39,10 @@ ColumnDataWriter::ColumnDataWriter(SegmentGroup* segment_group, bool is_push_wri _is_push_write(is_push_write), _compress_kind(compress_kind), _bloom_filter_fpp(bloom_filter_fpp), - _zone_maps(segment_group->get_num_zone_map_columns(), KeyRange(NULL, NULL)), + _zone_maps(segment_group->get_num_zone_map_columns(), KeyRange(nullptr, nullptr)), _row_index(0), - _row_block(NULL), - _segment_writer(NULL), + _row_block(nullptr), + _segment_writer(nullptr), _num_rows(0), _block_id(0), _max_segment_size(OLAP_MAX_COLUMN_SEGMENT_FILE_SIZE), @@ -79,7 +79,7 @@ OLAPStatus ColumnDataWriter::init() { _row_block = new (std::nothrow) RowBlock(&(_segment_group->get_tablet_schema())); - if (NULL == _row_block) { + if (nullptr == _row_block) { LOG(WARNING) << "fail to new RowBlock."; return OLAP_ERR_MALLOC_ERROR; } @@ -91,7 +91,7 @@ OLAPStatus ColumnDataWriter::init() { } VLOG_NOTICE << "init ColumnData writer. segment_group_id=" << _segment_group->segment_group_id() - << ", block_row_number=" << _segment_group->get_num_rows_per_row_block(); + << ", block_row_number=" << _segment_group->get_num_rows_per_row_block(); RowBlockInfo block_info(0U, _segment_group->get_num_rows_per_row_block()); block_info.null_supported = true; @@ -232,7 +232,7 @@ OLAPStatus ColumnDataWriter::_flush_row_block(bool finalize) { OLAPStatus ColumnDataWriter::_add_segment() { std::string file_name; - if (NULL != _segment_writer) { + if (nullptr != _segment_writer) { OLAP_LOG_WARNING("previous segment is not finalized before add new segment."); return OLAP_ERR_WRITER_SEGMENT_NOT_FINALIZED; } @@ -242,7 +242,7 @@ OLAPStatus ColumnDataWriter::_add_segment() { SegmentWriter(file_name, _segment_group, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, _compress_kind, _bloom_filter_fpp); - if (NULL == _segment_writer) { + if (nullptr == _segment_writer) { OLAP_LOG_WARNING("fail to allocate SegmentWriter"); return OLAP_ERR_MALLOC_ERROR; } diff --git a/be/src/olap/rowset/column_reader.cpp b/be/src/olap/rowset/column_reader.cpp index 25fc9e0b37..e870a3c947 100644 --- a/be/src/olap/rowset/column_reader.cpp +++ b/be/src/olap/rowset/column_reader.cpp @@ -25,7 +25,7 @@ namespace doris { IntegerColumnReader::IntegerColumnReader(uint32_t column_unique_id) - : _eof(false), _column_unique_id(column_unique_id), _data_reader(NULL) {} + : _eof(false), _column_unique_id(column_unique_id), _data_reader(nullptr) {} IntegerColumnReader::~IntegerColumnReader() { SAFE_DELETE(_data_reader); @@ -33,8 +33,8 @@ IntegerColumnReader::~IntegerColumnReader() { OLAPStatus IntegerColumnReader::init(std::map* streams, bool is_sign) { - if (NULL == streams) { - OLAP_LOG_WARNING("input streams is NULL"); + if (nullptr == streams) { + OLAP_LOG_WARNING("input streams is nullptr"); return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -42,14 +42,14 @@ OLAPStatus IntegerColumnReader::init(std::map* ReadOnlyFileStream* data_stream = extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams); - if (data_stream == NULL) { - OLAP_LOG_WARNING("specified stream is NULL"); + if (data_stream == nullptr) { + OLAP_LOG_WARNING("specified stream is nullptr"); return OLAP_ERR_COLUMN_STREAM_NOT_EXIST; } _data_reader = new (std::nothrow) RunLengthIntegerReader(data_stream, is_sign); - if (NULL == _data_reader) { + if (nullptr == _data_reader) { OLAP_LOG_WARNING("fail to malloc RunLengthIntegerReader"); return OLAP_ERR_MALLOC_ERROR; } @@ -74,9 +74,9 @@ StringColumnDirectReader::StringColumnDirectReader(uint32_t column_unique_id, uint32_t dictionary_size) : _eof(false), _column_unique_id(column_unique_id), - _values(NULL), - _data_stream(NULL), - _length_reader(NULL) {} + _values(nullptr), + _data_stream(nullptr), + _length_reader(nullptr) {} StringColumnDirectReader::~StringColumnDirectReader() { SAFE_DELETE(_length_reader); @@ -84,15 +84,15 @@ StringColumnDirectReader::~StringColumnDirectReader() { OLAPStatus StringColumnDirectReader::init(std::map* streams, int size, MemPool* mem_pool) { - if (NULL == streams) { - OLAP_LOG_WARNING("input streams is NULL"); + if (nullptr == streams) { + OLAP_LOG_WARNING("input streams is nullptr"); return OLAP_ERR_INPUT_PARAMETER_ERROR; } // Get data stream according to column id and type _data_stream = extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams); - if (NULL == _data_stream) { + if (nullptr == _data_stream) { OLAP_LOG_WARNING("specified stream not found. [unique_id = %u]", _column_unique_id); return OLAP_ERR_COLUMN_STREAM_NOT_EXIST; } @@ -102,14 +102,14 @@ OLAPStatus StringColumnDirectReader::init(std::map= schema.size()) { LOG(WARNING) << "invalid column_id, column_id=" << column_id << ", columns_size=" << schema.size(); - return NULL; + return nullptr; } const TabletColumn& column = schema[column_id]; - ColumnReader* reader = NULL; + ColumnReader* reader = nullptr; int32_t column_unique_id = column.unique_id(); if (0 == included.count(column_unique_id)) { - return NULL; + return nullptr; } if (0 == segment_included.count(column_unique_id)) { @@ -513,7 +513,7 @@ ColumnReader* ColumnReader::create(uint32_t column_id, const std::vector* streams, int size, MemPool* mem_pool, OlapReaderStatistics* stats) { - if (NULL == streams) { + if (nullptr == streams) { OLAP_LOG_WARNING("null parameters given."); return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -676,14 +676,14 @@ OLAPStatus ColumnReader::init(std::map* streams _is_null = reinterpret_cast(mem_pool->allocate(size)); memset(_is_null, 0, size); - if (NULL == present_stream) { - _present_reader = NULL; + if (nullptr == present_stream) { + _present_reader = nullptr; _value_present = false; } else { VLOG_TRACE << "create null present_stream for column_id:" << _column_unique_id; _present_reader = new (std::nothrow) BitFieldReader(present_stream); - if (NULL == _present_reader) { + if (nullptr == _present_reader) { OLAP_LOG_WARNING("malloc present reader failed."); return OLAP_ERR_MALLOC_ERROR; } @@ -700,7 +700,7 @@ OLAPStatus ColumnReader::init(std::map* streams } OLAPStatus ColumnReader::seek(PositionProvider* position) { - if (NULL != _present_reader) { + if (nullptr != _present_reader) { return _present_reader->seek(position); } @@ -715,7 +715,7 @@ OLAPStatus ColumnReader::next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool) { OLAPStatus res = OLAP_SUCCESS; column_vector->set_is_null(_is_null); - if (NULL != _present_reader) { + if (nullptr != _present_reader) { column_vector->set_no_nulls(false); for (uint32_t i = 0; i < size; ++i) { bool value = false; @@ -734,7 +734,7 @@ OLAPStatus ColumnReader::next_vector(ColumnVector* column_vector, uint32_t size, } uint64_t ColumnReader::_count_none_nulls(uint64_t rows) { - if (_present_reader != NULL) { + if (_present_reader != nullptr) { OLAPStatus res = OLAP_SUCCESS; uint64_t result = 0; @@ -757,8 +757,8 @@ uint64_t ColumnReader::_count_none_nulls(uint64_t rows) { TinyColumnReader::TinyColumnReader(uint32_t column_id, uint32_t column_unique_id) : ColumnReader(column_id, column_unique_id), _eof(false), - _values(NULL), - _data_reader(NULL) {} + _values(nullptr), + _data_reader(nullptr) {} TinyColumnReader::~TinyColumnReader() { SAFE_DELETE(_data_reader); @@ -766,8 +766,8 @@ TinyColumnReader::~TinyColumnReader() { OLAPStatus TinyColumnReader::init(std::map* streams, int size, MemPool* mem_pool, OlapReaderStatistics* stats) { - if (NULL == streams) { - OLAP_LOG_WARNING("input streams is NULL"); + if (nullptr == streams) { + OLAP_LOG_WARNING("input streams is nullptr"); return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -775,7 +775,7 @@ OLAPStatus TinyColumnReader::init(std::map* str ReadOnlyFileStream* data_stream = extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams); - if (NULL == data_stream) { + if (nullptr == data_stream) { OLAP_LOG_WARNING("specified stream not exist"); return OLAP_ERR_COLUMN_STREAM_NOT_EXIST; } @@ -783,7 +783,7 @@ OLAPStatus TinyColumnReader::init(std::map* str _values = reinterpret_cast(mem_pool->allocate(size)); _data_reader = new (std::nothrow) RunLengthByteReader(data_stream); - if (NULL == _data_reader) { + if (nullptr == _data_reader) { OLAP_LOG_WARNING("malloc data reader failed"); return OLAP_ERR_MALLOC_ERROR; } @@ -793,7 +793,7 @@ OLAPStatus TinyColumnReader::init(std::map* str OLAPStatus TinyColumnReader::seek(PositionProvider* positions) { OLAPStatus res; - if (NULL == _present_reader) { + if (nullptr == _present_reader) { res = _data_reader->seek(positions); if (OLAP_SUCCESS != res) { return res; @@ -859,9 +859,9 @@ OLAPStatus TinyColumnReader::next_vector(ColumnVector* column_vector, uint32_t s DecimalColumnReader::DecimalColumnReader(uint32_t column_id, uint32_t column_unique_id) : ColumnReader(column_id, column_unique_id), _eof(false), - _values(NULL), - _int_reader(NULL), - _frac_reader(NULL) {} + _values(nullptr), + _int_reader(nullptr), + _frac_reader(nullptr) {} DecimalColumnReader::~DecimalColumnReader() { SAFE_DELETE(_int_reader); @@ -870,8 +870,8 @@ DecimalColumnReader::~DecimalColumnReader() { OLAPStatus DecimalColumnReader::init(std::map* streams, int size, MemPool* mem_pool, OlapReaderStatistics* stats) { - if (NULL == streams) { - OLAP_LOG_WARNING("input streams is NULL"); + if (nullptr == streams) { + OLAP_LOG_WARNING("input streams is nullptr"); return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -884,7 +884,7 @@ OLAPStatus DecimalColumnReader::init(std::map* ReadOnlyFileStream* int_stream = extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams); - if (NULL == int_stream) { + if (nullptr == int_stream) { OLAP_LOG_WARNING("specified stream not found. [unique_id = %u]", _column_unique_id); return OLAP_ERR_COLUMN_STREAM_NOT_EXIST; } @@ -892,21 +892,21 @@ OLAPStatus DecimalColumnReader::init(std::map* ReadOnlyFileStream* frac_stream = extract_stream(_column_unique_id, StreamInfoMessage::SECONDARY, streams); - if (NULL == frac_stream) { + if (nullptr == frac_stream) { OLAP_LOG_WARNING("specified stream not found. [unique_id = %u]", _column_unique_id); return OLAP_ERR_COLUMN_STREAM_NOT_EXIST; } _int_reader = new (std::nothrow) RunLengthIntegerReader(int_stream, true); - if (NULL == _int_reader) { + if (nullptr == _int_reader) { OLAP_LOG_WARNING("fail to malloc RunLengthIntegerReader"); return OLAP_ERR_MALLOC_ERROR; } _frac_reader = new (std::nothrow) RunLengthIntegerReader(frac_stream, true); - if (NULL == _frac_reader) { + if (nullptr == _frac_reader) { OLAP_LOG_WARNING("fail to malloc RunLengthIntegerReader"); return OLAP_ERR_MALLOC_ERROR; } @@ -916,7 +916,7 @@ OLAPStatus DecimalColumnReader::init(std::map* OLAPStatus DecimalColumnReader::seek(PositionProvider* positions) { OLAPStatus res; - if (NULL == _present_reader) { + if (nullptr == _present_reader) { res = _int_reader->seek(positions); if (OLAP_SUCCESS != res) { return res; @@ -928,7 +928,7 @@ OLAPStatus DecimalColumnReader::seek(PositionProvider* positions) { return res; } } else { - //all field in the segment can be NULL, so the data stream is EOF + //all field in the segment can be nullptr, so the data stream is EOF res = ColumnReader::seek(positions); if (OLAP_SUCCESS != res) { return res; @@ -1025,9 +1025,9 @@ OLAPStatus DecimalColumnReader::next_vector(ColumnVector* column_vector, uint32_ LargeIntColumnReader::LargeIntColumnReader(uint32_t column_id, uint32_t column_unique_id) : ColumnReader(column_id, column_unique_id), _eof(false), - _values(NULL), - _high_reader(NULL), - _low_reader(NULL) {} + _values(nullptr), + _high_reader(nullptr), + _low_reader(nullptr) {} LargeIntColumnReader::~LargeIntColumnReader() { SAFE_DELETE(_high_reader); @@ -1036,8 +1036,8 @@ LargeIntColumnReader::~LargeIntColumnReader() { OLAPStatus LargeIntColumnReader::init(std::map* streams, int size, MemPool* mem_pool, OlapReaderStatistics* stats) { - if (NULL == streams) { - OLAP_LOG_WARNING("input streams is NULL"); + if (nullptr == streams) { + OLAP_LOG_WARNING("input streams is nullptr"); return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -1050,26 +1050,26 @@ OLAPStatus LargeIntColumnReader::init(std::map* // 从map中找到需要的流,LargeIntColumnReader的数据应该由一条DATA流组成 ReadOnlyFileStream* high_stream = extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams); - if (NULL == high_stream) { + if (nullptr == high_stream) { OLAP_LOG_WARNING("specified stream not found. [unique_id = %u]", _column_unique_id); return OLAP_ERR_COLUMN_STREAM_NOT_EXIST; } ReadOnlyFileStream* low_stream = extract_stream(_column_unique_id, StreamInfoMessage::SECONDARY, streams); - if (NULL == low_stream) { + if (nullptr == low_stream) { OLAP_LOG_WARNING("specified stream not found. [unique_id = %u]", _column_unique_id); return OLAP_ERR_COLUMN_STREAM_NOT_EXIST; } _high_reader = new (std::nothrow) RunLengthIntegerReader(high_stream, true); - if (NULL == _high_reader) { + if (nullptr == _high_reader) { OLAP_LOG_WARNING("fail to malloc RunLengthIntegerReader."); return OLAP_ERR_MALLOC_ERROR; } _low_reader = new (std::nothrow) RunLengthIntegerReader(low_stream, true); - if (NULL == _low_reader) { + if (nullptr == _low_reader) { OLAP_LOG_WARNING("fail to malloc RunLengthIntegerReader."); return OLAP_ERR_MALLOC_ERROR; } @@ -1079,7 +1079,7 @@ OLAPStatus LargeIntColumnReader::init(std::map* OLAPStatus LargeIntColumnReader::seek(PositionProvider* positions) { OLAPStatus res; - if (NULL == _present_reader) { + if (nullptr == _present_reader) { res = _high_reader->seek(positions); if (OLAP_SUCCESS != res) { return res; @@ -1090,7 +1090,7 @@ OLAPStatus LargeIntColumnReader::seek(PositionProvider* positions) { return res; } } else { - //all field in the segment can be NULL, so the data stream is EOF + //all field in the segment can be nullptr, so the data stream is EOF res = ColumnReader::seek(positions); if (OLAP_SUCCESS != res) { OLAP_LOG_WARNING("fail to seek null stream of largeint"); @@ -1144,7 +1144,7 @@ OLAPStatus LargeIntColumnReader::next_vector(ColumnVector* column_vector, uint32 if (column_vector->no_nulls()) { for (uint32_t i = 0; i < size; ++i) { - int64_t* value = NULL; + int64_t* value = nullptr; value = (int64_t*)(_values + i); res = _high_reader->next(value); if (OLAP_SUCCESS != res) { @@ -1160,7 +1160,7 @@ OLAPStatus LargeIntColumnReader::next_vector(ColumnVector* column_vector, uint32 } } else { for (uint32_t i = 0; i < size; ++i) { - int64_t* value = NULL; + int64_t* value = nullptr; if (!is_null[i]) { value = (int64_t*)(_values + i); res = _high_reader->next(value); diff --git a/be/src/olap/rowset/column_reader.h b/be/src/olap/rowset/column_reader.h index a57a3eb0a7..b585a2453c 100644 --- a/be/src/olap/rowset/column_reader.h +++ b/be/src/olap/rowset/column_reader.h @@ -48,7 +48,7 @@ inline ReadOnlyFileStream* extract_stream(uint32_t column_unique_id, StreamInfoM return (*it).second; } - return NULL; + return nullptr; } // Unique id -> PositionProvider @@ -228,7 +228,7 @@ public: FieldType type, int length) : ColumnReader(column_id, column_unique_id), _default_value(default_value), - _values(NULL), + _values(nullptr), _type(type), _length(length) {} @@ -432,7 +432,7 @@ public: IntegerColumnReaderWrapper(uint32_t column_id, uint32_t column_unique_id) : ColumnReader(column_id, column_unique_id), _reader(column_unique_id), - _values(NULL), + _values(nullptr), _eof(false) {} virtual ~IntegerColumnReaderWrapper() {} @@ -451,13 +451,13 @@ public: } virtual OLAPStatus seek(PositionProvider* positions) { OLAPStatus res; - if (NULL == _present_reader) { + if (nullptr == _present_reader) { res = _reader.seek(positions); if (OLAP_SUCCESS != res) { return res; } } else { - //all field in the segment can be NULL, so the data stream is EOF + //all field in the segment can be nullptr, so the data stream is EOF res = ColumnReader::seek(positions); if (OLAP_SUCCESS != res) { return res; @@ -550,13 +550,13 @@ public: virtual OLAPStatus seek(PositionProvider* positions) { OLAPStatus res; - if (NULL == _present_reader) { + if (nullptr == _present_reader) { res = _reader.seek(positions); if (OLAP_SUCCESS != res) { return res; } } else { - //all field in the segment can be NULL, so the data stream is EOF + //all field in the segment can be nullptr, so the data stream is EOF res = ColumnReader::seek(positions); if (OLAP_SUCCESS != res) { return res; @@ -616,13 +616,13 @@ public: virtual OLAPStatus seek(PositionProvider* position) { OLAPStatus res; - if (NULL == _present_reader) { + if (nullptr == _present_reader) { res = _reader.seek(position); if (OLAP_SUCCESS != res) { return res; } } else { - //all field in the segment can be NULL, so the data stream is EOF + //all field in the segment can be nullptr, so the data stream is EOF res = ColumnReader::seek(position); if (OLAP_SUCCESS != res) { return res; @@ -666,15 +666,15 @@ public: FloatintPointColumnReader(uint32_t column_id, uint32_t column_unique_id) : ColumnReader(column_id, column_unique_id), _eof(false), - _data_stream(NULL), - _values(NULL) {} + _data_stream(nullptr), + _values(nullptr) {} virtual ~FloatintPointColumnReader() {} virtual OLAPStatus init(std::map* streams, int size, MemPool* mem_pool, OlapReaderStatistics* stats) { - if (NULL == streams) { - OLAP_LOG_WARNING("input streams is NULL"); + if (nullptr == streams) { + OLAP_LOG_WARNING("input streams is nullptr"); return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -682,7 +682,7 @@ public: ColumnReader::init(streams, size, mem_pool, stats); _data_stream = extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams); - if (NULL == _data_stream) { + if (nullptr == _data_stream) { OLAP_LOG_WARNING("specified stream not exist"); return OLAP_ERR_COLUMN_STREAM_NOT_EXIST; } @@ -692,24 +692,24 @@ public: return OLAP_SUCCESS; } virtual OLAPStatus seek(PositionProvider* position) { - if (NULL == position) { - OLAP_LOG_WARNING("input positions is NULL"); + if (nullptr == position) { + OLAP_LOG_WARNING("input positions is nullptr"); return OLAP_ERR_INPUT_PARAMETER_ERROR; } - if (NULL == _data_stream) { + if (nullptr == _data_stream) { OLAP_LOG_WARNING("reader not init."); return OLAP_ERR_NOT_INITED; } OLAPStatus res; - if (NULL == _present_reader) { + if (nullptr == _present_reader) { res = _data_stream->seek(position); if (OLAP_SUCCESS != res) { return res; } } else { - //all field in the segment can be NULL, so the data stream is EOF + //all field in the segment can be nullptr, so the data stream is EOF res = ColumnReader::seek(position); if (OLAP_SUCCESS != res) { return res; @@ -724,7 +724,7 @@ public: return OLAP_SUCCESS; } virtual OLAPStatus skip(uint64_t row_count) { - if (NULL == _data_stream) { + if (nullptr == _data_stream) { OLAP_LOG_WARNING("reader not init."); return OLAP_ERR_NOT_INITED; } @@ -734,7 +734,7 @@ public: } virtual OLAPStatus next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool) { - if (NULL == _data_stream) { + if (nullptr == _data_stream) { OLAP_LOG_WARNING("reader not init."); return OLAP_ERR_NOT_INITED; } diff --git a/be/src/olap/rowset/column_writer.cpp b/be/src/olap/rowset/column_writer.cpp index 60a901778b..8c7b3b5153 100644 --- a/be/src/olap/rowset/column_writer.cpp +++ b/be/src/olap/rowset/column_writer.cpp @@ -133,11 +133,11 @@ ColumnWriter::ColumnWriter(uint32_t column_id, OutStreamFactory* stream_factory, _column(column), _stream_factory(stream_factory), _index(column.type()), - _is_present(NULL), - _is_present_stream(NULL), - _index_stream(NULL), + _is_present(nullptr), + _is_present_stream(nullptr), + _index_stream(nullptr), _is_found_nulls(false), - _bf(NULL), + _bf(nullptr), _num_rows_per_row_block(num_rows_per_row_block), _bf_fpp(bf_fpp) {} @@ -156,14 +156,14 @@ OLAPStatus ColumnWriter::init() { _is_present_stream = _stream_factory->create_stream(unique_column_id(), StreamInfoMessage::PRESENT); - if (NULL == _is_present_stream) { + if (nullptr == _is_present_stream) { OLAP_LOG_WARNING("fail to allocate IS PRESENT STREAM"); return OLAP_ERR_MALLOC_ERROR; } _is_present = new (std::nothrow) BitFieldWriter(_is_present_stream); - if (NULL == _is_present) { + if (nullptr == _is_present) { OLAP_LOG_WARNING("fail to allocate IS PRESENT Writer"); return OLAP_ERR_MALLOC_ERROR; } @@ -191,7 +191,7 @@ OLAPStatus ColumnWriter::init() { _index_stream = _stream_factory->create_stream(unique_column_id(), StreamInfoMessage::ROW_INDEX); - if (NULL == _index_stream) { + if (nullptr == _index_stream) { OLAP_LOG_WARNING("fail to allocate Index STREAM"); return OLAP_ERR_MALLOC_ERROR; } @@ -200,13 +200,13 @@ OLAPStatus ColumnWriter::init() { if (is_bf_column()) { _bf_index_stream = _stream_factory->create_stream(unique_column_id(), StreamInfoMessage::BLOOM_FILTER); - if (NULL == _bf_index_stream) { + if (nullptr == _bf_index_stream) { OLAP_LOG_WARNING("fail to allocate bloom filter index stream"); return OLAP_ERR_MALLOC_ERROR; } _bf = new (std::nothrow) BloomFilter(); - if (NULL == _bf) { + if (nullptr == _bf) { OLAP_LOG_WARNING("fail to allocate bloom filter"); return OLAP_ERR_MALLOC_ERROR; } @@ -238,15 +238,14 @@ OLAPStatus ColumnWriter::write(RowCursor* row_cursor) { if (!is_null) { if (_column.type() == OLAP_FIELD_TYPE_CHAR || _column.type() == OLAP_FIELD_TYPE_VARCHAR || - _column.type() == OLAP_FIELD_TYPE_HLL || - _column.type() == OLAP_FIELD_TYPE_STRING) { + _column.type() == OLAP_FIELD_TYPE_HLL || _column.type() == OLAP_FIELD_TYPE_STRING) { Slice* slice = reinterpret_cast(buf); _bf->add_bytes(slice->data, slice->size); } else { _bf->add_bytes(buf, row_cursor->column_size(_column_id)); } } else { - _bf->add_bytes(NULL, 0); + _bf->add_bytes(nullptr, 0); } } @@ -270,7 +269,7 @@ OLAPStatus ColumnWriter::create_row_index_entry() { _bf_index.add_bloom_filter(_bf); _bf = new (std::nothrow) BloomFilter(); - if (NULL == _bf) { + if (nullptr == _bf) { OLAP_LOG_WARNING("fail to allocate bloom filter"); return OLAP_ERR_MALLOC_ERROR; } @@ -324,7 +323,7 @@ void ColumnWriter::_remove_is_present_positions() { OLAPStatus ColumnWriter::finalize(ColumnDataHeaderMessage* header) { OLAPStatus res = OLAP_SUCCESS; - if (NULL != _is_present) { + if (nullptr != _is_present) { if (OLAP_SUCCESS != (res = _is_present->flush())) { return res; } @@ -335,12 +334,12 @@ OLAPStatus ColumnWriter::finalize(ColumnDataHeaderMessage* header) { } } - char* index_buf = NULL; + char* index_buf = nullptr; // char* index_statistic_buf = NULL; // Write index pb size_t pb_size = _index.output_size(); index_buf = new (std::nothrow) char[pb_size]; - ColumnMessage* column = NULL; + ColumnMessage* column = nullptr; if (OLAP_SUCCESS != _index.write_to_buffer(index_buf, pb_size)) { OLAP_LOG_WARNING("fail to serialize index"); @@ -399,7 +398,7 @@ FINALIZE_EXIT: } void ColumnWriter::record_position() { - if (NULL != _is_present) { + if (nullptr != _is_present) { _is_present->get_position(&_index_entry); } } @@ -432,7 +431,7 @@ ByteColumnWriter::ByteColumnWriter(uint32_t column_id, OutStreamFactory* stream_ const TabletColumn& column, size_t num_rows_per_row_block, double bf_fpp) : ColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp), - _writer(NULL) {} + _writer(nullptr) {} ByteColumnWriter::~ByteColumnWriter() { SAFE_DELETE(_writer); @@ -448,14 +447,14 @@ OLAPStatus ByteColumnWriter::init() { OutStreamFactory* factory = stream_factory(); OutStream* stream = factory->create_stream(unique_column_id(), StreamInfoMessage::DATA); - if (NULL == stream) { + if (nullptr == stream) { OLAP_LOG_WARNING("fail to allocate DATA STREAM"); return OLAP_ERR_MALLOC_ERROR; } _writer = new (std::nothrow) RunLengthByteWriter(stream); - if (NULL == _writer) { + if (nullptr == _writer) { OLAP_LOG_WARNING("fail to allocate RunLengthByteWriter"); return OLAP_ERR_MALLOC_ERROR; } @@ -492,7 +491,7 @@ IntegerColumnWriter::IntegerColumnWriter(uint32_t column_id, uint32_t unique_col : _column_id(column_id), _unique_column_id(unique_column_id), _stream_factory(stream_factory), - _writer(NULL), + _writer(nullptr), _is_signed(is_singed) {} IntegerColumnWriter::~IntegerColumnWriter() { @@ -502,14 +501,14 @@ IntegerColumnWriter::~IntegerColumnWriter() { OLAPStatus IntegerColumnWriter::init() { OutStream* stream = _stream_factory->create_stream(_unique_column_id, StreamInfoMessage::DATA); - if (NULL == stream) { + if (nullptr == stream) { OLAP_LOG_WARNING("fail to allocate DATA STREAM"); return OLAP_ERR_MALLOC_ERROR; } _writer = new (std::nothrow) RunLengthIntegerWriter(stream, _is_signed); - if (NULL == _writer) { + if (nullptr == _writer) { OLAP_LOG_WARNING("fail to allocate RunLengthIntegerWriter"); return OLAP_ERR_MALLOC_ERROR; } @@ -525,10 +524,10 @@ VarStringColumnWriter::VarStringColumnWriter(uint32_t column_id, OutStreamFactor : ColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp), _use_dictionary_encoding(false), _dict_total_size(0), - _dict_stream(NULL), - _length_writer(NULL), - _data_stream(NULL), - _id_writer(NULL) {} + _dict_stream(nullptr), + _length_writer(nullptr), + _data_stream(nullptr), + _id_writer(nullptr) {} VarStringColumnWriter::~VarStringColumnWriter() { SAFE_DELETE(_length_writer); @@ -548,7 +547,7 @@ OLAPStatus VarStringColumnWriter::init() { OutStream* length_stream = stream_factory()->create_stream(unique_column_id(), StreamInfoMessage::LENGTH); - if (NULL == _dict_stream || NULL == length_stream || NULL == _data_stream) { + if (nullptr == _dict_stream || nullptr == length_stream || nullptr == _data_stream) { OLAP_LOG_WARNING("fail to create stream."); return OLAP_ERR_MALLOC_ERROR; } @@ -556,7 +555,7 @@ OLAPStatus VarStringColumnWriter::init() { _length_writer = new (std::nothrow) RunLengthIntegerWriter(length_stream, false); _id_writer = new (std::nothrow) RunLengthIntegerWriter(_data_stream, false); - if (NULL == _length_writer || NULL == _id_writer) { + if (nullptr == _length_writer || nullptr == _id_writer) { OLAP_LOG_WARNING("fail to create writer."); return OLAP_ERR_MALLOC_ERROR; } @@ -754,8 +753,8 @@ DecimalColumnWriter::DecimalColumnWriter(uint32_t column_id, OutStreamFactory* s const TabletColumn& column, size_t num_rows_per_row_block, double bf_fpp) : ColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp), - _int_writer(NULL), - _frac_writer(NULL) {} + _int_writer(nullptr), + _frac_writer(nullptr) {} DecimalColumnWriter::~DecimalColumnWriter() { SAFE_DELETE(_int_writer); @@ -775,7 +774,7 @@ OLAPStatus DecimalColumnWriter::init() { OutStream* frac_stream = stream_factory()->create_stream(unique_column_id(), StreamInfoMessage::SECONDARY); - if (NULL == int_stream || NULL == frac_stream) { + if (nullptr == int_stream || nullptr == frac_stream) { OLAP_LOG_WARNING("fail to create stream."); return OLAP_ERR_MALLOC_ERROR; } @@ -783,7 +782,7 @@ OLAPStatus DecimalColumnWriter::init() { _int_writer = new (std::nothrow) RunLengthIntegerWriter(int_stream, true); _frac_writer = new (std::nothrow) RunLengthIntegerWriter(frac_stream, true); - if (NULL == _int_writer || NULL == _frac_writer) { + if (nullptr == _int_writer || nullptr == _frac_writer) { OLAP_LOG_WARNING("fail to create writer."); return OLAP_ERR_MALLOC_ERROR; } @@ -828,8 +827,8 @@ LargeIntColumnWriter::LargeIntColumnWriter(uint32_t column_id, OutStreamFactory* const TabletColumn& column, size_t num_rows_per_row_block, double bf_fpp) : ColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp), - _high_writer(NULL), - _low_writer(NULL) {} + _high_writer(nullptr), + _low_writer(nullptr) {} LargeIntColumnWriter::~LargeIntColumnWriter() { SAFE_DELETE(_high_writer); @@ -849,7 +848,7 @@ OLAPStatus LargeIntColumnWriter::init() { OutStream* low_stream = stream_factory()->create_stream(unique_column_id(), StreamInfoMessage::SECONDARY); - if (NULL == high_stream || NULL == low_stream) { + if (nullptr == high_stream || nullptr == low_stream) { OLAP_LOG_WARNING("fail to create stream."); return OLAP_ERR_MALLOC_ERROR; } @@ -857,7 +856,7 @@ OLAPStatus LargeIntColumnWriter::init() { _high_writer = new (std::nothrow) RunLengthIntegerWriter(high_stream, true); _low_writer = new (std::nothrow) RunLengthIntegerWriter(low_stream, true); - if (NULL == _high_writer || NULL == _low_writer) { + if (nullptr == _high_writer || nullptr == _low_writer) { OLAP_LOG_WARNING("fail to create writer."); return OLAP_ERR_MALLOC_ERROR; } diff --git a/be/src/olap/rowset/column_writer.h b/be/src/olap/rowset/column_writer.h index 9ec02617d9..e40ad38f31 100644 --- a/be/src/olap/rowset/column_writer.h +++ b/be/src/olap/rowset/column_writer.h @@ -287,7 +287,7 @@ public: DoubleColumnWriterBase(uint32_t column_id, OutStreamFactory* stream_factory, const TabletColumn& column, size_t num_rows_per_row_block, double bf_fpp) : ColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp), - _stream(NULL) {} + _stream(nullptr) {} virtual ~DoubleColumnWriterBase() {} @@ -302,7 +302,7 @@ public: OutStreamFactory* factory = stream_factory(); _stream = factory->create_stream(unique_column_id(), StreamInfoMessage::DATA); - if (NULL == _stream) { + if (nullptr == _stream) { OLAP_LOG_WARNING("fail to allocate DATA STREAM"); return OLAP_ERR_MALLOC_ERROR; } diff --git a/be/src/olap/rowset/segment_group.cpp b/be/src/olap/rowset/segment_group.cpp index 9f7b79031e..59ce46042b 100644 --- a/be/src/olap/rowset/segment_group.cpp +++ b/be/src/olap/rowset/segment_group.cpp @@ -50,20 +50,20 @@ namespace doris { } \ } while (0); -#define POS_PARAM_VALIDATE(pos) \ - do { \ - if (NULL == pos) { \ - OLAP_LOG_WARNING("fail to find, NULL position parameter."); \ - return OLAP_ERR_INPUT_PARAMETER_ERROR; \ - } \ +#define POS_PARAM_VALIDATE(pos) \ + do { \ + if (nullptr == pos) { \ + OLAP_LOG_WARNING("fail to find, nullptr position parameter."); \ + return OLAP_ERR_INPUT_PARAMETER_ERROR; \ + } \ } while (0); -#define SLICE_PARAM_VALIDATE(slice) \ - do { \ - if (NULL == slice) { \ - OLAP_LOG_WARNING("fail to find, NULL slice parameter."); \ - return OLAP_ERR_INPUT_PARAMETER_ERROR; \ - } \ +#define SLICE_PARAM_VALIDATE(slice) \ + do { \ + if (nullptr == slice) { \ + OLAP_LOG_WARNING("fail to find, nullptr slice parameter."); \ + return OLAP_ERR_INPUT_PARAMETER_ERROR; \ + } \ } while (0); SegmentGroup::SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const TabletSchema* schema, @@ -94,7 +94,8 @@ SegmentGroup::SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const T const TabletColumn& column = _schema->column(i); _short_key_columns.push_back(column); _short_key_length += column.index_length() + 1; // 1 for null byte - if (column.type() == OLAP_FIELD_TYPE_CHAR || column.type() == OLAP_FIELD_TYPE_VARCHAR || column.type() == OLAP_FIELD_TYPE_STRING) { + if (column.type() == OLAP_FIELD_TYPE_CHAR || column.type() == OLAP_FIELD_TYPE_VARCHAR || + column.type() == OLAP_FIELD_TYPE_STRING) { _new_short_key_length += sizeof(Slice) + 1; } else { _new_short_key_length += column.index_length() + 1; @@ -124,7 +125,7 @@ SegmentGroup::SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const T _ref_count = 0; _short_key_length = 0; _new_short_key_length = 0; - _short_key_buf = NULL; + _short_key_buf = nullptr; _new_segment_created = false; _empty = false; @@ -132,7 +133,8 @@ SegmentGroup::SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const T const TabletColumn& column = _schema->column(i); _short_key_columns.push_back(column); _short_key_length += column.index_length() + 1; // 1 for null byte - if (column.type() == OLAP_FIELD_TYPE_CHAR || column.type() == OLAP_FIELD_TYPE_VARCHAR || column.type() == OLAP_FIELD_TYPE_STRING) { + if (column.type() == OLAP_FIELD_TYPE_CHAR || column.type() == OLAP_FIELD_TYPE_VARCHAR || + column.type() == OLAP_FIELD_TYPE_STRING) { _new_short_key_length += sizeof(Slice) + 1; } else { _new_short_key_length += column.index_length() + 1; @@ -272,7 +274,7 @@ OLAPStatus SegmentGroup::add_zone_maps_for_linked_schema_change( } const TabletColumn& column = _schema->column(i); - // nullptr is checked in olap_cond.cpp eval, note: When we apply column statistic, Field can be NULL when type is Varchar. + // nullptr is checked in olap_cond.cpp eval, note: When we apply column statistic, Field can be nullptr when type is Varchar. WrapperField* first = nullptr; WrapperField* second = nullptr; @@ -320,11 +322,11 @@ OLAPStatus SegmentGroup::add_zone_maps( for (size_t i = 0; i < zone_map_fields.size(); ++i) { const TabletColumn& column = _schema->column(i); WrapperField* first = WrapperField::create(column); - DCHECK(first != NULL) << "failed to allocate memory for field: " << i; + DCHECK(first != nullptr) << "failed to allocate memory for field: " << i; first->copy(zone_map_fields[i].first); WrapperField* second = WrapperField::create(column); - DCHECK(second != NULL) << "failed to allocate memory for field: " << i; + DCHECK(second != nullptr) << "failed to allocate memory for field: " << i; second->copy(zone_map_fields[i].second); _zone_maps.push_back(std::make_pair(first, second)); @@ -339,14 +341,14 @@ OLAPStatus SegmentGroup::add_zone_maps( for (size_t i = 0; i < zone_map_strings.size(); ++i) { const TabletColumn& column = _schema->column(i); WrapperField* first = WrapperField::create(column); - DCHECK(first != NULL) << "failed to allocate memory for field: " << i; + DCHECK(first != nullptr) << "failed to allocate memory for field: " << i; RETURN_NOT_OK(first->from_string(zone_map_strings[i].first)); if (null_vec[i]) { - //[min, max] -> [NULL, max] + //[min, max] -> [nullptr, max] first->set_null(); } WrapperField* second = WrapperField::create(column); - DCHECK(first != NULL) << "failed to allocate memory for field: " << i; + DCHECK(first != nullptr) << "failed to allocate memory for field: " << i; RETURN_NOT_OK(second->from_string(zone_map_strings[i].second)); _zone_maps.push_back(std::make_pair(first, second)); } @@ -577,7 +579,7 @@ OLAPStatus SegmentGroup::add_segment() { // 打开文件 ++_num_segments; - OLAPIndexHeaderMessage* index_header = NULL; + OLAPIndexHeaderMessage* index_header = nullptr; // 构造Proto格式的Header index_header = _file_header.mutable_message(); index_header->set_start_version(_version.first); @@ -589,9 +591,9 @@ OLAPStatus SegmentGroup::add_segment() { index_header->set_null_supported(true); // 分配一段存储short key的内存, 初始化index_row - if (_short_key_buf == NULL) { + if (_short_key_buf == nullptr) { _short_key_buf = new (std::nothrow) char[_short_key_length]; - if (_short_key_buf == NULL) { + if (_short_key_buf == nullptr) { OLAP_LOG_WARNING("malloc short_key_buf error."); return OLAP_ERR_MALLOC_ERROR; } diff --git a/be/src/olap/rowset/segment_reader.cpp b/be/src/olap/rowset/segment_reader.cpp index 8058d67d55..192caaa878 100644 --- a/be/src/olap/rowset/segment_reader.cpp +++ b/be/src/olap/rowset/segment_reader.cpp @@ -53,14 +53,14 @@ SegmentReader::SegmentReader(const std::string file, SegmentGroup* segment_group _block_count(0), _num_rows_in_block(0), _null_supported(false), - _mmap_buffer(NULL), - _include_blocks(NULL), + _mmap_buffer(nullptr), + _include_blocks(nullptr), _is_using_mmap(false), _is_data_loaded(false), _buffer_size(0), _tracker(MemTracker::CreateTracker(-1, "SegmentReader:" + file, parent_tracker, false)), _mem_pool(new MemPool(_tracker.get())), - _shared_buffer(NULL), + _shared_buffer(nullptr), _lru_cache(lru_cache), _runtime_state(runtime_state), _stats(stats) {} @@ -83,10 +83,10 @@ SegmentReader::~SegmentReader() { } } - _lru_cache = NULL; + _lru_cache = nullptr; _file_handler.close(); - if (_is_data_loaded && _runtime_state != NULL) { + if (_is_data_loaded && _runtime_state != nullptr) { MemTracker::update_limits(_buffer_size * -1, _runtime_state->mem_trackers()); } @@ -145,7 +145,7 @@ OLAPStatus SegmentReader::_load_segment_file() { if (_is_using_mmap) { _mmap_buffer = StorageByteBuffer::mmap(&_file_handler, 0, PROT_READ, MAP_PRIVATE); - if (NULL == _mmap_buffer) { + if (nullptr == _mmap_buffer) { OLAP_LOG_WARNING("fail to call mmap, using default mode"); return OLAP_ERR_MALLOC_ERROR; } @@ -157,7 +157,7 @@ OLAPStatus SegmentReader::_load_segment_file() { OLAPStatus SegmentReader::_set_decompressor() { switch (_header_message().compress_kind()) { case COMPRESS_NONE: { - _decompressor = NULL; + _decompressor = nullptr; break; } #ifdef DORIS_WITH_LZO @@ -211,7 +211,7 @@ OLAPStatus SegmentReader::init(bool is_using_cache) { _shared_buffer = StorageByteBuffer::create(_header_message().stream_buffer_size() + sizeof(StreamHead)); - if (_shared_buffer == NULL) { + if (_shared_buffer == nullptr) { OLAP_LOG_WARNING("fail to create shared buffer. [size=%lu]", sizeof(StorageByteBuffer)); return OLAP_ERR_MALLOC_ERROR; } @@ -249,7 +249,7 @@ OLAPStatus SegmentReader::seek_to_block(uint32_t first_block, uint32_t last_bloc return res; } - if (_runtime_state != NULL) { + if (_runtime_state != nullptr) { MemTracker::update_limits(_buffer_size, _runtime_state->mem_trackers()); if (MemTracker::limit_exceeded(_runtime_state->mem_trackers())) { return OLAP_ERR_FETCH_MEMORY_EXCEEDED; @@ -441,9 +441,9 @@ OLAPStatus SegmentReader::_pick_delete_row_groups(uint32_t first_block, uint32_t } OLAPStatus SegmentReader::_init_include_blocks(uint32_t first_block, uint32_t last_block) { - if (NULL == _include_blocks) { + if (nullptr == _include_blocks) { _include_blocks = new (std::nothrow) uint8_t[_block_count]; - if (NULL == _include_blocks) { + if (nullptr == _include_blocks) { OLAP_LOG_WARNING("fail to malloc include block array"); return OLAP_ERR_MALLOC_ERROR; } @@ -471,7 +471,7 @@ OLAPStatus SegmentReader::_pick_row_groups(uint32_t first_block, uint32_t last_b _pick_delete_row_groups(first_block, last_block); - if (NULL == _conditions || _conditions->columns().size() == 0) { + if (nullptr == _conditions || _conditions->columns().size() == 0) { return OLAP_SUCCESS; } @@ -512,8 +512,8 @@ OLAPStatus SegmentReader::_pick_row_groups(uint32_t first_block, uint32_t last_b if (_remain_block < MIN_FILTER_BLOCK_NUM) { VLOG_TRACE << "bloom filter is ignored for too few block remained. " - << "remain_block=" << _remain_block - << ", const_time=" << timer.get_elapse_time_us(); + << "remain_block=" << _remain_block + << ", const_time=" << timer.get_elapse_time_us(); return OLAP_SUCCESS; } @@ -549,7 +549,7 @@ OLAPStatus SegmentReader::_pick_row_groups(uint32_t first_block, uint32_t last_b } VLOG_TRACE << "pick row groups finished. remain_block=" << _remain_block - << ", const_time=" << timer.get_elapse_time_us(); + << ", const_time=" << timer.get_elapse_time_us(); return OLAP_SUCCESS; } @@ -616,14 +616,14 @@ OLAPStatus SegmentReader::_load_index(bool is_using_cache) { ColumnId table_column_id = _unique_id_to_tablet_id_map[unique_column_id]; FieldType type = _get_field_type_by_index(table_column_id); - char* stream_buffer = NULL; + char* stream_buffer = nullptr; char key_buf[OLAP_LRU_CACHE_MAX_KEY_LENGTH]; CacheKey key = _construct_index_stream_key(key_buf, sizeof(key_buf), _file_handler.file_name(), unique_column_id, message.kind()); _cache_handle[cache_handle_index] = _lru_cache->lookup(key); - if (NULL != _cache_handle[cache_handle_index]) { + if (nullptr != _cache_handle[cache_handle_index]) { // 1. If you are in lru, take out the buffer and use it to initialize the index reader is_using_cache = true; stream_buffer = @@ -631,7 +631,7 @@ OLAPStatus SegmentReader::_load_index(bool is_using_cache) { } else { // 2. If it is not in lru, you need to create an index stream. stream_buffer = new (std::nothrow) char[stream_length]; - if (NULL == stream_buffer) { + if (nullptr == stream_buffer) { OLAP_LOG_WARNING( "fail to malloc index stream. " "[column_unique_id = %u, offset = %lu]", @@ -651,7 +651,7 @@ OLAPStatus SegmentReader::_load_index(bool is_using_cache) { // Put the read index into lru. _cache_handle[cache_handle_index] = _lru_cache->insert( key, stream_buffer, stream_length, &_delete_cached_index_stream); - if (NULL == _cache_handle[cache_handle_index]) { + if (nullptr == _cache_handle[cache_handle_index]) { // It may be that malloc in cache insert failed, first return success LOG(FATAL) << "fail to insert lru cache."; } @@ -661,7 +661,7 @@ OLAPStatus SegmentReader::_load_index(bool is_using_cache) { if (message.kind() == StreamInfoMessage::ROW_INDEX) { StreamIndexReader* index_message = new (std::nothrow) StreamIndexReader; - if (index_message == NULL) { + if (index_message == nullptr) { OLAP_LOG_WARNING("fail to malloc memory. [size=%lu]", sizeof(StreamIndexReader)); return OLAP_ERR_MALLOC_ERROR; } @@ -679,7 +679,7 @@ OLAPStatus SegmentReader::_load_index(bool is_using_cache) { _block_count = index_message->entry_count(); } else { BloomFilterIndexReader* bf_message = new (std::nothrow) BloomFilterIndexReader; - if (bf_message == NULL) { + if (bf_message == nullptr) { OLAP_LOG_WARNING("fail to malloc memory. [size=%lu]", sizeof(BloomFilterIndexReader)); return OLAP_ERR_MALLOC_ERROR; @@ -813,8 +813,8 @@ OLAPStatus SegmentReader::_seek_to_block_directly(int64_t block_id, if (OLAP_SUCCESS != (res = _column_readers[cid]->seek(&position))) { if (OLAP_ERR_COLUMN_STREAM_EOF == res) { VLOG_TRACE << "Stream EOF. tablet_id=" << _segment_group->get_tablet_id() - << ", column_id=" << _column_readers[cid]->column_unique_id() - << ", block_id=" << block_id; + << ", column_id=" << _column_readers[cid]->column_unique_id() + << ", block_id=" << block_id; return OLAP_ERR_DATA_EOF; } else { OLAP_LOG_WARNING( @@ -836,7 +836,7 @@ OLAPStatus SegmentReader::_reset_readers() { for (std::map::iterator it = _streams.begin(); it != _streams.end(); ++it) { - if (_runtime_state != NULL) { + if (_runtime_state != nullptr) { MemTracker::update_limits(-1 * it->second->get_buffer_size(), _runtime_state->mem_trackers()); } @@ -850,7 +850,7 @@ OLAPStatus SegmentReader::_reset_readers() { if ((*it) == nullptr) { continue; } - if (_runtime_state != NULL) { + if (_runtime_state != nullptr) { MemTracker::update_limits(-1 * (*it)->get_buffer_size(), _runtime_state->mem_trackers()); } diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp index b00c443d47..7ac50a7b34 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/column_reader.cpp @@ -479,7 +479,7 @@ Status FileColumnIterator::seek_to_ordinal(ordinal_t ord) { Status FileColumnIterator::seek_to_page_start() { if (_page == nullptr) { - return Status::NotSupported("Can not seek to page first when page is NULL"); + return Status::NotSupported("Can not seek to page first when page is nullptr"); } return seek_to_ordinal(_page->first_ordinal); } diff --git a/be/src/olap/rowset/segment_v2/column_writer.cpp b/be/src/olap/rowset/segment_v2/column_writer.cpp index d1afdf48f9..752537556f 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.cpp +++ b/be/src/olap/rowset/segment_v2/column_writer.cpp @@ -55,7 +55,7 @@ public: _rle_encoder.Put(value, run); } - // Returns whether the building nullmap contains NULL + // Returns whether the building nullmap contains nullptr bool has_null() const { return _has_null; } OwnedSlice finish() { diff --git a/be/src/olap/rowset/segment_writer.cpp b/be/src/olap/rowset/segment_writer.cpp index e165e39f91..6aaceb91ec 100644 --- a/be/src/olap/rowset/segment_writer.cpp +++ b/be/src/olap/rowset/segment_writer.cpp @@ -34,7 +34,7 @@ SegmentWriter::SegmentWriter(const std::string& file_name, SegmentGroup* segment _stream_buffer_size(stream_buffer_size), _compress_kind(compress_kind), _bloom_filter_fpp(bloom_filter_fpp), - _stream_factory(NULL), + _stream_factory(nullptr), _row_count(0), _block_count(0) {} @@ -52,7 +52,7 @@ OLAPStatus SegmentWriter::init(uint32_t write_mbytes_per_sec) { // 创建factory _stream_factory = new (std::nothrow) OutStreamFactory(_compress_kind, _stream_buffer_size); - if (NULL == _stream_factory) { + if (nullptr == _stream_factory) { OLAP_LOG_WARNING("fail to allocate out stream factory"); return OLAP_ERR_MALLOC_ERROR; } @@ -63,7 +63,7 @@ OLAPStatus SegmentWriter::init(uint32_t write_mbytes_per_sec) { i, _segment_group->get_tablet_schema(), _stream_factory, _segment_group->get_num_rows_per_row_block(), _bloom_filter_fpp); - if (NULL == writer) { + if (nullptr == writer) { OLAP_LOG_WARNING("fail to create writer"); return OLAP_ERR_MALLOC_ERROR; } else { @@ -191,7 +191,7 @@ OLAPStatus SegmentWriter::_make_file_header(ColumnDataHeaderMessage* file_header } VLOG_TRACE << "stream id=" << it->first.unique_column_id() << ", type=" << it->first.kind() - << ", length=" << stream->get_stream_length(); + << ", length=" << stream->get_stream_length(); } file_header->set_index_length(index_length); @@ -265,7 +265,7 @@ OLAPStatus SegmentWriter::finalize(uint32_t* segment_file_size) { if (!stream->is_suppressed()) { checksum = stream->crc32(checksum); VLOG_TRACE << "stream id=" << it->first.unique_column_id() - << ", type=" << it->first.kind(); + << ", type=" << it->first.kind(); res = stream->write_to_file(&file_handle, _write_mbytes_per_sec); if (OLAP_SUCCESS != res) { OLAP_LOG_WARNING("fail to write stream to file. [res=%d]", res); diff --git a/be/src/olap/short_key_index.h b/be/src/olap/short_key_index.h index 727d7da773..ce3b2f91a3 100644 --- a/be/src/olap/short_key_index.h +++ b/be/src/olap/short_key_index.h @@ -31,7 +31,7 @@ namespace doris { // In our system, we have more complicated situation. -// First, our keys can be NULL. +// First, our keys can be nullptr. // Second, when key columns are not complete we want to distinguish GT and GE. For example, // there are two key columns a and b, we have only one condition a > 1. We can only encode // a prefix key 1, which is less than 1|2. This will make our read more data than diff --git a/be/src/olap/skiplist.h b/be/src/olap/skiplist.h index 1cd5bee71e..f2676de561 100644 --- a/be/src/olap/skiplist.h +++ b/be/src/olap/skiplist.h @@ -145,9 +145,9 @@ private: bool KeyIsAfterNode(const Key& key, Node* n) const; // Return the earliest node that comes at or after key. - // Return NULL if there is no such node. + // Return nullptr if there is no such node. // - // If prev is non-NULL, fills prev[level] with pointer to previous + // If prev is non-nullptr, fills prev[level] with pointer to previous // node at "level" for every level in [0..max_height_-1]. Node* FindGreaterOrEqual(const Key& key, Node** prev) const; @@ -212,12 +212,12 @@ typename SkipList::Node* SkipList::NewNode(con template inline SkipList::Iterator::Iterator(const SkipList* list) { list_ = list; - node_ = NULL; + node_ = nullptr; } template inline bool SkipList::Iterator::Valid() const { - return node_ != NULL; + return node_ != nullptr; } template @@ -239,13 +239,13 @@ inline void SkipList::Iterator::Prev() { DCHECK(Valid()); node_ = list_->FindLessThan(node_->key); if (node_ == list_->head_) { - node_ = NULL; + node_ = nullptr; } } template inline void SkipList::Iterator::Seek(const Key& target) { - node_ = list_->FindGreaterOrEqual(target, NULL); + node_ = list_->FindGreaterOrEqual(target, nullptr); } template @@ -257,7 +257,7 @@ template inline void SkipList::Iterator::SeekToLast() { node_ = list_->FindLast(); if (node_ == list_->head_) { - node_ = NULL; + node_ = nullptr; } } @@ -276,8 +276,8 @@ int SkipList::RandomHeight() { template bool SkipList::KeyIsAfterNode(const Key& key, Node* n) const { - // NULL n is considered infinite - return (n != NULL) && (compare_(n->key, key) < 0); + // nullptr n is considered infinite + return (n != nullptr) && (compare_(n->key, key) < 0); } template @@ -291,7 +291,7 @@ typename SkipList::Node* SkipList::FindGreater // Keep searching in this list x = next; } else { - if (prev != NULL) prev[level] = x; + if (prev != nullptr) prev[level] = x; if (level == 0) { return next; } else { @@ -310,7 +310,7 @@ typename SkipList::Node* SkipList::FindLessTha while (true) { DCHECK(x == head_ || compare_(x->key, key) < 0); Node* next = x->Next(level); - if (next == NULL || compare_(next->key, key) >= 0) { + if (next == nullptr || compare_(next->key, key) >= 0) { if (level == 0) { return x; } else { @@ -329,7 +329,7 @@ typename SkipList::Node* SkipList::FindLast() int level = GetMaxHeight() - 1; while (true) { Node* next = x->Next(level); - if (next == NULL) { + if (next == nullptr) { if (level == 0) { return x; } else { @@ -351,7 +351,7 @@ SkipList::SkipList(Comparator cmp, MemPool* mem_pool, bool can_ max_height_(1), rnd_(0xdeadbeef) { for (int i = 0; i < kMaxHeight; i++) { - head_->SetNext(i, NULL); + head_->SetNext(i, nullptr); } } @@ -382,9 +382,9 @@ void SkipList::Insert(const Key& key, bool* overwritten) { // It is ok to mutate max_height_ without any synchronization // with concurrent readers. A concurrent reader that observes // the new value of max_height_ will see either the old value of - // new level pointers from head_ (NULL), or a new value set in + // new level pointers from head_ (nullptr), or a new value set in // the loop below. In the former case the reader will - // immediately drop to the next level since NULL sorts after all + // immediately drop to the next level since nullptr sorts after all // keys. In the latter case the reader will use the new node. max_height_.store(height, std::memory_order_relaxed); } @@ -423,9 +423,9 @@ void SkipList::InsertWithHint(const Key& key, bool is_exist, Hi // It is ok to mutate max_height_ without any synchronization // with concurrent readers. A concurrent reader that observes // the new value of max_height_ will see either the old value of - // new level pointers from head_ (NULL), or a new value set in + // new level pointers from head_ (nullptr), or a new value set in // the loop below. In the former case the reader will - // immediately drop to the next level since NULL sorts after all + // immediately drop to the next level since nullptr sorts after all // keys. In the latter case the reader will use the new node. max_height_.store(height, std::memory_order_relaxed); } @@ -441,8 +441,8 @@ void SkipList::InsertWithHint(const Key& key, bool is_exist, Hi template bool SkipList::Contains(const Key& key) const { - Node* x = FindGreaterOrEqual(key, NULL); - if (x != NULL && Equal(key, x->key)) { + Node* x = FindGreaterOrEqual(key, nullptr); + if (x != nullptr && Equal(key, x->key)) { return true; } else { return false; @@ -453,7 +453,7 @@ template bool SkipList::Find(const Key& key, Hint* hint) const { Node* x = FindGreaterOrEqual(key, hint->prev); hint->curr = x; - if (x != NULL && Equal(key, x->key)) { + if (x != nullptr && Equal(key, x->key)) { return true; } else { return false; diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp index 9db0fa73d8..0573d1d395 100644 --- a/be/src/olap/snapshot_manager.cpp +++ b/be/src/olap/snapshot_manager.cpp @@ -22,8 +22,6 @@ #include #include -#include -#include #include #include #include @@ -65,14 +63,11 @@ SnapshotManager* SnapshotManager::instance() { return _s_instance; } -OLAPStatus SnapshotManager::make_snapshot( - const TSnapshotRequest& request, - string* snapshot_path, - bool* allow_incremental_clone) { - +OLAPStatus SnapshotManager::make_snapshot(const TSnapshotRequest& request, string* snapshot_path, + bool* allow_incremental_clone) { OLAPStatus res = OLAP_SUCCESS; if (snapshot_path == nullptr) { - LOG(WARNING) << "output parameter cannot be NULL"; + LOG(WARNING) << "output parameter cannot be null"; return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -253,7 +248,7 @@ OLAPStatus SnapshotManager::_calc_snapshot_id_path(const TabletSharedPtr& tablet string* out_path) { OLAPStatus res = OLAP_SUCCESS; if (out_path == nullptr) { - LOG(WARNING) << "output parameter cannot be NULL"; + LOG(WARNING) << "output parameter cannot be null"; return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -312,7 +307,7 @@ OLAPStatus SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_ta << ", snapshot_version is " << snapshot_version; OLAPStatus res = OLAP_SUCCESS; if (snapshot_path == nullptr) { - LOG(WARNING) << "output parameter cannot be NULL"; + LOG(WARNING) << "output parameter cannot be null"; return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -371,10 +366,10 @@ OLAPStatus SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_ta if (rowset != nullptr) { consistent_rowsets.push_back(rowset); } else { - LOG(WARNING) << "failed to find missed version when snapshot. " - << " tablet=" << request.tablet_id - << " schema_hash=" << request.schema_hash - << " version=" << version; + LOG(WARNING) + << "failed to find missed version when snapshot. " + << " tablet=" << request.tablet_id + << " schema_hash=" << request.schema_hash << " version=" << version; res = OLAP_ERR_VERSION_NOT_EXIST; break; } @@ -383,7 +378,7 @@ OLAPStatus SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_ta if (res != OLAP_SUCCESS || !request.__isset.missing_version) { /// not all missing versions are found, fall back to full snapshot. - res = OLAP_SUCCESS; // reset res + res = OLAP_SUCCESS; // reset res consistent_rowsets.clear(); // reset vector ReadLock rdlock(ref_tablet->get_header_lock_ptr()); @@ -410,7 +405,8 @@ OLAPStatus SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_ta // get shortest version path // it very important!!!! // it means 0-version has to be a readable version graph - res = ref_tablet->capture_consistent_rowsets(Version(0, version), &consistent_rowsets); + res = ref_tablet->capture_consistent_rowsets(Version(0, version), + &consistent_rowsets); if (res != OLAP_SUCCESS) { LOG(WARNING) << "fail to select versions to span. res=" << res; break; @@ -433,8 +429,9 @@ OLAPStatus SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_ta } rs_metas.push_back(rs->rowset_meta()); VLOG_NOTICE << "add rowset meta to clone list. " - << " start version " << rs->rowset_meta()->start_version() << " end version " - << rs->rowset_meta()->end_version() << " empty " << rs->rowset_meta()->empty(); + << " start version " << rs->rowset_meta()->start_version() + << " end version " << rs->rowset_meta()->end_version() << " empty " + << rs->rowset_meta()->empty(); } if (res != OLAP_SUCCESS) { LOG(WARNING) << "fail to create hard link. [path=" << snapshot_id_path << "]"; @@ -448,8 +445,8 @@ OLAPStatus SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_ta if (snapshot_version == g_Types_constants.TSNAPSHOT_REQ_VERSION1) { // convert beta rowset to alpha rowset - res = _convert_beta_rowsets_to_alpha( - new_tablet_meta, new_tablet_meta->all_rs_metas(), schema_full_path); + res = _convert_beta_rowsets_to_alpha(new_tablet_meta, new_tablet_meta->all_rs_metas(), + schema_full_path); if (res != OLAP_SUCCESS) { break; } diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index 8e6aab6fff..ba3320f719 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -111,7 +111,7 @@ StorageEngine::StorageEngine(const EngineOptions& options) _available_storage_medium_type_count(0), _effective_cluster_id(-1), _is_all_cluster_id_exist(true), - _index_stream_lru_cache(NULL), + _index_stream_lru_cache(nullptr), _file_cache(nullptr), _compaction_mem_tracker(MemTracker::CreateTracker(-1, "AutoCompaction", nullptr, false, false, MemTrackerLevel::OVERVIEW)), @@ -638,8 +638,8 @@ void StorageEngine::_start_clean_cache() { OLAPStatus StorageEngine::start_trash_sweep(double* usage, bool ignore_guard) { OLAPStatus res = OLAP_SUCCESS; - std::unique_lock l(_trash_sweep_lock,std::defer_lock); - if(!l.try_lock()) { + std::unique_lock l(_trash_sweep_lock, std::defer_lock); + if (!l.try_lock()) { LOG(INFO) << "trash and snapshot sweep is running."; return res; } @@ -895,7 +895,7 @@ OLAPStatus StorageEngine::obtain_shard_path(TStorageMedium::type storage_medium, std::string* shard_path, DataDir** store) { LOG(INFO) << "begin to process obtain root path. storage_medium=" << storage_medium; - if (shard_path == NULL) { + if (shard_path == nullptr) { LOG(WARNING) << "invalid output parameter which is null pointer."; return OLAP_ERR_CE_CMD_PARAMS_ERROR; } diff --git a/be/src/olap/stream_index_common.cpp b/be/src/olap/stream_index_common.cpp index e831904e19..7b4caaa4ea 100644 --- a/be/src/olap/stream_index_common.cpp +++ b/be/src/olap/stream_index_common.cpp @@ -23,7 +23,7 @@ namespace doris { ColumnStatistics::ColumnStatistics() - : _minimum(NULL), _maximum(NULL), _ignored(true), _null_supported(false) {} + : _minimum(nullptr), _maximum(nullptr), _ignored(true), _null_supported(false) {} ColumnStatistics::~ColumnStatistics() { SAFE_DELETE(_minimum); diff --git a/be/src/olap/stream_index_reader.cpp b/be/src/olap/stream_index_reader.cpp index 3b2aad565d..02bf1e4880 100644 --- a/be/src/olap/stream_index_reader.cpp +++ b/be/src/olap/stream_index_reader.cpp @@ -20,11 +20,11 @@ namespace doris { PositionEntryReader::PositionEntryReader() - : _positions(NULL), _positions_count(0), _statistics_offset(0) {} + : _positions(nullptr), _positions_count(0), _statistics_offset(0) {} OLAPStatus PositionEntryReader::init(StreamIndexHeader* header, FieldType type, bool null_supported) { - if (NULL == header) { + if (nullptr == header) { return OLAP_ERR_INIT_FAILED; } @@ -68,7 +68,7 @@ int32_t PositionEntryReader::positions_count() const { } StreamIndexReader::StreamIndexReader() - : _buffer(NULL), + : _buffer(nullptr), _buffer_size(0), _start_offset(0), _step_size(0), @@ -85,7 +85,7 @@ StreamIndexReader::~StreamIndexReader() { OLAPStatus StreamIndexReader::init(char* buffer, size_t buffer_size, FieldType type, bool is_using_cache, bool null_supported) { - if (NULL == buffer) { + if (nullptr == buffer) { OLAP_LOG_WARNING("buffer given is invalid."); return OLAP_ERR_INPUT_PARAMETER_ERROR; } diff --git a/be/src/olap/stream_index_reader.h b/be/src/olap/stream_index_reader.h index 8dfdbd57bc..4354c237fb 100644 --- a/be/src/olap/stream_index_reader.h +++ b/be/src/olap/stream_index_reader.h @@ -54,7 +54,7 @@ private: class PositionProvider { public: - PositionProvider() : _entry(NULL), _index(0) {} + PositionProvider() : _entry(nullptr), _index(0) {} explicit PositionProvider(const PositionEntryReader* entry) : _entry(entry), _index(0) {} ~PositionProvider() {} int64_t get_next() { return _entry->positions(_index++); } diff --git a/be/src/olap/stream_index_writer.cpp b/be/src/olap/stream_index_writer.cpp index 9de3f15834..76f80f2a01 100644 --- a/be/src/olap/stream_index_writer.cpp +++ b/be/src/olap/stream_index_writer.cpp @@ -109,7 +109,7 @@ PositionEntryWriter* StreamIndexWriter::mutable_entry(uint32_t index) { return &_index_to_write[index]; } - return NULL; + return nullptr; } size_t StreamIndexWriter::entry_size() { @@ -136,8 +136,8 @@ size_t StreamIndexWriter::output_size() { } OLAPStatus StreamIndexWriter::write_to_buffer(char* buffer, size_t buffer_size) { - if (NULL == buffer) { - OLAP_LOG_WARNING("given buffer is NULL"); + if (nullptr == buffer) { + OLAP_LOG_WARNING("given buffer is null"); return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -162,7 +162,7 @@ OLAPStatus StreamIndexWriter::write_to_buffer(char* buffer, size_t buffer_size) _header.block_count = _index_to_write.size(); VLOG_TRACE << "header info. pos: " << _header.position_format - << ", stat:" << _header.statistic_format << ", entry_size:" << entry_size; + << ", stat:" << _header.statistic_format << ", entry_size:" << entry_size; memcpy(buffer, reinterpret_cast(&_header), sizeof(_header)); // set offset, write data char* write_offset = buffer + sizeof(_header); diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index 4ede471c81..71794edb11 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -60,7 +60,7 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id tablet_meta_pb.set_tablet_id(tablet_id); tablet_meta_pb.set_schema_hash(schema_hash); tablet_meta_pb.set_shard_id(shard_id); - tablet_meta_pb.set_creation_time(time(NULL)); + tablet_meta_pb.set_creation_time(time(nullptr)); tablet_meta_pb.set_cumulative_layer_point(-1); tablet_meta_pb.set_tablet_state(PB_RUNNING); *(tablet_meta_pb.mutable_tablet_uid()) = tablet_uid.to_proto(); @@ -155,8 +155,7 @@ TabletMeta::TabletMeta(const TabletMeta& b) _stale_rs_metas(b._stale_rs_metas), _del_pred_array(b._del_pred_array), _in_restore_mode(b._in_restore_mode), - _preferred_rowset_type(b._preferred_rowset_type) { -} + _preferred_rowset_type(b._preferred_rowset_type) {} void TabletMeta::_init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, ColumnPB* column) { diff --git a/be/src/olap/task/engine_batch_load_task.cpp b/be/src/olap/task/engine_batch_load_task.cpp index 2b6a95acdb..a275015541 100644 --- a/be/src/olap/task/engine_batch_load_task.cpp +++ b/be/src/olap/task/engine_batch_load_task.cpp @@ -197,7 +197,7 @@ AgentStatus EngineBatchLoadTask::_process() { bool is_timeout = false; auto download_cb = [this, estimate_time_out, file_size, &is_timeout](HttpClient* client) { // Check timeout and set timeout - time_t now = time(NULL); + time_t now = time(nullptr); if (_push_req.timeout > 0 && _push_req.timeout < now) { // return status to break this callback VLOG_NOTICE << "check time out. time_out:" << _push_req.timeout << ", now:" << now; @@ -258,9 +258,9 @@ AgentStatus EngineBatchLoadTask::_process() { if (status == DORIS_SUCCESS) { // Load delta file - time_t push_begin = time(NULL); + time_t push_begin = time(nullptr); OLAPStatus push_status = _push(_push_req, _tablet_infos); - time_t push_finish = time(NULL); + time_t push_finish = time(nullptr); LOG(INFO) << "Push finish, cost time: " << (push_finish - push_begin); if (push_status == OLAPStatus::OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST) { status = DORIS_PUSH_HAD_LOADED; diff --git a/be/src/olap/task/engine_checksum_task.cpp b/be/src/olap/task/engine_checksum_task.cpp index 9ccd7ae410..a1b9a7371d 100644 --- a/be/src/olap/task/engine_checksum_task.cpp +++ b/be/src/olap/task/engine_checksum_task.cpp @@ -42,14 +42,14 @@ OLAPStatus EngineChecksumTask::_compute_checksum() { << ", version=" << _version; OLAPStatus res = OLAP_SUCCESS; - if (_checksum == NULL) { + if (_checksum == nullptr) { OLAP_LOG_WARNING("invalid output parameter which is null pointer."); return OLAP_ERR_CE_CMD_PARAMS_ERROR; } TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(_tablet_id, _schema_hash); - if (NULL == tablet.get()) { + if (nullptr == tablet.get()) { OLAP_LOG_WARNING("can't find tablet. [tablet_id=%ld schema_hash=%d]", _tablet_id, _schema_hash); return OLAP_ERR_TABLE_NOT_FOUND; @@ -64,7 +64,7 @@ OLAPStatus EngineChecksumTask::_compute_checksum() { { ReadLock rdlock(tablet->get_header_lock_ptr()); const RowsetSharedPtr message = tablet->rowset_with_max_version(); - if (message == NULL) { + if (message == nullptr) { LOG(FATAL) << "fail to get latest version. tablet_id=" << _tablet_id; return OLAP_ERR_VERSION_NOT_EXIST; } diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp index edb421d876..5af17e7581 100644 --- a/be/src/olap/task/engine_clone_task.cpp +++ b/be/src/olap/task/engine_clone_task.cpp @@ -92,7 +92,8 @@ OLAPStatus EngineCloneTask::_do_clone() { // if missed version size is 0, then it is useless to clone from remote be, it means local data is // completed. Or remote be will just return header not the rowset files. clone will failed. if (missed_versions.size() == 0) { - LOG(INFO) << "missed version size = 0, skip clone and return success. tablet id=" << _clone_req.tablet_id; + LOG(INFO) << "missed version size = 0, skip clone and return success. tablet id=" + << _clone_req.tablet_id; _set_tablet_info(DORIS_SUCCESS, is_new_tablet); return OLAP_SUCCESS; } @@ -100,8 +101,9 @@ OLAPStatus EngineCloneTask::_do_clone() { // try to download missing version from src backend. // if tablet on src backend does not contains missing version, it will download all versions, // and set allow_incremental_clone to false - status = _make_and_download_snapshots(*(tablet->data_dir()), local_data_path, &src_host, &src_file_path, - _error_msgs, &missed_versions, &allow_incremental_clone); + status = _make_and_download_snapshots(*(tablet->data_dir()), local_data_path, &src_host, + &src_file_path, _error_msgs, &missed_versions, + &allow_incremental_clone); LOG(INFO) << "tablet exist with number of missing version: " << missed_versions.size() << ", try to incremental clone succeed: " << allow_incremental_clone @@ -144,8 +146,9 @@ OLAPStatus EngineCloneTask::_do_clone() { if (status == DORIS_SUCCESS) { bool allow_incremental_clone = false; - status = _make_and_download_snapshots(*store, tablet_dir_stream.str(), &src_host, &src_file_path, - _error_msgs, nullptr, &allow_incremental_clone); + status = _make_and_download_snapshots(*store, tablet_dir_stream.str(), &src_host, + &src_file_path, _error_msgs, nullptr, + &allow_incremental_clone); } if (status == DORIS_SUCCESS) { @@ -193,7 +196,7 @@ OLAPStatus EngineCloneTask::_do_clone() { if (std::filesystem::exists(local_path)) { std::filesystem::remove_all(local_path); } - } catch (std::filesystem::filesystem_error &e) { + } catch (std::filesystem::filesystem_error& e) { // Ignore the error, OLAP will delete it LOG(WARNING) << "clone delete useless dir failed. " << " error: " << e.what() << " local dir: " << local_data_path.c_str() @@ -260,11 +263,10 @@ void EngineCloneTask::_set_tablet_info(AgentStatus status, bool is_new_tablet) { /// 2. Download all snapshots to CLONE dir. /// 3. Convert rowset ids of downloaded snapshots. /// 4. Release the snapshots on source BE. -AgentStatus EngineCloneTask::_make_and_download_snapshots(DataDir& data_dir, const string& local_data_path, - TBackend* src_host, string* snapshot_path, - std::vector* error_msgs, - const std::vector* missed_versions, - bool* allow_incremental_clone) { +AgentStatus EngineCloneTask::_make_and_download_snapshots( + DataDir& data_dir, const string& local_data_path, TBackend* src_host, string* snapshot_path, + std::vector* error_msgs, const std::vector* missed_versions, + bool* allow_incremental_clone) { AgentStatus status = DORIS_SUCCESS; std::string local_path = local_data_path + "/"; @@ -551,7 +553,7 @@ OLAPStatus EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_di // check all files in /clone and /tablet set clone_files; - Status ret = FileUtils::list_dirs_files(clone_dir, NULL, &clone_files, Env::Default()); + Status ret = FileUtils::list_dirs_files(clone_dir, nullptr, &clone_files, Env::Default()); if (!ret.ok()) { LOG(WARNING) << "failed to list clone dir when clone. [clone_dir=" << clone_dir << "]" << " error: " << ret.to_string(); @@ -561,9 +563,10 @@ OLAPStatus EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_di set local_files; string tablet_dir = tablet->tablet_path(); - ret = FileUtils::list_dirs_files(tablet_dir, NULL, &local_files, Env::Default()); + ret = FileUtils::list_dirs_files(tablet_dir, nullptr, &local_files, Env::Default()); if (!ret.ok()) { - LOG(WARNING) << "failed to list local tablet dir when clone. [tablet_dir=" << tablet_dir << "]" + LOG(WARNING) << "failed to list local tablet dir when clone. [tablet_dir=" << tablet_dir + << "]" << " error: " << ret.to_string(); res = OLAP_ERR_DISK_FAILURE; break; @@ -575,7 +578,7 @@ OLAPStatus EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_di for (const string& clone_file : clone_files) { if (local_files.find(clone_file) != local_files.end()) { VLOG_NOTICE << "find same file when clone, skip it. " - << "tablet=" << tablet->full_name() << ", clone_file=" << clone_file; + << "tablet=" << tablet->full_name() << ", clone_file=" << clone_file; continue; } @@ -630,8 +633,8 @@ OLAPStatus EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_di /// 1. Get missing version from local tablet again and check if they exist in cloned tablet. /// 2. Revise the local tablet meta to add all incremental cloned rowset's meta. OLAPStatus EngineCloneTask::_finish_incremental_clone(Tablet* tablet, - const TabletMeta& cloned_tablet_meta, - int64_t committed_version) { + const TabletMeta& cloned_tablet_meta, + int64_t committed_version) { LOG(INFO) << "begin to finish incremental clone. tablet=" << tablet->full_name() << ", clone version=" << committed_version; @@ -640,14 +643,13 @@ OLAPStatus EngineCloneTask::_finish_incremental_clone(Tablet* tablet, std::vector missed_versions; tablet->calc_missed_versions_unlocked(committed_version, &missed_versions); VLOG_NOTICE << "get missed versions again when finish incremental clone. " - << "tablet=" << tablet->full_name() << ", clone version=" << committed_version - << ", missed_versions_size=" << missed_versions.size(); + << "tablet=" << tablet->full_name() << ", clone version=" << committed_version + << ", missed_versions_size=" << missed_versions.size(); // check missing versions exist in clone src std::vector rowsets_to_clone; for (Version version : missed_versions) { - RowsetMetaSharedPtr rs_meta = - cloned_tablet_meta.acquire_rs_meta_by_version(version); + RowsetMetaSharedPtr rs_meta = cloned_tablet_meta.acquire_rs_meta_by_version(version); if (rs_meta == nullptr) { LOG(WARNING) << "missed version is not found in cloned tablet meta." << ", missed_version=" << version.first << "-" << version.second; @@ -705,8 +707,7 @@ OLAPStatus EngineCloneTask::_finish_full_clone(Tablet* tablet, TabletMeta* clone if (local_version.first <= cloned_max_version.second && local_version.second > cloned_max_version.second) { LOG(WARNING) << "stop to full clone, version cross src latest." - << "tablet=" << tablet->full_name() - << ", local_version=" << local_version; + << "tablet=" << tablet->full_name() << ", local_version=" << local_version; return OLAP_ERR_TABLE_VERSION_DUPLICATE_ERROR; } else if (local_version.second <= cloned_max_version.second) { @@ -726,11 +727,13 @@ OLAPStatus EngineCloneTask::_finish_full_clone(Tablet* tablet, TabletMeta* clone if (existed_in_src) { cloned_tablet_meta->delete_rs_meta_by_version(local_version, &rs_metas_found_in_src); - LOG(INFO) << "version exist in local tablet, no need to clone. delete it from clone tablet" + LOG(INFO) << "version exist in local tablet, no need to clone. delete it from " + "clone tablet" << ". tablet=" << tablet->full_name() << ", version='" << local_version; } else { versions_to_delete.push_back(local_version); - LOG(INFO) << "version not exist in local tablet. it will be replaced by other version. " + LOG(INFO) << "version not exist in local tablet. it will be replaced by other " + "version. " << "delete it from local tablet. " << "tablet=" << tablet->full_name() << "," << ", version=" << local_version; diff --git a/be/src/olap/types.h b/be/src/olap/types.h index d1ededb4bc..4dddf08d88 100644 --- a/be/src/olap/types.h +++ b/be/src/olap/types.h @@ -534,7 +534,7 @@ struct BaseFieldtypeTraits : public CppTypeTraits { static OLAPStatus from_string(void* buf, const std::string& scan_key) { CppType value = 0; if (scan_key.length() > 0) { - value = static_cast(strtol(scan_key.c_str(), NULL, 10)); + value = static_cast(strtol(scan_key.c_str(), nullptr, 10)); } *reinterpret_cast(buf) = value; return OLAP_SUCCESS; @@ -634,7 +634,7 @@ struct FieldTypeTraits int128_t value = 0; const char* value_string = scan_key.c_str(); - char* end = NULL; + char* end = nullptr; value = strtol(value_string, &end, 10); if (*end != 0) { value = 0; @@ -828,7 +828,7 @@ struct FieldTypeTraits : public BaseFieldtypeTraits(buf) = value; @@ -909,7 +909,7 @@ struct FieldTypeTraits tm time_tm; char* res = strptime(scan_key.c_str(), "%Y-%m-%d %H:%M:%S", &time_tm); - if (NULL != res) { + if (nullptr != res) { CppType value = ((time_tm.tm_year + 1900) * 10000L + (time_tm.tm_mon + 1) * 100L + time_tm.tm_mday) * 1000000L + diff --git a/be/src/olap/utils.cpp b/be/src/olap/utils.cpp index 70f9ffa5a5..8c8ab694ef 100644 --- a/be/src/olap/utils.cpp +++ b/be/src/olap/utils.cpp @@ -61,9 +61,10 @@ namespace doris { OLAPStatus olap_compress(const char* src_buf, size_t src_len, char* dest_buf, size_t dest_len, size_t* written_len, OLAPCompressionType compression_type) { - if (NULL == src_buf || NULL == dest_buf || NULL == written_len) { - OLAP_LOG_WARNING("input param with NULL pointer. [src_buf=%p dest_buf=%p written_len=%p]", - src_buf, dest_buf, written_len); + if (nullptr == src_buf || nullptr == dest_buf || nullptr == written_len) { + OLAP_LOG_WARNING( + "input param with nullptr pointer. [src_buf=%p dest_buf=%p written_len=%p]", + src_buf, dest_buf, written_len); return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -135,9 +136,10 @@ OLAPStatus olap_compress(const char* src_buf, size_t src_len, char* dest_buf, si OLAPStatus olap_decompress(const char* src_buf, size_t src_len, char* dest_buf, size_t dest_len, size_t* written_len, OLAPCompressionType compression_type) { - if (NULL == src_buf || NULL == dest_buf || NULL == written_len) { - OLAP_LOG_WARNING("input param with NULL pointer. [src_buf=%p dest_buf=%p written_len=%p]", - src_buf, dest_buf, written_len); + if (nullptr == src_buf || nullptr == dest_buf || nullptr == written_len) { + OLAP_LOG_WARNING( + "input param with nullptr pointer. [src_buf=%p dest_buf=%p written_len=%p]", + src_buf, dest_buf, written_len); return OLAP_ERR_INPUT_PARAMETER_ERROR; } @@ -148,7 +150,7 @@ OLAPStatus olap_decompress(const char* src_buf, size_t src_len, char* dest_buf, case OLAP_COMP_TRANSPORT: { int lzo_res = lzo1x_decompress_safe(reinterpret_cast(src_buf), src_len, reinterpret_cast(dest_buf), written_len, - NULL); + nullptr); if (LZO_E_OK != lzo_res) { OLAP_LOG_WARNING( "decompress failed." @@ -167,7 +169,7 @@ OLAPStatus olap_decompress(const char* src_buf, size_t src_len, char* dest_buf, case OLAP_COMP_STORAGE: { int lzo_res = lzo1c_decompress_safe(reinterpret_cast(src_buf), src_len, reinterpret_cast(dest_buf), written_len, - NULL); + nullptr); if (LZO_E_OK != lzo_res) { OLAP_LOG_WARNING( "decompress failed." @@ -564,7 +566,7 @@ uint32_t olap_crc32(uint32_t crc32, const char* buf, size_t len) { } OLAPStatus gen_timestamp_string(string* out_string) { - time_t now = time(NULL); + time_t now = time(nullptr); tm local_tm; if (localtime_r(&now, &local_tm) == nullptr) { @@ -752,7 +754,7 @@ OLAPStatus read_write_test_file(const string& test_file_path) { } std::unique_ptr read_buff(read_test_buff, &std::free); // generate random numbers - uint32_t rand_seed = static_cast(time(NULL)); + uint32_t rand_seed = static_cast(time(nullptr)); for (size_t i = 0; i < TEST_FILE_BUF_SIZE; ++i) { int32_t tmp_value = rand_r(&rand_seed); write_test_buff[i] = static_cast(tmp_value); @@ -878,7 +880,7 @@ int Errno::no() { template <> bool valid_signed_number(const std::string& value_str) { - char* endptr = NULL; + char* endptr = nullptr; const char* value_string = value_str.c_str(); int64_t value = strtol(value_string, &endptr, 10); if (*endptr != 0) { @@ -957,32 +959,32 @@ bool valid_datetime(const string& value_str) { return false; } - int month = strtol(what[2].str().c_str(), NULL, 10); + int month = strtol(what[2].str().c_str(), nullptr, 10); if (month < 1 || month > 12) { OLAP_LOG_WARNING("invalid month. [month=%d]", month); return false; } - int day = strtol(what[3].str().c_str(), NULL, 10); + int day = strtol(what[3].str().c_str(), nullptr, 10); if (day < 1 || day > 31) { OLAP_LOG_WARNING("invalid day. [day=%d]", day); return false; } if (what[4].length()) { - int hour = strtol(what[5].str().c_str(), NULL, 10); + int hour = strtol(what[5].str().c_str(), nullptr, 10); if (hour < 0 || hour > 23) { OLAP_LOG_WARNING("invalid hour. [hour=%d]", hour); return false; } - int minute = strtol(what[6].str().c_str(), NULL, 10); + int minute = strtol(what[6].str().c_str(), nullptr, 10); if (minute < 0 || minute > 59) { OLAP_LOG_WARNING("invalid minute. [minute=%d]", minute); return false; } - int second = strtol(what[7].str().c_str(), NULL, 10); + int second = strtol(what[7].str().c_str(), nullptr, 10); if (second < 0 || second > 59) { OLAP_LOG_WARNING("invalid second. [second=%d]", second); return false; diff --git a/be/src/olap/utils.h b/be/src/olap/utils.h index 4f7d6bd631..a90f9c0c29 100644 --- a/be/src/olap/utils.h +++ b/be/src/olap/utils.h @@ -23,10 +23,10 @@ #include #include -#include #include #include #include +#include #include #include #include @@ -207,7 +207,7 @@ OLAPStatus read_write_test_file(const std::string& test_file_path); //转换两个list template void static_cast_assign_vector(std::vector* v1, const std::vector& v2) { - if (NULL != v1) { + if (nullptr != v1) { //GCC3.4的模板展开貌似有问题, 这里如果使用迭代器会编译失败 for (size_t i = 0; i < v2.size(); i++) { v1->push_back(static_cast(v2[i])); @@ -239,7 +239,7 @@ inline bool is_io_error(OLAPStatus status) { // 检查int8_t, int16_t, int32_t, int64_t的值是否溢出 template bool valid_signed_number(const std::string& value_str) { - char* endptr = NULL; + char* endptr = nullptr; errno = 0; int64_t value = strtol(value_str.c_str(), &endptr, 10); @@ -265,7 +265,7 @@ bool valid_unsigned_number(const std::string& value_str) { return false; } - char* endptr = NULL; + char* endptr = nullptr; errno = 0; uint64_t value = strtoul(value_str.c_str(), &endptr, 10); diff --git a/be/src/plugin/plugin.h b/be/src/plugin/plugin.h index 521de30012..9a31f46aef 100644 --- a/be/src/plugin/plugin.h +++ b/be/src/plugin/plugin.h @@ -39,8 +39,8 @@ namespace doris { * init_method, * close_method, * PLUGIN_NOT_DYNAMIC_INSTALL | PLUGIN_NOT_DYNAMIC_UNINSTALL, - * NULL, - * NULL + * nullptr, + * nullptr * } declare_plugin_end * */ diff --git a/be/src/runtime/buffered_block_mgr2.cc b/be/src/runtime/buffered_block_mgr2.cc index 48d4a76895..64317c053b 100644 --- a/be/src/runtime/buffered_block_mgr2.cc +++ b/be/src/runtime/buffered_block_mgr2.cc @@ -43,8 +43,7 @@ using std::bind; using std::mem_fn; using std::lock_guard; using std::mutex; -using boost::scoped_array; -using boost::shared_ptr; +using std::shared_ptr; using std::unique_lock; namespace doris { @@ -98,7 +97,7 @@ public: int _num_pinned_buffers; void pin_buffer(BufferDescriptor* buffer) { - DCHECK(buffer != NULL); + DCHECK(buffer != nullptr); if (buffer->len == _mgr->max_block_size()) { ++_num_pinned_buffers; _tracker->ConsumeLocal(buffer->len, _query_tracker.get()); @@ -107,7 +106,7 @@ public: } void unpin_buffer(BufferDescriptor* buffer) { - DCHECK(buffer != NULL); + DCHECK(buffer != nullptr); if (buffer->len == _mgr->max_block_size()) { DCHECK_GT(_num_pinned_buffers, 0); --_num_pinned_buffers; @@ -128,11 +127,11 @@ public: // BufferedBlockMgr2::Block methods. BufferedBlockMgr2::Block::Block(BufferedBlockMgr2* block_mgr) - : _buffer_desc(NULL), + : _buffer_desc(nullptr), _block_mgr(block_mgr), - _client(NULL), - _write_range(NULL), - _tmp_file(NULL), + _client(nullptr), + _write_range(nullptr), + _tmp_file(nullptr), _valid_data_len(0), _num_rows(0) {} @@ -154,27 +153,27 @@ void BufferedBlockMgr2::Block::init() { _in_write = false; _is_deleted = false; _valid_data_len = 0; - _client = NULL; + _client = nullptr; _num_rows = 0; } bool BufferedBlockMgr2::Block::validate() const { - if (_is_deleted && (_is_pinned || (!_in_write && _buffer_desc != NULL))) { + if (_is_deleted && (_is_pinned || (!_in_write && _buffer_desc != nullptr))) { LOG(ERROR) << "Deleted block in use - " << debug_string(); return false; } - if (_buffer_desc == NULL && (_is_pinned || _in_write)) { + if (_buffer_desc == nullptr && (_is_pinned || _in_write)) { LOG(ERROR) << "Block without buffer in use - " << debug_string(); return false; } - if (_buffer_desc == NULL && _block_mgr->_unpinned_blocks.contains(this)) { + if (_buffer_desc == nullptr && _block_mgr->_unpinned_blocks.contains(this)) { LOG(ERROR) << "Unpersisted block without buffer - " << debug_string(); return false; } - if (_buffer_desc != NULL && (_buffer_desc->block != this)) { + if (_buffer_desc != nullptr && (_buffer_desc->block != this)) { LOG(ERROR) << "Block buffer inconsistency - " << debug_string(); return false; } @@ -183,7 +182,7 @@ bool BufferedBlockMgr2::Block::validate() const { } string BufferedBlockMgr2::Block::tmp_file_path() const { - if (_tmp_file == NULL) { + if (_tmp_file == nullptr) { return ""; } return _tmp_file->path(); @@ -225,7 +224,7 @@ BufferedBlockMgr2::BufferedBlockMgr2(RuntimeState* state, TmpFileMgr* tmp_file_m Status BufferedBlockMgr2::create(RuntimeState* state, const std::shared_ptr& parent, RuntimeProfile* profile, TmpFileMgr* tmp_file_mgr, int64_t mem_limit, int64_t block_size, - boost::shared_ptr* block_mgr) { + std::shared_ptr* block_mgr) { DCHECK(parent != nullptr); block_mgr->reset(); { @@ -239,8 +238,8 @@ Status BufferedBlockMgr2::create(RuntimeState* state, const std::shared_ptrsecond.lock(); // } - if (*block_mgr == NULL) { - // weak_ptr::lock returns NULL if the weak_ptr is expired. This means + if (*block_mgr == nullptr) { + // weak_ptr::lock returns nullptr if the weak_ptr is expired. This means // all shared_ptr references have gone to 0 and it is in the process of // being deleted. This can happen if the last shared reference is released // but before the weak ptr is removed from the map. @@ -358,15 +357,15 @@ bool BufferedBlockMgr2::consume_memory(Client* client, int64_t size) { // memory. int buffers_acquired = 0; do { - BufferDescriptor* buffer_desc = NULL; + BufferDescriptor* buffer_desc = nullptr; Status s = find_buffer(lock, &buffer_desc); // This waits on the lock. - if (buffer_desc == NULL) { + if (buffer_desc == nullptr) { break; } DCHECK(s.ok()); _all_io_buffers.erase(buffer_desc->all_buffers_it); - if (buffer_desc->block != NULL) { - buffer_desc->block->_buffer_desc = NULL; + if (buffer_desc->block != nullptr) { + buffer_desc->block->_buffer_desc = nullptr; } delete[] buffer_desc->buffer; ++buffers_acquired; @@ -452,8 +451,8 @@ Status BufferedBlockMgr2::get_new_block(Client* client, Block* unpin_block, Bloc int64_t len) { DCHECK_LE(len, _max_block_size) << "Cannot request block bigger than max_len"; DCHECK_NE(len, 0) << "Cannot request block of zero size"; - *block = NULL; - Block* new_block = NULL; + *block = nullptr; + Block* new_block = nullptr; { lock_guard lock(_lock); @@ -465,7 +464,7 @@ Status BufferedBlockMgr2::get_new_block(Client* client, Block* unpin_block, Bloc DCHECK_EQ(new_block->_client, client); if (len > 0 && len < _max_block_size) { - DCHECK(unpin_block == NULL); + DCHECK(unpin_block == nullptr); if (client->_tracker->TryConsume(len)) { // TODO: Have a cache of unused blocks of size 'len' (0, _max_block_size) uint8_t* buffer = new uint8_t[len]; @@ -487,38 +486,39 @@ Status BufferedBlockMgr2::get_new_block(Client* client, Block* unpin_block, Bloc bool in_mem = true; RETURN_IF_ERROR(find_buffer_for_block(new_block, &in_mem)); DCHECK(!in_mem) << "A new block cannot start in mem."; - DCHECK(!new_block->is_pinned() || new_block->_buffer_desc != NULL) << new_block->debug_string(); + DCHECK(!new_block->is_pinned() || new_block->_buffer_desc != nullptr) + << new_block->debug_string(); if (!new_block->is_pinned()) { - if (unpin_block == NULL) { + if (unpin_block == nullptr) { // We couldn't get a new block and no unpin block was provided. Can't return // a block. new_block->_is_deleted = true; return_unused_block(new_block); - new_block = NULL; + new_block = nullptr; } else { // We need to transfer the buffer from unpin_block to new_block. RETURN_IF_ERROR(transfer_buffer(new_block, unpin_block, true)); } - } else if (unpin_block != NULL) { + } else if (unpin_block != nullptr) { // Got a new block without needing to transfer. Just unpin this block. RETURN_IF_ERROR(unpin_block->unpin()); } - DCHECK(new_block == NULL || new_block->is_pinned()); + DCHECK(new_block == nullptr || new_block->is_pinned()); *block = new_block; return Status::OK(); } Status BufferedBlockMgr2::transfer_buffer(Block* dst, Block* src, bool unpin) { Status status = Status::OK(); - DCHECK(dst != NULL); - DCHECK(src != NULL); + DCHECK(dst != nullptr); + DCHECK(src != nullptr); // First write out the src block. DCHECK(src->_is_pinned); DCHECK(!dst->_is_pinned); - DCHECK(dst->_buffer_desc == NULL); + DCHECK(dst->_buffer_desc == nullptr); DCHECK_EQ(src->_buffer_desc->len, _max_block_size); src->_is_pinned = false; @@ -545,7 +545,7 @@ Status BufferedBlockMgr2::transfer_buffer(Block* dst, Block* src, bool unpin) { // Assign the buffer to the new block. dst->_buffer_desc = src->_buffer_desc; dst->_buffer_desc->block = dst; - src->_buffer_desc = NULL; + src->_buffer_desc = nullptr; dst->_is_pinned = true; if (!unpin) { src->_is_deleted = true; @@ -567,8 +567,8 @@ BufferedBlockMgr2::~BufferedBlockMgr2() { // distinguish between the two expired pointers), and when the other // ~BufferedBlockMgr2() call occurs, it won't find an entry for this _query_id. if (it != _s_query_to_block_mgrs.end()) { - shared_ptr mgr = it->second.lock(); - if (mgr.get() == NULL) { + std::shared_ptr mgr = it->second.lock(); + if (mgr.get() == nullptr) { // The BufferBlockMgr object referenced by this entry is being deconstructed. _s_query_to_block_mgrs.erase(it); } else { @@ -578,7 +578,7 @@ BufferedBlockMgr2::~BufferedBlockMgr2() { } } - if (_io_request_context != NULL) { + if (_io_request_context != nullptr) { _io_mgr->unregister_context(_io_request_context); } @@ -587,8 +587,8 @@ BufferedBlockMgr2::~BufferedBlockMgr2() { // See IMPALA-1890. DCHECK_EQ(_non_local_outstanding_writes, 0) << endl << debug_internal(); // Delete tmp files. - for (TmpFileMgr::File& file : _tmp_files) { - file.remove(); + for (auto& file : _tmp_files) { + file->remove(); } _tmp_files.clear(); @@ -620,7 +620,7 @@ std::shared_ptr BufferedBlockMgr2::get_tracker(Client* client) const // TODO: It would be good if we had a sync primitive that supports is_mine() calls, see // IMPALA-1884. Status BufferedBlockMgr2::delete_or_unpin_block(Block* block, bool unpin) { - if (block == NULL) { + if (block == nullptr) { return is_cancelled() ? Status::Cancelled("Cancelled") : Status::OK(); } if (unpin) { @@ -632,7 +632,7 @@ Status BufferedBlockMgr2::delete_or_unpin_block(Block* block, bool unpin) { } Status BufferedBlockMgr2::pin_block(Block* block, bool* pinned, Block* release_block, bool unpin) { - DCHECK(block != NULL); + DCHECK(block != nullptr); DCHECK(!block->_is_deleted); *pinned = false; if (block->_is_pinned) { @@ -650,11 +650,11 @@ Status BufferedBlockMgr2::pin_block(Block* block, bool* pinned, Block* release_b } if (!block->_is_pinned) { - if (release_block == NULL) { + if (release_block == nullptr) { return Status::OK(); } - if (block->_buffer_desc != NULL) { + if (block->_buffer_desc != nullptr) { { lock_guard lock(_lock); if (_free_io_buffers.contains(block->_buffer_desc)) { @@ -679,17 +679,17 @@ Status BufferedBlockMgr2::pin_block(Block* block, bool* pinned, Block* release_b RETURN_IF_ERROR(transfer_buffer(block, release_block, unpin)); DCHECK(!release_block->_is_pinned); - release_block = NULL; // Handled by transfer. + release_block = nullptr; // Handled by transfer. DCHECK(block->_is_pinned); *pinned = true; } // Read the block from disk if it was not in memory. - DCHECK(block->_write_range != NULL) << block->debug_string() << endl << release_block; + DCHECK(block->_write_range != nullptr) << block->debug_string() << endl << release_block; SCOPED_TIMER(_disk_read_timer); // Create a ScanRange to perform the read. DiskIoMgr::ScanRange* scan_range = _obj_pool.add(new DiskIoMgr::ScanRange()); - scan_range->reset(NULL, block->_write_range->file(), block->_write_range->len(), + scan_range->reset(nullptr, block->_write_range->file(), block->_write_range->len(), block->_write_range->offset(), block->_write_range->disk_id(), false, block, DiskIoMgr::ScanRange::NEVER_CACHE); vector ranges(1, scan_range); @@ -766,7 +766,7 @@ Status BufferedBlockMgr2::write_unpinned_block(Block* block) { DCHECK(!block->_in_write) << block->debug_string(); DCHECK_EQ(block->_buffer_desc->len, _max_block_size); - if (block->_write_range == NULL) { + if (block->_write_range == nullptr) { if (_tmp_files.empty()) { RETURN_IF_ERROR(init_tmp_files()); } @@ -789,7 +789,7 @@ Status BufferedBlockMgr2::write_unpinned_block(Block* block) { block->_tmp_file = tmp_file; } - uint8_t* outbuf = NULL; + uint8_t* outbuf = nullptr; outbuf = block->buffer(); block->_write_range->set_data(outbuf, block->_valid_data_len); @@ -812,7 +812,7 @@ Status BufferedBlockMgr2::allocate_scratch_space(int64_t block_size, TmpFileMgr: vector errs; // Find the next physical file in round-robin order and create a write range for it. for (int attempt = 0; attempt < _tmp_files.size(); ++attempt) { - *tmp_file = &_tmp_files[_next_block_index]; + *tmp_file = _tmp_files[_next_block_index].get(); _next_block_index = (_next_block_index + 1) % _tmp_files.size(); if ((*tmp_file)->is_blacklisted()) { continue; @@ -874,8 +874,8 @@ void BufferedBlockMgr2::write_complete(Block* block, const Status& write_status) _free_io_buffers.enqueue(block->_buffer_desc); // Finish the delete_block() work. if (block->_is_deleted) { - block->_buffer_desc->block = NULL; - block->_buffer_desc = NULL; + block->_buffer_desc->block = nullptr; + block->_buffer_desc = nullptr; return_unused_block(block); } // Multiple threads may be waiting for the same block in find_buffer(). Wake them @@ -896,7 +896,7 @@ void BufferedBlockMgr2::write_complete(Block* block, const Status& write_status) if (!write_status.is_cancelled() && !state->is_cancelled()) { if (!write_status.ok()) { // Report but do not attempt to recover from write error. - DCHECK(block->_tmp_file != NULL); + DCHECK(block->_tmp_file != nullptr); block->_tmp_file->report_io_error(write_status.get_error_msg()); VLOG_QUERY << "Query: " << _query_id << " write complete callback with error."; state->log_error(write_status.get_error_msg()); @@ -937,27 +937,27 @@ void BufferedBlockMgr2::delete_block(Block* block) { } if (block->_in_write) { - DCHECK(block->_buffer_desc != NULL && block->_buffer_desc->len == _max_block_size) + DCHECK(block->_buffer_desc != nullptr && block->_buffer_desc->len == _max_block_size) << "Should never be writing a small buffer"; // If a write is still pending, return. Cleanup will be done in write_complete(). DCHECK(block->validate()) << endl << block->debug_string(); return; } - if (block->_buffer_desc != NULL) { + if (block->_buffer_desc != nullptr) { if (block->_buffer_desc->len != _max_block_size) { // Just delete the block for now. delete[] block->_buffer_desc->buffer; block->_client->_tracker->Release(block->_buffer_desc->len); delete block->_buffer_desc; - block->_buffer_desc = NULL; + block->_buffer_desc = nullptr; } else { if (!_free_io_buffers.contains(block->_buffer_desc)) { _free_io_buffers.enqueue(block->_buffer_desc); _buffer_available_cv.notify_one(); } - block->_buffer_desc->block = NULL; - block->_buffer_desc = NULL; + block->_buffer_desc->block = nullptr; + block->_buffer_desc = nullptr; } } return_unused_block(block); @@ -969,15 +969,15 @@ void BufferedBlockMgr2::return_unused_block(Block* block) { DCHECK(block->_is_deleted) << block->debug_string(); DCHECK(!block->_is_pinned) << block->debug_string(); ; - DCHECK(block->_buffer_desc == NULL); + DCHECK(block->_buffer_desc == nullptr); block->init(); _unused_blocks.enqueue(block); } Status BufferedBlockMgr2::find_buffer_for_block(Block* block, bool* in_mem) { - DCHECK(block != NULL); + DCHECK(block != nullptr); Client* client = block->_client; - DCHECK(client != NULL); + DCHECK(client != nullptr); DCHECK(!block->_is_pinned && !block->_is_deleted) << "Pinned or deleted block " << endl << block->debug_string(); *in_mem = false; @@ -1011,7 +1011,7 @@ Status BufferedBlockMgr2::find_buffer_for_block(Block* block, bool* in_mem) { return Status::OK(); } - if (block->_buffer_desc != NULL) { + if (block->_buffer_desc != nullptr) { // The block is in memory. It may be in 3 states: // 1. In the unpinned list. The buffer will not be in the free list. // 2. _in_write == true. The buffer will not be in the free list. @@ -1030,10 +1030,10 @@ Status BufferedBlockMgr2::find_buffer_for_block(Block* block, bool* in_mem) { _buffered_pin_counter->update(1); *in_mem = true; } else { - BufferDescriptor* buffer_desc = NULL; + BufferDescriptor* buffer_desc = nullptr; RETURN_IF_ERROR(find_buffer(l, &buffer_desc)); - if (buffer_desc == NULL) { + if (buffer_desc == nullptr) { // There are no free buffers or blocks we can evict. We need to fail this request. // If this is an optional request, return OK. If it is required, return OOM. if (!is_reserved_request) { @@ -1053,18 +1053,18 @@ Status BufferedBlockMgr2::find_buffer_for_block(Block* block, bool* in_mem) { "buffers in the block manager."); } - DCHECK(buffer_desc != NULL); + DCHECK(buffer_desc != nullptr); DCHECK_EQ(buffer_desc->len, max_block_size()) << "Non-I/O buffer"; - if (buffer_desc->block != NULL) { + if (buffer_desc->block != nullptr) { // This buffer was assigned to a block but now we are reusing it. Reset the // previous block->buffer link. DCHECK(buffer_desc->block->validate()) << endl << buffer_desc->block->debug_string(); - buffer_desc->block->_buffer_desc = NULL; + buffer_desc->block->_buffer_desc = nullptr; } buffer_desc->block = block; block->_buffer_desc = buffer_desc; } - DCHECK(block->_buffer_desc != NULL); + DCHECK(block->_buffer_desc != nullptr); DCHECK(block->_buffer_desc->len < max_block_size() || !block->_is_pinned) << "Trying to pin already pinned block. " << block->_buffer_desc->len << " " << block->_is_pinned; @@ -1086,7 +1086,7 @@ Status BufferedBlockMgr2::find_buffer_for_block(Block* block, bool* in_mem) { // 2. Pick a buffer from the free list. // 3. Wait and evict an unpinned buffer. Status BufferedBlockMgr2::find_buffer(unique_lock& lock, BufferDescriptor** buffer_desc) { - *buffer_desc = NULL; + *buffer_desc = nullptr; // First, try to allocate a new buffer. if (_free_io_buffers.size() < _block_write_threshold && @@ -1131,8 +1131,8 @@ Status BufferedBlockMgr2::find_buffer(unique_lock& lock, BufferDescriptor } BufferedBlockMgr2::Block* BufferedBlockMgr2::get_unused_block(Client* client) { - DCHECK(client != NULL); - Block* new_block = NULL; + DCHECK(client != nullptr); + Block* new_block = nullptr; if (_unused_blocks.empty()) { new_block = _obj_pool.add(new Block(this)); new_block->init(); @@ -1141,7 +1141,7 @@ BufferedBlockMgr2::Block* BufferedBlockMgr2::get_unused_block(Client* client) { new_block = _unused_blocks.dequeue(); _recycled_blocks_counter->update(1); } - DCHECK(new_block != NULL); + DCHECK(new_block != nullptr); new_block->_client = client; return new_block; } @@ -1163,7 +1163,7 @@ bool BufferedBlockMgr2::validate() const { return false; } - if (buffer->block == NULL && !is_free) { + if (buffer->block == nullptr && !is_free) { LOG(ERROR) << "Buffer with no block not in free list." << endl << debug_internal(); return false; } @@ -1173,7 +1173,7 @@ bool BufferedBlockMgr2::validate() const { return false; } - if (buffer->block != NULL) { + if (buffer->block != nullptr) { if (buffer->block->_buffer_desc != buffer) { LOG(ERROR) << "buffer<->block pointers inconsistent. Buffer: " << buffer << endl << buffer->block->debug_string(); @@ -1208,7 +1208,7 @@ bool BufferedBlockMgr2::validate() const { } Block* block = _unpinned_blocks.head(); - while (block != NULL) { + while (block != nullptr) { if (!block->validate()) { LOG(ERROR) << "Block inconsistent in unpinned list." << endl << block->debug_string(); return false; @@ -1240,7 +1240,7 @@ string BufferedBlockMgr2::debug_string(Client* client) { stringstream ss; unique_lock l(_lock); ss << debug_internal(); - if (client != NULL) { + if (client != nullptr) { ss << endl << client->debug_string(); } return ss.str(); @@ -1272,7 +1272,7 @@ void BufferedBlockMgr2::init(DiskIoMgr* io_mgr, RuntimeProfile* parent_profile, io_mgr->register_context(&_io_request_context); _profile.reset(new RuntimeProfile("BlockMgr")); - parent_profile->add_child(_profile.get(), true, NULL); + parent_profile->add_child(_profile.get(), true, nullptr); _block_size_counter = ADD_COUNTER(_profile.get(), "MaxBlockSize", TUnit::BYTES); _block_size_counter->set(_max_block_size); @@ -1295,7 +1295,7 @@ void BufferedBlockMgr2::init(DiskIoMgr* io_mgr, RuntimeProfile* parent_profile, Status BufferedBlockMgr2::init_tmp_files() { DCHECK(_tmp_files.empty()); - DCHECK(_tmp_file_mgr != NULL); + DCHECK(_tmp_file_mgr != nullptr); vector tmp_devices = _tmp_file_mgr->active_tmp_devices(); // Initialize the tmp files and the initial file to use. @@ -1307,7 +1307,7 @@ Status BufferedBlockMgr2::init_tmp_files() { // by active_tmp_devices() - handle this gracefully. Status status = _tmp_file_mgr->get_file(tmp_device_id, _query_id, &tmp_file); if (status.ok()) { - _tmp_files.push_back(tmp_file); + _tmp_files.emplace_back(tmp_file); } } if (_tmp_files.empty()) { diff --git a/be/src/runtime/buffered_block_mgr2.h b/be/src/runtime/buffered_block_mgr2.h index 9c17e8b62f..493398e8d9 100644 --- a/be/src/runtime/buffered_block_mgr2.h +++ b/be/src/runtime/buffered_block_mgr2.h @@ -18,9 +18,6 @@ #ifndef DORIS_BE_SRC_RUNTIME_BUFFERED_BLOCK_MGR2_H #define DORIS_BE_SRC_RUNTIME_BUFFERED_BLOCK_MGR2_H -#include -#include -#include #include #include "runtime/disk_io_mgr.h" @@ -143,12 +140,12 @@ public: // 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 - // false and the block is not pinned. If 'release_block' is non-NULL, if there is + // false and the block is not pinned. If 'release_block' is non-nullptr, if there is // memory pressure, this block will be pinned using the buffer from 'release_block'. // If 'unpin' is true, 'release_block' will be unpinned (regardless of whether or not // the buffer was used for this block). If 'unpin' is false, 'release_block' is // deleted. 'release_block' must be pinned. - Status pin(bool* pinned, Block* release_block = NULL, bool unpin = true); + Status pin(bool* pinned, Block* release_block = nullptr, bool unpin = true); // Unpins a block by adding it to the list of unpinned blocks maintained by the block // manager. An unpinned block must be flushed before its buffer is released or @@ -173,7 +170,7 @@ public: // Return the number of remaining bytes that can be allocated in this block. int bytes_remaining() const { - DCHECK(_buffer_desc != NULL); + DCHECK(_buffer_desc != nullptr); return _buffer_desc->len - _valid_data_len; } @@ -186,7 +183,7 @@ public: // Pointer to start of the block data in memory. Only guaranteed to be valid if the // block is pinned. uint8_t* buffer() const { - DCHECK(_buffer_desc != NULL); + DCHECK(_buffer_desc != nullptr); return _buffer_desc->buffer; } @@ -228,7 +225,7 @@ public: // be taken. bool validate() const; - // Pointer to the buffer associated with the block. NULL if the block is not in + // Pointer to the buffer associated with the block. nullptr if the block is not in // memory and cannot be changed while the block is pinned or being written. BufferDescriptor* _buffer_desc; @@ -288,7 +285,7 @@ public: // - buffer_size: maximum size of each buffer. static Status create(RuntimeState* state, const std::shared_ptr& parent, RuntimeProfile* profile, TmpFileMgr* tmp_file_mgr, int64_t mem_limit, - int64_t buffer_size, boost::shared_ptr* block_mgr); + int64_t buffer_size, std::shared_ptr* block_mgr); ~BufferedBlockMgr2(); @@ -319,15 +316,15 @@ public: bool try_acquire_tmp_reservation(Client* client, int num_buffers); // Return a new pinned block. If there is no memory for this block, *block will be set - // to NULL. + // to nullptr. // If len > 0, get_new_block() will return a block with a buffer of size len. len // must be less than max_block_size and this block cannot be unpinned. // This function will try to allocate new memory for the block up to the limit. // Otherwise it will (conceptually) write out an unpinned block and use that memory. - // The caller can pass a non-NULL 'unpin_block' to transfer memory from 'unpin_block' - // to the new block. If 'unpin_block' is non-NULL, the new block can never fail to + // The caller can pass a non-nullptr 'unpin_block' to transfer memory from 'unpin_block' + // to the new block. If 'unpin_block' is non-nullptr, the new block can never fail to // get a buffer. The semantics of this are: - // - If 'unpin_block' is non-NULL, it must be pinned. + // - If 'unpin_block' is non-nullptr, it must be pinned. // - If the call succeeds, 'unpin_block' is unpinned. // - If there is no memory pressure, block will get a newly allocated buffer. // - If there is memory pressure, block will get the buffer from 'unpin_block'. @@ -340,8 +337,8 @@ public: // Returns true if the block manager was cancelled. bool is_cancelled(); - // Dumps block mgr state. Grabs lock. If client is not NULL, also dumps its state. - std::string debug_string(Client* client = NULL); + // Dumps block mgr state. Grabs lock. If client is not nullptr, also dumps its state. + std::string debug_string(Client* client = nullptr); // Consumes 'size' bytes from the buffered block mgr. This is used by callers that want // the memory to come from the block mgr pool (and therefore trigger spilling) but need @@ -397,13 +394,13 @@ private: // Length of the buffer. int64_t len; - // Block that this buffer is assigned to. May be NULL. + // Block that this buffer is assigned to. May be nullptr. Block* block; // Iterator into _all_io_buffers for this buffer. std::list::iterator all_buffers_it; - BufferDescriptor(uint8_t* buf, int64_t len) : buffer(buf), len(len), block(NULL) {} + BufferDescriptor(uint8_t* buf, int64_t len) : buffer(buf), len(len), block(nullptr) {} }; BufferedBlockMgr2(RuntimeState* state, TmpFileMgr* tmp_file_mgr, int64_t block_size); @@ -422,7 +419,7 @@ private: Status unpin_block(Block* block); void delete_block(Block* block); - // If the 'block' is NULL, checks if cancelled and returns. Otherwise, depending on + // If the 'block' is nullptr, checks if cancelled and returns. Otherwise, depending on // 'unpin' calls either delete_block() or unpin_block(), which both first check for // cancellation. It should be called without the _lock acquired. Status delete_or_unpin_block(Block* block, bool unpin); @@ -449,7 +446,7 @@ private: // Uses the _lock, the caller should not have already acquired the _lock. Status find_buffer_for_block(Block* block, bool* in_mem); - // Returns a new buffer that can be used. *buffer is set to NULL if there was no + // Returns a new buffer that can be used. *buffer is set to nullptr if there was no // memory. // Otherwise, this function gets a new buffer by: // 1. Allocating a new buffer if possible @@ -547,13 +544,13 @@ private: // List of blocks that have been deleted and are no longer in use. // Can be reused in get_new_block(). Blocks in this list must be in the Init'ed state, - // i.e. _buffer_desc = NULL, _is_pinned = false, _in_write = false, + // i.e. _buffer_desc = nullptr, _is_pinned = false, _in_write = false, // _is_deleted = false, valid_data_len = 0. InternalQueue _unused_blocks; // List of buffers that can be assigned to a block in pin() or get_new_block(). // These buffers either have no block associated with them or are associated with an - // an unpinned block that has been persisted. That is, either block = NULL or + // an unpinned block that has been persisted. That is, either block = nullptr or // (!block->_is_pinned && !block->_in_write && !_unpinned_blocks.Contains(block)). // All of these buffers are io sized. InternalQueue _free_io_buffers; @@ -563,7 +560,7 @@ private: // Temporary physical file handle, (one per tmp device) to which blocks may be written. // Blocks are round-robined across these files. - boost::ptr_vector _tmp_files; + std::vector> _tmp_files; // Index into _tmp_files denoting the file to which the next block to be persisted will // be written. @@ -579,7 +576,7 @@ private: bool _is_cancelled; // Counters and timers to track behavior. - boost::scoped_ptr _profile; + std::unique_ptr _profile; // These have a fixed value for the lifetime of the manager and show memory usage. RuntimeProfile::Counter* _mem_tracker_counter; @@ -622,7 +619,7 @@ private: // map contains only weak ptrs. BufferedBlockMgr2s that are handed out are shared ptrs. // When all the shared ptrs are no longer referenced, the BufferedBlockMgr2 // d'tor will be called at which point the weak ptr will be removed from the map. - typedef std::unordered_map> BlockMgrsMap; + typedef std::unordered_map> BlockMgrsMap; static BlockMgrsMap _s_query_to_block_mgrs; // Unowned. diff --git a/be/src/runtime/buffered_tuple_stream2.cc b/be/src/runtime/buffered_tuple_stream2.cc index b127e9f79f..69fc2c440e 100644 --- a/be/src/runtime/buffered_tuple_stream2.cc +++ b/be/src/runtime/buffered_tuple_stream2.cc @@ -29,7 +29,7 @@ using std::string; using std::vector; using std::list; -using boost::scoped_ptr; +using std::unique_ptr; namespace doris { @@ -59,20 +59,20 @@ BufferedTupleStream2::BufferedTupleStream2(RuntimeState* state, const RowDescrip _block_mgr(block_mgr), _block_mgr_client(client), _total_byte_size(0), - _read_ptr(NULL), + _read_ptr(nullptr), _read_tuple_idx(0), _read_bytes(0), _rows_returned(0), _read_block_idx(-1), - _write_block(NULL), + _write_block(nullptr), _num_pinned(0), _num_small_blocks(0), _closed(false), _num_rows(0), _pinned(true), - _pin_timer(NULL), - _unpin_timer(NULL), - _get_new_block_timer(NULL) { + _pin_timer(nullptr), + _unpin_timer(nullptr), + _get_new_block_timer(nullptr) { _null_indicators_read_block = _null_indicators_write_block = -1; _read_block = _blocks.end(); _fixed_tuple_row_size = 0; @@ -127,7 +127,7 @@ string BufferedTupleStream2::debug_string() const { } Status BufferedTupleStream2::init(int node_id, RuntimeProfile* profile, bool pinned) { - if (profile != NULL) { + if (profile != nullptr) { _pin_timer = ADD_TIMER(profile, "PinTime"); _unpin_timer = ADD_TIMER(profile, "UnpinTime"); _get_new_block_timer = ADD_TIMER(profile, "GetNewBlockTime"); @@ -142,7 +142,7 @@ Status BufferedTupleStream2::init(int node_id, RuntimeProfile* profile, bool pin if (!got_block) { return _block_mgr->mem_limit_too_low_error(_block_mgr_client, node_id); } - DCHECK(_write_block != NULL); + DCHECK(_write_block != nullptr); if (!pinned) { RETURN_IF_ERROR(unpin_stream()); } @@ -151,7 +151,7 @@ Status BufferedTupleStream2::init(int node_id, RuntimeProfile* profile, bool pin Status BufferedTupleStream2::switch_to_io_buffers(bool* got_buffer) { if (!_use_small_buffers) { - *got_buffer = (_write_block != NULL); + *got_buffer = (_write_block != nullptr); return Status::OK(); } _use_small_buffers = false; @@ -216,11 +216,11 @@ Status BufferedTupleStream2::new_block_for_write(int64_t min_size, bool* got_blo } BufferedBlockMgr2::Block* unpin_block = _write_block; - if (_write_block != NULL) { + if (_write_block != nullptr) { DCHECK(_write_block->is_pinned()); if (_pinned || _write_block == *_read_block || !_write_block->is_max_size()) { // In these cases, don't unpin the current write block. - unpin_block = NULL; + unpin_block = nullptr; } } @@ -239,20 +239,20 @@ Status BufferedTupleStream2::new_block_for_write(int64_t min_size, bool* got_blo } } - BufferedBlockMgr2::Block* new_block = NULL; + BufferedBlockMgr2::Block* new_block = nullptr; { SCOPED_TIMER(_get_new_block_timer); RETURN_IF_ERROR( _block_mgr->get_new_block(_block_mgr_client, unpin_block, &new_block, block_len)); } - *got_block = (new_block != NULL); + *got_block = (new_block != nullptr); if (!*got_block) { - DCHECK(unpin_block == NULL); + DCHECK(unpin_block == nullptr); return Status::OK(); } - if (unpin_block != NULL) { + if (unpin_block != nullptr) { DCHECK(unpin_block == _write_block); DCHECK(!_write_block->is_pinned()); --_num_pinned; @@ -284,10 +284,11 @@ Status BufferedTupleStream2::next_block_for_read() { DCHECK(_read_block != _blocks.end()); DCHECK_EQ(_num_pinned, num_pinned(_blocks)) << _pinned; - // If non-NULL, this will be the current block if we are going to free it while - // grabbing the next block. This will stay NULL if we don't want to free the + // If non-nullptr, this will be the current block if we are going to free it while + // grabbing the next block. This will stay nullptr if we don't want to free the // current block. - BufferedBlockMgr2::Block* block_to_free = (!_pinned || _delete_on_read) ? *_read_block : NULL; + BufferedBlockMgr2::Block* block_to_free = + (!_pinned || _delete_on_read) ? *_read_block : nullptr; if (_delete_on_read) { // TODO: this is weird. We are deleting even if it is pinned. The analytic // eval node needs this. @@ -296,27 +297,27 @@ Status BufferedTupleStream2::next_block_for_read() { _blocks.pop_front(); _read_block = _blocks.begin(); _read_block_idx = 0; - if (block_to_free != NULL && !block_to_free->is_max_size()) { + if (block_to_free != nullptr && !block_to_free->is_max_size()) { block_to_free->del(); - block_to_free = NULL; + block_to_free = nullptr; DCHECK_EQ(_num_pinned, num_pinned(_blocks)) << debug_string(); } } else { ++_read_block; ++_read_block_idx; - if (block_to_free != NULL && !block_to_free->is_max_size()) { - block_to_free = NULL; + if (block_to_free != nullptr && !block_to_free->is_max_size()) { + block_to_free = nullptr; } } - _read_ptr = NULL; + _read_ptr = nullptr; _read_tuple_idx = 0; _read_bytes = 0; bool pinned = false; if (_read_block == _blocks.end() || (*_read_block)->is_pinned()) { // End of the blocks or already pinned, just handle block_to_free - if (block_to_free != NULL) { + if (block_to_free != nullptr) { SCOPED_TIMER(_unpin_timer); if (_delete_on_read) { block_to_free->del(); @@ -331,10 +332,10 @@ Status BufferedTupleStream2::next_block_for_read() { SCOPED_TIMER(_pin_timer); RETURN_IF_ERROR((*_read_block)->pin(&pinned, block_to_free, !_delete_on_read)); if (!pinned) { - DCHECK(block_to_free == NULL) << "Should have been able to pin." << std::endl - << _block_mgr->debug_string(_block_mgr_client); + DCHECK(block_to_free == nullptr) << "Should have been able to pin." << std::endl + << _block_mgr->debug_string(_block_mgr_client); } - if (block_to_free == NULL && pinned) { + if (block_to_free == nullptr && pinned) { ++_num_pinned; } } @@ -354,12 +355,12 @@ Status BufferedTupleStream2::prepare_for_read(bool delete_on_read, bool* got_buf return Status::OK(); } - if (!_read_write && _write_block != NULL) { + if (!_read_write && _write_block != nullptr) { DCHECK(_write_block->is_pinned()); if (!_pinned && _write_block != _blocks.front()) { RETURN_IF_ERROR(unpin_block(_write_block)); } - _write_block = NULL; + _write_block = nullptr; } // Walk the blocks and pin the first non-io sized block. @@ -371,7 +372,7 @@ Status BufferedTupleStream2::prepare_for_read(bool delete_on_read, bool* got_buf bool current_pinned = false; RETURN_IF_ERROR((*it)->pin(¤t_pinned)); if (!current_pinned) { - DCHECK(got_buffer != NULL) << "Should have reserved enough blocks"; + DCHECK(got_buffer != nullptr) << "Should have reserved enough blocks"; *got_buffer = false; return Status::OK(); } @@ -392,7 +393,7 @@ Status BufferedTupleStream2::prepare_for_read(bool delete_on_read, bool* got_buf _rows_returned = 0; _read_block_idx = 0; _delete_on_read = delete_on_read; - if (got_buffer != NULL) { + if (got_buffer != nullptr) { *got_buffer = true; } return Status::OK(); @@ -400,7 +401,7 @@ Status BufferedTupleStream2::prepare_for_read(bool delete_on_read, bool* got_buf Status BufferedTupleStream2::pin_stream(bool already_reserved, bool* pinned) { DCHECK(!_closed); - DCHECK(pinned != NULL); + DCHECK(pinned != nullptr); if (!already_reserved) { // If we can't get all the blocks, don't try at all. if (!_block_mgr->try_acquire_tmp_reservation(_block_mgr_client, blocks_unpinned())) { @@ -456,7 +457,7 @@ Status BufferedTupleStream2::unpin_stream(bool all) { } if (all) { _read_block = _blocks.end(); - _write_block = NULL; + _write_block = nullptr; } _pinned = false; return Status::OK(); @@ -465,7 +466,7 @@ Status BufferedTupleStream2::unpin_stream(bool all) { int BufferedTupleStream2::compute_num_null_indicator_bytes(int block_size) const { if (_nullable_tuple) { // We assume that all rows will use their max size, so we may be underutilizing the - // space, i.e. we may have some unused space in case of rows with NULL tuples. + // space, i.e. we may have some unused space in case of rows with nullptr tuples. const uint32_t tuples_per_row = _desc.tuple_descriptors().size(); const uint32_t min_row_size_in_bits = 8 * _fixed_tuple_row_size + tuples_per_row; const uint32_t block_size_in_bits = 8 * block_size; @@ -477,7 +478,7 @@ int BufferedTupleStream2::compute_num_null_indicator_bytes(int block_size) const } } -Status BufferedTupleStream2::get_rows(scoped_ptr* batch, bool* got_rows) { +Status BufferedTupleStream2::get_rows(unique_ptr* batch, bool* got_rows) { RETURN_IF_ERROR(pin_stream(false, got_rows)); if (!*got_rows) { return Status::OK(); @@ -533,7 +534,7 @@ Status BufferedTupleStream2::get_next_internal(RowBatch* batch, bool* eos, DCHECK(_read_block != _blocks.end()); DCHECK((*_read_block)->is_pinned()) << debug_string(); - DCHECK(_read_ptr != NULL); + DCHECK(_read_ptr != nullptr); int64_t rows_left = _num_rows - _rows_returned; int rows_to_fill = @@ -545,7 +546,7 @@ Status BufferedTupleStream2::get_next_internal(RowBatch* batch, bool* eos, // Produce tuple rows from the current block and the corresponding position on the // null tuple indicator. vector local_indices; - if (indices == NULL) { + if (indices == nullptr) { // A hack so that we do not need to check whether 'indices' is not null in the // tight loop. indices = &local_indices; @@ -558,7 +559,7 @@ Status BufferedTupleStream2::get_next_internal(RowBatch* batch, bool* eos, indices->reserve(rows_to_fill); int i = 0; - uint8_t* null_word = NULL; + uint8_t* null_word = nullptr; uint32_t null_pos = 0; // Start reading from position _read_tuple_idx in the block. uint64_t last_read_ptr = 0; @@ -580,12 +581,12 @@ Status BufferedTupleStream2::get_next_internal(RowBatch* batch, bool* eos, last_read_row); if (HasNullableTuple) { for (int j = 0; j < tuples_per_row; ++j) { - // Stitch together the tuples from the block and the NULL ones. + // Stitch together the tuples from the block and the nullptr ones. null_word = (*_read_block)->buffer() + (_read_tuple_idx >> 3); null_pos = _read_tuple_idx & 7; ++_read_tuple_idx; const bool is_not_null = ((*null_word & (1 << (7 - null_pos))) == 0); - // Copy tuple and advance _read_ptr. If it is a NULL tuple, it calls set_tuple + // Copy tuple and advance _read_ptr. If it is a nullptr tuple, it calls set_tuple // with Tuple* being 0x0. To do that we multiply the current _read_ptr with // false (0x0). row->set_tuple(j, reinterpret_cast(reinterpret_cast(_read_ptr) * @@ -611,7 +612,7 @@ Status BufferedTupleStream2::get_next_internal(RowBatch* batch, bool* eos, // Update string slot ptrs. for (int j = 0; j < _string_slots.size(); ++j) { Tuple* tuple = row->get_tuple(_string_slots[j].first); - if (HasNullableTuple && tuple == NULL) { + if (HasNullableTuple && tuple == nullptr) { continue; } read_strings(_string_slots[j].second, data_len, tuple); @@ -622,7 +623,7 @@ Status BufferedTupleStream2::get_next_internal(RowBatch* batch, bool* eos, // length of collections and strings. // for (int j = 0; j < _collection_slots.size(); ++j) { // Tuple* tuple = row->get_tuple(_collection_slots[j].first); - // if (HasNullableTuple && tuple == NULL) { + // if (HasNullableTuple && tuple == nullptr) { // continue; // } // ReadCollections(_collection_slots[j].second, data_len, tuple); @@ -646,7 +647,7 @@ Status BufferedTupleStream2::get_next_internal(RowBatch* batch, bool* eos, void BufferedTupleStream2::read_strings(const vector& string_slots, int data_len, Tuple* tuple) { - DCHECK(tuple != NULL); + DCHECK(tuple != nullptr); for (int i = 0; i < string_slots.size(); ++i) { const SlotDescriptor* slot_desc = string_slots[i]; if (tuple->is_null(slot_desc->null_indicator_offset())) { @@ -664,7 +665,7 @@ void BufferedTupleStream2::read_strings(const vector& string_sl #if 0 void BufferedTupleStream2::ReadCollections(const vector& collection_slots, int data_len, Tuple* tuple) { - DCHECK(tuple != NULL); + DCHECK(tuple != nullptr); for (int i = 0; i < collection_slots.size(); ++i) { const SlotDescriptor* slot_desc = collection_slots[i]; if (tuple->IsNull(slot_desc->null_indicator_offset())) continue; @@ -694,8 +695,8 @@ int64_t BufferedTupleStream2::compute_row_size(TupleRow* row) const { for (int i = 0; i < _desc.tuple_descriptors().size(); ++i) { const TupleDescriptor* tuple_desc = _desc.tuple_descriptors()[i]; Tuple* tuple = row->get_tuple(i); - DCHECK(_nullable_tuple || tuple_desc->byte_size() == 0 || tuple != NULL); - if (tuple == NULL) { + DCHECK(_nullable_tuple || tuple_desc->byte_size() == 0 || tuple != nullptr); + if (tuple == nullptr) { continue; } size += tuple->total_byte_size(*tuple_desc); diff --git a/be/src/runtime/buffered_tuple_stream2.h b/be/src/runtime/buffered_tuple_stream2.h index 061375287e..5f23fc8870 100644 --- a/be/src/runtime/buffered_tuple_stream2.h +++ b/be/src/runtime/buffered_tuple_stream2.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_RUNTIME_BUFFERED_TUPLE_STREAM2_H #define DORIS_BE_SRC_RUNTIME_BUFFERED_TUPLE_STREAM2_H -#include #include #include "common/status.h" @@ -72,7 +71,7 @@ class Tuple; // directly followed by the var len portion. (Fixed len and var len are interleaved). // If any tuple in the row is nullable, then there is a bitstring of null tuple // indicators at the header of the block. The order of bits in the null indicators -// bitstring corresponds to the order of tuples in the block. The NULL tuples are not +// bitstring corresponds to the order of tuples in the block. The nullptr tuples are not // stored in the body of the block, only as set bits in the null indicators bitsting. // // The behavior of reads and writes is as follows: @@ -104,7 +103,7 @@ class Tuple; // from the end of the same block. Don't interleave fixed and var len data. // - It would be good to allocate the null indicators at the end of each block and grow // this array as new rows are inserted in the block. If we do so, then there will be -// fewer gaps in case of many rows with NULL tuples. +// fewer gaps in case of many rows with nullptr tuples. // - We will want to multithread this. Add a AddBlock() call so the synchronization // happens at the block level. This is a natural extension. // - Instead of allocating all blocks from the block_mgr, allocate some blocks that @@ -171,7 +170,7 @@ public: // Initializes the tuple stream object on behalf of node 'node_id'. Must be called // once before any of the other APIs. // If 'pinned' is true, the tuple stream starts of pinned, otherwise it is unpinned. - // If 'profile' is non-NULL, counters are created. + // If 'profile' is non-nullptr, counters are created. // 'node_id' is only used for error reporting. Status init(int node_id, RuntimeProfile* profile, bool pinned); @@ -186,8 +185,8 @@ public: bool add_row(TupleRow* row, Status* status); // Allocates space to store a row of size 'size' and returns a pointer to the memory - // when successful. Returns NULL if there is not enough memory or an error occurred. - // When returning NULL, sets *status. The returned memory is guaranteed to fit on one + // when successful. Returns nullptr if there is not enough memory or an error occurred. + // When returning nullptr, sets *status. The returned memory is guaranteed to fit on one // block. uint8_t* allocate_row(int size, Status* status); @@ -199,10 +198,10 @@ public: // begin reading. Otherwise this must be called after the last AddRow() and // before get_next(). // delete_on_read: Blocks are deleted after they are read. - // If got_buffer is NULL, this function will fail (with a bad status) if no buffer + // If got_buffer is nullptr, this function will fail (with a bad status) if no buffer // is available. If got_buffer is non-null, this function will not fail on OOM and // *got_buffer is true if a buffer was pinned. - Status prepare_for_read(bool delete_on_read, bool* got_buffer = NULL); + Status prepare_for_read(bool delete_on_read, bool* got_buffer = nullptr); // Pins all blocks in this stream and switches to pinned mode. // If there is not enough memory, *pinned is set to false and the stream is unmodified. @@ -216,14 +215,14 @@ public: // Get the next batch of output rows. Memory is still owned by the BufferedTupleStream2 // and must be copied out by the caller. - // If 'indices' is non-NULL, that is also populated for each returned row with the + // If 'indices' is non-nullptr, that is also populated for each returned row with the // index for that row. - Status get_next(RowBatch* batch, bool* eos, std::vector* indices = NULL); + Status get_next(RowBatch* batch, bool* eos, std::vector* indices = nullptr); // Returns all the rows in the stream in batch. This pins the entire stream // in the process. // *got_rows is false if the stream could not be pinned. - Status get_rows(boost::scoped_ptr* batch, bool* got_rows); + Status get_rows(std::unique_ptr* batch, bool* got_rows); // Must be called once at the end to cleanup all resources. Idempotent. void close(); @@ -245,7 +244,7 @@ public: int blocks_pinned() const { return _num_pinned; } int blocks_unpinned() const { return _blocks.size() - _num_pinned - _num_small_blocks; } bool has_read_block() const { return _read_block != _blocks.end(); } - bool has_write_block() const { return _write_block != NULL; } + bool has_write_block() const { return _write_block != nullptr; } bool using_small_buffers() const { return _use_small_buffers; } bool has_tuple_footprint() const { return _fixed_tuple_row_size > 0 || !_string_slots.empty() || _nullable_tuple; @@ -330,7 +329,7 @@ private: // The block index of the current read block. int _read_block_idx; - // The current block for writing. NULL if there is no available block to write to. + // The current block for writing. nullptr if there is no available block to write to. BufferedBlockMgr2::Block* _write_block; // Number of pinned blocks in _blocks, stored to avoid iterating over the list diff --git a/be/src/runtime/buffered_tuple_stream2.inline.h b/be/src/runtime/buffered_tuple_stream2.inline.h index 090411200f..d55f25552f 100644 --- a/be/src/runtime/buffered_tuple_stream2.inline.h +++ b/be/src/runtime/buffered_tuple_stream2.inline.h @@ -40,14 +40,14 @@ inline bool BufferedTupleStream2::add_row(TupleRow* row, Status* status) { inline uint8_t* BufferedTupleStream2::allocate_row(int size, Status* status) { DCHECK(!_closed); - if (UNLIKELY(_write_block == NULL || _write_block->bytes_remaining() < size)) { + if (UNLIKELY(_write_block == nullptr || _write_block->bytes_remaining() < size)) { bool got_block; *status = new_block_for_write(size, &got_block); if (!status->ok() || !got_block) { - return NULL; + return nullptr; } } - DCHECK(_write_block != NULL); + DCHECK(_write_block != nullptr); DCHECK(_write_block->is_pinned()); DCHECK_GE(_write_block->bytes_remaining(), size); ++_num_rows; @@ -56,7 +56,7 @@ inline uint8_t* BufferedTupleStream2::allocate_row(int size, Status* status) { } inline void BufferedTupleStream2::get_tuple_row(const RowIdx& idx, TupleRow* row) const { - DCHECK(row != NULL); + DCHECK(row != nullptr); DCHECK(!_closed); DCHECK(is_pinned()); DCHECK(!_delete_on_read); @@ -65,7 +65,7 @@ inline void BufferedTupleStream2::get_tuple_row(const RowIdx& idx, TupleRow* row uint8_t* data = _block_start_idx[idx.block()] + idx.offset(); if (_nullable_tuple) { - // Stitch together the tuples from the block and the NULL ones. + // Stitch together the tuples from the block and the nullptr ones. const int tuples_per_row = _desc.tuple_descriptors().size(); uint32_t tuple_idx = idx.idx() * tuples_per_row; for (int i = 0; i < tuples_per_row; ++i) { diff --git a/be/src/runtime/buffered_tuple_stream2_ir.cc b/be/src/runtime/buffered_tuple_stream2_ir.cc index 74df46f477..07c356414f 100644 --- a/be/src/runtime/buffered_tuple_stream2_ir.cc +++ b/be/src/runtime/buffered_tuple_stream2_ir.cc @@ -37,7 +37,7 @@ bool BufferedTupleStream2::deep_copy(TupleRow* row) { // TODO: in case of duplicate tuples, this can redundantly serialize data. template bool BufferedTupleStream2::deep_copy_internal(TupleRow* row) { - if (UNLIKELY(_write_block == NULL)) { + if (UNLIKELY(_write_block == nullptr)) { return false; } DCHECK_GE(_null_indicators_write_block, 0); @@ -56,11 +56,11 @@ bool BufferedTupleStream2::deep_copy_internal(TupleRow* row) { // if this row doesn't fit. int bytes_allocated = _fixed_tuple_row_size; - // Copy the not NULL fixed len tuples. For the NULL tuples just update the NULL tuple + // Copy the not nullptr fixed len tuples. For the nullptr tuples just update the nullptr tuple // indicator. if (HasNullableTuple) { DCHECK_GT(_null_indicators_write_block, 0); - uint8_t* null_word = NULL; + uint8_t* null_word = nullptr; uint32_t null_pos = 0; // Calculate how much space it should return. int to_return = 0; @@ -71,7 +71,7 @@ bool BufferedTupleStream2::deep_copy_internal(TupleRow* row) { const int tuple_size = _desc.tuple_descriptors()[i]->byte_size(); Tuple* t = row->get_tuple(i); const uint8_t mask = 1 << (7 - null_pos); - if (t != NULL) { + if (t != nullptr) { *null_word &= ~mask; memcpy(tuple_buf, t, tuple_size); tuple_buf += tuple_size; @@ -90,8 +90,8 @@ bool BufferedTupleStream2::deep_copy_internal(TupleRow* row) { const int tuple_size = _desc.tuple_descriptors()[i]->byte_size(); Tuple* t = row->get_tuple(i); // TODO: Once IMPALA-1306 (Avoid passing empty tuples of non-materialized slots) - // is delivered, the check below should become DCHECK(t != NULL). - DCHECK(t != NULL || tuple_size == 0); + // is delivered, the check below should become DCHECK(t != nullptr). + DCHECK(t != nullptr || tuple_size == 0); memcpy(tuple_buf, t, tuple_size); tuple_buf += tuple_size; } @@ -102,7 +102,7 @@ bool BufferedTupleStream2::deep_copy_internal(TupleRow* row) { // by the string data so only the len information is necessary. for (int i = 0; i < _string_slots.size(); ++i) { Tuple* tuple = row->get_tuple(_string_slots[i].first); - if (HasNullableTuple && tuple == NULL) { + if (HasNullableTuple && tuple == nullptr) { continue; } if (UNLIKELY(!copy_strings(tuple, _string_slots[i].second, &bytes_allocated))) { @@ -115,7 +115,7 @@ bool BufferedTupleStream2::deep_copy_internal(TupleRow* row) { // need to convert pointers to offsets on the write path. // for (int i = 0; i < _collection_slots.size(); ++i) { // Tuple* tuple = row->get_tuple(_collection_slots[i].first); - // if (HasNullableTuple && tuple == NULL) continue; + // if (HasNullableTuple && tuple == nullptr) continue; // if (UNLIKELY(!copy_collections(tuple, _collection_slots[i].second, // &bytes_allocated))) { // _write_block->return_allocation(bytes_allocated); diff --git a/be/src/runtime/buffered_tuple_stream3.cc b/be/src/runtime/buffered_tuple_stream3.cc index 325c30a2d9..2c7ce31992 100644 --- a/be/src/runtime/buffered_tuple_stream3.cc +++ b/be/src/runtime/buffered_tuple_stream3.cc @@ -671,7 +671,7 @@ void BufferedTupleStream3::UnpinStream(UnpinMode mode) { } */ Status BufferedTupleStream3::GetRows(const std::shared_ptr& tracker, - boost::scoped_ptr* batch, bool* got_rows) { + std::unique_ptr* batch, bool* got_rows) { if (num_rows() > numeric_limits::max()) { // RowBatch::num_rows_ is a 32-bit int, avoid an overflow. return Status::InternalError( @@ -942,7 +942,7 @@ bool BufferedTupleStream3::DeepCopyInternal(TupleRow* row, uint8_t** data, const uint8_t* data_end) noexcept { uint8_t* pos = *data; const uint64_t tuples_per_row = desc_->tuple_descriptors().size(); - // Copy the not NULL fixed len tuples. For the NULL tuples just update the NULL tuple + // Copy the not nullptr fixed len tuples. For the nullptr tuples just update the nullptr tuple // indicator. if (HAS_NULLABLE_TUPLE) { int null_indicator_bytes = NullIndicatorBytesPerRow(); @@ -1063,7 +1063,7 @@ void BufferedTupleStream3::UnflattenTupleRow(uint8_t** data, TupleRow* row) cons const int tuples_per_row = desc_->tuple_descriptors().size(); uint8_t* ptr = *data; if (has_nullable_tuple_) { - // Stitch together the tuples from the page and the NULL ones. + // Stitch together the tuples from the page and the nullptr ones. const uint8_t* null_indicators = ptr; ptr += NullIndicatorBytesPerRow(); for (int i = 0; i < tuples_per_row; ++i) { diff --git a/be/src/runtime/buffered_tuple_stream3.h b/be/src/runtime/buffered_tuple_stream3.h index 65b32309f1..7d8f053d03 100644 --- a/be/src/runtime/buffered_tuple_stream3.h +++ b/be/src/runtime/buffered_tuple_stream3.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_RUNTIME_BUFFERED_TUPLE_STREAM_H #define DORIS_BE_RUNTIME_BUFFERED_TUPLE_STREAM_H -#include #include #include #include @@ -137,7 +136,7 @@ class TupleRow; /// +--------+-----------+-----------+-----------+-------------+ /// <--4b--> <---12b---> <----8b---> <---12b---> <----10b----> /// -/// Example layout for a row with the second tuple nullable ((1, "hello"), NULL) +/// Example layout for a row with the second tuple nullable ((1, "hello"), nullptr) /// with all var len data stored in the stream: /// <- null tuple bitstring -> <---- tuple 1 -----> <- var len -> <- next row ... /// +-------------------------+--------+-----------+------------+ @@ -334,10 +333,10 @@ public: /// process. If the current unused reservation is not sufficient to pin the stream in /// memory, this will try to increase the reservation. If that fails, 'got_rows' is set /// to false. - Status GetRows(const std::shared_ptr& tracker, boost::scoped_ptr* batch, + Status GetRows(const std::shared_ptr& tracker, std::unique_ptr* batch, bool* got_rows) WARN_UNUSED_RESULT; - /// Must be called once at the end to cleanup all resources. If 'batch' is non-NULL, + /// Must be called once at the end to cleanup all resources. If 'batch' is non-nullptr, /// attaches buffers from pinned pages that rows returned from GetNext() may reference. /// Otherwise deletes all pages. Does nothing if the stream was already closed. The /// 'flush' mode is forwarded to RowBatch::AddBuffer() when attaching buffers. @@ -475,7 +474,7 @@ private: /// True if there is currently an active write iterator into the stream. bool has_write_iterator_; - /// The current page for writing. NULL if there is no write iterator or no current + /// The current page for writing. nullptr if there is no write iterator or no current /// write page. Always pinned. Size is 'default_page_len_', except temporarily while /// appending a larger row between AddRowCustomBegin() and AddRowCustomEnd(). Page* write_page_; diff --git a/be/src/runtime/bufferpool/buffer_allocator.h b/be/src/runtime/bufferpool/buffer_allocator.h index 4dbb09fbf4..f2ab5e6324 100644 --- a/be/src/runtime/bufferpool/buffer_allocator.h +++ b/be/src/runtime/bufferpool/buffer_allocator.h @@ -18,8 +18,6 @@ #ifndef DORIS_BE_RUNTIME_BUFFER_ALLOCATOR_H #define DORIS_BE_RUNTIME_BUFFER_ALLOCATOR_H -#include - #include "runtime/bufferpool/buffer_pool_internal.h" #include "runtime/bufferpool/free_list.h" #include "util/aligned_new.h" @@ -168,7 +166,7 @@ private: /// 'min_buffer_len' so that there is at least one valid buffer size. static int64_t CalcMaxBufferLen(int64_t min_buffer_len, int64_t system_bytes_limit); - /// Same as Allocate() but leaves 'buffer->client_' NULL and does not update counters. + /// Same as Allocate() but leaves 'buffer->client_' nullptr and does not update counters. Status AllocateInternal(int64_t len, BufferPool::BufferHandle* buffer) WARN_UNUSED_RESULT; /// Tries to reclaim enough memory from various sources so that the caller can allocate @@ -193,7 +191,7 @@ private: BufferPool* const pool_; /// System allocator that is ultimately used to allocate and free buffers. - const boost::scoped_ptr system_allocator_; + const std::unique_ptr system_allocator_; /// The minimum power-of-two buffer length that can be allocated. const int64_t min_buffer_len_; diff --git a/be/src/runtime/bufferpool/buffer_pool.cc b/be/src/runtime/bufferpool/buffer_pool.cc index 3f3011d5a8..c0660bd77b 100644 --- a/be/src/runtime/bufferpool/buffer_pool.cc +++ b/be/src/runtime/bufferpool/buffer_pool.cc @@ -71,8 +71,8 @@ void BufferPool::PageHandle::Open(Page* page, ClientHandle* client) { } void BufferPool::PageHandle::Reset() { - page_ = NULL; - client_ = NULL; + page_ = nullptr; + client_ = nullptr; } int BufferPool::PageHandle::pin_count() const { @@ -119,7 +119,7 @@ Status BufferPool::RegisterClient(const string& name, ReservationTracker* parent int64_t reservation_limit, RuntimeProfile* profile, ClientHandle* client) { DCHECK(!client->is_registered()); - DCHECK(parent_reservation != NULL); + DCHECK(parent_reservation != nullptr); client->impl_ = new Client(this, //file_group, name, parent_reservation, mem_tracker, reservation_limit, profile); return Status::OK(); @@ -129,7 +129,7 @@ void BufferPool::DeregisterClient(ClientHandle* client) { if (!client->is_registered()) return; client->impl_->Close(); // Will DCHECK if any remaining buffers or pinned pages. delete client->impl_; // Will DCHECK if there are any remaining pages. - client->impl_ = NULL; + client->impl_ = nullptr; } Status BufferPool::CreatePage(ClientHandle* client, int64_t len, PageHandle* handle, @@ -405,7 +405,7 @@ BufferPool::Page* BufferPool::Client::CreatePinnedPage(BufferHandle&& buffer) { } void BufferPool::Client::DestroyPageInternal(PageHandle* handle, BufferHandle* out_buffer) { - DCHECK(handle->is_pinned() || out_buffer == NULL); + DCHECK(handle->is_pinned() || out_buffer == nullptr); Page* page = handle->page_; // Remove the page from the list that it is currently present in (if any). { @@ -422,12 +422,12 @@ void BufferPool::Client::DestroyPageInternal(PageHandle* handle, BufferHandle* o --num_pages_; } - //if (page->write_handle != NULL) { + //if (page->write_handle != nullptr) { // Discard any on-disk data. //file_group_->DestroyWriteHandle(move(page->write_handle)); //} // - if (out_buffer != NULL) { + if (out_buffer != nullptr) { DCHECK(page->buffer.is_open()); *out_buffer = std::move(page->buffer); buffers_allocated_bytes_ += out_buffer->len(); @@ -482,7 +482,7 @@ Status BufferPool::Client::StartMoveToPinned(ClientHandle* client, Page* page) { // back to the pinned state. pinned_pages_.enqueue(page); DCHECK(page->buffer.is_open()); - DCHECK(page->write_handle != NULL); + DCHECK(page->write_handle != nullptr); // Don't need on-disk data. cl.unlock(); // Don't block progress for other threads operating on other pages. return file_group_->RestoreData(move(page->write_handle), page->buffer.mem_range()); @@ -600,7 +600,7 @@ Status BufferPool::Client::CleanPages(std::unique_lock* client_lock, void BufferPool::Client::WriteDirtyPagesAsync(int64_t min_bytes_to_write) { DCHECK_GE(min_bytes_to_write, 0); DCHECK_LE(min_bytes_to_write, dirty_unpinned_pages_.bytes()); - // if (file_group_ == NULL) { + // if (file_group_ == nullptr) { // Spilling disabled - there should be no unpinned pages to write. DCHECK_EQ(0, min_bytes_to_write); DCHECK_EQ(0, dirty_unpinned_pages_.bytes()); @@ -623,10 +623,10 @@ void BufferPool::Client::WriteDirtyPagesAsync(int64_t min_bytes_to_write) { && (bytes_written < min_bytes_to_write || in_flight_write_pages_.size() < target_writes)) { Page* page = dirty_unpinned_pages_.tail(); // LIFO. - DCHECK(page != NULL) << "Should have been enough dirty unpinned pages"; + DCHECK(page != nullptr) << "Should have been enough dirty unpinned pages"; { std::lock_guard pl(page->buffer_lock); - DCHECK(file_group_ != NULL); + DCHECK(file_group_ != nullptr); DCHECK(page->buffer.is_open()); COUNTER_ADD(counters().bytes_written, page->len); COUNTER_ADD(counters().write_io_ops, 1); diff --git a/be/src/runtime/bufferpool/buffer_pool.h b/be/src/runtime/bufferpool/buffer_pool.h index f7cc1529ae..3dae928782 100644 --- a/be/src/runtime/bufferpool/buffer_pool.h +++ b/be/src/runtime/bufferpool/buffer_pool.h @@ -20,7 +20,6 @@ #include -#include #include #include @@ -166,9 +165,9 @@ public: /// Register a client. Returns an error status and does not register the client if the /// arguments are invalid. 'name' is an arbitrary name used to identify the client in - /// any errors messages or logging. If 'file_group' is non-NULL, it is used to allocate - /// scratch space to write unpinned pages to disk. If it is NULL, unpinning of pages is - /// not allowed for this client. Counters for this client are added to the (non-NULL) + /// any errors messages or logging. If 'file_group' is non-nullptr, it is used to allocate + /// scratch space to write unpinned pages to disk. If it is nullptr, unpinning of pages is + /// not allowed for this client. Counters for this client are added to the (non-nullptr) /// 'profile'. 'client' is the client to register. 'client' must not already be /// registered. /// @@ -189,7 +188,7 @@ public: /// sufficient unused reservation to pin the new page (otherwise it will DCHECK). /// CreatePage() only fails when a system error prevents the buffer pool from fulfilling /// the reservation. - /// On success, the handle is mapped to the new page and 'buffer', if non-NULL, is set + /// On success, the handle is mapped to the new page and 'buffer', if non-nullptr, is set /// to the page's buffer. Status CreatePage(ClientHandle* client, int64_t len, PageHandle* handle, const BufferHandle** buffer = nullptr) WARN_UNUSED_RESULT; @@ -300,7 +299,7 @@ private: /// Allocator for allocating and freeing all buffer memory and managing lists of free /// buffers and clean pages. - boost::scoped_ptr allocator_; + std::unique_ptr allocator_; /// The minimum length of a buffer in bytes. All buffers and pages are a power-of-two /// multiple of this length. This is always a power of two. @@ -315,7 +314,7 @@ private: /// Client methods or BufferPool methods with the Client as an argument is not supported. class BufferPool::ClientHandle { public: - ClientHandle() : impl_(NULL) {} + ClientHandle() : impl_(nullptr) {} /// Client must be deregistered. ~ClientHandle() { DCHECK(!is_registered()); } @@ -362,7 +361,7 @@ public: /// Call SetDebugDenyIncreaseReservation() on this client's ReservationTracker. void SetDebugDenyIncreaseReservation(double probability); - bool is_registered() const { return impl_ != NULL; } + bool is_registered() const { return impl_ != nullptr; } /// Return true if there are any unpinned pages for this client. bool has_unpinned_pages() const; @@ -375,7 +374,7 @@ private: friend class SubReservation; DISALLOW_COPY_AND_ASSIGN(ClientHandle); - /// Internal state for the client. NULL means the client isn't registered. + /// Internal state for the client. nullptr means the client isn't registered. /// Owned by BufferPool. Client* impl_; }; @@ -402,7 +401,7 @@ private: /// Child of the client's tracker used to track the sub-reservation. Usage is not /// tracked against this tracker - instead the reservation is always transferred back /// to the client's tracker before use. - boost::scoped_ptr tracker_; + std::unique_ptr tracker_; }; /// A handle to a buffer allocated from the buffer pool. Each BufferHandle should only @@ -421,7 +420,7 @@ public: /// Destination must be uninitialized. Inline to make moving efficient. inline BufferHandle& operator=(BufferHandle&& src); - bool is_open() const { return data_ != NULL; } + bool is_open() const { return data_ != nullptr; } int64_t len() const { DCHECK(is_open()); return len_; @@ -457,10 +456,10 @@ private: inline void Reset(); /// The client the buffer handle belongs to, used to validate that the correct client - /// is provided in BufferPool method calls. Set to NULL if the buffer is in a free list. + /// is provided in BufferPool method calls. Set to nullptr if the buffer is in a free list. const ClientHandle* client_; - /// Pointer to the start of the buffer. Non-NULL if open, NULL if closed. + /// Pointer to the start of the buffer. Non-nullptr if open, nullptr if closed. uint8_t* data_; /// Length of the buffer in bytes. @@ -486,7 +485,7 @@ public: // Destination must be closed. PageHandle& operator=(PageHandle&& src); - bool is_open() const { return page_ != NULL; } + bool is_open() const { return page_ != nullptr; } bool is_pinned() const { return pin_count() > 0; } int pin_count() const; int64_t len() const; @@ -514,7 +513,7 @@ private: /// Internal helper to reset the handle to an unopened state. void Reset(); - /// The internal page structure. NULL if the handle is not open. + /// The internal page structure. nullptr if the handle is not open. Page* page_; /// The client the page handle belongs to. @@ -538,8 +537,8 @@ inline BufferPool::BufferHandle& BufferPool::BufferHandle::operator=(BufferHandl } inline void BufferPool::BufferHandle::Reset() { - client_ = NULL; - data_ = NULL; + client_ = nullptr; + data_ = nullptr; len_ = -1; home_core_ = -1; } diff --git a/be/src/runtime/bufferpool/buffer_pool_internal.h b/be/src/runtime/bufferpool/buffer_pool_internal.h index b2362d0756..5c483c06b0 100644 --- a/be/src/runtime/bufferpool/buffer_pool_internal.h +++ b/be/src/runtime/bufferpool/buffer_pool_internal.h @@ -155,7 +155,7 @@ public: /// and the page's buffer will be returned. /// Neither the client's lock nor handle->page_->buffer_lock should be held by the /// caller. - void DestroyPageInternal(PageHandle* handle, BufferHandle* out_buffer = NULL); + void DestroyPageInternal(PageHandle* handle, BufferHandle* out_buffer = nullptr); /// Updates client state to reflect that 'page' is now a dirty unpinned page. May /// initiate writes for this or other dirty unpinned pages. @@ -213,7 +213,7 @@ public: ReservationTracker* reservation() { return &reservation_; } const BufferPoolClientCounters& counters() const { return counters_; } - //bool spilling_enabled() const { return file_group_ != NULL; } + //bool spilling_enabled() const { return file_group_ != nullptr; } void set_debug_write_delay_ms(int val) { debug_write_delay_ms_ = val; } bool has_unpinned_pages() const { // Safe to read without lock since other threads should not be calling BufferPool @@ -265,7 +265,7 @@ private: /// The buffer pool that owns the client. BufferPool* const pool_; - /// The file group that should be used for allocating scratch space. If NULL, spilling + /// The file group that should be used for allocating scratch space. If nullptr, spilling /// is disabled. //TmpFileMgr::FileGroup* const file_group_; @@ -277,7 +277,7 @@ private: ReservationTracker reservation_; /// The RuntimeProfile counters for this client, owned by the client's RuntimeProfile. - /// All non-NULL. + /// All non-nullptr. BufferPoolClientCounters counters_; /// Debug option to delay write completion. diff --git a/be/src/runtime/bufferpool/reservation_tracker.h b/be/src/runtime/bufferpool/reservation_tracker.h index 0c860f61cf..df81b9e9e5 100644 --- a/be/src/runtime/bufferpool/reservation_tracker.h +++ b/be/src/runtime/bufferpool/reservation_tracker.h @@ -20,7 +20,6 @@ #include -#include #include #include "common/status.h" @@ -93,15 +92,15 @@ public: /// Initializes the root tracker with the given reservation limit in bytes. The initial /// reservation is 0. - /// if 'profile' is not NULL, the counters defined in ReservationTrackerCounters are + /// if 'profile' is not nullptr, the counters defined in ReservationTrackerCounters are /// added to 'profile'. void InitRootTracker(RuntimeProfile* profile, int64_t reservation_limit); /// Initializes a new ReservationTracker with a parent. - /// If 'mem_tracker' is not NULL, reservations for this ReservationTracker and its + /// If 'mem_tracker' is not nullptr, reservations for this ReservationTracker and its /// children will be counted as consumption against 'mem_tracker'. /// 'reservation_limit' is the maximum reservation for this tracker in bytes. - /// if 'profile' is not NULL, the counters in 'counters_' are added to 'profile'. + /// if 'profile' is not nullptr, the counters in 'counters_' are added to 'profile'. void InitChildTracker(RuntimeProfile* profile, ReservationTracker* parent, MemTracker* mem_tracker, int64_t reservation_limit); @@ -178,13 +177,13 @@ private: return reservation_ - used_reservation_ - child_reservations_; } - /// Returns the parent's memtracker if 'parent_' is non-NULL, or NULL otherwise. + /// Returns the parent's memtracker if 'parent_' is non-nullptr, or nullptr otherwise. MemTracker* GetParentMemTracker() const { return parent_ == nullptr ? nullptr : parent_->mem_tracker_; } /// Initializes 'counters_', storing the counters in 'profile'. - /// If 'profile' is NULL, creates a dummy profile to store the counters. + /// If 'profile' is nullptr, creates a dummy profile to store the counters. void InitCounters(RuntimeProfile* profile, int64_t max_reservation); /// Internal helper for IncreaseReservation(). If 'use_existing_reservation' is true, @@ -261,16 +260,16 @@ private: /// A dummy profile to hold the counters in 'counters_' in the case that no profile /// is provided. - boost::scoped_ptr dummy_profile_; + std::unique_ptr dummy_profile_; /// The RuntimeProfile counters for this tracker. - /// All non-NULL if 'initialized_' is true. + /// All non-nullptr if 'initialized_' is true. ReservationTrackerCounters counters_; /// The parent of this tracker in the hierarchy. Does not change after initialization. ReservationTracker* parent_ = nullptr; - /// If non-NULL, reservations are counted as memory consumption against this tracker. + /// If non-nullptr, reservations are counted as memory consumption against this tracker. /// Does not change after initialization. Not owned. /// TODO: remove once all memory is accounted via ReservationTrackers. MemTracker* mem_tracker_ = nullptr; diff --git a/be/src/runtime/cache/cache_utils.h b/be/src/runtime/cache/cache_utils.h index c6cdab2e7e..ea413ff4da 100644 --- a/be/src/runtime/cache/cache_utils.h +++ b/be/src/runtime/cache/cache_utils.h @@ -22,7 +22,6 @@ #include #include -#include #include #include #include @@ -31,6 +30,7 @@ #include #include #include +#include namespace doris { @@ -51,7 +51,7 @@ struct CacheStat { inline long cache_time_second() { struct timeval tv; - gettimeofday(&tv, NULL); + gettimeofday(&tv, nullptr); return tv.tv_sec; } diff --git a/be/src/runtime/cache/result_cache.cpp b/be/src/runtime/cache/result_cache.cpp index b649491e55..da4c1fc283 100644 --- a/be/src/runtime/cache/result_cache.cpp +++ b/be/src/runtime/cache/result_cache.cpp @@ -130,7 +130,7 @@ void ResultCache::fetch(const PFetchCacheRequest* request, PFetchCacheResult* re for (auto part_it = part_rowbatch_list.begin(); part_it != part_rowbatch_list.end(); part_it++) { PCacheValue* srcValue = (*part_it)->get_value(); - if (srcValue != NULL) { + if (srcValue != nullptr) { PCacheValue* value = result->add_values(); value->CopyFrom(*srcValue); LOG(INFO) << "fetch cache partition key:" << srcValue->param().partition_key(); @@ -228,7 +228,7 @@ void ResultCache::prune() { << ", elasticity_size : " << _elasticity_size; ResultNode* result_node = _node_list.get_head(); while (_cache_size > _max_size) { - if (result_node == NULL) { + if (result_node == nullptr) { break; } result_node = find_min_time_node(result_node); diff --git a/be/src/runtime/cache/result_cache.h b/be/src/runtime/cache/result_cache.h index ffd3b6d83d..910cc191ee 100644 --- a/be/src/runtime/cache/result_cache.h +++ b/be/src/runtime/cache/result_cache.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_RUNTIME_RESULT_CACHE_H #define DORIS_BE_SRC_RUNTIME_RESULT_CACHE_H -#include #include #include #include @@ -45,7 +44,7 @@ typedef std::unordered_map ResultNodeMap; // a doubly linked list class, point to result node class ResultNodeList { public: - ResultNodeList() : _head(NULL), _tail(NULL), _node_count(0) {} + ResultNodeList() : _head(nullptr), _tail(nullptr), _node_count(0) {} virtual ~ResultNodeList() {} ResultNode* new_node(const UniqueId& sql_key) { return new ResultNode(sql_key); } diff --git a/be/src/runtime/cache/result_node.cpp b/be/src/runtime/cache/result_node.cpp index 4a56349e40..6ed0ab016d 100644 --- a/be/src/runtime/cache/result_node.cpp +++ b/be/src/runtime/cache/result_node.cpp @@ -27,7 +27,7 @@ bool compare_partition(const PartitionRowBatch* left_node, const PartitionRowBat //return new batch size,only include the size of PRowBatch void PartitionRowBatch::set_row_batch(const PCacheValue& value) { - if (_cache_value != NULL && !check_newer(value.param())) { + if (_cache_value != nullptr && !check_newer(value.param())) { LOG(WARNING) << "set old version data, cache ver:" << _cache_value->param().last_version() << ",cache time:" << _cache_value->param().last_version_time() << ", setdata ver:" << value.param().last_version() @@ -84,8 +84,9 @@ PCacheStatus ResultNode::update_partition(const PUpdateCacheRequest* request, } } -PCacheStatus ResultNode::update_sql_cache(const PUpdateCacheRequest *request, bool &is_update_firstkey) { - PartitionRowBatch* partition = NULL; +PCacheStatus ResultNode::update_sql_cache(const PUpdateCacheRequest* request, + bool& is_update_firstkey) { + PartitionRowBatch* partition = nullptr; if (request->values_size() > 1) { return PCacheStatus::PARAM_ERROR; } @@ -120,14 +121,15 @@ PCacheStatus ResultNode::update_sql_cache(const PUpdateCacheRequest *request, bo return PCacheStatus::CACHE_OK; } -PCacheStatus ResultNode::update_partition_cache(const PUpdateCacheRequest *request, bool &is_update_firstkey) { +PCacheStatus ResultNode::update_partition_cache(const PUpdateCacheRequest* request, + bool& is_update_firstkey) { PartitionKey first_key = kint64max; if (_partition_list.size() == 0) { is_update_firstkey = true; } else { first_key = (*(_partition_list.begin()))->get_partition_key(); } - PartitionRowBatch* partition = NULL; + PartitionRowBatch* partition = nullptr; for (int i = 0; i < request->values_size(); i++) { const PCacheValue& value = request->values(i); PartitionKey partition_key = value.param().partition_key(); @@ -322,8 +324,8 @@ void ResultNode::unlink() { if (_prev) { _prev->set_next(_next); } - _next = NULL; - _prev = NULL; + _next = nullptr; + _prev = nullptr; } } // namespace doris diff --git a/be/src/runtime/cache/result_node.h b/be/src/runtime/cache/result_node.h index d2c5d1d614..7add78f1d8 100644 --- a/be/src/runtime/cache/result_node.h +++ b/be/src/runtime/cache/result_node.h @@ -55,7 +55,7 @@ class PClearCacheRequest; class PartitionRowBatch { public: PartitionRowBatch(int64 partition_key) - : _partition_key(partition_key), _cache_value(NULL), _data_size(0) {} + : _partition_key(partition_key), _cache_value(nullptr), _data_size(0) {} ~PartitionRowBatch() {} @@ -118,10 +118,10 @@ typedef std::unordered_map PartitionRowBatchMa */ class ResultNode { public: - ResultNode() : _sql_key(0, 0), _prev(NULL), _next(NULL), _data_size(0) {} + ResultNode() : _sql_key(0, 0), _prev(nullptr), _next(nullptr), _data_size(0) {} ResultNode(const UniqueId& sql_key) - : _sql_key(sql_key), _prev(NULL), _next(NULL), _data_size(0) {} + : _sql_key(sql_key), _prev(nullptr), _next(nullptr), _data_size(0) {} virtual ~ResultNode() {} @@ -129,7 +129,8 @@ public: PCacheStatus fetch_partition(const PFetchCacheRequest* request, PartitionRowBatchList& rowBatchList, bool& is_hit_firstkey); PCacheStatus update_sql_cache(const PUpdateCacheRequest* request, bool& is_update_firstkey); - PCacheStatus update_partition_cache(const PUpdateCacheRequest* request, bool& is_update_firstkey); + PCacheStatus update_partition_cache(const PUpdateCacheRequest* request, + bool& is_update_firstkey); size_t prune_first(); void clear(); @@ -166,14 +167,14 @@ public: const CacheStat* get_first_stat() const { if (_partition_list.size() == 0) { - return NULL; + return nullptr; } return (*(_partition_list.begin()))->get_stat(); } const CacheStat* get_last_stat() const { if (_partition_list.size() == 0) { - return NULL; + return nullptr; } return (*(_partition_list.end()--))->get_stat(); } diff --git a/be/src/runtime/data_spliter.cpp b/be/src/runtime/data_spliter.cpp index 9205eb8c2c..4194af1a3f 100644 --- a/be/src/runtime/data_spliter.cpp +++ b/be/src/runtime/data_spliter.cpp @@ -157,7 +157,7 @@ Status DataSpliter::process_partition(RuntimeState* state, TupleRow* row, Partit void* partition_val = ctx->get_value(row); // construct a PartRangeKey PartRangeKey tmpPartKey; - if (NULL != partition_val) { + if (nullptr != partition_val) { RETURN_IF_ERROR( PartRangeKey::from_value(ctx->root()->type().type, partition_val, &tmpPartKey)); } else { @@ -183,10 +183,10 @@ Status DataSpliter::process_distribute(RuntimeState* state, TupleRow* row, for (auto& ctx : part->distributed_expr_ctxs()) { void* partition_val = ctx->get_value(row); - if (partition_val != NULL) { + if (partition_val != nullptr) { hash_val = RawValue::zlib_crc32(partition_val, ctx->root()->type(), hash_val); } else { - //NULL is treat as 0 when hash + //nullptr is treat as 0 when hash static const int INT_VALUE = 0; static const TypeDescriptor INT_TYPE(TYPE_INT); hash_val = RawValue::zlib_crc32(&INT_VALUE, INT_TYPE, hash_val); diff --git a/be/src/runtime/data_spliter.h b/be/src/runtime/data_spliter.h index 2d59be0fdd..1e26b03e9c 100644 --- a/be/src/runtime/data_spliter.h +++ b/be/src/runtime/data_spliter.h @@ -75,7 +75,7 @@ private: Status send_row(RuntimeState* state, const TabletDesc& desc, TupleRow* row, DppSink* dpp_sink); Status process_one_row(RuntimeState* state, TupleRow* row); - boost::scoped_ptr _obj_pool; + std::unique_ptr _obj_pool; const RowDescriptor& _row_desc; // Information used to partition data diff --git a/be/src/runtime/data_stream_mgr.cpp b/be/src/runtime/data_stream_mgr.cpp index 32ff126a6b..fbe2af1ca7 100644 --- a/be/src/runtime/data_stream_mgr.cpp +++ b/be/src/runtime/data_stream_mgr.cpp @@ -17,8 +17,8 @@ #include "runtime/data_stream_mgr.h" -#include #include +#include #include "gen_cpp/BackendService.h" #include "gen_cpp/PaloInternalService_types.h" @@ -36,7 +36,7 @@ DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(data_stream_receiver_count, MetricUnit::NOUNI DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(fragment_endpoint_count, MetricUnit::NOUNIT); using std::mutex; -using boost::shared_ptr; +using std::shared_ptr; using std::unique_lock; using std::lock_guard; @@ -67,7 +67,7 @@ shared_ptr DataStreamMgr::create_recvr( RuntimeState* state, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, int buffer_size, RuntimeProfile* profile, bool is_merging, std::shared_ptr sub_plan_query_statistics_recvr) { - DCHECK(profile != NULL); + DCHECK(profile != nullptr); VLOG_FILE << "creating receiver for fragment=" << fragment_instance_id << ", node=" << dest_node_id; shared_ptr recvr(new DataStreamRecvr( @@ -116,7 +116,7 @@ Status DataStreamMgr::transmit_data(const PTransmitDataParams* request, if (recvr == nullptr) { // The receiver may remove itself from the receiver map via deregister_recvr() // at any time without considering the remaining number of senders. - // As a consequence, find_recvr() may return an innocuous NULL if a thread + // As a consequence, find_recvr() may return an innocuous nullptr if a thread // calling deregister_recvr() beat the thread calling find_recvr() // in acquiring _lock. // TODO: Rethink the lifecycle of DataStreamRecvr to distinguish @@ -144,7 +144,7 @@ Status DataStreamMgr::transmit_data(const PTransmitDataParams* request, } Status DataStreamMgr::deregister_recvr(const TUniqueId& fragment_instance_id, PlanNodeId node_id) { - boost::shared_ptr targert_recvr; + std::shared_ptr targert_recvr; VLOG_QUERY << "deregister_recvr(): fragment_instance_id=" << fragment_instance_id << ", node=" << node_id; size_t hash_value = get_hash_value(fragment_instance_id, node_id); @@ -182,14 +182,14 @@ Status DataStreamMgr::deregister_recvr(const TUniqueId& fragment_instance_id, Pl void DataStreamMgr::cancel(const TUniqueId& fragment_instance_id) { VLOG_QUERY << "cancelling all streams for fragment=" << fragment_instance_id; - std::vector> recvrs; + std::vector> recvrs; { lock_guard l(_lock); FragmentStreamSet::iterator i = _fragment_stream_set.lower_bound(std::make_pair(fragment_instance_id, 0)); while (i != _fragment_stream_set.end() && i->first == fragment_instance_id) { shared_ptr recvr = find_recvr(i->first, i->second, false); - if (recvr == NULL) { + if (recvr == nullptr) { // keep going but at least log it std::stringstream err; err << "cancel(): missing in stream_map: fragment=" << i->first diff --git a/be/src/runtime/data_stream_mgr.h b/be/src/runtime/data_stream_mgr.h index 6bbeda843e..be370603c0 100644 --- a/be/src/runtime/data_stream_mgr.h +++ b/be/src/runtime/data_stream_mgr.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_RUNTIME_DATA_STREAM_MGR_H #define DORIS_BE_SRC_RUNTIME_DATA_STREAM_MGR_H -#include #include #include #include @@ -75,7 +74,7 @@ public: // single stream. // Ownership of the receiver is shared between this DataStream mgr instance and the // caller. - boost::shared_ptr create_recvr( + std::shared_ptr create_recvr( RuntimeState* state, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, int buffer_size, RuntimeProfile* profile, bool is_merging, @@ -98,7 +97,7 @@ private: // create_recvr(). // we don't want to create a map, DataStreamRecvr*>, // because that requires a bunch of copying of ids for lookup - typedef std::unordered_multimap> StreamMap; + typedef std::unordered_multimap> StreamMap; StreamMap _receiver_map; // less-than ordering for pair @@ -123,10 +122,10 @@ private: FragmentStreamSet _fragment_stream_set; // Return the receiver for given fragment_instance_id/node_id, - // or NULL if not found. If 'acquire_lock' is false, assumes _lock is already being + // or nullptr if not found. If 'acquire_lock' is false, assumes _lock is already being // held and won't try to acquire it. - boost::shared_ptr find_recvr(const TUniqueId& fragment_instance_id, - PlanNodeId node_id, bool acquire_lock = true); + std::shared_ptr find_recvr(const TUniqueId& fragment_instance_id, + PlanNodeId node_id, bool acquire_lock = true); // Remove receiver block for fragment_instance_id/node_id from the map. Status deregister_recvr(const TUniqueId& fragment_instance_id, PlanNodeId node_id); diff --git a/be/src/runtime/data_stream_recvr.cc b/be/src/runtime/data_stream_recvr.cc index 49b2495004..962395ad26 100644 --- a/be/src/runtime/data_stream_recvr.cc +++ b/be/src/runtime/data_stream_recvr.cc @@ -40,7 +40,7 @@ using std::make_pair; using std::condition_variable; using std::mutex; -using boost::scoped_ptr; +using std::unique_ptr; using std::unique_lock; using std::try_lock; using std::lock_guard; @@ -127,7 +127,7 @@ private: // The batch that was most recently returned via get_batch(), i.e. the current batch // from this queue being processed by a consumer. Is destroyed when the next batch // is retrieved. - scoped_ptr _current_batch; + unique_ptr _current_batch; // Set to true when the first batch has been received bool _received_first_batch; @@ -154,14 +154,14 @@ Status DataStreamRecvr::SenderQueue::get_batch(RowBatch** next_batch) { // Don't count time spent waiting on the sender as active time. CANCEL_SAFE_SCOPED_TIMER(_recvr->_data_arrival_timer, &_is_cancelled); CANCEL_SAFE_SCOPED_TIMER( - _received_first_batch ? NULL : _recvr->_first_batch_wait_total_timer, + _received_first_batch ? nullptr : _recvr->_first_batch_wait_total_timer, &_is_cancelled); _data_arrival_cv.wait(l); } // _cur_batch must be replaced with the returned batch. _current_batch.reset(); - *next_batch = NULL; + *next_batch = nullptr; if (_is_cancelled) { return Status::Cancelled("Cancelled"); } @@ -242,7 +242,7 @@ void DataStreamRecvr::SenderQueue::add_batch(const PRowBatch& pb_batch, int be_n return; } - RowBatch* batch = NULL; + RowBatch* batch = nullptr; { SCOPED_TIMER(_recvr->_deserialize_row_batch_timer); // Note: if this function makes a row batch, the batch *must* be added @@ -424,7 +424,7 @@ void DataStreamRecvr::transfer_all_resources(RowBatch* transfer_batch) { _merger->transfer_all_resources(transfer_batch); } else { for (SenderQueue* sender_queue : _sender_queues) { - if (sender_queue->current_batch() != NULL) { + if (sender_queue->current_batch() != nullptr) { sender_queue->current_batch()->transfer_resource_ownership(transfer_batch); } } @@ -467,7 +467,7 @@ DataStreamRecvr::DataStreamRecvr( } Status DataStreamRecvr::get_next(RowBatch* output_batch, bool* eos) { - DCHECK(_merger.get() != NULL); + DCHECK(_merger.get() != nullptr); return _merger->get_next(output_batch, eos); } @@ -501,14 +501,14 @@ void DataStreamRecvr::close() { // Remove this receiver from the DataStreamMgr that created it. // TODO: log error msg _mgr->deregister_recvr(fragment_instance_id(), dest_node_id()); - _mgr = NULL; + _mgr = nullptr; _merger.reset(); // TODO: Maybe shared tracker doesn't need to be reset manually _mem_tracker.reset(); } DataStreamRecvr::~DataStreamRecvr() { - DCHECK(_mgr == NULL) << "Must call close()"; + DCHECK(_mgr == nullptr) << "Must call close()"; } Status DataStreamRecvr::get_batch(RowBatch** next_batch) { diff --git a/be/src/runtime/data_stream_recvr.h b/be/src/runtime/data_stream_recvr.h index 6347017be2..3e4806c6c2 100644 --- a/be/src/runtime/data_stream_recvr.h +++ b/be/src/runtime/data_stream_recvr.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_RUNTIME_DATA_STREAM_RECVR_H #define DORIS_BE_SRC_RUNTIME_DATA_STREAM_RECVR_H -#include #include #include "common/object_pool.h" @@ -73,7 +72,7 @@ public: // Returns next row batch in data stream; blocks if there aren't any. // Retains ownership of the returned batch. The caller must acquire data from the - // returned batch before the next call to get_batch(). A NULL returned batch indicated + // returned batch before the next call to get_batch(). A nullptr returned batch indicated // eos. Must only be called if _is_merging is false. // TODO: This is currently only exposed to the non-merging version of the exchange. // Refactor so both merging and non-merging exchange use get_next(RowBatch*, bool* eos). @@ -167,7 +166,7 @@ private: std::vector _sender_queues; // SortedRunMerger used to merge rows from different senders. - boost::scoped_ptr _merger; + std::unique_ptr _merger; std::vector> _child_mergers; diff --git a/be/src/runtime/data_stream_sender.cpp b/be/src/runtime/data_stream_sender.cpp index d651159e1f..31c21ab8cb 100644 --- a/be/src/runtime/data_stream_sender.cpp +++ b/be/src/runtime/data_stream_sender.cpp @@ -21,9 +21,8 @@ #include #include -#include -#include #include +#include #include "common/config.h" #include "common/logging.h" @@ -171,8 +170,8 @@ Status DataStreamSender::Channel::add_row(TupleRow* row) { const std::vector& descs = _row_desc.tuple_descriptors(); for (int i = 0; i < descs.size(); ++i) { - if (UNLIKELY(row->get_tuple(i) == NULL)) { - dest->set_tuple(i, NULL); + if (UNLIKELY(row->get_tuple(i) == nullptr)) { + dest->set_tuple(i, nullptr); } else { dest->set_tuple(i, row->get_tuple(i)->deep_copy(*descs[i], _batch->tuple_data_pool())); } @@ -198,9 +197,8 @@ Status DataStreamSender::Channel::send_current_batch(bool eos) { } Status DataStreamSender::Channel::send_local_batch(bool eos) { - boost::shared_ptr recvr = - _parent->state()->exec_env()->stream_mgr()->find_recvr(_fragment_instance_id, - _dest_node_id); + std::shared_ptr recvr = _parent->state()->exec_env()->stream_mgr()->find_recvr( + _fragment_instance_id, _dest_node_id); if (recvr != nullptr) { recvr->add_batch(_batch.get(), _parent->_sender_id, true); if (eos) { @@ -213,9 +211,8 @@ Status DataStreamSender::Channel::send_local_batch(bool eos) { } Status DataStreamSender::Channel::send_local_batch(RowBatch* batch, bool use_move) { - boost::shared_ptr recvr = - _parent->state()->exec_env()->stream_mgr()->find_recvr(_fragment_instance_id, - _dest_node_id); + std::shared_ptr recvr = _parent->state()->exec_env()->stream_mgr()->find_recvr( + _fragment_instance_id, _dest_node_id); if (recvr != nullptr) { recvr->add_batch(batch, _parent->_sender_id, use_move); COUNTER_UPDATE(_parent->_local_bytes_send_counter, batch->total_byte_size()); @@ -230,7 +227,7 @@ Status DataStreamSender::Channel::close_internal() { VLOG_RPC << "Channel::close() instance_id=" << _fragment_instance_id << " dest_node=" << _dest_node_id << " #rows= " << ((_batch == nullptr) ? 0 : _batch->num_rows()); - if (_batch != NULL && _batch->num_rows() > 0) { + if (_batch != nullptr && _batch->num_rows() > 0) { RETURN_IF_ERROR(send_current_batch(true)); } else { RETURN_IF_ERROR(send_batch(nullptr, true)); @@ -265,9 +262,9 @@ DataStreamSender::DataStreamSender(ObjectPool* pool, int sender_id, const RowDes _current_pb_batch(&_pb_batch1), _pool(pool), _sender_id(sender_id), - _serialize_batch_timer(NULL), - _bytes_sent_counter(NULL), - _local_bytes_send_counter(NULL) {} + _serialize_batch_timer(nullptr), + _bytes_sent_counter(nullptr), + _local_bytes_send_counter(nullptr) {} DataStreamSender::DataStreamSender(ObjectPool* pool, int sender_id, const RowDescriptor& row_desc, const TDataStreamSink& sink, @@ -275,13 +272,13 @@ DataStreamSender::DataStreamSender(ObjectPool* pool, int sender_id, const RowDes int per_channel_buffer_size, bool send_query_statistics_with_every_batch) : _row_desc(row_desc), - _profile(NULL), + _profile(nullptr), _current_pb_batch(&_pb_batch1), _pool(pool), _sender_id(sender_id), - _serialize_batch_timer(NULL), - _bytes_sent_counter(NULL), - _local_bytes_send_counter(NULL), + _serialize_batch_timer(nullptr), + _bytes_sent_counter(nullptr), + _local_bytes_send_counter(nullptr), _current_channel_idx(0), _part_type(sink.output_partition.type), _ignore_not_found(sink.__isset.ignore_not_found ? sink.ignore_not_found : true), @@ -413,7 +410,7 @@ DataStreamSender::~DataStreamSender() { } Status DataStreamSender::open(RuntimeState* state) { - DCHECK(state != NULL); + DCHECK(state != nullptr); RETURN_IF_ERROR(Expr::open(_partition_expr_ctxs, state)); for (auto iter : _partition_infos) { RETURN_IF_ERROR(iter->open(state)); @@ -550,7 +547,7 @@ Status DataStreamSender::find_partition(RuntimeState* state, TupleRow* row, Part void* partition_val = ctx->get_value(row); // construct a PartRangeKey PartRangeKey tmpPartKey; - if (NULL != partition_val) { + if (nullptr != partition_val) { RETURN_IF_ERROR( PartRangeKey::from_value(ctx->root()->type().type, partition_val, &tmpPartKey)); } else { @@ -584,10 +581,10 @@ Status DataStreamSender::process_distribute(RuntimeState* state, TupleRow* row, uint32_t hash_val = 0; for (auto& ctx : part->distributed_expr_ctxs()) { void* partition_val = ctx->get_value(row); - if (partition_val != NULL) { + if (partition_val != nullptr) { hash_val = RawValue::zlib_crc32(partition_val, ctx->root()->type(), hash_val); } else { - //NULL is treat as 0 when hash + //nullptr is treat as 0 when hash static const int INT_VALUE = 0; static const TypeDescriptor INT_TYPE(TYPE_INT); hash_val = RawValue::zlib_crc32(&INT_VALUE, INT_TYPE, hash_val); diff --git a/be/src/runtime/data_stream_sender.h b/be/src/runtime/data_stream_sender.h index 71bc41f23d..145b3a7f4e 100644 --- a/be/src/runtime/data_stream_sender.h +++ b/be/src/runtime/data_stream_sender.h @@ -195,7 +195,7 @@ protected: int64_t _packet_seq; // we're accumulating rows into this batch - boost::scoped_ptr _batch; + std::unique_ptr _batch; bool _need_close; int _be_number; diff --git a/be/src/runtime/datetime_value.cpp b/be/src/runtime/datetime_value.cpp index 45f5374906..9ec8ccbb6a 100644 --- a/be/src/runtime/datetime_value.cpp +++ b/be/src/runtime/datetime_value.cpp @@ -35,9 +35,9 @@ const uint64_t log_10_int[] = {1, 10, 100, 1000, static int s_days_in_month[13] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; static const char* s_ab_month_name[] = {"", "Jan", "Feb", "Mar", "Apr", "May", "Jun", - "Jul", "Aug", "Sep", "Oct", "Nov", "Dec", NULL}; + "Jul", "Aug", "Sep", "Oct", "Nov", "Dec", nullptr}; -static const char* s_ab_day_name[] = {"Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun", NULL}; +static const char* s_ab_day_name[] = {"Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun", nullptr}; uint8_t mysql_week_mode(uint32_t mode) { mode &= 7; @@ -638,7 +638,7 @@ int DateTimeValue::compute_format_len(const char* format, int len) { bool DateTimeValue::to_format_string(const char* format, int len, char* to) const { char buf[64]; - char* pos = NULL; + char* pos = nullptr; const char* ptr = format; const char* end = format + len; char ch = '\0'; @@ -960,7 +960,7 @@ uint8_t DateTimeValue::calc_weekday(uint64_t day_nr, bool is_sunday_first_day) { return (day_nr + 5L + (is_sunday_first_day ? 1L : 0L)) % 7; } -// TODO(zhaochun): Think endptr is NULL +// TODO(zhaochun): Think endptr is nullptr // Return true if convert to a integer success. Otherwise false. static bool str_to_int64(const char* ptr, const char** endptr, int64_t* ret) { const static uint64_t MAX_NEGATIVE_NUMBER = 0x8000000000000000; @@ -1035,7 +1035,7 @@ static int find_in_lib(const char* lib[], const char* str, const char* end) { int pos = 0; int find_count = 0; int find_pos = 0; - for (; lib[pos] != NULL; ++pos) { + for (; lib[pos] != nullptr; ++pos) { const char* i = str; const char* j = lib[pos]; while (i < end && *j) { @@ -1105,7 +1105,7 @@ bool DateTimeValue::from_date_format_str(const char* format, int format_len, con } // Check switch if (*ptr == '%' && ptr + 1 < end) { - const char* tmp = NULL; + const char* tmp = nullptr; int64_t int_value = 0; ptr++; switch (*ptr++) { @@ -1600,7 +1600,7 @@ bool DateTimeValue::from_unixtime(int64_t timestamp, const cctz::time_zone& ctz) const char* DateTimeValue::month_name() const { if (_month < 1 || _month > 12) { - return NULL; + return nullptr; } return s_month_name[_month]; } @@ -1608,14 +1608,14 @@ const char* DateTimeValue::month_name() const { const char* DateTimeValue::day_name() const { int day = weekday(); if (day < 0 || day >= 7) { - return NULL; + return nullptr; } return s_day_name[day]; } DateTimeValue DateTimeValue::local_time() { DateTimeValue value; - value.from_unixtime(time(NULL), TimezoneUtils::default_time_zone); + value.from_unixtime(time(nullptr), TimezoneUtils::default_time_zone); return value; } diff --git a/be/src/runtime/datetime_value.h b/be/src/runtime/datetime_value.h index 9ddb1fab46..096a24d096 100644 --- a/be/src/runtime/datetime_value.h +++ b/be/src/runtime/datetime_value.h @@ -151,10 +151,10 @@ constexpr size_t max_char_length(const char* const* name, size_t end) { static constexpr const char* s_month_name[] = { "", "January", "February", "March", "April", "May", "June", - "July", "August", "September", "October", "November", "December", NULL}; + "July", "August", "September", "October", "November", "December", nullptr}; static constexpr const char* s_day_name[] = {"Monday", "Tuesday", "Wednesday", "Thursday", - "Friday", "Saturday", "Sunday", NULL}; + "Friday", "Saturday", "Sunday", nullptr}; static constexpr size_t MAX_DAY_NAME_LEN = max_char_length(s_day_name, std::size(s_day_name)); static constexpr size_t MAX_MONTH_NAME_LEN = max_char_length(s_month_name, std::size(s_month_name)); @@ -177,8 +177,8 @@ public: explicit DateTimeValue(int64_t t) { from_date_int64(t); } - void set_time(uint32_t year, uint32_t month, uint32_t day, uint32_t hour, - uint32_t minute, uint32_t second, uint32_t microsecond); + void set_time(uint32_t year, uint32_t month, uint32_t day, uint32_t hour, uint32_t minute, + uint32_t second, uint32_t microsecond); // Converted from Olap Date or Datetime bool from_olap_datetime(uint64_t datetime) { @@ -187,7 +187,8 @@ public: uint64_t date = datetime / 1000000; uint64_t time = datetime % 1000000; - auto [year, month, day, hour, minute, second, microsecond] = std::tuple{0,0,0,0,0,0,0}; + auto [year, month, day, hour, minute, second, microsecond] = + std::tuple {0, 0, 0, 0, 0, 0, 0}; year = date / 10000; date %= 10000; month = date / 100; @@ -211,7 +212,8 @@ public: _neg = 0; _type = TIME_DATE; - auto [year, month, day, hour, minute, second, microsecond] = std::tuple{0,0,0,0,0,0,0}; + auto [year, month, day, hour, minute, second, microsecond] = + std::tuple {0, 0, 0, 0, 0, 0, 0}; day = date & 0x1f; date >>= 5; @@ -275,7 +277,7 @@ public: // Return true if range or date is invalid static bool check_range(uint32_t year, uint32_t month, uint32_t day, uint32_t hour, - uint32_t minute, uint32_t second, uint32_t microsecond, uint16_t type); + uint32_t minute, uint32_t second, uint32_t microsecond, uint16_t type); static bool check_date(uint32_t year, uint32_t month, uint32_t day); @@ -346,7 +348,8 @@ public: int64_t to_int64() const; bool check_range_and_set_time(uint32_t year, uint32_t month, uint32_t day, uint32_t hour, - uint32_t minute, uint32_t second, uint32_t microsecond, uint16_t type) { + uint32_t minute, uint32_t second, uint32_t microsecond, + uint16_t type) { if (check_range(year, month, day, hour, minute, second, microsecond, type)) { return false; } @@ -555,8 +558,10 @@ public: int type() const { return _type; } - bool is_valid_date() const { return !check_range(_year, _month, _day, - _hour, _minute, _second, _microsecond, _type) && _month > 0 && _day > 0; } + bool is_valid_date() const { + return !check_range(_year, _month, _day, _hour, _minute, _second, _microsecond, _type) && + _month > 0 && _day > 0; + } private: // Used to make sure sizeof DateTimeValue diff --git a/be/src/runtime/descriptors.cpp b/be/src/runtime/descriptors.cpp index 9b3a28ecc8..ab4fa972c2 100644 --- a/be/src/runtime/descriptors.cpp +++ b/be/src/runtime/descriptors.cpp @@ -177,14 +177,14 @@ std::string ODBCTableDescriptor::debug_string() const { TupleDescriptor::TupleDescriptor(const TTupleDescriptor& tdesc) : _id(tdesc.id), - _table_desc(NULL), + _table_desc(nullptr), _byte_size(tdesc.byteSize), _num_null_bytes(tdesc.numNullBytes), _num_materialized_slots(0), _slots(), _has_varlen_slots(false) { if (false == tdesc.__isset.numNullSlots) { - //be compatible for existing tables with no NULL value + //be compatible for existing tables with no nullptr value _num_null_slots = 0; } else { _num_null_slots = tdesc.numNullSlots; @@ -193,14 +193,14 @@ TupleDescriptor::TupleDescriptor(const TTupleDescriptor& tdesc) TupleDescriptor::TupleDescriptor(const PTupleDescriptor& pdesc) : _id(pdesc.id()), - _table_desc(NULL), + _table_desc(nullptr), _byte_size(pdesc.byte_size()), _num_null_bytes(pdesc.num_null_bytes()), _num_materialized_slots(0), _slots(), _has_varlen_slots(false) { if (!pdesc.has_num_null_slots()) { - //be compatible for existing tables with no NULL value + //be compatible for existing tables with no nullptr value _num_null_slots = 0; } else { _num_null_slots = pdesc.num_null_slots(); @@ -257,7 +257,7 @@ void TupleDescriptor::to_protobuf(PTupleDescriptor* ptuple) const { std::string TupleDescriptor::debug_string() const { std::stringstream out; out << "Tuple(id=" << _id << " size=" << _byte_size; - if (_table_desc != NULL) { + if (_table_desc != nullptr) { //out << " " << _table_desc->debug_string(); } @@ -288,7 +288,7 @@ RowDescriptor::RowDescriptor(const DescriptorTbl& desc_tbl, const std::vectornum_materialized_slots(); _num_null_slots += tupleDesc->num_null_slots(); _tuple_desc_map.push_back(tupleDesc); - DCHECK(_tuple_desc_map.back() != NULL); + DCHECK(_tuple_desc_map.back() != nullptr); } _num_null_bytes = (_num_null_slots + 7) / 8; @@ -461,12 +461,11 @@ std::string RowDescriptor::debug_string() const { return ss.str(); } - int RowDescriptor::get_column_id(int slot_id) const { int column_id_counter = 0; - for(const auto tuple_desc:_tuple_desc_map) { - for(const auto slot:tuple_desc->slots()) { - if(slot->id() == slot_id) { + for (const auto tuple_desc : _tuple_desc_map) { + for (const auto slot : tuple_desc->slots()) { + if (slot->id() == slot_id) { return column_id_counter; } column_id_counter++; @@ -482,7 +481,7 @@ Status DescriptorTbl::create(ObjectPool* pool, const TDescriptorTable& thrift_tb // deserialize table descriptors first, they are being referenced by tuple descriptors for (size_t i = 0; i < thrift_tbl.tableDescriptors.size(); ++i) { const TTableDescriptor& tdesc = thrift_tbl.tableDescriptors[i]; - TableDescriptor* desc = NULL; + TableDescriptor* desc = nullptr; switch (tdesc.tableType) { case TTableType::MYSQL_TABLE: @@ -520,7 +519,7 @@ Status DescriptorTbl::create(ObjectPool* pool, const TDescriptorTable& thrift_tb // fix up table pointer if (tdesc.__isset.tableId) { desc->_table_desc = (*tbl)->get_table_descriptor(tdesc.tableId); - DCHECK(desc->_table_desc != NULL); + DCHECK(desc->_table_desc != nullptr); } (*tbl)->_tuple_desc_map[tdesc.id] = desc; @@ -548,7 +547,7 @@ TableDescriptor* DescriptorTbl::get_table_descriptor(TableId id) const { TableDescriptorMap::const_iterator i = _tbl_desc_map.find(id); if (i == _tbl_desc_map.end()) { - return NULL; + return nullptr; } else { return i->second; } @@ -559,7 +558,7 @@ TupleDescriptor* DescriptorTbl::get_tuple_descriptor(TupleId id) const { TupleDescriptorMap::const_iterator i = _tuple_desc_map.find(id); if (i == _tuple_desc_map.end()) { - return NULL; + return nullptr; } else { return i->second; } @@ -570,7 +569,7 @@ SlotDescriptor* DescriptorTbl::get_slot_descriptor(SlotId id) const { SlotDescriptorMap::const_iterator i = _slot_desc_map.find(id); if (i == _slot_desc_map.end()) { - return NULL; + return nullptr; } else { return i->second; } diff --git a/be/src/runtime/disk_io_mgr.cc b/be/src/runtime/disk_io_mgr.cc index 9841022115..9b9b350fee 100644 --- a/be/src/runtime/disk_io_mgr.cc +++ b/be/src/runtime/disk_io_mgr.cc @@ -30,8 +30,7 @@ using std::endl; using std::lock_guard; using std::unique_lock; using std::mutex; -using boost::thread; -using boost::thread_group; +using std::thread; // Returns the ceil of value/divisor static int64_t bit_ceil(int64_t value, int64_t divisor) { @@ -92,12 +91,12 @@ const int DiskIoMgr::DEFAULT_QUEUE_CAPACITY = 2; // } // DiskIoMgr::HdfsCachedFileHandle::~HdfsCachedFileHandle() { -// if (_hdfs_file != NULL && _fs != NULL) { +// if (_hdfs_file != nullptr && _fs != nullptr) { // VLOG_FILE << "hdfsCloseFile() fid=" << _hdfs_file; // hdfsCloseFile(_fs, _hdfs_file); // } -// _fs = NULL; -// _hdfs_file = NULL; +// _fs = nullptr; +// _hdfs_file = nullptr; // } // This class provides a cache of RequestContext objects. RequestContexts are recycled. @@ -199,14 +198,14 @@ string DiskIoMgr::debug_string() { } DiskIoMgr::BufferDescriptor::BufferDescriptor(DiskIoMgr* io_mgr) - : _io_mgr(io_mgr), _reader(NULL), _buffer(NULL) {} + : _io_mgr(io_mgr), _reader(nullptr), _buffer(nullptr) {} void DiskIoMgr::BufferDescriptor::reset(RequestContext* reader, ScanRange* range, char* buffer, int64_t buffer_len) { - DCHECK(_io_mgr != NULL); - DCHECK(_buffer == NULL); - DCHECK(range != NULL); - DCHECK(buffer != NULL); + DCHECK(_io_mgr != nullptr); + DCHECK(_buffer == nullptr); + DCHECK(range != nullptr); + DCHECK(buffer != nullptr); DCHECK_GE(buffer_len, 0); _reader = reader; _scan_range = range; @@ -218,13 +217,13 @@ void DiskIoMgr::BufferDescriptor::reset(RequestContext* reader, ScanRange* range } void DiskIoMgr::BufferDescriptor::return_buffer() { - DCHECK(_io_mgr != NULL); + DCHECK(_io_mgr != nullptr); _io_mgr->return_buffer(this); } void DiskIoMgr::BufferDescriptor::set_mem_tracker(std::shared_ptr tracker) { // Cached buffers don't count towards mem usage. - if (_scan_range->_cached_buffer != NULL) { + if (_scan_range->_cached_buffer != nullptr) { return; } if (_mem_tracker.get() == tracker.get()) { @@ -267,7 +266,7 @@ DiskIoMgr::DiskIoMgr() : _num_threads_per_disk(config::num_threads_per_disk), _max_buffer_size(config::read_size), _min_buffer_size(config::min_buffer_size), - _cached_read_options(NULL), + _cached_read_options(nullptr), _shut_down(false), _total_bytes_read_counter(TUnit::BYTES), _read_timer(TUnit::TIME_NS) @@ -288,7 +287,7 @@ DiskIoMgr::DiskIoMgr(int num_local_disks, int threads_per_disk, int min_buffer_s : _num_threads_per_disk(threads_per_disk), _max_buffer_size(max_buffer_size), _min_buffer_size(min_buffer_size), - _cached_read_options(NULL), + _cached_read_options(nullptr), _shut_down(false), _total_bytes_read_counter(TUnit::BYTES), _read_timer(TUnit::TIME_NS) @@ -309,7 +308,7 @@ DiskIoMgr::~DiskIoMgr() { _shut_down = true; // Notify all worker threads and shut them down. for (int i = 0; i < _disk_queues.size(); ++i) { - if (_disk_queues[i] == NULL) { + if (_disk_queues[i] == nullptr) { continue; } { @@ -323,7 +322,7 @@ DiskIoMgr::~DiskIoMgr() { _disk_thread_group.join_all(); for (int i = 0; i < _disk_queues.size(); ++i) { - if (_disk_queues[i] == NULL) { + if (_disk_queues[i] == nullptr) { continue; } int disk_id = _disk_queues[i]->disk_id; @@ -335,7 +334,8 @@ DiskIoMgr::~DiskIoMgr() { } } - DCHECK(_request_context_cache.get() == NULL || _request_context_cache->validate_all_inactive()) + DCHECK(_request_context_cache.get() == nullptr || + _request_context_cache->validate_all_inactive()) << endl << debug_string(); DCHECK_EQ(_num_buffers_in_readers, 0); @@ -353,7 +353,7 @@ DiskIoMgr::~DiskIoMgr() { } /* - * if (_cached_read_options != NULL) { + * if (_cached_read_options != nullptr) { * hadoopRzOptionsFree(_cached_read_options); * } */ @@ -387,18 +387,18 @@ Status DiskIoMgr::init(const std::shared_ptr& process_mem_tracker) { // _disk_thread_group.AddThread(new Thread("disk-io-mgr", ss.str(), // &DiskIoMgr::work_loop, this, _disk_queues[i])); _disk_thread_group.add_thread( - new boost::thread(std::bind(&DiskIoMgr::work_loop, this, _disk_queues[i]))); + new std::thread(std::bind(&DiskIoMgr::work_loop, this, _disk_queues[i]))); } } _request_context_cache.reset(new RequestContextCache(this)); // _cached_read_options = hadoopRzOptionsAlloc(); - // DCHECK(_cached_read_options != NULL); + // DCHECK(_cached_read_options != nullptr); // Disable checksum for cached reads. // int ret = hadoopRzOptionsSetSkipChecksum(_cached_read_options, true); // DCHECK_EQ(ret, 0); // Disable automatic fallback for cached reads. - // ret = hadoopRzOptionsSetByteBufferPool(_cached_read_options, NULL); + // ret = hadoopRzOptionsSetByteBufferPool(_cached_read_options, nullptr); // DCHECK_EQ(ret, 0); return Status::OK(); @@ -572,9 +572,9 @@ Status DiskIoMgr::add_scan_ranges(RequestContext* reader, const vector reader_lock(reader->_lock); @@ -604,7 +604,7 @@ Status DiskIoMgr::get_next_range(RequestContext* reader, ScanRange** range) { // // This range ended up not being cached. Loop again and pick up a new range. // reader->add_request_range(*range, false); // DCHECK(reader->validate()) << endl << reader->debug_string(); - // *range = NULL; + // *range = nullptr; // continue; // } @@ -612,12 +612,12 @@ Status DiskIoMgr::get_next_range(RequestContext* reader, ScanRange** range) { reader->_ready_to_start_ranges_cv.wait(reader_lock); } else { *range = reader->_ready_to_start_ranges.dequeue(); - DCHECK(*range != NULL); + DCHECK(*range != nullptr); int disk_id = (*range)->disk_id(); DCHECK_EQ(*range, reader->_disk_states[disk_id].next_scan_range_to_start()); - // Set this to NULL, the next time this disk runs for this reader, it will + // Set this to nullptr, the next time this disk runs for this reader, it will // get another range ready. - reader->_disk_states[disk_id].set_next_scan_range_to_start(NULL); + reader->_disk_states[disk_id].set_next_scan_range_to_start(nullptr); reader->schedule_scan_range(*range); break; } @@ -626,9 +626,9 @@ Status DiskIoMgr::get_next_range(RequestContext* reader, ScanRange** range) { } Status DiskIoMgr::read(RequestContext* reader, ScanRange* range, BufferDescriptor** buffer) { - DCHECK(range != NULL); - DCHECK(buffer != NULL); - *buffer = NULL; + DCHECK(range != nullptr); + DCHECK(buffer != nullptr); + *buffer = nullptr; if (range->len() > _max_buffer_size) { stringstream error_msg; @@ -641,28 +641,28 @@ Status DiskIoMgr::read(RequestContext* reader, ScanRange* range, BufferDescripto ranges.push_back(range); RETURN_IF_ERROR(add_scan_ranges(reader, ranges, true)); RETURN_IF_ERROR(range->get_next(buffer)); - DCHECK((*buffer) != NULL); + DCHECK((*buffer) != nullptr); DCHECK((*buffer)->eosr()); return Status::OK(); } void DiskIoMgr::return_buffer(BufferDescriptor* buffer_desc) { - DCHECK(buffer_desc != NULL); + DCHECK(buffer_desc != nullptr); if (!buffer_desc->_status.ok()) { - DCHECK(buffer_desc->_buffer == NULL); + DCHECK(buffer_desc->_buffer == nullptr); } RequestContext* reader = buffer_desc->_reader; - if (buffer_desc->_buffer != NULL) { - if (buffer_desc->_scan_range->_cached_buffer == NULL) { + if (buffer_desc->_buffer != nullptr) { + if (buffer_desc->_scan_range->_cached_buffer == nullptr) { // Not a cached buffer. Return the io buffer and update mem tracking. return_free_buffer(buffer_desc); } - buffer_desc->_buffer = NULL; + buffer_desc->_buffer = nullptr; --_num_buffers_in_readers; --reader->_num_buffers_in_reader; } else { - // A NULL buffer means there was an error in which case there is no buffer + // A nullptr buffer means there was an error in which case there is no buffer // to return. } @@ -676,7 +676,7 @@ void DiskIoMgr::return_buffer(BufferDescriptor* buffer_desc) { } void DiskIoMgr::return_buffer_desc(BufferDescriptor* desc) { - DCHECK(desc != NULL); + DCHECK(desc != nullptr); unique_lock lock(_free_buffers_lock); DCHECK(find(_free_buffer_descs.begin(), _free_buffer_descs.end(), desc) == _free_buffer_descs.end()); @@ -685,7 +685,7 @@ void DiskIoMgr::return_buffer_desc(BufferDescriptor* desc) { DiskIoMgr::BufferDescriptor* DiskIoMgr::get_buffer_desc(RequestContext* reader, ScanRange* range, char* buffer, int64_t buffer_size) { - BufferDescriptor* buffer_desc = NULL; + BufferDescriptor* buffer_desc = nullptr; { unique_lock lock(_free_buffers_lock); if (_free_buffer_descs.empty()) { @@ -710,7 +710,7 @@ char* DiskIoMgr::get_free_buffer(int64_t* buffer_size) { *buffer_size = (1 << idx) * _min_buffer_size; unique_lock lock(_free_buffers_lock); - char* buffer = NULL; + char* buffer = nullptr; if (_free_buffers[idx].empty()) { ++_num_allocated_buffers; // Update the process mem usage. This is checked the next time we start @@ -721,7 +721,7 @@ char* DiskIoMgr::get_free_buffer(int64_t* buffer_size) { buffer = _free_buffers[idx].front(); _free_buffers[idx].pop_front(); } - DCHECK(buffer != NULL); + DCHECK(buffer != nullptr); return buffer; } @@ -747,11 +747,11 @@ void DiskIoMgr::gc_io_buffers() { void DiskIoMgr::return_free_buffer(BufferDescriptor* desc) { return_free_buffer(desc->_buffer, desc->_buffer_len); desc->set_mem_tracker(nullptr); - desc->_buffer = NULL; + desc->_buffer = nullptr; } void DiskIoMgr::return_free_buffer(char* buffer, int64_t buffer_size) { - DCHECK(buffer != NULL); + DCHECK(buffer != nullptr); int idx = free_buffers_idx(buffer_size); DCHECK_EQ(bit_ceil(buffer_size, _min_buffer_size) & ~(1 << idx), 0) << "_buffer_size / _min_buffer_size should be power of 2, got buffer_size = " @@ -778,12 +778,12 @@ void DiskIoMgr::return_free_buffer(char* buffer, int64_t buffer_size) { bool DiskIoMgr::get_next_request_range(DiskQueue* disk_queue, RequestRange** range, RequestContext** request_context) { int disk_id = disk_queue->disk_id; - *range = NULL; + *range = nullptr; // This loops returns either with work to do or when the disk IoMgr shuts down. while (!_shut_down) { - *request_context = NULL; - RequestContext::PerDiskState* request_disk_state = NULL; + *request_context = nullptr; + RequestContext::PerDiskState* request_disk_state = nullptr; { unique_lock disk_lock(disk_queue->lock); @@ -803,7 +803,7 @@ bool DiskIoMgr::get_next_request_range(DiskQueue* disk_queue, RequestRange** ran // TODO: revisit. *request_context = disk_queue->request_contexts.front(); disk_queue->request_contexts.pop_front(); - DCHECK(*request_context != NULL); + DCHECK(*request_context != nullptr); request_disk_state = &((*request_context)->_disk_states[disk_id]); request_disk_state->increment_request_thread_and_dequeue(); } @@ -822,7 +822,7 @@ bool DiskIoMgr::get_next_request_range(DiskQueue* disk_queue, RequestRange** ran (*request_context)->_mem_tracker != nullptr ? (*request_context)->_mem_tracker->AnyLimitExceeded(MemLimit::HARD) : false; - // bool reader_limit_exceeded = (*request_context)->_mem_tracker != NULL + // bool reader_limit_exceeded = (*request_context)->_mem_tracker != nullptr // ? (*request_context)->_mem_tracker->limit_exceeded() : false; if (process_limit_exceeded || reader_limit_exceeded) { @@ -842,7 +842,7 @@ bool DiskIoMgr::get_next_request_range(DiskQueue* disk_queue, RequestRange** ran DCHECK_EQ((*request_context)->_state, RequestContext::Active) << (*request_context)->debug_string(); - if (request_disk_state->next_scan_range_to_start() == NULL && + if (request_disk_state->next_scan_range_to_start() == nullptr && !request_disk_state->unstarted_scan_ranges()->empty()) { // We don't have a range queued for this disk for what the caller should // read next. Populate that. We want to have one range waiting to minimize @@ -854,7 +854,7 @@ bool DiskIoMgr::get_next_request_range(DiskQueue* disk_queue, RequestRange** ran if ((*request_context)->_num_unstarted_scan_ranges == 0) { // All the ranges have been started, notify everyone blocked on get_next_range. - // Only one of them will get work so make sure to return NULL to the other + // Only one of them will get work so make sure to return nullptr to the other // caller threads. (*request_context)->_ready_to_start_ranges_cv.notify_all(); } else { @@ -883,7 +883,7 @@ bool DiskIoMgr::get_next_request_range(DiskQueue* disk_queue, RequestRange** ran } DCHECK_GT(request_disk_state->num_remaining_ranges(), 0); *range = request_disk_state->in_flight_ranges()->dequeue(); - DCHECK(*range != NULL); + DCHECK(*range != nullptr); // Now that we've picked a request range, put the context back on the queue so // another thread can pick up another request range for this context. @@ -924,13 +924,13 @@ void DiskIoMgr::handle_read_finished(DiskQueue* disk_queue, RequestContext* read RequestContext::PerDiskState& state = reader->_disk_states[disk_queue->disk_id]; DCHECK(reader->validate()) << endl << reader->debug_string(); DCHECK_GT(state.num_threads_in_op(), 0); - DCHECK(buffer->_buffer != NULL); + DCHECK(buffer->_buffer != nullptr); if (reader->_state == RequestContext::Cancelled) { state.decrement_request_thread_and_check_done(reader); DCHECK(reader->validate()) << endl << reader->debug_string(); return_free_buffer(buffer); - buffer->_buffer = NULL; + buffer->_buffer = nullptr; buffer->_scan_range->cancel(reader->_status); // Enqueue the buffer to use the scan range's buffer cleanup path. buffer->_scan_range->enqueue_buffer(buffer); @@ -938,7 +938,7 @@ void DiskIoMgr::handle_read_finished(DiskQueue* disk_queue, RequestContext* read } DCHECK_EQ(reader->_state, RequestContext::Active); - DCHECK(buffer->_buffer != NULL); + DCHECK(buffer->_buffer != nullptr); // Update the reader's scan ranges. There are a three cases here: // 1. Read error @@ -963,7 +963,7 @@ void DiskIoMgr::handle_read_finished(DiskQueue* disk_queue, RequestContext* read // For cached buffers, we can't close the range until the cached buffer is returned. // close() is called from DiskIoMgr::return_buffer(). /* - * if (scan_range->_cached_buffer == NULL) { + * if (scan_range->_cached_buffer == nullptr) { * scan_range->close(); * } */ @@ -990,9 +990,9 @@ void DiskIoMgr::work_loop(DiskQueue* disk_queue) { // 3. Perform the read or write as specified. // Cancellation checking needs to happen in both steps 1 and 3. while (!_shut_down) { - RequestContext* worker_context = NULL; + RequestContext* worker_context = nullptr; ; - RequestRange* range = NULL; + RequestRange* range = nullptr; if (!get_next_request_range(disk_queue, &range, &worker_context)) { DCHECK(_shut_down); @@ -1013,7 +1013,7 @@ void DiskIoMgr::work_loop(DiskQueue* disk_queue) { // This function reads the specified scan range associated with the // specified reader context and disk queue. void DiskIoMgr::read_range(DiskQueue* disk_queue, RequestContext* reader, ScanRange* range) { - char* buffer = NULL; + char* buffer = nullptr; int64_t bytes_remaining = range->_len - range->_bytes_read; DCHECK_GT(bytes_remaining, 0); int64_t buffer_size = std::min(bytes_remaining, static_cast(_max_buffer_size)); @@ -1059,12 +1059,12 @@ void DiskIoMgr::read_range(DiskQueue* disk_queue, RequestContext* reader, ScanRa // Validate more invariants. DCHECK_GT(reader->_num_used_buffers, 0); - DCHECK(range != NULL); - DCHECK(reader != NULL); - DCHECK(buffer != NULL); + DCHECK(range != nullptr); + DCHECK(reader != nullptr); + DCHECK(buffer != nullptr); BufferDescriptor* buffer_desc = get_buffer_desc(reader, range, buffer, buffer_size); - DCHECK(buffer_desc != NULL); + DCHECK(buffer_desc != nullptr); // No locks in this section. Only working on local vars. We don't want to hold a // lock across the read call. @@ -1084,7 +1084,7 @@ void DiskIoMgr::read_range(DiskQueue* disk_queue, RequestContext* reader, ScanRa buffer_desc->_status = range->read(buffer, &buffer_desc->_len, &buffer_desc->_eosr); buffer_desc->_scan_range_offset = range->_bytes_read - buffer_desc->_len; - if (reader->_bytes_read_counter != NULL) { + if (reader->_bytes_read_counter != nullptr) { COUNTER_UPDATE(reader->_bytes_read_counter, buffer_desc->_len); } @@ -1101,7 +1101,7 @@ void DiskIoMgr::read_range(DiskQueue* disk_queue, RequestContext* reader, ScanRa void DiskIoMgr::write(RequestContext* writer_context, WriteRange* write_range) { FILE* file_handle = fopen(write_range->file(), "rb+"); Status ret_status; - if (file_handle == NULL) { + if (file_handle == nullptr) { stringstream error_msg; error_msg << "fopen(" << write_range->_file << ", \"rb+\") failed with errno=" << errno << " description=" << get_str_err_msg(); @@ -1186,7 +1186,7 @@ Status DiskIoMgr::add_write_range(RequestContext* writer, WriteRange* write_rang /* * DiskIoMgr::HdfsCachedFileHandle* DiskIoMgr::OpenHdfsFile(const hdfsFS& fs, * const char* fname, int64_t mtime) { - * HdfsCachedFileHandle* fh = NULL; + * HdfsCachedFileHandle* fh = nullptr; * * // Check if a cached file handle exists and validate the mtime, if the mtime of the * // cached handle is not matching the mtime of the requested file, reopen. @@ -1204,7 +1204,7 @@ Status DiskIoMgr::add_write_range(RequestContext* writer, WriteRange* write_rang * if (!fh->ok()) { * VLOG_FILE << "Opening the file " << fname << " failed."; * delete fh; - * return NULL; + * return nullptr; * } * * return fh; diff --git a/be/src/runtime/disk_io_mgr.h b/be/src/runtime/disk_io_mgr.h index 89c487b4e1..9b0d330b94 100644 --- a/be/src/runtime/disk_io_mgr.h +++ b/be/src/runtime/disk_io_mgr.h @@ -18,11 +18,10 @@ #ifndef DORIS_BE_SRC_QUERY_RUNTIME_DISK_IO_MGR_H #define DORIS_BE_SRC_QUERY_RUNTIME_DISK_IO_MGR_H -#include -#include #include #include #include +#include #include #include @@ -36,6 +35,7 @@ #include "util/internal_queue.h" #include "util/metrics.h" #include "util/runtime_profile.h" +#include "util/thread_group.h" namespace doris { @@ -217,7 +217,7 @@ public: // is evicted. static void release(HdfsCachedFileHandle** h); - bool ok() const { return _hdfs_file != NULL; } + bool ok() const { return _hdfs_file != nullptr; } private: hdfsFS _fs; @@ -281,7 +281,7 @@ public: // true if the current scan range is complete bool _eosr; - // Status of the read to this buffer. if status is not ok, 'buffer' is NULL + // Status of the read to this buffer. if status is not ok, 'buffer' is nullptr Status _status; int64_t _scan_range_offset; @@ -308,7 +308,7 @@ public: RequestType::type request_type() const { return _request_type; } protected: - // Hadoop filesystem that contains _file, or set to NULL for local filesystem. + // Hadoop filesystem that contains _file, or set to nullptr for local filesystem. hdfsFS _fs; // Path to file being read or written. @@ -345,7 +345,7 @@ public: // must fall within the file bounds (offset >= 0 and offset + len <= file_length). // Resets this scan range object with the scan range description. void reset(hdfsFS fs, const char* file, int64_t len, int64_t offset, int disk_id, - bool try_cache, bool expected_local, int64_t mtime, void* metadata = NULL); + bool try_cache, bool expected_local, int64_t mtime, void* metadata = nullptr); void* meta_data() const { return _meta_data; } // bool try_cache() const { return _try_cache; } @@ -354,7 +354,7 @@ public: // Returns the next buffer for this scan range. buffer is an output parameter. // This function blocks until a buffer is ready or an error occurred. If this is - // called when all buffers have been returned, *buffer is set to NULL and Status::OK() + // called when all buffers have been returned, *buffer is set to nullptr and Status::OK() // is returned. // Only one thread can be in get_next() at any time. Status get_next(BufferDescriptor** buffer); @@ -591,7 +591,7 @@ public: // Returns the next unstarted scan range for this reader. When the range is returned, // the disk threads in the IoMgr will already have started reading from it. The // caller is expected to call ScanRange::get_next on the returned range. - // If there are no more unstarted ranges, NULL is returned. + // If there are no more unstarted ranges, nullptr is returned. // This call is blocking. Status get_next_range(RequestContext* reader, ScanRange** range); @@ -666,7 +666,7 @@ public: bool validate() const; // Given a FS handle, name and last modified time of the file, tries to open that file - // and return an instance of HdfsCachedFileHandle. In case of an error returns NULL. + // and return an instance of HdfsCachedFileHandle. In case of an error returns nullptr. // HdfsCachedFileHandle* OpenHdfsFile(const hdfsFS& fs, const char* fname, int64_t mtime); // When the file handle is no longer in use by the scan range, return it and try to @@ -707,7 +707,7 @@ private: // Thread group containing all the worker threads. // ThreadGroup _disk_thread_group; - boost::thread_group _disk_thread_group; + ThreadGroup _disk_thread_group; // Options object for cached hdfs reads. Set on startup and never modified. struct hadoopRzOptions* _cached_read_options; @@ -726,7 +726,7 @@ private: // active as well as those in the process of being cancelled. This is a cache // of context objects that get recycled to minimize object allocations and lock // contention. - boost::scoped_ptr _request_context_cache; + std::unique_ptr _request_context_cache; // Protects _free_buffers and _free_buffer_descs std::mutex _free_buffers_lock; @@ -798,7 +798,7 @@ private: // if buffer was acquired via get_free_buffer() (which it should have been). void return_free_buffer(char* buffer, int64_t buffer_size); - // Returns the buffer in desc (cannot be NULL), sets buffer to NULL and clears the + // Returns the buffer in desc (cannot be nullptr), sets buffer to nullptr and clears the // mem tracker. void return_free_buffer(BufferDescriptor* desc); diff --git a/be/src/runtime/disk_io_mgr_internal.h b/be/src/runtime/disk_io_mgr_internal.h index 897ab5df59..0d933f93ba 100644 --- a/be/src/runtime/disk_io_mgr_internal.h +++ b/be/src/runtime/disk_io_mgr_internal.h @@ -156,14 +156,14 @@ public: // in_flight_queue AND have not prepared a range by setting next_range_to_start. // The rule to make sure readers are scheduled correctly is to ensure anytime a // range is put on the in_flight_queue or anytime next_range_to_start is set to - // NULL, the reader is scheduled. + // nullptr, the reader is scheduled. // Adds range to in_flight_ranges, scheduling this reader on the disk threads // if necessary. // Reader lock must be taken before this. void schedule_scan_range(DiskIoMgr::ScanRange* range) { DCHECK_EQ(_state, Active); - DCHECK(range != NULL); + DCHECK(range != nullptr); RequestContext::PerDiskState& state = _disk_states[range->disk_id()]; state.in_flight_ranges()->enqueue(range); state.schedule_context(this, range->disk_id()); @@ -379,7 +379,7 @@ private: _num_remaining_ranges = 0; _is_on_queue = false; _num_threads_in_op = 0; - _next_scan_range_to_start = NULL; + _next_scan_range_to_start = nullptr; } private: @@ -392,7 +392,7 @@ private: // For each disk, keeps track if the context is on this disk's queue, indicating // the disk must do some work for this context. The disk needs to do work in 4 cases: // 1) in_flight_ranges is not empty, the disk needs to read for this reader. - // 2) next_range_to_start is NULL, the disk needs to prepare a scan range to be + // 2) next_range_to_start is nullptr, the disk needs to prepare a scan range to be // read next. // 3) the reader has been cancelled and this disk needs to participate in the // cleanup. @@ -425,7 +425,7 @@ private: // picks the next range to start. The range is set here and also added to the // _ready_to_start_ranges queue. The reader pulls from the queue in FIFO order, // so the ranges from different disks are round-robined. When the range is pulled - // off the _ready_to_start_ranges queue, it sets this variable to NULL, so the disk + // off the _ready_to_start_ranges queue, it sets this variable to nullptr, so the disk // knows to populate it again and add it to _ready_to_start_ranges i.e. it is used // as a flag by DiskIoMgr::GetNextScanRange to determine if it needs to add another // range to _ready_to_start_ranges. diff --git a/be/src/runtime/disk_io_mgr_reader_context.cc b/be/src/runtime/disk_io_mgr_reader_context.cc index dbd5aafeb2..984ae748de 100644 --- a/be/src/runtime/disk_io_mgr_reader_context.cc +++ b/be/src/runtime/disk_io_mgr_reader_context.cc @@ -53,8 +53,8 @@ void DiskIoMgr::RequestContext::cancel(const Status& status) { // four queues. for (int i = 0; i < _disk_states.size(); ++i) { RequestContext::PerDiskState& state = _disk_states[i]; - RequestRange* range = NULL; - while ((range = state.in_flight_ranges()->dequeue()) != NULL) { + RequestRange* range = nullptr; + while ((range = state.in_flight_ranges()->dequeue()) != nullptr) { if (range->request_type() == RequestType::READ) { static_cast(range)->cancel(status); } else { @@ -63,24 +63,24 @@ void DiskIoMgr::RequestContext::cancel(const Status& status) { } } - ScanRange* scan_range = NULL; - while ((scan_range = state.unstarted_scan_ranges()->dequeue()) != NULL) { + ScanRange* scan_range = nullptr; + while ((scan_range = state.unstarted_scan_ranges()->dequeue()) != nullptr) { scan_range->cancel(status); } - WriteRange* write_range = NULL; - while ((write_range = state.unstarted_write_ranges()->dequeue()) != NULL) { + WriteRange* write_range = nullptr; + while ((write_range = state.unstarted_write_ranges()->dequeue()) != nullptr) { write_callbacks.push_back(write_range->_callback); } } - ScanRange* range = NULL; - while ((range = _ready_to_start_ranges.dequeue()) != NULL) { + ScanRange* range = nullptr; + while ((range = _ready_to_start_ranges.dequeue()) != nullptr) { range->cancel(status); } - while ((range = _blocked_ranges.dequeue()) != NULL) { + while ((range = _blocked_ranges.dequeue()) != nullptr) { range->cancel(status); } - while ((range = _cached_ranges.dequeue()) != NULL) { + while ((range = _cached_ranges.dequeue()) != nullptr) { range->cancel(status); } @@ -120,10 +120,10 @@ void DiskIoMgr::RequestContext::add_request_range(DiskIoMgr::RequestRange* range state.unstarted_scan_ranges()->enqueue(scan_range); ++_num_unstarted_scan_ranges; } - // If next_scan_range_to_start is NULL, schedule this RequestContext so that it will - // be set. If it's not NULL, this context will be scheduled when GetNextRange() is + // If next_scan_range_to_start is nullptr, schedule this RequestContext so that it will + // be set. If it's not nullptr, this context will be scheduled when GetNextRange() is // invoked. - schedule_context = state.next_scan_range_to_start() == NULL; + schedule_context = state.next_scan_range_to_start() == nullptr; } else { DCHECK(range->request_type() == RequestType::WRITE); DCHECK(!schedule_immediately); @@ -143,10 +143,10 @@ void DiskIoMgr::RequestContext::add_request_range(DiskIoMgr::RequestRange* range DiskIoMgr::RequestContext::RequestContext(DiskIoMgr* parent, int num_disks) : _parent(parent), - _bytes_read_counter(NULL), - _read_timer(NULL), - _active_read_thread_counter(NULL), - _disks_accessed_bitmap(NULL), + _bytes_read_counter(nullptr), + _read_timer(nullptr), + _active_read_thread_counter(nullptr), + _disks_accessed_bitmap(nullptr), _state(Inactive), _disk_states(num_disks) {} @@ -155,10 +155,10 @@ void DiskIoMgr::RequestContext::reset(std::shared_ptr tracker) { DCHECK_EQ(_state, Inactive); _status = Status::OK(); - _bytes_read_counter = NULL; - _read_timer = NULL; - _active_read_thread_counter = NULL; - _disks_accessed_bitmap = NULL; + _bytes_read_counter = nullptr; + _read_timer = nullptr; + _active_read_thread_counter = nullptr; + _disks_accessed_bitmap = nullptr; _state = Active; _mem_tracker = std::move(tracker); diff --git a/be/src/runtime/disk_io_mgr_scan_range.cc b/be/src/runtime/disk_io_mgr_scan_range.cc index a7ffff7638..03a5437135 100644 --- a/be/src/runtime/disk_io_mgr_scan_range.cc +++ b/be/src/runtime/disk_io_mgr_scan_range.cc @@ -52,7 +52,7 @@ bool DiskIoMgr::ScanRange::enqueue_buffer(BufferDescriptor* buffer) { DCHECK(!_eosr_queued); if (_is_cancelled) { // Return the buffer, this range has been cancelled - if (buffer->_buffer != NULL) { + if (buffer->_buffer != nullptr) { ++_io_mgr->_num_buffers_in_readers; ++_reader->_num_buffers_in_reader; } @@ -79,7 +79,7 @@ bool DiskIoMgr::ScanRange::enqueue_buffer(BufferDescriptor* buffer) { } Status DiskIoMgr::ScanRange::get_next(BufferDescriptor** buffer) { - *buffer = NULL; + *buffer = nullptr; { unique_lock scan_range_lock(_lock); @@ -121,7 +121,7 @@ Status DiskIoMgr::ScanRange::get_next(BufferDescriptor** buffer) { Status status = (*buffer)->_status; if (!status.ok()) { (*buffer)->return_buffer(); - *buffer = NULL; + *buffer = nullptr; return status; } @@ -138,7 +138,7 @@ Status DiskIoMgr::ScanRange::get_next(BufferDescriptor** buffer) { _reader->_blocked_ranges.remove(this); cancel(_reader->_status); (*buffer)->return_buffer(); - *buffer = NULL; + *buffer = nullptr; return _status; } @@ -155,7 +155,7 @@ Status DiskIoMgr::ScanRange::get_next(BufferDescriptor** buffer) { void DiskIoMgr::ScanRange::cancel(const Status& status) { // Cancelling a range that was never started, ignore. - if (_io_mgr == NULL) { + if (_io_mgr == nullptr) { return; } @@ -176,7 +176,7 @@ void DiskIoMgr::ScanRange::cancel(const Status& status) { // For cached buffers, we can't close the range until the cached buffer is returned. // close() is called from DiskIoMgr::return_buffer(). - if (_cached_buffer == NULL) { + if (_cached_buffer == nullptr) { close(); } } @@ -219,12 +219,12 @@ bool DiskIoMgr::ScanRange::validate() { DiskIoMgr::ScanRange::ScanRange(int capacity) : _ready_buffers_capacity(capacity) { _request_type = RequestType::READ; - reset(NULL, "", -1, -1, -1, false, false, NEVER_CACHE); + reset(nullptr, "", -1, -1, -1, false, false, NEVER_CACHE); } DiskIoMgr::ScanRange::~ScanRange() { - DCHECK(_hdfs_file == NULL) << "File was not closed."; - DCHECK(_cached_buffer == NULL) << "Cached buffer was not released."; + DCHECK(_hdfs_file == nullptr) << "File was not closed."; + DCHECK(_cached_buffer == nullptr) << "Cached buffer was not released."; } void DiskIoMgr::ScanRange::reset(hdfsFS fs, const char* file, int64_t len, int64_t offset, @@ -239,19 +239,19 @@ void DiskIoMgr::ScanRange::reset(hdfsFS fs, const char* file, int64_t len, int64 _try_cache = try_cache; _expected_local = expected_local; _meta_data = meta_data; - _cached_buffer = NULL; - _io_mgr = NULL; - _reader = NULL; - _hdfs_file = NULL; + _cached_buffer = nullptr; + _io_mgr = nullptr; + _reader = nullptr; + _hdfs_file = nullptr; _mtime = mtime; } void DiskIoMgr::ScanRange::init_internal(DiskIoMgr* io_mgr, RequestContext* reader) { - DCHECK(_hdfs_file == NULL); + DCHECK(_hdfs_file == nullptr); _io_mgr = io_mgr; _reader = reader; - _local_file = NULL; - _hdfs_file = NULL; + _local_file = nullptr; + _hdfs_file = nullptr; _bytes_read = 0; _is_cancelled = false; _eosr_queued = false; @@ -270,30 +270,30 @@ Status DiskIoMgr::ScanRange::open() { return Status::Cancelled("Cancelled"); } - // if (_fs != NULL) { - // if (_hdfs_file != NULL) { + // if (_fs != nullptr) { + // if (_hdfs_file != nullptr) { // return Status::OK(); // } // _hdfs_file = _io_mgr->OpenHdfsFile(_fs, file(), mtime()); - // if (_hdfs_file == NULL) { + // if (_hdfs_file == nullptr) { // return Status::InternalError("GetHdfsErrorMsg("Failed to open HDFS file ", _file)); // } // if (hdfsSeek(_fs, _hdfs_file->file(), _offset) != 0) { // _io_mgr->cache_or_close_file_handle(file(), _hdfs_file, false); - // _hdfs_file = NULL; + // _hdfs_file = nullptr; // string error_msg = GetHdfsErrorMsg(""); // stringstream ss; // ss << "Error seeking to " << _offset << " in file: " << _file << " " << error_msg; // return Status::InternalError(ss.str()); // } // } else { - if (_local_file != NULL) { + if (_local_file != nullptr) { return Status::OK(); } _local_file = fopen(file(), "r"); - if (_local_file == NULL) { + if (_local_file == nullptr) { string error_msg = get_str_err_msg(); stringstream ss; ss << "Could not open file: " << _file << ": " << error_msg; @@ -301,7 +301,7 @@ Status DiskIoMgr::ScanRange::open() { } if (fseek(_local_file, _offset, SEEK_SET) == -1) { fclose(_local_file); - _local_file = NULL; + _local_file = nullptr; string error_msg = get_str_err_msg(); stringstream ss; ss << "Could not seek to " << _offset << " for file: " << _file << ": " << error_msg; @@ -314,8 +314,8 @@ Status DiskIoMgr::ScanRange::open() { void DiskIoMgr::ScanRange::close() { unique_lock hdfs_lock(_hdfs_lock); /* - * if (_fs != NULL) { - * if (_hdfs_file == NULL) return; + * if (_fs != nullptr) { + * if (_hdfs_file == nullptr) return; * * struct hdfsReadStatistics* stats; * if (IsDfsPath(file())) { @@ -337,21 +337,21 @@ void DiskIoMgr::ScanRange::close() { * hdfsFileFreeReadStatistics(stats); * } * } - * if (_cached_buffer != NULL) { + * if (_cached_buffer != nullptr) { * hadoopRzBufferFree(_hdfs_file->file(), _cached_buffer); - * _cached_buffer = NULL; + * _cached_buffer = nullptr; * } * _io_mgr->cache_or_close_file_handle(file(), _hdfs_file, false); * VLOG_FILE << "Cache HDFS file handle file=" << file(); - * _hdfs_file = NULL; + * _hdfs_file = nullptr; * } else { */ { - if (_local_file == NULL) { + if (_local_file == nullptr) { return; } fclose(_local_file); - _local_file = NULL; + _local_file = nullptr; } } @@ -390,8 +390,8 @@ Status DiskIoMgr::ScanRange::read(char* buffer, int64_t* bytes_read, bool* eosr) DCHECK_GE(bytes_to_read, 0); /* - * if (_fs != NULL) { - * DCHECK(_hdfs_file != NULL); + * if (_fs != nullptr) { + * DCHECK(_hdfs_file != nullptr); * int64_t max_chunk_size = max_read_chunk_size(); * while (*bytes_read < bytes_to_read) { * int chunk_size = min(bytes_to_read - *bytes_read, max_chunk_size); @@ -407,7 +407,7 @@ Status DiskIoMgr::ScanRange::read(char* buffer, int64_t* bytes_read, bool* eosr) * } * } else { */ - DCHECK(_local_file != NULL); + DCHECK(_local_file != nullptr); *bytes_read = fread(buffer, 1, bytes_to_read, _local_file); DCHECK_GE(*bytes_read, 0); DCHECK_LE(*bytes_read, bytes_to_read); @@ -442,19 +442,19 @@ Status DiskIoMgr::ScanRange::read(char* buffer, int64_t* bytes_read, bool* eosr) * if (!status.ok()) return status; * * // Cached reads not supported on local filesystem. - * if (_fs == NULL) return Status::OK(); + * if (_fs == nullptr) return Status::OK(); * * { * unique_lock hdfs_lock(_hdfs_lock); * if (_is_cancelled) return Status::Cancelled("Cancelled"); * - * DCHECK(_hdfs_file != NULL); - * DCHECK(_cached_buffer == NULL); + * DCHECK(_hdfs_file != nullptr); + * DCHECK(_cached_buffer == nullptr); * _cached_buffer = hadoopReadZero(_hdfs_file->file(), * _io_mgr->_cached_read_options, len()); * * // Data was not cached, caller will fall back to normal read path. - * if (_cached_buffer == NULL) return Status::OK(); + * if (_cached_buffer == nullptr) return Status::OK(); * } * * // Cached read succeeded. @@ -473,7 +473,7 @@ Status DiskIoMgr::ScanRange::read(char* buffer, int64_t* bytes_read, bool* eosr) * desc->_eosr = true; * _bytes_read = bytes_read; * enqueue_buffer(desc); - * if (_reader->_bytes_read_counter != NULL) { + * if (_reader->_bytes_read_counter != nullptr) { * COUNTER_ADD(_reader->_bytes_read_counter, bytes_read); * } * *read_succeeded = true; diff --git a/be/src/runtime/dpp_sink.cpp b/be/src/runtime/dpp_sink.cpp index bfcabf256f..1321a6a1b9 100644 --- a/be/src/runtime/dpp_sink.cpp +++ b/be/src/runtime/dpp_sink.cpp @@ -45,10 +45,10 @@ static void update_min(SlotRef* ref, TupleRow* agg_row, TupleRow* row) { void* slot = ref->get_slot(agg_row); bool agg_row_null = ref->is_null_bit_set(agg_row); void* value = SlotRef::get_value(ref, row); - if (!agg_row_null && value != NULL) { + if (!agg_row_null && value != nullptr) { T* t_slot = static_cast(slot); *t_slot = std::min(*t_slot, *static_cast(value)); - } else if (!agg_row_null && value == NULL) { + } else if (!agg_row_null && value == nullptr) { Tuple* agg_tuple = ref->get_tuple(agg_row); agg_tuple->set_null(ref->null_indicator_offset()); } @@ -59,10 +59,10 @@ static void update_max(SlotRef* ref, TupleRow* agg_row, TupleRow* row) { void* slot = ref->get_slot(agg_row); bool agg_row_null = ref->is_null_bit_set(agg_row); void* value = SlotRef::get_value(ref, row); - if (!agg_row_null && value != NULL) { + if (!agg_row_null && value != nullptr) { T* t_slot = static_cast(slot); *t_slot = std::max(*t_slot, *static_cast(value)); - } else if (agg_row_null && value != NULL) { + } else if (agg_row_null && value != nullptr) { T* t_slot = static_cast(slot); *t_slot = *static_cast(value); Tuple* agg_tuple = ref->get_tuple(agg_row); @@ -75,10 +75,10 @@ static void update_sum(SlotRef* ref, TupleRow* agg_row, TupleRow* row) { void* slot = ref->get_slot(agg_row); bool agg_row_null = ref->is_null_bit_set(agg_row); void* value = SlotRef::get_value(ref, row); - if (!agg_row_null && value != NULL) { + if (!agg_row_null && value != nullptr) { T* t_slot = static_cast(slot); *t_slot += *static_cast(value); - } else if (agg_row_null && value != NULL) { + } else if (agg_row_null && value != nullptr) { T* t_slot = static_cast(slot); *t_slot = *static_cast(value); Tuple* agg_tuple = ref->get_tuple(agg_row); @@ -91,13 +91,13 @@ void update_sum(SlotRef* ref, TupleRow* agg_row, TupleRow* row) { void* slot = ref->get_slot(agg_row); bool agg_row_null = ref->is_null_bit_set(agg_row); void* value = SlotRef::get_value(ref, row); - if (!agg_row_null && value != NULL) { + if (!agg_row_null && value != nullptr) { int128_t l_val, r_val; memcpy(&l_val, slot, sizeof(int128_t)); memcpy(&r_val, value, sizeof(int128_t)); l_val += r_val; memcpy(slot, &l_val, sizeof(int128_t)); - } else if (agg_row_null && value != NULL) { + } else if (agg_row_null && value != nullptr) { memcpy(slot, value, sizeof(int128_t)); Tuple* agg_tuple = ref->get_tuple(agg_row); agg_tuple->set_not_null(ref->null_indicator_offset()); diff --git a/be/src/runtime/dpp_writer.cpp b/be/src/runtime/dpp_writer.cpp index b2a4457441..7e0de66135 100644 --- a/be/src/runtime/dpp_writer.cpp +++ b/be/src/runtime/dpp_writer.cpp @@ -96,8 +96,8 @@ void DppWriter::append_to_buf(const void* ptr, int len) { } void DppWriter::increase_buf(int len) { - //increase buf to store NULL bytes - //len is the bytes of NULL + //increase buf to store nullptr bytes + //len is the bytes of nullptr if (_pos + len > _end) { int cur_len = _pos - _buf; int old_buf_len = _end - _buf; @@ -126,7 +126,7 @@ Status DppWriter::append_one_row(TupleRow* row) { if (true == _output_expr_ctxs[i]->is_nullable()) { int index = off % 8; if (item == nullptr) { - //store NULL bytes + //store nullptr bytes position[off / 8] |= 1 << (7 - index); off += 1; continue; @@ -192,7 +192,7 @@ Status DppWriter::append_one_row(TupleRow* row) { return Status::InternalError(ss.str()); } append_to_buf(&len, 2); - // passing a NULL pointer to memcpy may be core/ + // passing a nullptr pointer to memcpy may be core/ if (len == 0) { break; } diff --git a/be/src/runtime/export_sink.cpp b/be/src/runtime/export_sink.cpp index 843d13e83c..7ddc611fb0 100644 --- a/be/src/runtime/export_sink.cpp +++ b/be/src/runtime/export_sink.cpp @@ -180,7 +180,7 @@ Status ExportSink::gen_row_buffer(TupleRow* row, std::stringstream* ss) { case TYPE_STRING: { const StringValue* string_val = (const StringValue*)(item); - if (string_val->ptr == NULL) { + if (string_val->ptr == nullptr) { if (string_val->len == 0) { } else { (*ss) << "\\N"; @@ -274,7 +274,7 @@ std::string ExportSink::gen_file_name() { const TUniqueId& id = _state->fragment_instance_id(); struct timeval tv; - gettimeofday(&tv, NULL); + gettimeofday(&tv, nullptr); std::stringstream file_name; file_name << "export-data-" << print_id(id) << "-" << (tv.tv_sec * 1000 + tv.tv_usec / 1000); diff --git a/be/src/runtime/external_scan_context_mgr.cpp b/be/src/runtime/external_scan_context_mgr.cpp index 9df04f5f4f..71d79a6df4 100644 --- a/be/src/runtime/external_scan_context_mgr.cpp +++ b/be/src/runtime/external_scan_context_mgr.cpp @@ -55,7 +55,7 @@ ExternalScanContextMgr::~ExternalScanContextMgr() { Status ExternalScanContextMgr::create_scan_context(std::shared_ptr* p_context) { std::string context_id = generate_uuid_string(); std::shared_ptr context(new ScanContext(context_id)); - // context->last_access_time = time(NULL); + // context->last_access_time = time(nullptr); { std::lock_guard l(_lock); _active_contexts.insert(std::make_pair(context_id, context)); @@ -109,7 +109,7 @@ void ExternalScanContextMgr::gc_expired_context() { #ifndef BE_TEST while (!_stop_background_threads_latch.wait_for( MonoDelta::FromSeconds(doris::config::scan_context_gc_interval_min * 60))) { - time_t current_time = time(NULL); + time_t current_time = time(nullptr); std::vector> expired_contexts; { std::lock_guard l(_lock); diff --git a/be/src/runtime/file_result_writer.cpp b/be/src/runtime/file_result_writer.cpp index a71444dc66..23cd6b0a85 100644 --- a/be/src/runtime/file_result_writer.cpp +++ b/be/src/runtime/file_result_writer.cpp @@ -18,10 +18,11 @@ #include "runtime/file_result_writer.h" #include "exec/broker_writer.h" +#include "exec/hdfs_reader_writer.h" +#include "exec/hdfs_writer.h" #include "exec/local_file_writer.h" #include "exec/parquet_writer.h" #include "exec/s3_writer.h" -#include "exec/hdfs_reader_writer.h" #include "exprs/expr.h" #include "exprs/expr_context.h" #include "gen_cpp/PaloInternalService_types.h" @@ -30,8 +31,8 @@ #include "runtime/raw_value.h" #include "runtime/row_batch.h" #include "runtime/runtime_state.h" -#include "runtime/tuple_row.h" #include "runtime/string_value.h" +#include "runtime/tuple_row.h" #include "service/backend_options.h" #include "util/date_func.h" #include "util/file_utils.h" @@ -51,17 +52,17 @@ FileResultWriter::FileResultWriter(const ResultFileOptions* file_opts, _output_expr_ctxs(output_expr_ctxs), _parent_profile(parent_profile), _sinker(sinker) { - if (_file_opts->is_local_file) { - _storage_type = TStorageBackendType::LOCAL; - } else { - _storage_type = TStorageBackendType::BROKER; - } - // The new file writer needs to use fragment instance id as part of the file prefix. - // But during the upgrade process, the old version of fe will be called to the new version of be, - // resulting in no such attribute. So we need a mock here. - _fragment_instance_id.hi = 12345678987654321; - _fragment_instance_id.lo = 98765432123456789; + if (_file_opts->is_local_file) { + _storage_type = TStorageBackendType::LOCAL; + } else { + _storage_type = TStorageBackendType::BROKER; } + // The new file writer needs to use fragment instance id as part of the file prefix. + // But during the upgrade process, the old version of fe will be called to the new version of be, + // resulting in no such attribute. So we need a mock here. + _fragment_instance_id.hi = 12345678987654321; + _fragment_instance_id.lo = 98765432123456789; +} FileResultWriter::FileResultWriter(const ResultFileOptions* file_opts, const TStorageBackendType::type storage_type, @@ -139,7 +140,9 @@ Status FileResultWriter::_create_file_writer(const std::string& file_name) { } else if (_storage_type == TStorageBackendType::S3) { _file_writer = new S3Writer(_file_opts->broker_properties, file_name, 0 /* offset */); } else if (_storage_type == TStorageBackendType::HDFS) { - RETURN_IF_ERROR(HdfsReaderWriter::create_writer(const_cast&>(_file_opts->broker_properties), file_name, &_file_writer)); + RETURN_IF_ERROR(HdfsReaderWriter::create_writer( + const_cast&>(_file_opts->broker_properties), + file_name, &_file_writer)); } RETURN_IF_ERROR(_file_writer->open()); switch (_file_opts->file_format) { @@ -163,8 +166,8 @@ Status FileResultWriter::_create_file_writer(const std::string& file_name) { // file name format as: my_prefix_{fragment_instance_id}_0.csv Status FileResultWriter::_get_next_file_name(std::string* file_name) { std::stringstream ss; - ss << _file_opts->file_path << print_id(_fragment_instance_id) - << "_" << (_file_idx++) << "." << _file_format_to_name(); + ss << _file_opts->file_path << print_id(_fragment_instance_id) << "_" << (_file_idx++) << "." + << _file_format_to_name(); *file_name = ss.str(); if (_storage_type == TStorageBackendType::LOCAL) { // For local file writer, the file_path is a local dir. @@ -308,7 +311,7 @@ Status FileResultWriter::_write_one_row_as_csv(TupleRow* row) { case TYPE_CHAR: case TYPE_STRING: { const StringValue* string_val = (const StringValue*)(item); - if (string_val->ptr == NULL) { + if (string_val->ptr == nullptr) { if (string_val->len != 0) { _plain_text_outstream << NULL_IN_CSV; } @@ -459,13 +462,13 @@ Status FileResultWriter::_fill_result_batch() { RawValue::write(&written_data_bytes, tuple, tuple_desc->slots()[2], tuple_pool); StringValue* url_str_val = - reinterpret_cast(tuple->get_slot(tuple_desc->slots()[3]->tuple_offset())); + reinterpret_cast(tuple->get_slot(tuple_desc->slots()[3]->tuple_offset())); std::string file_url; _get_file_url(&file_url); url_str_val->ptr = (char*)_output_batch->tuple_data_pool()->allocate(file_url.length()); url_str_val->len = file_url.length(); memcpy(url_str_val->ptr, file_url.c_str(), url_str_val->len); - + _output_batch->commit_last_row(); return Status::OK(); } diff --git a/be/src/runtime/fold_constant_mgr.cpp b/be/src/runtime/fold_constant_mgr.cpp index 6c39fc533e..9c377ee1d4 100644 --- a/be/src/runtime/fold_constant_mgr.cpp +++ b/be/src/runtime/fold_constant_mgr.cpp @@ -15,21 +15,21 @@ // specific language governing permissions and limitations // under the License. +#include "runtime/fold_constant_mgr.h" + #include #include -#include "runtime/fold_constant_mgr.h" -#include "runtime/tuple_row.h" -#include "runtime/exec_env.h" -#include "runtime/runtime_state.h" -#include "runtime/mem_tracker.h" -#include "exprs/expr_context.h" -#include "exprs/expr.h" #include "common/object_pool.h" #include "common/status.h" - -#include "gen_cpp/internal_service.pb.h" +#include "exprs/expr.h" +#include "exprs/expr_context.h" #include "gen_cpp/PaloInternalService_types.h" +#include "gen_cpp/internal_service.pb.h" +#include "runtime/exec_env.h" +#include "runtime/mem_tracker.h" +#include "runtime/runtime_state.h" +#include "runtime/tuple_row.h" using std::string; using std::map; @@ -38,14 +38,11 @@ namespace doris { TUniqueId FoldConstantMgr::_dummy_id; -FoldConstantMgr::FoldConstantMgr(ExecEnv* exec_env) - : _exec_env(exec_env), _pool(){ - -} +FoldConstantMgr::FoldConstantMgr(ExecEnv* exec_env) : _exec_env(exec_env), _pool() {} -Status FoldConstantMgr::fold_constant_expr( - const TFoldConstantParams& params, PConstantExprResult* response) { - auto expr_map = params.expr_map; +Status FoldConstantMgr::fold_constant_expr(const TFoldConstantParams& params, + PConstantExprResult* response) { + auto expr_map = params.expr_map; auto expr_result_map = response->mutable_expr_result_map(); TQueryGlobals query_globals = params.query_globals; @@ -90,7 +87,7 @@ Status FoldConstantMgr::fold_constant_expr( expr_result.set_content(std::move(result)); expr_result.mutable_type()->set_type(t_type); - + pexpr_result_map.mutable_map()->insert({n.first, expr_result}); // close context expr @@ -101,7 +98,6 @@ Status FoldConstantMgr::fold_constant_expr( } return Status::OK(); - } Status FoldConstantMgr::init(TQueryGlobals query_globals) { @@ -115,7 +111,7 @@ Status FoldConstantMgr::init(TQueryGlobals query_globals) { TQueryOptions query_options; _runtime_state.reset(new RuntimeState(fragment_params.params, query_options, query_globals, ExecEnv::GetInstance())); - DescriptorTbl* desc_tbl = NULL; + DescriptorTbl* desc_tbl = nullptr; TDescriptorTable* t_desc_tbl = new TDescriptorTable(); Status status = DescriptorTbl::create(_runtime_state->obj_pool(), *t_desc_tbl, &desc_tbl); if (UNLIKELY(!status.ok())) { @@ -131,7 +127,8 @@ Status FoldConstantMgr::init(TQueryGlobals query_globals) { _runtime_profile = _runtime_state->runtime_profile(); _runtime_profile->set_name("FoldConstantExpr"); - _mem_tracker = MemTracker::CreateTracker(-1, "FoldConstantExpr", _runtime_state->instance_mem_tracker()); + _mem_tracker = MemTracker::CreateTracker(-1, "FoldConstantExpr", + _runtime_state->instance_mem_tracker()); _mem_pool.reset(new MemPool(_mem_tracker.get())); return Status::OK(); @@ -139,14 +136,14 @@ Status FoldConstantMgr::init(TQueryGlobals query_globals) { Status FoldConstantMgr::prepare_and_open(ExprContext* ctx) { RowDescriptor* desc = new RowDescriptor(); - ctx -> prepare(_runtime_state.get(), *desc, _mem_tracker); - return ctx -> open(_runtime_state.get()); + ctx->prepare(_runtime_state.get(), *desc, _mem_tracker); + return ctx->open(_runtime_state.get()); } -string FoldConstantMgr::get_result(void* src, PrimitiveType slot_type){ +string FoldConstantMgr::get_result(void* src, PrimitiveType slot_type) { switch (slot_type) { case TYPE_NULL: { - return NULL; + return nullptr; } case TYPE_BOOLEAN: { bool val = *reinterpret_cast(src); @@ -199,10 +196,8 @@ string FoldConstantMgr::get_result(void* src, PrimitiveType slot_type){ } default: DCHECK(false) << "Type not implemented: " << slot_type; - return NULL; + return nullptr; } } - -} - +} // namespace doris diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 4178e2190a..9719e6cf60 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -303,7 +303,7 @@ void FragmentExecState::coordinator_callback(const Status& status, RuntimeProfil params.__set_done(done); RuntimeState* runtime_state = _executor.runtime_state(); - DCHECK(runtime_state != NULL); + DCHECK(runtime_state != nullptr); if (runtime_state->query_options().query_type == TQueryType::LOAD && !done && status.ok()) { // this is a load plan, and load is not finished, just make a brief report params.__set_loaded_rows(runtime_state->num_rows_load_total()); @@ -726,7 +726,7 @@ Status FragmentMgr::exec_external_plan_fragment(const TScanOpenParams& params, } // set up desc tbl - DescriptorTbl* desc_tbl = NULL; + DescriptorTbl* desc_tbl = nullptr; ObjectPool obj_pool; st = DescriptorTbl::create(&obj_pool, t_query_plan_info.desc_tbl, &desc_tbl); if (!st.ok()) { diff --git a/be/src/runtime/mem_pool.h b/be/src/runtime/mem_pool.h index 3a3750eda7..04d6236831 100644 --- a/be/src/runtime/mem_pool.h +++ b/be/src/runtime/mem_pool.h @@ -88,7 +88,7 @@ class MemTracker; /// delete p; class MemPool { public: - /// 'tracker' tracks the amount of memory allocated by this pool. Must not be NULL. + /// 'tracker' tracks the amount of memory allocated by this pool. Must not be nullptr. MemPool(MemTracker* mem_tracker) : current_chunk_idx_(-1), next_chunk_size_(INITIAL_CHUNK_SIZE), @@ -114,8 +114,8 @@ public: } /// Same as Allocate() except the mem limit is checked before the allocation and - /// this call will fail (returns NULL) if it does. - /// The caller must handle the NULL case. This should be used for allocations + /// this call will fail (returns nullptr) if it does. + /// The caller must handle the nullptr case. This should be used for allocations /// where the size can be very big to bound the amount by which we exceed mem limits. uint8_t* try_allocate(int64_t size) { return allocate(size, DEFAULT_ALIGNMENT); } @@ -179,7 +179,7 @@ private: ChunkInfo() : allocated_bytes(0) {} }; - /// A static field used as non-NULL pointer for zero length allocations. NULL is + /// A static field used as non-nullptr pointer for zero length allocations. nullptr is /// reserved for allocation failures. It must be as aligned as max_align_t for /// TryAllocateAligned(). static uint32_t k_zero_length_region_; @@ -225,12 +225,12 @@ private: } } - // If we couldn't allocate a new chunk, return NULL. malloc() guarantees alignment + // If we couldn't allocate a new chunk, return nullptr. malloc() guarantees alignment // of alignof(std::max_align_t), so we do not need to do anything additional to // guarantee alignment. //static_assert( //INITIAL_CHUNK_SIZE >= config::FLAGS_MEMORY_MAX_ALIGNMENT, "Min chunk size too low"); - if (UNLIKELY(!find_chunk(size, CHECK_LIMIT_FIRST))) return NULL; + if (UNLIKELY(!find_chunk(size, CHECK_LIMIT_FIRST))) return nullptr; ChunkInfo& info = chunks_[current_chunk_idx_]; uint8_t* result = info.chunk.data + info.allocated_bytes; diff --git a/be/src/runtime/mem_tracker.h b/be/src/runtime/mem_tracker.h index a8db786e2b..172ff20538 100644 --- a/be/src/runtime/mem_tracker.h +++ b/be/src/runtime/mem_tracker.h @@ -42,7 +42,7 @@ enum class MemLimit { HARD, SOFT }; /// The Level use to decide whether to show it in web page /// each MemTracker have a Level equals to parent, only be set explicit -enum class MemTrackerLevel {OVERVIEW = 0, TASK, VERBOSE}; +enum class MemTrackerLevel { OVERVIEW = 0, TASK, VERBOSE }; class ObjectPool; class MemTracker; @@ -92,11 +92,13 @@ public: static std::shared_ptr CreateTracker( int64_t byte_limit = -1, const std::string& label = std::string(), std::shared_ptr parent = std::shared_ptr(), - bool log_usage_if_zero = true, bool reset_label_name = true, MemTrackerLevel level = MemTrackerLevel::VERBOSE); + bool log_usage_if_zero = true, bool reset_label_name = true, + MemTrackerLevel level = MemTrackerLevel::VERBOSE); static std::shared_ptr CreateTracker( RuntimeProfile* profile, int64_t byte_limit, const std::string& label = std::string(), - const std::shared_ptr& parent = std::shared_ptr(), bool reset_label_name = true, MemTrackerLevel level = MemTrackerLevel::VERBOSE); + const std::shared_ptr& parent = std::shared_ptr(), + bool reset_label_name = true, MemTrackerLevel level = MemTrackerLevel::VERBOSE); // this is used for creating an orphan mem tracker, or for unit test. // If a mem tracker has parent, it should be created by `CreateTracker()` @@ -341,7 +343,7 @@ public: //void RegisterMetrics(MetricGroup* metrics, const std::string& prefix); /// Logs the usage of this tracker and optionally its children (recursively). - /// If 'logged_consumption' is non-NULL, sets the consumption value logged. + /// If 'logged_consumption' is non-nullptr, sets the consumption value logged. /// 'max_recursive_depth' specifies the maximum number of levels of children /// to include in the dump. If it is zero, then no children are dumped. /// Limiting the recursive depth reduces the cost of dumping, particularly @@ -367,7 +369,7 @@ public: /// details of the allocation which caused the limit to be exceeded. /// If 'failed_allocation_size' is greater than zero, logs the allocation size. If /// 'failed_allocation_size' is zero, nothing about the allocation size is logged. - /// If 'state' is non-NULL, logs the error to 'state'. + /// If 'state' is non-nullptr, logs the error to 'state'. Status MemLimitExceeded(RuntimeState* state, const std::string& details, int64_t failed_allocation = 0) WARN_UNUSED_RESULT { return MemLimitExceeded(this, state, details, failed_allocation); @@ -469,7 +471,7 @@ private: int limit); /// If an ancestor of this tracker is a query MemTracker, return that tracker. - /// Otherwise return NULL. + /// Otherwise return nullptr. MemTracker* GetQueryMemTracker(); /// Increases/Decreases the consumption of this tracker and the ancestors up to (but @@ -494,7 +496,7 @@ private: /// 0 if the query is still executing or 1 if it has finished executing. Before /// it has finished executing, the tracker limit is treated as "reserved memory" /// for the purpose of admission control - see GetPoolMemReserved(). - std::atomic query_exec_finished_{0}; + std::atomic query_exec_finished_ {0}; /// Only valid for MemTrackers returned from GetRequestPoolMemTracker() std::string pool_name_; @@ -516,12 +518,12 @@ private: /// in bytes std::shared_ptr consumption_; - /// If non-NULL, used to measure consumption (in bytes) rather than the values provided + /// If non-nullptr, used to measure consumption (in bytes) rather than the values provided /// to Consume()/Release(). Only used for the process tracker, thus parent_ should be - /// NULL if consumption_metric_ is set. + /// nullptr if consumption_metric_ is set. IntGauge* consumption_metric_; - /// If non-NULL, counters from a corresponding ReservationTracker that should be + /// If non-nullptr, counters from a corresponding ReservationTracker that should be /// reported in logs and other diagnostics. Owned by this MemTracker. The counters /// are owned by the fragment's RuntimeProfile. AtomicPtr reservation_counters_; diff --git a/be/src/runtime/memory_scratch_sink.cpp b/be/src/runtime/memory_scratch_sink.cpp index ab1d0c15cc..ae3c02003c 100644 --- a/be/src/runtime/memory_scratch_sink.cpp +++ b/be/src/runtime/memory_scratch_sink.cpp @@ -73,7 +73,7 @@ Status MemoryScratchSink::prepare(RuntimeState* state) { } Status MemoryScratchSink::send(RuntimeState* state, RowBatch* batch) { - if (NULL == batch || 0 == batch->num_rows()) { + if (nullptr == batch || 0 == batch->num_rows()) { return Status::OK(); } std::shared_ptr result; diff --git a/be/src/runtime/mysql_result_writer.cpp b/be/src/runtime/mysql_result_writer.cpp index 759cbd29ce..7d3c5229fc 100644 --- a/be/src/runtime/mysql_result_writer.cpp +++ b/be/src/runtime/mysql_result_writer.cpp @@ -32,11 +32,12 @@ namespace doris { MysqlResultWriter::MysqlResultWriter(BufferControlBlock* sinker, - const std::vector& output_expr_ctxs, RuntimeProfile* parent_profile) + const std::vector& output_expr_ctxs, + RuntimeProfile* parent_profile) : ResultWriter(), _sinker(sinker), _output_expr_ctxs(output_expr_ctxs), - _row_buffer(NULL), + _row_buffer(nullptr), _parent_profile(parent_profile) {} MysqlResultWriter::~MysqlResultWriter() { @@ -45,12 +46,12 @@ MysqlResultWriter::~MysqlResultWriter() { Status MysqlResultWriter::init(RuntimeState* state) { _init_profile(); - if (NULL == _sinker) { - return Status::InternalError("sinker is NULL pointer."); + if (nullptr == _sinker) { + return Status::InternalError("sinker is nullptr pointer."); } - _row_buffer = new(std::nothrow) MysqlRowBuffer(); - if (NULL == _row_buffer) { + _row_buffer = new (std::nothrow) MysqlRowBuffer(); + if (nullptr == _row_buffer) { return Status::InternalError("no memory to alloc."); } @@ -123,7 +124,7 @@ int MysqlResultWriter::_add_row_value(int index, const TypeDescriptor& type, voi case TYPE_STRING: { const StringValue* string_val = (const StringValue*)(item); - if (string_val->ptr == NULL) { + if (string_val->ptr == nullptr) { if (string_val->len == 0) { // 0x01 is a magic num, not useful actually, just for present "" char* tmp_val = reinterpret_cast(0x01); @@ -212,7 +213,7 @@ Status MysqlResultWriter::_add_one_row(TupleRow* row) { Status MysqlResultWriter::append_row_batch(const RowBatch* batch) { SCOPED_TIMER(_append_row_batch_timer); - if (NULL == batch || 0 == batch->num_rows()) { + if (nullptr == batch || 0 == batch->num_rows()) { return Status::OK(); } @@ -240,7 +241,8 @@ Status MysqlResultWriter::append_row_batch(const RowBatch* batch) { if (status.ok()) { SCOPED_TIMER(_result_send_timer); // push this batch to back - RETURN_NOT_OK_STATUS_WITH_WARN(_sinker->add_batch(result), "fappend result batch to sink failed."); + RETURN_NOT_OK_STATUS_WITH_WARN(_sinker->add_batch(result), + "fappend result batch to sink failed."); _written_rows += num_rows; } return Status::OK(); diff --git a/be/src/runtime/mysql_table_writer.cpp b/be/src/runtime/mysql_table_writer.cpp index eceafd4a36..a2ae2af060 100644 --- a/be/src/runtime/mysql_table_writer.cpp +++ b/be/src/runtime/mysql_table_writer.cpp @@ -54,8 +54,8 @@ Status MysqlTableWriter::open(const MysqlConnInfo& conn_info, const std::string& MYSQL* res = mysql_real_connect(_mysql_conn, conn_info.host.c_str(), conn_info.user.c_str(), conn_info.passwd.c_str(), conn_info.db.c_str(), conn_info.port, - NULL, // unix socket - 0); // flags + nullptr, // unix socket + 0); // flags if (res == nullptr) { std::stringstream ss; ss << "mysql_real_connect failed because " << mysql_error(_mysql_conn); @@ -122,7 +122,7 @@ Status MysqlTableWriter::insert_row(TupleRow* row) { case TYPE_STRING: { const StringValue* string_val = (const StringValue*)(item); - if (string_val->ptr == NULL) { + if (string_val->ptr == nullptr) { if (string_val->len == 0) { ss << "\'\'"; } else { diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index ecb3f43957..4b6280ebf6 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -119,7 +119,8 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, _mem_tracker = MemTracker::CreateTracker(bytes_limit, "PlanFragmentExecutor:" + print_id(_query_id) + ":" + print_id(params.fragment_instance_id), - _exec_env->process_mem_tracker(), true, false, MemTrackerLevel::TASK); + _exec_env->process_mem_tracker(), true, false, + MemTrackerLevel::TASK); _runtime_state->set_fragment_mem_tracker(_mem_tracker); RETURN_IF_ERROR(_runtime_state->create_block_mgr()); @@ -147,7 +148,6 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, _plan); } - // set #senders of exchange nodes before calling Prepare() std::vector exch_nodes; _plan->collect_nodes(TPlanNodeType::EXCHANGE_NODE, &exch_nodes); @@ -157,7 +157,7 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, DCHECK_GT(num_senders, 0); if (_runtime_state->enable_vectorized_exec()) { } else { - static_cast(exch_node)->set_num_senders(num_senders); + static_cast(exch_node)->set_num_senders(num_senders); } } @@ -184,16 +184,15 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, // set up sink, if required if (request.fragment.__isset.output_sink) { - RETURN_IF_ERROR(DataSink::create_data_sink(obj_pool(), request.fragment.output_sink, - request.fragment.output_exprs, params, - row_desc(), runtime_state()->enable_vectorized_exec(), - &_sink, *desc_tbl)); + RETURN_IF_ERROR(DataSink::create_data_sink( + obj_pool(), request.fragment.output_sink, request.fragment.output_exprs, params, + row_desc(), runtime_state()->enable_vectorized_exec(), &_sink, *desc_tbl)); RETURN_IF_ERROR(_sink->prepare(runtime_state())); RuntimeProfile* sink_profile = _sink->profile(); - if (sink_profile != NULL) { - profile()->add_child(sink_profile, true, NULL); + if (sink_profile != nullptr) { + profile()->add_child(sink_profile, true, nullptr); } _collect_query_statistics_with_every_batch = @@ -201,12 +200,12 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, ? params.send_query_statistics_with_every_batch : false; } else { - // _sink is set to NULL - _sink.reset(NULL); + // _sink is set to nullptr + _sink.reset(nullptr); } // set up profile counters - profile()->add_child(_plan->runtime_profile(), true, NULL); + profile()->add_child(_plan->runtime_profile(), true, nullptr); _rows_produced_counter = ADD_COUNTER(profile(), "RowsProduced", TUnit::UNIT); _fragment_cpu_timer = ADD_TIMER(profile(), "FragmentCpuTime"); @@ -217,7 +216,7 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, _prepared = true; _query_statistics.reset(new QueryStatistics()); - if (_sink.get() != NULL) { + if (_sink.get() != nullptr) { _sink->set_query_statistics(_query_statistics); } return Status::OK(); @@ -235,7 +234,7 @@ Status PlanFragmentExecutor::open() { // at end, otherwise the coordinator hangs in case we finish w/ an error if (_is_report_success && _report_status_cb && config::status_report_interval > 0) { std::unique_lock l(_report_thread_lock); - _report_thread = boost::thread(&PlanFragmentExecutor::report_profile, this); + _report_thread = std::thread(&PlanFragmentExecutor::report_profile, this); // make sure the thread started up, otherwise report_profile() might get into a race // with stop_report_thread() _report_thread_started_cv.wait(l); @@ -265,7 +264,7 @@ Status PlanFragmentExecutor::open_internal() { RETURN_IF_ERROR(_plan->open(_runtime_state.get())); } - if (_sink.get() == NULL) { + if (_sink.get() == nullptr) { return Status::OK(); } { @@ -275,14 +274,14 @@ Status PlanFragmentExecutor::open_internal() { // If there is a sink, do all the work of driving it here, so that // when this returns the query has actually finished - RowBatch* batch = NULL; + RowBatch* batch = nullptr; while (true) { { SCOPED_CPU_TIMER(_fragment_cpu_timer); RETURN_IF_ERROR(get_next_internal(&batch)); } - if (batch == NULL) { + if (batch == nullptr) { break; } @@ -328,8 +327,8 @@ Status PlanFragmentExecutor::open_internal() { RETURN_IF_ERROR(status); } - // Setting to NULL ensures that the d'tor won't double-close the sink. - _sink.reset(NULL); + // Setting to nullptr ensures that the d'tor won't double-close the sink. + _sink.reset(nullptr); _done = true; stop_report_thread(); @@ -368,7 +367,8 @@ void PlanFragmentExecutor::report_profile() { _stop_report_thread_cv.wait_for(l, std::chrono::seconds(config::status_report_interval)); } else { - LOG(WARNING) << "config::status_report_interval is equal to or less than zero, exiting reporting thread."; + LOG(WARNING) << "config::status_report_interval is equal to or less than zero, exiting " + "reporting thread."; break; } @@ -458,7 +458,7 @@ Status PlanFragmentExecutor::get_next(RowBatch** batch) { Status PlanFragmentExecutor::get_next_internal(RowBatch** batch) { if (_done) { - *batch = NULL; + *batch = nullptr; return Status::OK(); } @@ -473,7 +473,7 @@ Status PlanFragmentExecutor::get_next_internal(RowBatch** batch) { break; } - *batch = NULL; + *batch = nullptr; } return Status::OK(); @@ -530,16 +530,16 @@ void PlanFragmentExecutor::close() { return; } - _row_batch.reset(NULL); + _row_batch.reset(nullptr); // Prepare may not have been called, which sets _runtime_state - if (_runtime_state.get() != NULL) { + if (_runtime_state.get() != nullptr) { // _runtime_state init failed if (_plan != nullptr) { _plan->close(_runtime_state.get()); } - if (_sink.get() != NULL) { + if (_sink.get() != nullptr) { if (_prepared) { Status status; { @@ -564,7 +564,8 @@ void PlanFragmentExecutor::close() { _runtime_state->runtime_profile()->pretty_print(&ss); LOG(INFO) << ss.str(); } - LOG(INFO) << "Close() fragment_instance_id=" << print_id(_runtime_state->fragment_instance_id()); + LOG(INFO) << "Close() fragment_instance_id=" + << print_id(_runtime_state->fragment_instance_id()); } // _mem_tracker init failed diff --git a/be/src/runtime/plan_fragment_executor.h b/be/src/runtime/plan_fragment_executor.h index f2d1f2ae96..b4c9725e0f 100644 --- a/be/src/runtime/plan_fragment_executor.h +++ b/be/src/runtime/plan_fragment_executor.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_RUNTIME_PLAN_FRAGMENT_EXECUTOR_H #define DORIS_BE_RUNTIME_PLAN_FRAGMENT_EXECUTOR_H -#include #include #include #include @@ -111,9 +110,9 @@ public: // time when open() returns, and the status-reporting thread will have been stopped. Status open(); - // Return results through 'batch'. Sets '*batch' to NULL if no more results. + // Return results through 'batch'. Sets '*batch' to nullptr if no more results. // '*batch' is owned by PlanFragmentExecutor and must not be deleted. - // When *batch == NULL, get_next() should not be called anymore. Also, report_status_cb + // When *batch == nullptr, get_next() should not be called anymore. Also, report_status_cb // will have been called for the final time and the status-reporting thread // will have been stopped. Status get_next(RowBatch** batch); @@ -151,7 +150,7 @@ private: // profile reporting-related report_status_callback _report_status_cb; - boost::thread _report_thread; + std::thread _report_thread; std::mutex _report_thread_lock; // Indicates that profile reporting thread should stop. @@ -190,12 +189,12 @@ private: // note that RuntimeState should be constructed before and destructed after `_sink' and `_row_batch', // therefore we declare it before `_sink' and `_row_batch' - boost::scoped_ptr _runtime_state; + std::unique_ptr _runtime_state; // Output sink for rows sent to this fragment. May not be set, in which case rows are // returned via get_next's row batch // Created in prepare (if required), owned by this object. - boost::scoped_ptr _sink; - boost::scoped_ptr _row_batch; + std::unique_ptr _sink; + std::unique_ptr _row_batch; // Number of rows returned by this fragment RuntimeProfile::Counter* _rows_produced_counter; @@ -235,7 +234,7 @@ private: // If this plan fragment has a sink and open_internal() returns without an // error condition, all rows will have been sent to the sink, the sink will // have been closed, a final report will have been sent and the report thread will - // have been stopped. _sink will be set to NULL after successful execution. + // have been stopped. _sink will be set to nullptr after successful execution. Status open_internal(); // Executes get_next() logic and returns resulting status. diff --git a/be/src/runtime/qsorter.cpp b/be/src/runtime/qsorter.cpp index f01f1f54a4..951b35a63a 100644 --- a/be/src/runtime/qsorter.cpp +++ b/be/src/runtime/qsorter.cpp @@ -52,7 +52,7 @@ bool TupleRowLessThan::operator()(TupleRow* const& lhs, TupleRow* const& rhs) co void* lhs_value = _lhs_expr_ctxs[i]->get_value(lhs); void* rhs_value = _rhs_expr_ctxs[i]->get_value(rhs); - // NULL's always go at the end regardless of asc/desc + // nullptr's always go at the end regardless of asc/desc if (lhs_value == nullptr && rhs_value == nullptr) { continue; } @@ -93,7 +93,7 @@ Status QSorter::prepare(RuntimeState* state) { Status QSorter::insert_tuple_row(TupleRow* input_row) { TupleRow* insert_tuple_row = input_row->deep_copy(_row_desc.tuple_descriptors(), _tuple_pool.get()); - if (insert_tuple_row == NULL) { + if (insert_tuple_row == nullptr) { return Status::InternalError("deep copy failed."); } _sorted_rows.push_back(insert_tuple_row); diff --git a/be/src/runtime/qsorter.h b/be/src/runtime/qsorter.h index 23beecafab..24e3ac93e0 100644 --- a/be/src/runtime/qsorter.h +++ b/be/src/runtime/qsorter.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_RUNTIME_QSORTER_H #define DORIS_BE_RUNTIME_QSORTER_H -#include #include #include "common/status.h" @@ -71,7 +70,7 @@ private: std::vector::iterator _next_iter; // Stores everything referenced in _priority_queue - boost::scoped_ptr _tuple_pool; + std::unique_ptr _tuple_pool; }; } // namespace doris diff --git a/be/src/runtime/raw_value.cpp b/be/src/runtime/raw_value.cpp index e9778bbda5..b6647c9ead 100644 --- a/be/src/runtime/raw_value.cpp +++ b/be/src/runtime/raw_value.cpp @@ -31,12 +31,12 @@ const int RawValue::ASCII_PRECISION = 16; // print 16 digits for double/float void RawValue::print_value_as_bytes(const void* value, const TypeDescriptor& type, std::stringstream* stream) { - if (value == NULL) { + if (value == nullptr) { return; } const char* chars = reinterpret_cast(value); - const StringValue* string_val = NULL; + const StringValue* string_val = nullptr; switch (type.type) { case TYPE_NULL: @@ -97,7 +97,7 @@ void RawValue::print_value_as_bytes(const void* value, const TypeDescriptor& typ void RawValue::print_value(const void* value, const TypeDescriptor& type, int scale, std::stringstream* stream) { - if (value == NULL) { + if (value == nullptr) { *stream << "NULL"; return; } @@ -113,7 +113,7 @@ void RawValue::print_value(const void* value, const TypeDescriptor& type, int sc } std::string tmp; - const StringValue* string_val = NULL; + const StringValue* string_val = nullptr; switch (type.type) { case TYPE_BOOLEAN: { @@ -194,14 +194,14 @@ void RawValue::print_value(const void* value, const TypeDescriptor& type, int sc void RawValue::print_value(const void* value, const TypeDescriptor& type, int scale, std::string* str) { - if (value == NULL) { + if (value == nullptr) { *str = "NULL"; return; } std::stringstream out; out.precision(ASCII_PRECISION); - const StringValue* string_val = NULL; + const StringValue* string_val = nullptr; std::string tmp; bool val = false; @@ -239,7 +239,7 @@ void RawValue::print_value(const void* value, const TypeDescriptor& type, int sc } void RawValue::write(const void* value, void* dst, const TypeDescriptor& type, MemPool* pool) { - DCHECK(value != NULL); + DCHECK(value != nullptr); switch (type.type) { case TYPE_NULL: @@ -297,13 +297,13 @@ void RawValue::write(const void* value, void* dst, const TypeDescriptor& type, M case TYPE_OBJECT: case TYPE_HLL: case TYPE_VARCHAR: - case TYPE_CHAR: + case TYPE_CHAR: case TYPE_STRING: { const StringValue* src = reinterpret_cast(value); StringValue* dest = reinterpret_cast(dst); dest->len = src->len; - if (pool != NULL) { + if (pool != nullptr) { dest->ptr = reinterpret_cast(pool->allocate(dest->len)); memcpy(dest->ptr, src->ptr, dest->len); } else { @@ -318,7 +318,7 @@ void RawValue::write(const void* value, void* dst, const TypeDescriptor& type, M const CollectionValue* src = reinterpret_cast(value); CollectionValue* val = reinterpret_cast(dst); - if (pool != NULL) { + if (pool != nullptr) { auto children_type = type.children.at(0).type; CollectionValue::init_collection(pool, src->size(), children_type, val); ArrayIterator src_iter = src->iterator(children_type); @@ -346,7 +346,7 @@ void RawValue::write(const void* value, void* dst, const TypeDescriptor& type, M // TODO: can we remove some of this code duplication? Templated allocator? void RawValue::write(const void* value, const TypeDescriptor& type, void* dst, uint8_t** buf) { - DCHECK(value != NULL); + DCHECK(value != nullptr); switch (type.type) { case TYPE_BOOLEAN: *reinterpret_cast(dst) = *reinterpret_cast(value); @@ -379,7 +379,7 @@ void RawValue::write(const void* value, const TypeDescriptor& type, void* dst, u case TYPE_VARCHAR: case TYPE_CHAR: case TYPE_STRING: { - DCHECK(buf != NULL); + DCHECK(buf != nullptr); const StringValue* src = reinterpret_cast(value); StringValue* dest = reinterpret_cast(dst); dest->len = src->len; @@ -400,7 +400,7 @@ void RawValue::write(const void* value, const TypeDescriptor& type, void* dst, u void RawValue::write(const void* value, Tuple* tuple, const SlotDescriptor* slot_desc, MemPool* pool) { - if (value == NULL) { + if (value == nullptr) { tuple->set_null(slot_desc->null_indicator_offset()); } else { void* slot = tuple->get_slot(slot_desc->tuple_offset()); diff --git a/be/src/runtime/raw_value.h b/be/src/runtime/raw_value.h index fee63f9f94..e9e270e85b 100644 --- a/be/src/runtime/raw_value.h +++ b/be/src/runtime/raw_value.h @@ -38,7 +38,7 @@ public: // Ascii output precision for double/float static const int ASCII_PRECISION; - // Convert 'value' into ascii and write to 'stream'. NULL turns into "NULL". 'scale' + // Convert 'value' into ascii and write to 'stream'. nullptr turns into NULL. 'scale' // determines how many digits after the decimal are printed for floating point numbers, // -1 indicates to use the stream's current formatting. static void print_value(const void* value, const TypeDescriptor& type, int scale, @@ -93,13 +93,13 @@ public: // Writes the bytes of a given value into the slot of a tuple. // For string values, the string data is copied into memory allocated from 'pool' - // only if pool is non-NULL. + // only if pool is non-nullptr. static void write(const void* value, Tuple* tuple, const SlotDescriptor* slot_desc, MemPool* pool); // Writes 'src' into 'dst' for type. - // For string values, the string data is copied into 'pool' if pool is non-NULL. - // src must be non-NULL. + // For string values, the string data is copied into 'pool' if pool is non-nullptr. + // src must be non-nullptr. static void write(const void* src, void* dst, const TypeDescriptor& type, MemPool* pool); // Writes 'src' into 'dst' for type. @@ -226,7 +226,7 @@ inline bool RawValue::eq(const void* v1, const void* v2, const TypeDescriptor& t // seed ^= v + 0x9e3779b9 + (seed << 6) + (seed >> 2); inline uint32_t RawValue::get_hash_value(const void* v, const PrimitiveType& type, uint32_t seed) { // Hash_combine with v = 0 - if (v == NULL) { + if (v == nullptr) { uint32_t value = 0x9e3779b9; return seed ^ (value + (seed << 6) + (seed >> 2)); } @@ -234,7 +234,7 @@ inline uint32_t RawValue::get_hash_value(const void* v, const PrimitiveType& typ switch (type) { case TYPE_VARCHAR: case TYPE_CHAR: - case TYPE_HLL: + case TYPE_HLL: case TYPE_STRING: { const StringValue* string_value = reinterpret_cast(v); return HashUtil::hash(string_value->ptr, string_value->len, seed); @@ -282,7 +282,7 @@ inline uint32_t RawValue::get_hash_value(const void* v, const PrimitiveType& typ inline uint32_t RawValue::get_hash_value_fvn(const void* v, const PrimitiveType& type, uint32_t seed) { // Hash_combine with v = 0 - if (v == NULL) { + if (v == nullptr) { uint32_t value = 0x9e3779b9; return seed ^ (value + (seed << 6) + (seed >> 2)); } @@ -339,7 +339,7 @@ inline uint32_t RawValue::get_hash_value_fvn(const void* v, const PrimitiveType& // Because crc32 hardware is not equal with zlib crc32 inline uint32_t RawValue::zlib_crc32(const void* v, const TypeDescriptor& type, uint32_t seed) { // Hash_combine with v = 0 - if (v == NULL) { + if (v == nullptr) { uint32_t value = 0x9e3779b9; return seed ^ (value + (seed << 6) + (seed >> 2)); } diff --git a/be/src/runtime/raw_value_ir.cpp b/be/src/runtime/raw_value_ir.cpp index 944c3fc605..2837575915 100644 --- a/be/src/runtime/raw_value_ir.cpp +++ b/be/src/runtime/raw_value_ir.cpp @@ -34,11 +34,11 @@ int RawValue::compare(const void* v1, const void* v2, const TypeDescriptor& type int64_t b1; int64_t b2; - if (NULL == v1 && NULL == v2) { + if (nullptr == v1 && nullptr == v2) { return 0; - } else if (NULL == v1 && NULL != v2) { + } else if (nullptr == v1 && nullptr != v2) { return -1; - } else if (NULL != v1 && NULL == v2) { + } else if (nullptr != v1 && nullptr == v2) { return 1; } diff --git a/be/src/runtime/result_buffer_mgr.cpp b/be/src/runtime/result_buffer_mgr.cpp index 3076e0f891..a62a061ca5 100644 --- a/be/src/runtime/result_buffer_mgr.cpp +++ b/be/src/runtime/result_buffer_mgr.cpp @@ -59,14 +59,14 @@ Status ResultBufferMgr::init() { } Status ResultBufferMgr::create_sender(const TUniqueId& query_id, int buffer_size, - boost::shared_ptr* sender) { + std::shared_ptr* sender) { *sender = find_control_block(query_id); if (*sender != nullptr) { LOG(WARNING) << "already have buffer control block for this instance " << query_id; return Status::OK(); } - boost::shared_ptr control_block( + std::shared_ptr control_block( new BufferControlBlock(query_id, buffer_size)); { std::lock_guard l(_lock); @@ -76,8 +76,7 @@ Status ResultBufferMgr::create_sender(const TUniqueId& query_id, int buffer_size return Status::OK(); } -boost::shared_ptr ResultBufferMgr::find_control_block( - const TUniqueId& query_id) { +std::shared_ptr ResultBufferMgr::find_control_block(const TUniqueId& query_id) { // TODO(zhaochun): this lock can be bottleneck? std::lock_guard l(_lock); BufferMap::iterator iter = _buffer_map.find(query_id); @@ -86,13 +85,13 @@ boost::shared_ptr ResultBufferMgr::find_control_block( return iter->second; } - return boost::shared_ptr(); + return std::shared_ptr(); } Status ResultBufferMgr::fetch_data(const TUniqueId& query_id, TFetchDataResult* result) { - boost::shared_ptr cb = find_control_block(query_id); + std::shared_ptr cb = find_control_block(query_id); - if (NULL == cb) { + if (nullptr == cb) { // the sender tear down its buffer block return Status::InternalError("no result for this query."); } @@ -104,7 +103,7 @@ void ResultBufferMgr::fetch_data(const PUniqueId& finst_id, GetResultBatchCtx* c TUniqueId tid; tid.__set_hi(finst_id.hi()); tid.__set_lo(finst_id.lo()); - boost::shared_ptr cb = find_control_block(tid); + std::shared_ptr cb = find_control_block(tid); if (cb == nullptr) { LOG(WARNING) << "no result for this query, id=" << tid; ctx->on_failure(Status::InternalError("no result for this query")); @@ -145,7 +144,7 @@ void ResultBufferMgr::cancel_thread() { do { // get query std::vector query_to_cancel; - time_t now_time = time(NULL); + time_t now_time = time(nullptr); { std::lock_guard l(_timeout_lock); TimeoutMap::iterator end = _timeout_map.upper_bound(now_time + 1); diff --git a/be/src/runtime/result_buffer_mgr.h b/be/src/runtime/result_buffer_mgr.h index 8a1ef99547..1af035ac6e 100644 --- a/be/src/runtime/result_buffer_mgr.h +++ b/be/src/runtime/result_buffer_mgr.h @@ -18,11 +18,9 @@ #ifndef DORIS_BE_RUNTIME_RESULT_BUFFER_MGR_H #define DORIS_BE_RUNTIME_RESULT_BUFFER_MGR_H -#include -#include -#include #include #include +#include #include #include @@ -50,7 +48,7 @@ public: // the returned sender do not need release // sender is not used when call cancel or unregister Status create_sender(const TUniqueId& query_id, int buffer_size, - boost::shared_ptr* sender); + std::shared_ptr* sender); // fetch data, used by RPC Status fetch_data(const TUniqueId& fragment_id, TFetchDataResult* result); @@ -63,10 +61,10 @@ public: Status cancel_at_time(time_t cancel_time, const TUniqueId& query_id); private: - typedef std::unordered_map> BufferMap; + typedef std::unordered_map> BufferMap; typedef std::map> TimeoutMap; - boost::shared_ptr find_control_block(const TUniqueId& query_id); + std::shared_ptr find_control_block(const TUniqueId& query_id); // used to erase the buffer that fe not clears // when fe crush, this thread clear the buffer avoid memory leak in this backend diff --git a/be/src/runtime/result_file_sink.cpp b/be/src/runtime/result_file_sink.cpp index 41eedaca7d..9ee71eb344 100644 --- a/be/src/runtime/result_file_sink.cpp +++ b/be/src/runtime/result_file_sink.cpp @@ -31,9 +31,9 @@ namespace doris { -ResultFileSink::ResultFileSink(const RowDescriptor& row_desc, const std::vector& t_output_expr, - const TResultFileSink& sink) - : DataStreamSender(nullptr, 0, row_desc), _t_output_expr(t_output_expr){ +ResultFileSink::ResultFileSink(const RowDescriptor& row_desc, + const std::vector& t_output_expr, const TResultFileSink& sink) + : DataStreamSender(nullptr, 0, row_desc), _t_output_expr(t_output_expr) { CHECK(sink.__isset.file_options); _file_opts.reset(new ResultFileOptions(sink.file_options)); CHECK(sink.__isset.storage_backend_type); @@ -43,22 +43,22 @@ ResultFileSink::ResultFileSink(const RowDescriptor& row_desc, const std::vector< _name = "ResultFileSink"; } -ResultFileSink::ResultFileSink(const RowDescriptor& row_desc, const std::vector& t_output_expr, - const TResultFileSink& sink, - const std::vector& destinations, - ObjectPool* pool, int sender_id, DescriptorTbl& descs) - : DataStreamSender(pool, sender_id, row_desc), _t_output_expr(t_output_expr), - _output_row_descriptor(descs.get_tuple_descriptor(sink.output_tuple_id), false) { +ResultFileSink::ResultFileSink(const RowDescriptor& row_desc, + const std::vector& t_output_expr, const TResultFileSink& sink, + const std::vector& destinations, + ObjectPool* pool, int sender_id, DescriptorTbl& descs) + : DataStreamSender(pool, sender_id, row_desc), + _t_output_expr(t_output_expr), + _output_row_descriptor(descs.get_tuple_descriptor(sink.output_tuple_id), false) { CHECK(sink.__isset.file_options); _file_opts.reset(new ResultFileOptions(sink.file_options)); CHECK(sink.__isset.storage_backend_type); _storage_type = sink.storage_backend_type; _is_top_sink = false; DCHECK_EQ(destinations.size(), 1); - _channel_shared_ptrs.emplace_back( - new Channel(this, _output_row_descriptor, destinations[0].brpc_server, - destinations[0].fragment_instance_id, - sink.dest_node_id, _buf_size, true, true)); + _channel_shared_ptrs.emplace_back(new Channel( + this, _output_row_descriptor, destinations[0].brpc_server, + destinations[0].fragment_instance_id, sink.dest_node_id, _buf_size, true, true)); _channels.push_back(_channel_shared_ptrs.back().get()); _name = "ResultFileSink"; @@ -91,12 +91,12 @@ Status ResultFileSink::prepare(RuntimeState* state) { CHECK(_file_opts.get() != nullptr); if (_is_top_sink) { // create sender - RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(state->fragment_instance_id(), - _buf_size, &_sender)); + RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender( + state->fragment_instance_id(), _buf_size, &_sender)); // create writer - _writer.reset(new (std::nothrow) FileResultWriter(_file_opts.get(), _storage_type, - state->fragment_instance_id(), _output_expr_ctxs, - _profile, _sender.get(), nullptr)); + _writer.reset(new (std::nothrow) FileResultWriter( + _file_opts.get(), _storage_type, state->fragment_instance_id(), _output_expr_ctxs, + _profile, _sender.get(), nullptr)); } else { // init channel _profile = _pool->add(new RuntimeProfile(title.str())); @@ -104,16 +104,16 @@ Status ResultFileSink::prepare(RuntimeState* state) { _serialize_batch_timer = ADD_TIMER(profile(), "SerializeBatchTime"); _bytes_sent_counter = ADD_COUNTER(profile(), "BytesSent", TUnit::BYTES); _local_bytes_send_counter = ADD_COUNTER(profile(), "LocalBytesSent", TUnit::BYTES); - _uncompressed_bytes_counter = ADD_COUNTER(profile(), "UncompressedRowBatchSize", TUnit::BYTES); + _uncompressed_bytes_counter = + ADD_COUNTER(profile(), "UncompressedRowBatchSize", TUnit::BYTES); _mem_tracker = MemTracker::CreateTracker( _profile, -1, "ResultFileSink:" + print_id(state->fragment_instance_id()), state->instance_mem_tracker()); // create writer _output_batch = new RowBatch(_output_row_descriptor, 1024, _mem_tracker.get()); - _writer.reset(new (std::nothrow) FileResultWriter(_file_opts.get(), _storage_type, - state->fragment_instance_id(), _output_expr_ctxs, - _profile, nullptr, _output_batch)); - + _writer.reset(new (std::nothrow) FileResultWriter( + _file_opts.get(), _storage_type, state->fragment_instance_id(), _output_expr_ctxs, + _profile, nullptr, _output_batch)); } RETURN_IF_ERROR(_writer->init(state)); for (int i = 0; i < _channels.size(); ++i) { @@ -152,7 +152,7 @@ Status ResultFileSink::close(RuntimeState* state, Status exec_status) { _sender->close(final_status); } state->exec_env()->result_mgr()->cancel_at_time( - time(NULL) + config::result_buffer_cancelled_interval_time, + time(nullptr) + config::result_buffer_cancelled_interval_time, state->fragment_instance_id()); } else { if (final_status.ok()) { diff --git a/be/src/runtime/result_file_sink.h b/be/src/runtime/result_file_sink.h index 34d038cde0..f6d60ac224 100644 --- a/be/src/runtime/result_file_sink.h +++ b/be/src/runtime/result_file_sink.h @@ -19,11 +19,11 @@ #define DORIS_BE_RUNTIME_RESULT_FILE_SINK_H #include "common/status.h" -#include "runtime/data_stream_sender.h" -#include "runtime/descriptors.h" #include "gen_cpp/PaloInternalService_types.h" #include "gen_cpp/PlanNodes_types.h" #include "gen_cpp/Types_types.h" +#include "runtime/data_stream_sender.h" +#include "runtime/descriptors.h" namespace doris { @@ -43,11 +43,11 @@ public: // row_desc used for convert RowBatch to TRowBatch // buffer_size is the buffer size allocated to each query ResultFileSink(const RowDescriptor& row_desc, const std::vector& select_exprs, - const TResultFileSink& sink); + const TResultFileSink& sink); ResultFileSink(const RowDescriptor& row_desc, const std::vector& select_exprs, - const TResultFileSink& sink, - const std::vector& destinations, - ObjectPool* pool, int sender_id, DescriptorTbl& descs); + const TResultFileSink& sink, + const std::vector& destinations, ObjectPool* pool, + int sender_id, DescriptorTbl& descs); virtual ~ResultFileSink(); virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); @@ -72,12 +72,11 @@ private: std::vector _output_expr_ctxs; RowDescriptor _output_row_descriptor; - boost::shared_ptr _sender; - boost::shared_ptr _writer; + std::shared_ptr _sender; + std::shared_ptr _writer; RowBatch* _output_batch = nullptr; - int _buf_size = 1024; // Allocated from _pool + int _buf_size = 1024; // Allocated from _pool bool _is_top_sink = true; - }; } // namespace doris diff --git a/be/src/runtime/result_sink.cpp b/be/src/runtime/result_sink.cpp index 3a04138f9c..49870e6cd7 100644 --- a/be/src/runtime/result_sink.cpp +++ b/be/src/runtime/result_sink.cpp @@ -121,7 +121,7 @@ Status ResultSink::close(RuntimeState* state, Status exec_status) { _sender->close(final_status); } state->exec_env()->result_mgr()->cancel_at_time( - time(NULL) + config::result_buffer_cancelled_interval_time, + time(nullptr) + config::result_buffer_cancelled_interval_time, state->fragment_instance_id()); Expr::close(_output_expr_ctxs, state); diff --git a/be/src/runtime/result_sink.h b/be/src/runtime/result_sink.h index a7c1da4c51..516c9946e7 100644 --- a/be/src/runtime/result_sink.h +++ b/be/src/runtime/result_sink.h @@ -37,7 +37,7 @@ class MemTracker; class ResultFileOptions; namespace vectorized { - class VExprContext; +class VExprContext; } class ResultSink : public DataSink { @@ -74,8 +74,8 @@ private: const std::vector& _t_output_expr; std::vector _output_expr_ctxs; - boost::shared_ptr _sender; - boost::shared_ptr _writer; + std::shared_ptr _sender; + std::shared_ptr _writer; RuntimeProfile* _profile; // Allocated from _pool int _buf_size; // Allocated from _pool diff --git a/be/src/runtime/row_batch.cpp b/be/src/runtime/row_batch.cpp index 25855f630f..b5d38748c1 100644 --- a/be/src/runtime/row_batch.cpp +++ b/be/src/runtime/row_batch.cpp @@ -52,7 +52,7 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, int capacity, MemTracker* mem_ _need_to_return(false), _tuple_data_pool(new MemPool(_mem_tracker)), _agg_object_pool(new ObjectPool()) { - DCHECK(_mem_tracker != NULL); + DCHECK(_mem_tracker != nullptr); DCHECK_GT(capacity, 0); _tuple_ptrs_size = _capacity * _num_tuples_per_row * sizeof(Tuple*); DCHECK_GT(_tuple_ptrs_size, 0); @@ -60,7 +60,7 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, int capacity, MemTracker* mem_ if (config::enable_partitioned_aggregation) { _mem_tracker->Consume(_tuple_ptrs_size); _tuple_ptrs = reinterpret_cast(malloc(_tuple_ptrs_size)); - DCHECK(_tuple_ptrs != NULL); + DCHECK(_tuple_ptrs != nullptr); } else { _tuple_ptrs = reinterpret_cast(_tuple_data_pool->allocate(_tuple_ptrs_size)); } @@ -219,19 +219,19 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, const TRowBatch& input_batch, _need_to_return(false), _tuple_data_pool(new MemPool(_mem_tracker)), _agg_object_pool(new ObjectPool()) { - DCHECK(_mem_tracker != NULL); + DCHECK(_mem_tracker != nullptr); _tuple_ptrs_size = _num_rows * input_batch.row_tuples.size() * sizeof(Tuple*); DCHECK_GT(_tuple_ptrs_size, 0); // TODO: switch to Init() pattern so we can check memory limit and return Status. if (config::enable_partitioned_aggregation) { _mem_tracker->Consume(_tuple_ptrs_size); _tuple_ptrs = reinterpret_cast(malloc(_tuple_ptrs_size)); - DCHECK(_tuple_ptrs != NULL); + DCHECK(_tuple_ptrs != nullptr); } else { _tuple_ptrs = reinterpret_cast(_tuple_data_pool->allocate(_tuple_ptrs_size)); } - uint8_t* tuple_data = NULL; + uint8_t* tuple_data = nullptr; if (input_batch.is_compressed) { // Decompress tuple data into data pool const char* compressed_data = input_batch.tuple_data.c_str(); @@ -255,7 +255,7 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, const TRowBatch& input_batch, for (vector::const_iterator offset = input_batch.tuple_offsets.begin(); offset != input_batch.tuple_offsets.end(); ++offset) { if (*offset == -1) { - _tuple_ptrs[tuple_idx++] = NULL; + _tuple_ptrs[tuple_idx++] = nullptr; } else { // _tuple_ptrs[tuple_idx++] = // reinterpret_cast(_tuple_data_pool->get_data_ptr(*offset)); @@ -282,7 +282,7 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, const TRowBatch& input_batch, } Tuple* tuple = row->get_tuple(j); - if (tuple == NULL) { + if (tuple == nullptr) { continue; } @@ -358,10 +358,10 @@ void RowBatch::clear() { _blocks[i]->del(); } if (config::enable_partitioned_aggregation) { - DCHECK(_tuple_ptrs != NULL); + DCHECK(_tuple_ptrs != nullptr); free(_tuple_ptrs); _mem_tracker->Release(_tuple_ptrs_size); - _tuple_ptrs = NULL; + _tuple_ptrs = nullptr; } _cleared = true; } @@ -394,7 +394,7 @@ size_t RowBatch::serialize(TRowBatch* output_batch) { std::vector::const_iterator desc = tuple_descs.begin(); for (int j = 0; desc != tuple_descs.end(); ++desc, ++j) { - if (row->get_tuple(j) == NULL) { + if (row->get_tuple(j) == nullptr) { // NULLs are encoded as -1 output_batch->tuple_offsets.push_back(-1); continue; @@ -503,7 +503,7 @@ size_t RowBatch::serialize(PRowBatch* output_batch) { } void RowBatch::add_io_buffer(DiskIoMgr::BufferDescriptor* buffer) { - DCHECK(buffer != NULL); + DCHECK(buffer != nullptr); _io_buffers.push_back(buffer); _auxiliary_mem_usage += buffer->buffer_len(); buffer->set_mem_tracker(std::shared_ptr(_mem_tracker)); // TODO(yingchun): fixme @@ -519,7 +519,7 @@ Status RowBatch::resize_and_allocate_tuple_buffer(RuntimeState* state, int64_t* *tuple_buffer_size = static_cast(row_size) * _capacity; // TODO(dhc): change allocate to try_allocate? *buffer = _tuple_data_pool->allocate(*tuple_buffer_size); - if (*buffer == NULL) { + if (*buffer == nullptr) { std::stringstream ss; ss << "Failed to allocate tuple buffer" << *tuple_buffer_size; LOG(WARNING) << ss.str(); @@ -529,19 +529,19 @@ Status RowBatch::resize_and_allocate_tuple_buffer(RuntimeState* state, int64_t* } void RowBatch::add_tuple_stream(BufferedTupleStream2* stream) { - DCHECK(stream != NULL); + DCHECK(stream != nullptr); _tuple_streams.push_back(stream); _auxiliary_mem_usage += stream->byte_size(); } void RowBatch::add_block(BufferedBlockMgr2::Block* block) { - DCHECK(block != NULL); + DCHECK(block != nullptr); _blocks.push_back(block); _auxiliary_mem_usage += block->buffer_len(); } void RowBatch::reset() { - DCHECK(_tuple_data_pool.get() != NULL); + DCHECK(_tuple_data_pool.get() != nullptr); _num_rows = 0; _capacity = _tuple_ptrs_size / (_num_tuples_per_row * sizeof(Tuple*)); _has_in_flight_row = false; @@ -668,7 +668,7 @@ void RowBatch::acquire_state(RowBatch* src) { if (!config::enable_partitioned_aggregation) { // Tuple pointers are allocated from tuple_data_pool_ so are transferred. _tuple_ptrs = src->_tuple_ptrs; - src->_tuple_ptrs = NULL; + src->_tuple_ptrs = nullptr; } else { // tuple_ptrs_ were allocated with malloc so can be swapped between batches. std::swap(_tuple_ptrs, src->_tuple_ptrs); @@ -701,7 +701,7 @@ size_t RowBatch::total_byte_size() { for (int j = 0; desc != tuple_descs.end(); ++desc, ++j) { Tuple* tuple = row->get_tuple(j); - if (tuple == NULL) { + if (tuple == nullptr) { continue; } result += (*desc)->byte_size(); diff --git a/be/src/runtime/row_batch.h b/be/src/runtime/row_batch.h index b0733e6ce3..1474f79056 100644 --- a/be/src/runtime/row_batch.h +++ b/be/src/runtime/row_batch.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_RUNTIME_ROW_BATCH_H #define DORIS_BE_RUNTIME_ROW_BATCH_H -#include #include #include @@ -154,7 +153,7 @@ public: // that will eventually be attached to this row batch. We need to make sure // the tuple pool does not accumulate excessive memory. bool at_capacity(MemPool* tuple_pool) { - DCHECK(tuple_pool != NULL); + DCHECK(tuple_pool != nullptr); return at_capacity() || tuple_pool->total_allocated_bytes() > AT_CAPACITY_MEM_USAGE; } @@ -173,7 +172,7 @@ public: size_t total_byte_size(); TupleRow* get_row(int row_idx) const { - DCHECK(_tuple_ptrs != NULL); + DCHECK(_tuple_ptrs != nullptr); DCHECK_GE(row_idx, 0); //DCHECK_LT(row_idx, _num_rows + (_has_in_flight_row ? 1 : 0)); return reinterpret_cast(_tuple_ptrs + row_idx * _num_tuples_per_row); @@ -377,7 +376,7 @@ public: /// Allocates a buffer large enough for the fixed-length portion of 'capacity_' rows in /// this batch from 'tuple_data_pool_'. 'capacity_' is reduced if the allocation would /// exceed FIXED_LEN_BUFFER_LIMIT. Always returns enough space for at least one row. - /// Returns Status::MemoryLimitExceeded("Memory limit exceeded") and sets 'buffer' to NULL if a memory limit would + /// Returns Status::MemoryLimitExceeded("Memory limit exceeded") and sets 'buffer' to nullptr if a memory limit would /// have been exceeded. 'state' is used to log the error. /// On success, sets 'buffer_size' to the size in bytes and 'buffer' to the buffer. Status resize_and_allocate_tuple_buffer(RuntimeState* state, int64_t* buffer_size, @@ -445,7 +444,7 @@ private: bool _need_to_return; // holding (some of the) data referenced by rows - boost::scoped_ptr _tuple_data_pool; + std::unique_ptr _tuple_data_pool; // holding some complex agg object data (bitmap, hll) std::unique_ptr _agg_object_pool; diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index ceedbc86d7..4b6d1e8c44 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -266,14 +266,14 @@ Status RuntimeState::init_buffer_poolstate() { VLOG_QUERY << "Buffer pool limit for " << print_id(_query_id) << ": " << max_reservation; _buffer_reservation = _obj_pool->add(new ReservationTracker); - _buffer_reservation->InitChildTracker(NULL, exec_env->buffer_reservation(), + _buffer_reservation->InitChildTracker(nullptr, exec_env->buffer_reservation(), _query_mem_tracker.get(), max_reservation); return Status::OK(); } Status RuntimeState::create_block_mgr() { - DCHECK(_block_mgr2.get() == NULL); + DCHECK(_block_mgr2.get() == nullptr); int64_t block_mgr_limit = _query_mem_tracker->limit(); if (block_mgr_limit < 0) { @@ -339,11 +339,11 @@ Status RuntimeState::set_mem_limit_exceeded(MemTracker* tracker, int64_t failed_ } } - DCHECK(_query_mem_tracker.get() != NULL); + DCHECK(_query_mem_tracker.get() != nullptr); std::stringstream ss; ss << "Memory Limit Exceeded\n"; if (failed_allocation_size != 0) { - DCHECK(tracker != NULL); + DCHECK(tracker != nullptr); ss << " " << tracker->label() << " could not allocate " << PrettyPrinter::print(failed_allocation_size, TUnit::BYTES) << " without exceeding limit." << std::endl; diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index c30baf72f0..30179ebfd8 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -19,7 +19,6 @@ #define DORIS_BE_SRC_QUERY_RUNTIME_RUNTIME_STATE_H #include -#include #include #include #include @@ -74,7 +73,7 @@ public: // RuntimeState for executing expr in fe-support. RuntimeState(const TQueryGlobals& query_globals); - // Empty d'tor to avoid issues with scoped_ptr. + // Empty d'tor to avoid issues with unique_ptr. ~RuntimeState(); // Set per-query state. @@ -150,7 +149,7 @@ public: Status create_codegen(); BufferedBlockMgr2* block_mgr2() { - DCHECK(_block_mgr2.get() != NULL); + DCHECK(_block_mgr2.get() != nullptr); return _block_mgr2.get(); } @@ -229,13 +228,13 @@ public: // If 'failed_allocation_size' is not 0, then it is the size of the allocation (in // bytes) that would have exceeded the limit allocated for 'tracker'. // This value and tracker are only used for error reporting. - // If 'msg' is non-NULL, it will be appended to query_status_ in addition to the + // If 'msg' is non-nullptr, it will be appended to query_status_ in addition to the // generic "Memory limit exceeded" error. - Status set_mem_limit_exceeded(MemTracker* tracker = NULL, int64_t failed_allocation_size = 0, - const std::string* msg = NULL); + Status set_mem_limit_exceeded(MemTracker* tracker = nullptr, int64_t failed_allocation_size = 0, + const std::string* msg = nullptr); Status set_mem_limit_exceeded(const std::string& msg) { - return set_mem_limit_exceeded(NULL, 0, &msg); + return set_mem_limit_exceeded(nullptr, 0, &msg); } // Returns a non-OK status if query execution should stop (e.g., the query was cancelled @@ -372,18 +371,14 @@ public: int64_t get_load_mem_limit(); RuntimeFilterMgr* runtime_filter_mgr() { return _runtime_filter_mgr.get(); } - - void set_query_fragments_ctx(QueryFragmentsCtx* ctx) { - _query_ctx = ctx; - } - QueryFragmentsCtx* get_query_fragments_ctx() { - return _query_ctx; - } + void set_query_fragments_ctx(QueryFragmentsCtx* ctx) { _query_ctx = ctx; } + + QueryFragmentsCtx* get_query_fragments_ctx() { return _query_ctx; } private: // Use a custom block manager for the query for testing purposes. - void set_block_mgr2(const boost::shared_ptr& block_mgr) { + void set_block_mgr2(const std::shared_ptr& block_mgr) { _block_mgr2 = block_mgr; } @@ -422,7 +417,7 @@ private: // Receivers depend on the descriptor table and we need to guarantee that their control // blocks are removed from the data stream manager before the objects in the // descriptor table are destroyed. - boost::scoped_ptr _data_stream_recvrs_pool; + std::unique_ptr _data_stream_recvrs_pool; // Lock protecting _error_log and _unreported_error_idx std::mutex _error_log_lock; @@ -464,12 +459,12 @@ private: // will not necessarily be set in all error cases. std::mutex _process_status_lock; Status _process_status; - //boost::scoped_ptr _udf_pool; + //std::unique_ptr _udf_pool; // BufferedBlockMgr object used to allocate and manage blocks of input data in memory // with a fixed memory budget. // The block mgr is shared by all fragments for this query. - boost::shared_ptr _block_mgr2; + std::shared_ptr _block_mgr2; // This is the node id of the root node for this plan fragment. This is used as the // hash seed and has two useful properties: @@ -512,8 +507,8 @@ private: ReservationTracker* _buffer_reservation = nullptr; /// Buffer reservation for this fragment instance - a child of the query buffer - /// reservation. Non-NULL if 'query_state_' is not NULL. - boost::scoped_ptr _instance_buffer_reservation; + /// reservation. Non-nullptr if 'query_state_' is not nullptr. + std::unique_ptr _instance_buffer_reservation; /// Pool of buffer reservations used to distribute initial reservations to operators /// in the query. Contains a ReservationTracker that is a child of diff --git a/be/src/runtime/sorted_run_merger.cc b/be/src/runtime/sorted_run_merger.cc index 7f98556a46..28baab462e 100644 --- a/be/src/runtime/sorted_run_merger.cc +++ b/be/src/runtime/sorted_run_merger.cc @@ -44,7 +44,7 @@ public: // Construct an instance from a sorted input run. BatchedRowSupplier(SortedRunMerger* parent, const RunBatchSupplier& sorted_run) : _sorted_run(sorted_run), - _input_row_batch(NULL), + _input_row_batch(nullptr), _input_row_batch_index(-1), _parent(parent) {} @@ -54,30 +54,30 @@ public: virtual Status init(bool* done) { *done = false; RETURN_IF_ERROR(_sorted_run(&_input_row_batch)); - if (_input_row_batch == NULL) { + if (_input_row_batch == nullptr) { *done = true; return Status::OK(); } - RETURN_IF_ERROR(next(NULL, done)); + RETURN_IF_ERROR(next(nullptr, done)); return Status::OK(); } // Increment the current row index. If the current input batch is exhausted fetch the - // next one from the sorted run. Transfer ownership to transfer_batch if not NULL. + // next one from the sorted run. Transfer ownership to transfer_batch if not nullptr. virtual Status next(RowBatch* transfer_batch, bool* done) { - DCHECK(_input_row_batch != NULL); + DCHECK(_input_row_batch != nullptr); ++_input_row_batch_index; if (_input_row_batch_index < _input_row_batch->num_rows()) { *done = false; } else { ScopedTimer timer(_parent->_get_next_batch_timer); - if (transfer_batch != NULL) { + if (transfer_batch != nullptr) { _input_row_batch->transfer_resource_ownership(transfer_batch); } RETURN_IF_ERROR(_sorted_run(&_input_row_batch)); - DCHECK(_input_row_batch == NULL || _input_row_batch->num_rows() > 0); - *done = _input_row_batch == NULL; + DCHECK(_input_row_batch == nullptr || _input_row_batch->num_rows() > 0); + *done = _input_row_batch == nullptr; _input_row_batch_index = 0; } return Status::OK(); @@ -126,19 +126,19 @@ public: _pull_task_thread = std::thread( &SortedRunMerger::ParallelBatchedRowSupplier::process_sorted_run_task, this); - RETURN_IF_ERROR(next(NULL, done)); + RETURN_IF_ERROR(next(nullptr, done)); return Status::OK(); } // Increment the current row index. If the current input batch is exhausted fetch the - // next one from the sorted run. Transfer ownership to transfer_batch if not NULL. + // next one from the sorted run. Transfer ownership to transfer_batch if not nullptr. Status next(RowBatch* transfer_batch, bool* done) override { ++_input_row_batch_index; if (_input_row_batch && _input_row_batch_index < _input_row_batch->num_rows()) { *done = false; } else { ScopedTimer timer(_parent->_get_next_batch_timer); - if (_input_row_batch && transfer_batch != NULL) { + if (_input_row_batch && transfer_batch != nullptr) { _input_row_batch->transfer_resource_ownership(transfer_batch); } // release the mem of child merge @@ -164,9 +164,9 @@ private: // The backup row batch input be backup batch from _sort_run. RowBatch* _input_row_batch_backup; - std::atomic_bool _backup_ready{false}; + std::atomic_bool _backup_ready {false}; - std::atomic_bool _cancel{false}; + std::atomic_bool _cancel {false}; std::thread _pull_task_thread; @@ -188,7 +188,7 @@ private: // do merge from sender queue data _status_backup = _sorted_run(&_input_row_batch_backup); _backup_ready = true; - Defer defer_op{[this]() { _batch_prepared_cv.notify_one(); }}; + Defer defer_op {[this]() { _batch_prepared_cv.notify_one(); }}; if (!_status_backup.ok() || _input_row_batch_backup == nullptr || _cancel) { if (!_status_backup.ok()) _input_row_batch_backup = nullptr; @@ -240,7 +240,7 @@ Status SortedRunMerger::prepare(const vector& input_runs, bool BatchedRowSupplier* new_elem = _pool.add(parallel ? new ParallelBatchedRowSupplier(this, input_run) : new BatchedRowSupplier(this, input_run)); - DCHECK(new_elem != NULL); + DCHECK(new_elem != nullptr); bool empty = false; RETURN_IF_ERROR(new_elem->init(&empty)); if (!empty) { @@ -290,7 +290,7 @@ Status SortedRunMerger::get_next(RowBatch* output_batch, bool* eos) { bool min_run_complete = false; // Advance to the next element in min. output_batch is supplied to transfer // resource ownership if the input batch in min is exhausted. - RETURN_IF_ERROR(min->next(_deep_copy_input ? NULL : output_batch, &min_run_complete)); + RETURN_IF_ERROR(min->next(_deep_copy_input ? nullptr : output_batch, &min_run_complete)); if (min_run_complete) { // Remove the element from the heap. iter_swap(_min_heap.begin(), _min_heap.end() - 1); diff --git a/be/src/runtime/sorted_run_merger.h b/be/src/runtime/sorted_run_merger.h index a08ea8c0f5..b73cdc1b0c 100644 --- a/be/src/runtime/sorted_run_merger.h +++ b/be/src/runtime/sorted_run_merger.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_RUNTIME_SORTED_RUN_MERGER_H #define DORIS_BE_SRC_RUNTIME_SORTED_RUN_MERGER_H -#include #include #include "common/object_pool.h" @@ -44,7 +43,7 @@ class RuntimeProfile; class SortedRunMerger { public: // Function that returns the next batch of rows from an input sorted run. The batch - // is owned by the supplier (i.e. not SortedRunMerger). eos is indicated by an NULL + // is owned by the supplier (i.e. not SortedRunMerger). eos is indicated by an nullptr // batch being returned. typedef std::function RunBatchSupplier; diff --git a/be/src/runtime/spill_sorter.cc b/be/src/runtime/spill_sorter.cc index 4d5d655514..a461ebe7fa 100644 --- a/be/src/runtime/spill_sorter.cc +++ b/be/src/runtime/spill_sorter.cc @@ -34,7 +34,7 @@ using std::vector; using std::bind; using std::function; using std::mem_fn; -using boost::scoped_ptr; +using std::unique_ptr; namespace doris { @@ -197,13 +197,13 @@ private: // tuples comprising this run. The data pointed to by the var-len slots are in // _var_len_blocks. // If _is_sorted is true, the tuples in _fixed_len_blocks will be in sorted order. - // _fixed_len_blocks[i] is NULL iff it has been deleted. + // _fixed_len_blocks[i] is nullptr iff it has been deleted. vector _fixed_len_blocks; // Sequence of blocks in this run containing the var-length data corresponding to the // var-length columns from _fixed_len_blocks. These are reconstructed to be in sorted // order in unpin_all_blocks(). - // _var_len_blocks[i] is NULL iff it has been deleted. + // _var_len_blocks[i] is nullptr iff it has been deleted. vector _var_len_blocks; // If there are var-len slots, an extra pinned block is used to copy out var-len data @@ -219,7 +219,7 @@ private: // _buffered_batch is used to return TupleRows to the merger when this run is being // merged. _buffered_batch is returned in calls to get_next_batch(). - scoped_ptr _buffered_batch; + unique_ptr _buffered_batch; // Members used when a run is read in get_next(). // The index into the _fixed_len_blocks and _var_len_blocks vectors of the current blocks being @@ -261,7 +261,7 @@ private: class TupleIterator { public: TupleIterator(TupleSorter* parent, int64_t index) - : _parent(parent), _index(index), _current_tuple(NULL) { + : _parent(parent), _index(index), _current_tuple(nullptr) { DCHECK_GE(index, 0); DCHECK_LE(index, _parent->_run->_num_tuples); // If the run is empty, only _index and _current_tuple are initialized. @@ -390,29 +390,30 @@ SpillSorter::Run::Run(SpillSorter* parent, TupleDescriptor* sort_tuple_desc, boo _materialize_slots(materialize_slots), _is_sorted(!materialize_slots), _is_pinned(true), - _var_len_copy_block(NULL), + _var_len_copy_block(nullptr), _num_tuples(0) {} Status SpillSorter::Run::init() { - BufferedBlockMgr2::Block* block = NULL; - RETURN_IF_ERROR(_sorter->_block_mgr->get_new_block(_sorter->_block_mgr_client, NULL, &block)); - if (block == NULL) { + BufferedBlockMgr2::Block* block = nullptr; + RETURN_IF_ERROR( + _sorter->_block_mgr->get_new_block(_sorter->_block_mgr_client, nullptr, &block)); + if (block == nullptr) { return Status::MemoryLimitExceeded(get_mem_alloc_failed_error_msg("fixed")); } _fixed_len_blocks.push_back(block); if (_has_var_len_slots) { RETURN_IF_ERROR( - _sorter->_block_mgr->get_new_block(_sorter->_block_mgr_client, NULL, &block)); - if (block == NULL) { + _sorter->_block_mgr->get_new_block(_sorter->_block_mgr_client, nullptr, &block)); + if (block == nullptr) { return Status::MemoryLimitExceeded(get_mem_alloc_failed_error_msg("variable")); } _var_len_blocks.push_back(block); if (!_is_sorted) { - RETURN_IF_ERROR(_sorter->_block_mgr->get_new_block(_sorter->_block_mgr_client, NULL, + RETURN_IF_ERROR(_sorter->_block_mgr->get_new_block(_sorter->_block_mgr_client, nullptr, &_var_len_copy_block)); - if (_var_len_copy_block == NULL) { + if (_var_len_copy_block == nullptr) { return Status::MemoryLimitExceeded(get_mem_alloc_failed_error_msg("variable")); } } @@ -462,7 +463,7 @@ Status SpillSorter::Run::add_batch(RowBatch* batch, int start_index, int* num_pr Tuple* new_tuple = cur_fixed_len_block->allocate(_sort_tuple_size); if (_materialize_slots) { new_tuple->materialize_exprs( - input_row, *_sort_tuple_desc, _sorter->_sort_tuple_slot_expr_ctxs, NULL, + input_row, *_sort_tuple_desc, _sorter->_sort_tuple_slot_expr_ctxs, nullptr, &string_values, &total_var_len); if (total_var_len > _sorter->_block_mgr->max_block_size()) { std::stringstream error_msg; @@ -533,41 +534,41 @@ Status SpillSorter::Run::add_batch(RowBatch* batch, int start_index, int* num_pr } void SpillSorter::Run::transfer_resources(RowBatch* row_batch) { - DCHECK(row_batch != NULL); + DCHECK(row_batch != nullptr); for (BufferedBlockMgr2::Block* block : _fixed_len_blocks) { - if (block != NULL) { + if (block != nullptr) { row_batch->add_block(block); } } _fixed_len_blocks.clear(); for (BufferedBlockMgr2::Block* block : _var_len_blocks) { - if (block != NULL) { + if (block != nullptr) { row_batch->add_block(block); } } _var_len_blocks.clear(); - if (_var_len_copy_block != NULL) { + if (_var_len_copy_block != nullptr) { row_batch->add_block(_var_len_copy_block); - _var_len_copy_block = NULL; + _var_len_copy_block = nullptr; } } void SpillSorter::Run::delete_all_blocks() { for (BufferedBlockMgr2::Block* block : _fixed_len_blocks) { - if (block != NULL) { + if (block != nullptr) { block->del(); } } _fixed_len_blocks.clear(); for (BufferedBlockMgr2::Block* block : _var_len_blocks) { - if (block != NULL) { + if (block != nullptr) { block->del(); } } _var_len_blocks.clear(); - if (_var_len_copy_block != NULL) { + if (_var_len_copy_block != nullptr) { _var_len_copy_block->del(); - _var_len_copy_block = NULL; + _var_len_copy_block = nullptr; } } @@ -578,13 +579,13 @@ Status SpillSorter::Run::unpin_all_blocks() { int64_t var_data_offset = 0; int total_var_len = 0; string_values.reserve(_sort_tuple_desc->string_slots().size()); - BufferedBlockMgr2::Block* cur_sorted_var_len_block = NULL; + BufferedBlockMgr2::Block* cur_sorted_var_len_block = nullptr; if (has_var_len_blocks()) { - DCHECK(_var_len_copy_block != NULL); + DCHECK(_var_len_copy_block != nullptr); sorted_var_len_blocks.push_back(_var_len_copy_block); cur_sorted_var_len_block = _var_len_copy_block; } else { - DCHECK(_var_len_copy_block == NULL); + DCHECK(_var_len_copy_block == nullptr); } for (int i = 0; i < _fixed_len_blocks.size(); ++i) { @@ -595,7 +596,7 @@ Status SpillSorter::Run::unpin_all_blocks() { Tuple* cur_tuple = reinterpret_cast(cur_fixed_block->buffer() + block_offset); collect_non_null_varslots(cur_tuple, &string_values, &total_var_len); - DCHECK(cur_sorted_var_len_block != NULL); + DCHECK(cur_sorted_var_len_block != nullptr); if (cur_sorted_var_len_block->bytes_remaining() < total_var_len) { bool added = false; RETURN_IF_ERROR(try_add_block(&sorted_var_len_blocks, &added)); @@ -623,9 +624,9 @@ Status SpillSorter::Run::unpin_all_blocks() { } _var_len_blocks.clear(); sorted_var_len_blocks.swap(_var_len_blocks); - // Set _var_len_copy_block to NULL since it's now in _var_len_blocks and is no longer + // Set _var_len_copy_block to nullptr since it's now in _var_len_blocks and is no longer // needed. - _var_len_copy_block = NULL; + _var_len_copy_block = nullptr; _is_pinned = false; return Status::OK(); } @@ -673,7 +674,7 @@ Status SpillSorter::Run::prepare_read() { } Status SpillSorter::Run::get_next_batch(RowBatch** output_batch) { - if (_buffered_batch.get() != NULL) { + if (_buffered_batch.get() != nullptr) { _buffered_batch->reset(); // Fill more rows into _buffered_batch. bool eos = false; @@ -694,15 +695,15 @@ Status SpillSorter::Run::get_next_batch(RowBatch** output_batch) { _buffered_batch.reset(); // The merge is complete. Delete the last blocks in the run. _fixed_len_blocks.back()->del(); - _fixed_len_blocks[_fixed_len_blocks.size() - 1] = NULL; + _fixed_len_blocks[_fixed_len_blocks.size() - 1] = nullptr; if (has_var_len_blocks()) { _var_len_blocks.back()->del(); - _var_len_blocks[_var_len_blocks.size() - 1] = NULL; + _var_len_blocks[_var_len_blocks.size() - 1] = nullptr; } } } - // *output_batch == NULL indicates eos. + // *output_batch == nullptr indicates eos. *output_batch = _buffered_batch.get(); return Status::OK(); } @@ -724,7 +725,7 @@ Status SpillSorter::Run::get_next(RowBatch* output_batch, bool* eos) { // get_next(). if (_pin_next_fixed_len_block) { _fixed_len_blocks[_fixed_len_blocks_index - 1]->del(); - _fixed_len_blocks[_fixed_len_blocks_index - 1] = NULL; + _fixed_len_blocks[_fixed_len_blocks_index - 1] = nullptr; bool pinned; RETURN_IF_ERROR(fixed_len_block->pin(&pinned)); // Temporary work-around for IMPALA-2344. Fail the query with OOM rather than @@ -736,7 +737,7 @@ Status SpillSorter::Run::get_next(RowBatch* output_batch, bool* eos) { } if (_pin_next_var_len_block) { _var_len_blocks[_var_len_blocks_index - 1]->del(); - _var_len_blocks[_var_len_blocks_index - 1] = NULL; + _var_len_blocks[_var_len_blocks_index - 1] = nullptr; bool pinned; RETURN_IF_ERROR(_var_len_blocks[_var_len_blocks_index]->pin(&pinned)); // Temporary work-around for IMPALA-2344. Fail the query with OOM rather than @@ -749,10 +750,10 @@ Status SpillSorter::Run::get_next(RowBatch* output_batch, bool* eos) { } // get_next fills rows into the output batch until a block boundary is reached. - DCHECK(fixed_len_block != NULL); + DCHECK(fixed_len_block != nullptr); while (!output_batch->at_capacity() && _fixed_len_block_offset < fixed_len_block->valid_data_len()) { - DCHECK(fixed_len_block != NULL); + DCHECK(fixed_len_block != nullptr); Tuple* input_tuple = reinterpret_cast(fixed_len_block->buffer() + _fixed_len_block_offset); @@ -835,7 +836,7 @@ Status SpillSorter::Run::try_add_block(vector* block_ BufferedBlockMgr2::Block* last_block = block_sequence->back(); if (!_is_sorted) { _sorter->_sorted_data_size->update(last_block->valid_data_len()); - last_block = NULL; + last_block = nullptr; } else { // If the run is sorted, we will unpin the last block and extend the run. } @@ -843,7 +844,7 @@ Status SpillSorter::Run::try_add_block(vector* block_ BufferedBlockMgr2::Block* new_block; RETURN_IF_ERROR( _sorter->_block_mgr->get_new_block(_sorter->_block_mgr_client, last_block, &new_block)); - if (new_block != NULL) { + if (new_block != nullptr) { *added = true; block_sequence->push_back(new_block); } else { @@ -997,7 +998,7 @@ void SpillSorter::TupleSorter::sort_helper(TupleIterator first, TupleIterator la while (last._index - first._index > INSERTION_THRESHOLD) { TupleIterator mid(this, first._index + (last._index - first._index) / 2); - DCHECK(mid._current_tuple != NULL); + DCHECK(mid._current_tuple != nullptr); // partition() splits the tuples in [first, last) into two groups (<= mid iter // and >= mid iter) in-place. 'cut' is the index of the first tuple in the second group. TupleIterator cut = partition(first, last, mid); @@ -1033,19 +1034,19 @@ SpillSorter::SpillSorter(const TupleRowComparator& compare_less_than, RuntimeState* state) : _state(state), _compare_less_than(compare_less_than), - _in_mem_tuple_sorter(NULL), + _in_mem_tuple_sorter(nullptr), _block_mgr(state->block_mgr2()), - _block_mgr_client(NULL), + _block_mgr_client(nullptr), _has_var_len_slots(false), _sort_tuple_slot_expr_ctxs(slot_materialize_expr_ctxs), _mem_tracker(mem_tracker), _output_row_desc(output_row_desc), - _unsorted_run(NULL), + _unsorted_run(nullptr), _profile(profile), - _initial_runs_counter(NULL), - _num_merges_counter(NULL), - _in_mem_sort_timer(NULL), - _sorted_data_size(NULL), + _initial_runs_counter(nullptr), + _num_merges_counter(nullptr), + _in_mem_sort_timer(nullptr), + _sorted_data_size(nullptr), _spilled(false) {} SpillSorter::~SpillSorter() { @@ -1056,14 +1057,14 @@ SpillSorter::~SpillSorter() { for (deque::iterator it = _merging_runs.begin(); it != _merging_runs.end(); ++it) { (*it)->delete_all_blocks(); } - if (_unsorted_run != NULL) { + if (_unsorted_run != nullptr) { _unsorted_run->delete_all_blocks(); } _block_mgr->clear_reservations(_block_mgr_client); } Status SpillSorter::init() { - DCHECK(_unsorted_run == NULL) << "Already initialized"; + DCHECK(_unsorted_run == nullptr) << "Already initialized"; TupleDescriptor* sort_tuple_desc = _output_row_desc->tuple_descriptors()[0]; _has_var_len_slots = sort_tuple_desc->has_varlen_slots(); _in_mem_tuple_sorter.reset(new TupleSorter(_compare_less_than, _block_mgr->max_block_size(), @@ -1084,14 +1085,14 @@ Status SpillSorter::init() { RETURN_IF_ERROR(_block_mgr->register_client(min_blocks_required, _mem_tracker, _state, &_block_mgr_client)); - DCHECK(_unsorted_run != NULL); + DCHECK(_unsorted_run != nullptr); RETURN_IF_ERROR(_unsorted_run->init()); return Status::OK(); } Status SpillSorter::add_batch(RowBatch* batch) { - DCHECK(_unsorted_run != NULL); - DCHECK(batch != NULL); + DCHECK(_unsorted_run != nullptr); + DCHECK(batch != nullptr); int num_processed = 0; int cur_batch_index = 0; while (cur_batch_index < batch->num_rows()) { @@ -1180,7 +1181,7 @@ Status SpillSorter::reset() { _merging_runs.clear(); _sorted_runs.clear(); _obj_pool.clear(); - DCHECK(_unsorted_run == NULL); + DCHECK(_unsorted_run == nullptr); _unsorted_run = _obj_pool.add(new Run(this, _output_row_desc->tuple_descriptors()[0], true)); RETURN_IF_ERROR(_unsorted_run->init()); return Status::OK(); @@ -1195,7 +1196,7 @@ Status SpillSorter::sort_run() { _unsorted_run->_fixed_len_blocks.pop_back(); } if (_has_var_len_slots) { - DCHECK(_unsorted_run->_var_len_copy_block != NULL); + DCHECK(_unsorted_run->_var_len_copy_block != nullptr); last_block = _unsorted_run->_var_len_blocks.back(); if (last_block->valid_data_len() > 0) { _sorted_data_size->update(last_block->valid_data_len()); @@ -1204,7 +1205,7 @@ Status SpillSorter::sort_run() { _unsorted_run->_var_len_blocks.pop_back(); if (_unsorted_run->_var_len_blocks.size() == 0) { _unsorted_run->_var_len_copy_block->del(); - _unsorted_run->_var_len_copy_block = NULL; + _unsorted_run->_var_len_copy_block = nullptr; } } } @@ -1214,7 +1215,7 @@ Status SpillSorter::sort_run() { RETURN_IF_CANCELLED(_state); } _sorted_runs.push_back(_unsorted_run); - _unsorted_run = NULL; + _unsorted_run = nullptr; return Status::OK(); } @@ -1243,7 +1244,7 @@ Status SpillSorter::merge_intermediate_runs() { // For an intermediate merge, intermediate_merge_batch contains deep-copied rows from // the input runs. If (_sorted_runs.size() > max_runs_per_final_merge), // one or more intermediate merges are required. - scoped_ptr intermediate_merge_batch; + unique_ptr intermediate_merge_batch; while (_sorted_runs.size() > max_runs_per_final_merge) { // An intermediate merge adds one merge to _unmerged_sorted_runs. // Merging 'runs - (_max_runs_final - 1)' number of runs is sufficient to guarantee diff --git a/be/src/runtime/spill_sorter.h b/be/src/runtime/spill_sorter.h index 5d52acc16d..2e67d10d90 100644 --- a/be/src/runtime/spill_sorter.h +++ b/be/src/runtime/spill_sorter.h @@ -157,7 +157,7 @@ private: // In memory sorter and less-than comparator. TupleRowComparator _compare_less_than; - boost::scoped_ptr _in_mem_tuple_sorter; + std::unique_ptr _in_mem_tuple_sorter; // Block manager object used to allocate, pin and release runs. Not owned by SpillSorter. BufferedBlockMgr2* _block_mgr; @@ -184,7 +184,7 @@ private: // The current unsorted run that is being collected. Is sorted and added to // _sorted_runs after it is full (i.e. number of blocks allocated == max available // buffers) or after the input is complete. Owned and placed in _obj_pool. - // When it is added to _sorted_runs, it is set to NULL. + // When it is added to _sorted_runs, it is set to nullptr. Run* _unsorted_run; // List of sorted runs that have been produced but not merged. _unsorted_run is added @@ -195,7 +195,7 @@ private: // Merger object (intermediate or final) currently used to produce sorted runs. // Only one merge is performed at a time. Will never be used if the input fits in // memory. - boost::scoped_ptr _merger; + std::unique_ptr _merger; // Runs that are currently processed by the _merge. // These runs can be deleted when we are done with the current merge. diff --git a/be/src/runtime/string_search.hpp b/be/src/runtime/string_search.hpp index 218d369c0f..2525321d55 100644 --- a/be/src/runtime/string_search.hpp +++ b/be/src/runtime/string_search.hpp @@ -54,9 +54,8 @@ #ifndef DORIS_BE_SRC_QUERY_BE_RUNTIME_STRING_SEARCH_H #define DORIS_BE_SRC_QUERY_BE_RUNTIME_STRING_SEARCH_H -#include #include -#include +#include #include "common/logging.h" #include "runtime/string_value.h" @@ -66,7 +65,6 @@ namespace doris { // TODO: This can be sped up with SIDD_CMP_EQUAL_ORDERED or at the very least rewritten // from published algorithms. class StringSearch { - public: virtual ~StringSearch() {} StringSearch() : _pattern(NULL), _mask(0) {} @@ -158,19 +156,15 @@ public: private: static const int BLOOM_WIDTH = 64; - void bloom_add(char c) { - _mask |= (1UL << (c & (BLOOM_WIDTH - 1))); - } + void bloom_add(char c) { _mask |= (1UL << (c & (BLOOM_WIDTH - 1))); } - bool bloom_query(char c) const { - return _mask & (1UL << (c & (BLOOM_WIDTH - 1))); - } + bool bloom_query(char c) const { return _mask & (1UL << (c & (BLOOM_WIDTH - 1))); } const StringValue* _pattern; int64_t _mask; int64_t _skip; }; -} +} // namespace doris #endif diff --git a/be/src/runtime/string_value.h b/be/src/runtime/string_value.h index 160ff179f9..285189e8bc 100644 --- a/be/src/runtime/string_value.h +++ b/be/src/runtime/string_value.h @@ -43,7 +43,7 @@ struct StringValue { size_t len; StringValue(char* ptr, int len) : ptr(ptr), len(len) {} - StringValue() : ptr(NULL), len(0) {} + StringValue() : ptr(nullptr), len(0) {} /// Construct a StringValue from 's'. 's' must be valid for as long as /// this object is valid. diff --git a/be/src/runtime/thread_resource_mgr.cpp b/be/src/runtime/thread_resource_mgr.cpp index 7575a4d87f..be6d21a027 100644 --- a/be/src/runtime/thread_resource_mgr.cpp +++ b/be/src/runtime/thread_resource_mgr.cpp @@ -67,7 +67,7 @@ void ThreadResourceMgr::ResourcePool::reserve_optional_tokens(int num) { ThreadResourceMgr::ResourcePool* ThreadResourceMgr::register_pool() { std::unique_lock l(_lock); - ResourcePool* pool = NULL; + ResourcePool* pool = nullptr; if (_free_pool_objs.empty()) { pool = new ResourcePool(this); @@ -76,7 +76,7 @@ ThreadResourceMgr::ResourcePool* ThreadResourceMgr::register_pool() { _free_pool_objs.pop_front(); } - DCHECK(pool != NULL); + DCHECK(pool != nullptr); DCHECK(_pools.find(pool) == _pools.end()); _pools.insert(pool); pool->reset(); @@ -87,7 +87,7 @@ ThreadResourceMgr::ResourcePool* ThreadResourceMgr::register_pool() { } void ThreadResourceMgr::unregister_pool(ResourcePool* pool) { - DCHECK(pool != NULL); + DCHECK(pool != nullptr); std::unique_lock l(_lock); // this may be double unregistered after pr #3326 by LaiYingChun, so check if the pool is already unregisted if (_pools.find(pool) != _pools.end()) { diff --git a/be/src/runtime/thread_resource_mgr.h b/be/src/runtime/thread_resource_mgr.h index 02c3584f31..ef5442d2ac 100644 --- a/be/src/runtime/thread_resource_mgr.h +++ b/be/src/runtime/thread_resource_mgr.h @@ -20,12 +20,10 @@ #include -#include -#include -#include #include #include #include +#include #include "common/status.h" @@ -86,7 +84,7 @@ public: // that have 1+ thread usage). class ResourcePool { public: - virtual ~ResourcePool(){}; + virtual ~ResourcePool() {}; // Acquire a thread for the pool. This will always succeed; the // pool will go over the quota. // Pools should use this API to reserve threads they need in order diff --git a/be/src/runtime/tuple.cpp b/be/src/runtime/tuple.cpp index 6073598caf..bf5dc16270 100644 --- a/be/src/runtime/tuple.cpp +++ b/be/src/runtime/tuple.cpp @@ -286,7 +286,7 @@ void Tuple::materialize_exprs(TupleRow* row, const TupleDescriptor& desc, DCHECK(slot_type == TYPE_NULL || slot_type == expr_type); } void* src = materialize_expr_ctxs[mat_expr_index]->get_value(row); - if (src != NULL) { + if (src != nullptr) { void* dst = get_slot(slot_desc->tuple_offset()); RawValue::write(src, dst, slot_desc->type(), pool); if (collect_string_vals) { diff --git a/be/src/runtime/tuple.h b/be/src/runtime/tuple.h index eeb4edf6b4..36a19b2b94 100644 --- a/be/src/runtime/tuple.h +++ b/be/src/runtime/tuple.h @@ -107,7 +107,7 @@ public: // Materialize this by evaluating the expressions in materialize_exprs // over the specified 'row'. 'pool' is used to allocate var-length data. // (Memory for this tuple itself must already be allocated.) - // If collect_string_vals is true, the materialized non-NULL string value + // If collect_string_vals is true, the materialized non-nullptr string value // slots and the total length of the string slots are returned in var_values // and total_var_len. template diff --git a/be/src/runtime/tuple_row.h b/be/src/runtime/tuple_row.h index 7c5099e355..82aa5d44f5 100644 --- a/be/src/runtime/tuple_row.h +++ b/be/src/runtime/tuple_row.h @@ -54,8 +54,8 @@ public: void deep_copy(TupleRow* dst, const std::vector& descs, MemPool* pool, bool reuse_tuple_mem) { for (int i = 0; i < descs.size(); ++i) { - if (this->get_tuple(i) != NULL) { - if (reuse_tuple_mem && dst->get_tuple(i) != NULL) { + if (this->get_tuple(i) != nullptr) { + if (reuse_tuple_mem && dst->get_tuple(i) != nullptr) { this->get_tuple(i)->deep_copy(dst->get_tuple(i), *descs[i], pool); } else { dst->set_tuple(i, this->get_tuple(i)->deep_copy(*descs[i], pool)); @@ -63,7 +63,7 @@ public: } else { // TODO: this is wasteful. If we have 'reuse_tuple_mem', we should be able // to save the tuple buffer and reuse it (i.e. freelist). - dst->set_tuple(i, NULL); + dst->set_tuple(i, nullptr); } } } @@ -81,8 +81,8 @@ public: int64_t bytes = 0; for (int i = 0; i < descs.size(); ++i) { Tuple* old_tuple = dst->get_tuple(i); - if (_tuples[i] != NULL) { - if (reuse_tuple_mem && old_tuple != NULL) { + if (_tuples[i] != nullptr) { + if (reuse_tuple_mem && old_tuple != nullptr) { bytes += _tuples[i]->dcopy_with_new(dst->get_tuple(i), *descs[i]); } else { int64_t new_bytes = 0; @@ -90,7 +90,7 @@ public: bytes += new_bytes; } } else { - dst->set_tuple(i, NULL); + dst->set_tuple(i, nullptr); } } return bytes; diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index dee19c1101..3aea4271fe 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -263,7 +263,7 @@ void BackendService::open_scanner(TScanOpenResult& result_, const TScanOpenParam _exec_env->external_scan_context_mgr()->create_scan_context(&p_context); p_context->fragment_instance_id = fragment_instance_id; p_context->offset = 0; - p_context->last_access_time = time(NULL); + p_context->last_access_time = time(nullptr); if (params.__isset.keep_alive_min) { p_context->keep_alive_min = params.keep_alive_min; } else { @@ -331,7 +331,7 @@ void BackendService::get_next(TScanBatchResult& result_, const TScanNextBatchPar result_.status = t_status; } } - context->last_access_time = time(NULL); + context->last_access_time = time(nullptr); } void BackendService::close_scanner(TScanCloseResult& result_, const TScanCloseParams& params) { diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index 2b136a2279..430ac5e8c5 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -19,11 +19,9 @@ #include #include -#include -#include -#include #include #include +#include #include #if defined(LEAK_SANITIZER) diff --git a/be/src/testutil/desc_tbl_builder.cc b/be/src/testutil/desc_tbl_builder.cc index 705bd35f48..5a7fcb46e7 100644 --- a/be/src/testutil/desc_tbl_builder.cc +++ b/be/src/testutil/desc_tbl_builder.cc @@ -65,7 +65,7 @@ static TTupleDescriptor make_tuple_descriptor(int id, int byte_size, int num_nul } DescriptorTbl* DescriptorTblBuilder::build() { - DescriptorTbl* desc_tbl = NULL; + DescriptorTbl* desc_tbl = nullptr; TDescriptorTable thrift_desc_tbl; int tuple_id = 0; int slot_id = 0; diff --git a/be/src/udf/uda_test_harness.h b/be/src/udf/uda_test_harness.h index 95cc068516..906efe969f 100644 --- a/be/src/udf/uda_test_harness.h +++ b/be/src/udf/uda_test_harness.h @@ -18,8 +18,6 @@ #ifndef DORIS_BE_UDF_UDA_TEST_HARNESS_H #define DORIS_BE_UDF_UDA_TEST_HARNESS_H -#include -#include #include #include #include @@ -67,7 +65,7 @@ protected: _merge_fn(merge_fn), _serialize_fn(serialize_fn), _finalize_fn(finalize_fn), - _result_comparator_fn(NULL), + _result_comparator_fn(nullptr), _num_input_values(0) {} // Runs the UDA in all the modes, validating the result is 'expected' each time. @@ -101,7 +99,7 @@ private: SerializeFn _serialize_fn; FinalizeFn _finalize_fn; - // Customer comparator, NULL if default == should be used. + // Customer comparator, nullptr if default == should be used. ResultComparator _result_comparator_fn; // Set during execute() by subclass diff --git a/be/src/udf/udf.cpp b/be/src/udf/udf.cpp index dcc410dfbf..acdb66e70d 100644 --- a/be/src/udf/udf.cpp +++ b/be/src/udf/udf.cpp @@ -81,8 +81,8 @@ FunctionContextImpl::FunctionContextImpl(doris_udf::FunctionContext* parent) _num_updates(0), _num_removes(0), _context(parent), - _pool(NULL), - _state(NULL), + _pool(nullptr), + _state(nullptr), _debug(false), _version(doris_udf::FunctionContext::V2_0), _num_warnings(0), @@ -108,7 +108,7 @@ void FunctionContextImpl::close() { } free(_varargs_buffer); - _varargs_buffer = NULL; + _varargs_buffer = nullptr; _closed = true; } @@ -199,8 +199,8 @@ static const int MAX_WARNINGS = 1000; FunctionContext* FunctionContext::create_test_context() { FunctionContext* context = new FunctionContext(); context->impl()->_debug = true; - context->impl()->_state = NULL; - context->impl()->_pool = new doris::FreePool(NULL); + context->impl()->_state = nullptr; + context->impl()->_pool = new doris::FreePool(nullptr); return context; } @@ -220,8 +220,8 @@ FunctionContext::DorisVersion FunctionContext::version() const { } const char* FunctionContext::user() const { - if (_impl->_state == NULL) { - return NULL; + if (_impl->_state == nullptr) { + return nullptr; } return _impl->_state->user().c_str(); @@ -269,7 +269,7 @@ uint8_t* FunctionContext::reallocate(uint8_t* ptr, int byte_size) { } void FunctionContext::free(uint8_t* buffer) { - if (buffer == NULL) { + if (buffer == nullptr) { return; } @@ -338,7 +338,7 @@ bool FunctionContext::add_warning(const char* warning_msg) { std::stringstream ss; ss << "UDF WARNING: " << warning_msg; - if (_impl->_state != NULL) { + if (_impl->_state != nullptr) { return _impl->_state->log_error(ss.str()); } else { std::cerr << ss.str() << std::endl; @@ -388,7 +388,7 @@ void StringVal::append(FunctionContext* ctx, const uint8_t* buf, size_t buf_len) "Concatenated string length larger than allowed limit of " "1 GB character data."); ctx->free(ptr); - ptr = NULL; + ptr = nullptr; len = 0; is_null = true; } else { @@ -405,7 +405,7 @@ void StringVal::append(FunctionContext* ctx, const uint8_t* buf, size_t buf_len, "Concatenated string length larger than allowed limit of " "1 GB character data."); ctx->free(ptr); - ptr = NULL; + ptr = nullptr; len = 0; is_null = true; } else { @@ -418,7 +418,7 @@ void StringVal::append(FunctionContext* ctx, const uint8_t* buf, size_t buf_len, const FunctionContext::TypeDesc* FunctionContext::get_arg_type(int arg_idx) const { if (arg_idx < 0 || arg_idx >= _impl->_arg_types.size()) { - return NULL; + return nullptr; } return &_impl->_arg_types[arg_idx]; } diff --git a/be/src/udf/udf.h b/be/src/udf/udf.h index c0c5a2388e..b0817e48bf 100644 --- a/be/src/udf/udf.h +++ b/be/src/udf/udf.h @@ -33,8 +33,8 @@ class FunctionContextImpl; namespace doris_udf { // All input and output values will be one of the structs below. The struct is a simple -// object containing a boolean to store if the value is NULL and the value itself. The -// value is unspecified if the NULL boolean is set. +// object containing a boolean to store if the value is nullptr and the value itself. The +// value is unspecified if the nullptr boolean is set. struct AnyVal; struct BooleanVal; struct TinyIntVal; @@ -124,7 +124,7 @@ public: // Returns the version of Doris that's currently running. DorisVersion version() const; - // Returns the user that is running the query. Returns NULL if it is not + // Returns the user that is running the query. Returns nullptr if it is not // available. const char* user() const; @@ -149,7 +149,7 @@ public: // Returns true if there's been an error set. bool has_error() const; - // Returns the current error message. Returns NULL if there is no error. + // Returns the current error message. Returns nullptr if there is no error. const char* error_msg() const; // Allocates memory for UDAs. All UDA allocations should use this if possible instead of @@ -191,7 +191,7 @@ public: /// Methods for maintaining state across UDF/UDA function calls. SetFunctionState() can /// be used to store a pointer that can then be retreived via GetFunctionState(). If /// GetFunctionState() is called when no pointer is set, it will return - /// NULL. SetFunctionState() does not take ownership of 'ptr'; it is up to the UDF/UDA + /// nullptr. SetFunctionState() does not take ownership of 'ptr'; it is up to the UDF/UDA /// to clean up any function state if necessary. void set_function_state(FunctionStateScope scope, void* ptr); void* get_function_state(FunctionStateScope scope) const; @@ -212,7 +212,7 @@ public: int get_num_constant_args() const; // Returns the type information for the arg_idx-th argument (0-indexed, not including - // the FunctionContext* argument). Returns NULL if arg_idx is invalid. + // the FunctionContext* argument). Returns nullptr if arg_idx is invalid. const TypeDesc* get_arg_type(int arg_idx) const; // Returns true if the arg_idx-th input argument (0 indexed, not including the @@ -220,7 +220,7 @@ public: bool is_arg_constant(int arg_idx) const; // Returns a pointer to the value of the arg_idx-th input argument (0 indexed, not - // including the FunctionContext* argument). Returns NULL if the argument is not + // including the FunctionContext* argument). Returns nullptr if the argument is not // constant. This function can be used to obtain user-specified constants in a UDF's // Init() or Close() functions. AnyVal* get_constant_arg(int arg_idx) const; @@ -251,7 +251,7 @@ private: // // The UDF must return one of the *Val structs. The UDF must accept a pointer // to a FunctionContext object and then a const reference for each of the input arguments. -// NULL input arguments will have NULL passed in. +// nullptr input arguments will have nullptr passed in. // Examples of valid Udf signatures are: // 1) DoubleVal Example1(FunctionContext* context); // 2) IntVal Example2(FunctionContext* context, const IntVal& a1, const DoubleVal& a2); @@ -581,7 +581,7 @@ struct DateTimeVal : public AnyVal { bool operator!=(const DateTimeVal& other) const { return !(*this == other); } }; -// Note: there is a difference between a NULL string (is_null == true) and an +// Note: there is a difference between a nullptr string (is_null == true) and an // empty string (len == 0). struct StringVal : public AnyVal { static const int MAX_LENGTH = (1 << 30); @@ -591,13 +591,13 @@ struct StringVal : public AnyVal { // Construct a StringVal from ptr/len. Note: this does not make a copy of ptr // so the buffer must exist as long as this StringVal does. - StringVal() : len(0), ptr(NULL) {} + StringVal() : len(0), ptr(nullptr) {} // Construct a StringVal from ptr/len. Note: this does not make a copy of ptr // so the buffer must exist as long as this StringVal does. StringVal(uint8_t* ptr, int64_t len) : len(len), ptr(ptr) {} - // Construct a StringVal from NULL-terminated c-string. Note: this does not make a + // Construct a StringVal from nullptr-terminated c-string. Note: this does not make a // copy of ptr so the underlying string must exist as long as this StringVal does. StringVal(const char* ptr) : len(strlen(ptr)), ptr((uint8_t*)ptr) {} @@ -636,13 +636,13 @@ struct StringVal : public AnyVal { bool operator!=(const StringVal& other) const { return !(*this == other); } /// Will create a new StringVal with the given dimension and copy the data from the - /// parameters. In case of an error will return a NULL string and set an error on the + /// parameters. In case of an error will return a nullptr string and set an error on the /// function context. static StringVal copy_from(FunctionContext* ctx, const uint8_t* buf, size_t len); /// Append the passed buffer to this StringVal. Reallocate memory to fit the buffer. If /// the memory allocation becomes too large, will set an error on FunctionContext and - /// return a NULL string. + /// return a nullptr string. void append(FunctionContext* ctx, const uint8_t* buf, size_t len); void append(FunctionContext* ctx, const uint8_t* buf, size_t len, const uint8_t* buf2, size_t buf2_len); @@ -736,7 +736,7 @@ struct CollectionVal : public AnyVal { CollectionVal() = default; CollectionVal(void* data, uint32_t length, bool has_null, bool* null_signs) - : data(data), length(length), has_null(has_null), null_signs(null_signs){}; + : data(data), length(length), has_null(has_null), null_signs(null_signs) {}; static CollectionVal null() { CollectionVal val; diff --git a/be/src/udf/udf_internal.h b/be/src/udf/udf_internal.h index ed6b37de5f..2d2c318ed4 100644 --- a/be/src/udf/udf_internal.h +++ b/be/src/udf/udf_internal.h @@ -102,7 +102,7 @@ public: std::string& string_result() { return _string_result; } - const doris_udf::FunctionContext::TypeDesc& get_return_type() const { return _return_type; } + const doris_udf::FunctionContext::TypeDesc& get_return_type() const { return _return_type; } private: friend class doris_udf::FunctionContext; @@ -126,7 +126,7 @@ private: // Pool to service allocations from. FreePool* _pool; - // We use the query's runtime state to report errors and warnings. NULL for test + // We use the query's runtime state to report errors and warnings. nullptr for test // contexts. RuntimeState* _state; @@ -164,7 +164,7 @@ private: // Type descriptors for each argument of the function. std::vector _arg_types; - // Contains an AnyVal* for each argument of the function. If the AnyVal* is NULL, + // Contains an AnyVal* for each argument of the function. If the AnyVal* is nullptr, // indicates that the corresponding argument is non-constant. Otherwise contains the // value of the argument. std::vector _constant_args; diff --git a/be/src/udf/udf_ir.cpp b/be/src/udf/udf_ir.cpp index 2fa565c826..4eefd175a1 100644 --- a/be/src/udf/udf_ir.cpp +++ b/be/src/udf/udf_ir.cpp @@ -22,12 +22,12 @@ bool FunctionContext::is_arg_constant(int i) const { if (i < 0 || i >= _impl->_constant_args.size()) { return false; } - return _impl->_constant_args[i] != NULL; + return _impl->_constant_args[i] != nullptr; } AnyVal* FunctionContext::get_constant_arg(int i) const { if (i < 0 || i >= _impl->_constant_args.size()) { - return NULL; + return nullptr; } return _impl->_constant_args[i]; } @@ -55,7 +55,7 @@ void* FunctionContext::get_function_state(FunctionStateScope scope) const { break; default: // TODO: signal error somehow - return NULL; + return nullptr; } } diff --git a/be/src/util/aes_util.cpp b/be/src/util/aes_util.cpp index 1b4c7d1177..be418d0f12 100644 --- a/be/src/util/aes_util.cpp +++ b/be/src/util/aes_util.cpp @@ -45,7 +45,7 @@ const EVP_CIPHER* get_evp_type(const AesMode mode) { case AES_256_CBC: return EVP_aes_256_cbc(); default: - return NULL; + return nullptr; } } diff --git a/be/src/util/batch_process_thread_pool.hpp b/be/src/util/batch_process_thread_pool.hpp index 98822b583a..2f7cfd8df2 100644 --- a/be/src/util/batch_process_thread_pool.hpp +++ b/be/src/util/batch_process_thread_pool.hpp @@ -27,6 +27,7 @@ #include "common/config.h" #include "util/blocking_priority_queue.hpp" #include "util/stopwatch.hpp" +#include "util/thread_group.h" namespace doris { @@ -156,7 +157,7 @@ private: BlockingPriorityQueue _work_queue; // Collection of worker threads that process work from the queue. - boost::thread_group _threads; + ThreadGroup _threads; // Guards _shutdown and _empty_cv std::mutex _lock; diff --git a/be/src/util/bit_stream_utils.h b/be/src/util/bit_stream_utils.h index 902c288e57..d7c4e11266 100644 --- a/be/src/util/bit_stream_utils.h +++ b/be/src/util/bit_stream_utils.h @@ -90,7 +90,7 @@ public: // 'buffer' is the buffer to read from. The buffer's length is 'buffer_len'. BitReader(const uint8_t* buffer, int buffer_len); - BitReader() : buffer_(NULL), max_bytes_(0) {} + BitReader() : buffer_(nullptr), max_bytes_(0) {} // Gets the next value from the buffer. Returns true if 'v' could be read or false if // there are not enough bytes left. num_bits must be <= 32. @@ -124,7 +124,7 @@ public: // Maximum byte length of a vlq encoded int static const int MAX_VLQ_BYTE_LEN = 5; - bool is_initialized() const { return buffer_ != NULL; } + bool is_initialized() const { return buffer_ != nullptr; } private: // Used by SeekToBit() and GetValue() to fetch the diff --git a/be/src/util/bitmap_value.h b/be/src/util/bitmap_value.h index 34f0aba762..6a50b9d75b 100644 --- a/be/src/util/bitmap_value.h +++ b/be/src/util/bitmap_value.h @@ -108,7 +108,7 @@ public: /** * Construct a roaring object from the C struct. * - * Passing a NULL point is unsafe. + * Passing a nullptr point is unsafe. */ explicit Roaring64Map(roaring_bitmap_t* s) { roaring::Roaring r(s); @@ -539,7 +539,7 @@ public: /** * Iterate over the bitmap elements. The function iterator is called once - * for all the values with ptr (can be NULL) as the second parameter of each + * for all the values with ptr (can be nullptr) as the second parameter of each * call. * * roaring_iterator is simply a pointer to a function that returns bool diff --git a/be/src/util/buffer_builder.hpp b/be/src/util/buffer_builder.hpp index 7091bb7591..710b1e2c4c 100644 --- a/be/src/util/buffer_builder.hpp +++ b/be/src/util/buffer_builder.hpp @@ -18,29 +18,21 @@ #ifndef DORIS_BE_SRC_COMMON_UTIL_BUFFER_BUILDER_HPP #define DORIS_BE_SRC_COMMON_UTIL_BUFFER_BUILDER_HPP -#include -#include #include +#include namespace doris { // Utility class to build an in-memory buffer. class BufferBuilder { public: - BufferBuilder(uint8_t* dst_buffer, int dst_len) : - _buffer(dst_buffer), - _capacity(dst_len), - _size(0) { - } + BufferBuilder(uint8_t* dst_buffer, int dst_len) + : _buffer(dst_buffer), _capacity(dst_len), _size(0) {} - BufferBuilder(char* dst_buffer, int dst_len) : - _buffer(reinterpret_cast(dst_buffer)), - _capacity(dst_len), - _size(0) { - } + BufferBuilder(char* dst_buffer, int dst_len) + : _buffer(reinterpret_cast(dst_buffer)), _capacity(dst_len), _size(0) {} - ~BufferBuilder() { - } + ~BufferBuilder() {} inline void append(const void* buffer, int len) { DCHECK_LE(_size + len, _capacity); @@ -48,17 +40,13 @@ public: _size += len; } - template + template inline void append(const T& v) { append(&v, sizeof(T)); } - int capacity() const { - return _capacity; - } - int size() const { - return _size; - } + int capacity() const { return _capacity; } + int size() const { return _size; } private: uint8_t* _buffer; @@ -66,6 +54,6 @@ private: int _size; }; -} +} // namespace doris #endif diff --git a/be/src/util/cpu_info.cpp b/be/src/util/cpu_info.cpp index 304eec1007..9ea1cc8859 100644 --- a/be/src/util/cpu_info.cpp +++ b/be/src/util/cpu_info.cpp @@ -318,16 +318,16 @@ void CpuInfo::_get_cache_info(long cache_sizes[NUM_CACHE_LEVELS], #ifdef __APPLE__ // On Mac OS X use sysctl() to get the cache sizes size_t len = 0; - sysctlbyname("hw.cachesize", NULL, &len, NULL, 0); + sysctlbyname("hw.cachesize", nullptr, &len, nullptr, 0); uint64_t* data = static_cast(malloc(len)); - sysctlbyname("hw.cachesize", data, &len, NULL, 0); + sysctlbyname("hw.cachesize", data, &len, nullptr, 0); DCHECK(len / sizeof(uint64_t) >= 3); for (size_t i = 0; i < NUM_CACHE_LEVELS; ++i) { cache_sizes[i] = data[i]; } size_t linesize; size_t sizeof_linesize = sizeof(linesize); - sysctlbyname("hw.cachelinesize", &linesize, &sizeof_linesize, NULL, 0); + sysctlbyname("hw.cachelinesize", &linesize, &sizeof_linesize, nullptr, 0); for (size_t i = 0; i < NUM_CACHE_LEVELS; ++i) cache_line_sizes[i] = linesize; #else // Call sysconf to query for the cache sizes diff --git a/be/src/util/cpu_info.h b/be/src/util/cpu_info.h index 363b6d98b9..33dd776c22 100644 --- a/be/src/util/cpu_info.h +++ b/be/src/util/cpu_info.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_UTIL_CPU_INFO_H #define DORIS_BE_SRC_UTIL_CPU_INFO_H -#include #include #include #include diff --git a/be/src/util/date_func.cpp b/be/src/util/date_func.cpp index e51299326d..93559b684f 100644 --- a/be/src/util/date_func.cpp +++ b/be/src/util/date_func.cpp @@ -26,7 +26,7 @@ uint64_t timestamp_from_datetime(const std::string& datetime_str) { char* res = strptime(datetime_str.c_str(), "%Y-%m-%d %H:%M:%S", &time_tm); uint64_t value = 0; - if (NULL != res) { + if (nullptr != res) { value = ((time_tm.tm_year + 1900) * 10000L + (time_tm.tm_mon + 1) * 100L + time_tm.tm_mday) * 1000000L + @@ -44,7 +44,7 @@ uint24_t timestamp_from_date(const std::string& date_str) { char* res = strptime(date_str.c_str(), "%Y-%m-%d", &time_tm); int value = 0; - if (NULL != res) { + if (nullptr != res) { value = (time_tm.tm_year + 1900) * 16 * 32 + (time_tm.tm_mon + 1) * 32 + time_tm.tm_mday; } else { // 1400 - 01 - 01 @@ -56,7 +56,7 @@ uint24_t timestamp_from_date(const std::string& date_str) { // refer to https://dev.mysql.com/doc/refman/5.7/en/time.html // the time value between '-838:59:59' and '838:59:59' int32_t time_to_buffer_from_double(double time, char* buffer) { - char *begin = buffer; + char* begin = buffer; if (time < 0) { time = -time; *buffer++ = '-'; diff --git a/be/src/util/debug_util.h b/be/src/util/debug_util.h index 5f91f171bc..ead439d926 100644 --- a/be/src/util/debug_util.h +++ b/be/src/util/debug_util.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_COMMON_UTIL_DEBUG_UTIL_H #define DORIS_BE_SRC_COMMON_UTIL_DEBUG_UTIL_H -#include #include #include diff --git a/be/src/util/disk_info.h b/be/src/util/disk_info.h index 2adcd11763..5278a06c7d 100644 --- a/be/src/util/disk_info.h +++ b/be/src/util/disk_info.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_COMMON_UTIL_DISK_INFO_H #define DORIS_BE_SRC_COMMON_UTIL_DISK_INFO_H -#include #include #include #include diff --git a/be/src/util/dynamic_util.cpp b/be/src/util/dynamic_util.cpp index c85fb3afc3..85e37506ee 100644 --- a/be/src/util/dynamic_util.cpp +++ b/be/src/util/dynamic_util.cpp @@ -27,7 +27,7 @@ Status dynamic_lookup(void* handle, const char* symbol, void** fn_ptr) { *(void**)(fn_ptr) = dlsym(handle, symbol); char* error = dlerror(); - if (error != NULL) { + if (error != nullptr) { std::stringstream ss; ss << "Unable to find " << symbol << "\ndlerror: " << error; return Status::InternalError(ss.str()); @@ -41,7 +41,7 @@ Status dynamic_open(const char* library, void** handle) { *handle = dlopen(library, flags); - if (*handle == NULL) { + if (*handle == nullptr) { std::stringstream ss; ss << "Unable to load " << library << "\ndlerror: " << dlerror(); return Status::InternalError(ss.str()); diff --git a/be/src/util/dynamic_util.h b/be/src/util/dynamic_util.h index 403334b316..280937dadd 100644 --- a/be/src/util/dynamic_util.h +++ b/be/src/util/dynamic_util.h @@ -23,14 +23,14 @@ namespace doris { // Look up symbols in a dynamically linked library. -// handle -- handle to the library. NULL if loading from the current process. +// handle -- handle to the library. nullptr if loading from the current process. // symbol -- symbol to lookup. // fn_ptr -- pointer tor return address of function. Status dynamic_lookup(void* handle, const char* symbol, void** fn_ptr); // Open a dynamically loaded library. // library -- name of the library. The default paths will be searched. -// library can be NULL to get the handle for the current process. +// library can be nullptr to get the handle for the current process. // handle -- returned handle to the library. Status dynamic_open(const char* library, void** handle); diff --git a/be/src/util/error_util.h b/be/src/util/error_util.h index 35de1481fe..4c6bfabcd0 100644 --- a/be/src/util/error_util.h +++ b/be/src/util/error_util.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_UTIL_ERROR_UTIL_H #define DORIS_BE_SRC_UTIL_ERROR_UTIL_H -#include #include #include diff --git a/be/src/util/internal_queue.h b/be/src/util/internal_queue.h index 64d0c3ca8a..ed48183d5c 100644 --- a/be/src/util/internal_queue.h +++ b/be/src/util/internal_queue.h @@ -31,7 +31,7 @@ namespace doris { /// it's an internal queue, the list pointers are maintained in the Nodes which is memory /// owned by the user. The nodes cannot be deallocated while the queue has elements. /// The internal structure is a doubly-linked list. -/// NULL <-- N1 <--> N2 <--> N3 --> NULL +/// nullptr <-- N1 <--> N2 <--> N3 --> nullptr /// (head) (tail) /// /// InternalQueue instantiates a thread-safe queue where the queue is protected by an @@ -48,13 +48,13 @@ class InternalQueueBase { public: struct Node { public: - Node() : parent_queue(NULL), next_node(NULL), prev_node(NULL) {} + Node() : parent_queue(nullptr), next_node(nullptr), prev_node(nullptr) {} virtual ~Node() {} /// Returns true if the node is in a queue. - bool in_queue() const { return parent_queue != NULL; } + bool in_queue() const { return parent_queue != nullptr; } - /// Returns the Next/Prev node or NULL if this is the end/front. + /// Returns the Next/Prev node or nullptr if this is the end/front. T* next() const { std::lock_guard lock(parent_queue->lock_); return reinterpret_cast(next_node); @@ -67,88 +67,88 @@ public: private: friend class InternalQueueBase; - /// Pointer to the queue this Node is on. NULL if not on any queue. + /// Pointer to the queue this Node is on. nullptr if not on any queue. InternalQueueBase* parent_queue; Node* next_node; Node* prev_node; }; - InternalQueueBase() : head_(NULL), tail_(NULL), size_(0) {} + InternalQueueBase() : head_(nullptr), tail_(nullptr), size_(0) {} - /// Returns the element at the head of the list without dequeuing or NULL + /// Returns the element at the head of the list without dequeuing or nullptr /// if the queue is empty. This is O(1). T* head() const { std::lock_guard lock(lock_); - if (empty()) return NULL; + if (empty()) return nullptr; return reinterpret_cast(head_); } - /// Returns the element at the end of the list without dequeuing or NULL + /// Returns the element at the end of the list without dequeuing or nullptr /// if the queue is empty. This is O(1). T* tail() { std::lock_guard lock(lock_); - if (empty()) return NULL; + if (empty()) return nullptr; return reinterpret_cast(tail_); } /// Enqueue node onto the queue's tail. This is O(1). void enqueue(T* n) { Node* node = (Node*)n; - DCHECK(node->next_node == NULL); - DCHECK(node->prev_node == NULL); - DCHECK(node->parent_queue == NULL); + DCHECK(node->next_node == nullptr); + DCHECK(node->prev_node == nullptr); + DCHECK(node->parent_queue == nullptr); node->parent_queue = this; { std::lock_guard lock(lock_); - if (tail_ != NULL) tail_->next_node = node; + if (tail_ != nullptr) tail_->next_node = node; node->prev_node = tail_; tail_ = node; - if (head_ == NULL) head_ = node; + if (head_ == nullptr) head_ = node; ++size_; } } - /// Dequeues an element from the queue's head. Returns NULL if the queue + /// Dequeues an element from the queue's head. Returns nullptr if the queue /// is empty. This is O(1). T* dequeue() { - Node* result = NULL; + Node* result = nullptr; { std::lock_guard lock(lock_); - if (empty()) return NULL; + if (empty()) return nullptr; --size_; result = head_; head_ = head_->next_node; - if (head_ == NULL) { - tail_ = NULL; + if (head_ == nullptr) { + tail_ = nullptr; } else { - head_->prev_node = NULL; + head_->prev_node = nullptr; } } - DCHECK(result != NULL); - result->next_node = result->prev_node = NULL; - result->parent_queue = NULL; + DCHECK(result != nullptr); + result->next_node = result->prev_node = nullptr; + result->parent_queue = nullptr; return reinterpret_cast(result); } - /// Dequeues an element from the queue's tail. Returns NULL if the queue + /// Dequeues an element from the queue's tail. Returns nullptr if the queue /// is empty. This is O(1). T* pop_back() { - Node* result = NULL; + Node* result = nullptr; { std::lock_guard lock(lock_); - if (empty()) return NULL; + if (empty()) return nullptr; --size_; result = tail_; tail_ = tail_->prev_node; - if (tail_ == NULL) { - head_ = NULL; + if (tail_ == nullptr) { + head_ = nullptr; } else { - tail_->next_node = NULL; + tail_->next_node = nullptr; } } - DCHECK(result != NULL); - result->next_node = result->prev_node = NULL; - result->parent_queue = NULL; + DCHECK(result != nullptr); + result->next_node = result->prev_node = nullptr; + result->parent_queue = nullptr; return reinterpret_cast(result); } @@ -159,34 +159,34 @@ public: if (node->parent_queue != this) return false; { std::lock_guard lock(lock_); - if (node->next_node == NULL && node->prev_node == NULL) { + if (node->next_node == nullptr && node->prev_node == nullptr) { // Removing only node DCHECK(node == head_); DCHECK(tail_ == node); - head_ = tail_ = NULL; + head_ = tail_ = nullptr; --size_; - node->parent_queue = NULL; + node->parent_queue = nullptr; return true; } if (head_ == node) { - DCHECK(node->prev_node == NULL); + DCHECK(node->prev_node == nullptr); head_ = node->next_node; } else { - DCHECK(node->prev_node != NULL); + DCHECK(node->prev_node != nullptr); node->prev_node->next_node = node->next_node; } if (node == tail_) { - DCHECK(node->next_node == NULL); + DCHECK(node->next_node == nullptr); tail_ = node->prev_node; - } else if (node->next_node != NULL) { + } else if (node->next_node != nullptr) { node->next_node->prev_node = node->prev_node; } --size_; } - node->next_node = node->prev_node = NULL; - node->parent_queue = NULL; + node->next_node = node->prev_node = nullptr; + node->parent_queue = nullptr; return true; } @@ -194,18 +194,18 @@ public: void clear() { std::lock_guard lock(lock_); Node* cur = head_; - while (cur != NULL) { + while (cur != nullptr) { Node* tmp = cur; cur = cur->next_node; - tmp->prev_node = tmp->next_node = NULL; - tmp->parent_queue = NULL; + tmp->prev_node = tmp->next_node = nullptr; + tmp->parent_queue = nullptr; } size_ = 0; - head_ = tail_ = NULL; + head_ = tail_ = nullptr; } int size() const { return size_; } - bool empty() const { return head_ == NULL; } + bool empty() const { return head_ == nullptr; } /// Returns if the target is on the queue. This is O(1) and does not acquire any locks. bool contains(const T* target) const { return target->parent_queue == this; } @@ -214,19 +214,19 @@ public: bool validate() { int num_elements_found = 0; std::lock_guard lock(lock_); - if (head_ == NULL) { - if (tail_ != NULL) return false; + if (head_ == nullptr) { + if (tail_ != nullptr) return false; if (size() != 0) return false; return true; } - if (head_->prev_node != NULL) return false; + if (head_->prev_node != nullptr) return false; Node* current = head_; - while (current != NULL) { + while (current != nullptr) { if (current->parent_queue != this) return false; ++num_elements_found; Node* next_node = current->next_node; - if (next_node == NULL) { + if (next_node == nullptr) { if (current != tail_) return false; } else { if (next_node->prev_node != current) return false; @@ -242,7 +242,7 @@ public: // from 'fn'. void iterate(std::function fn) { std::lock_guard lock(lock_); - for (Node* current = head_; current != NULL; current = current->next_node) { + for (Node* current = head_; current != nullptr; current = current->next_node) { if (!fn(reinterpret_cast(current))) return; } } @@ -254,7 +254,7 @@ public: { std::lock_guard lock(lock_); Node* curr = head_; - while (curr != NULL) { + while (curr != nullptr) { ss << (void*)curr; curr = curr->next_node; } diff --git a/be/src/util/json_util.h b/be/src/util/json_util.h index 95c2184aa4..cdf04f055b 100644 --- a/be/src/util/json_util.h +++ b/be/src/util/json_util.h @@ -25,7 +25,6 @@ #include "common/status.h" #include "util/pretty_printer.h" -#include "util/template_util.h" namespace doris { @@ -35,9 +34,9 @@ namespace doris { /// 'value' (e.g. conversion to MB etc). Otherwise the value is directly copied into /// 'out_val'. template -ENABLE_IF_NOT_ARITHMETIC(T, void) -ToJsonValue(const T& value, const TUnit::type unit, rapidjson::Document* document, - rapidjson::Value* out_val) { +typename std::enable_if::value, void>::type ToJsonValue( + const T& value, const TUnit::type unit, rapidjson::Document* document, + rapidjson::Value* out_val) { *out_val = value; } @@ -50,9 +49,9 @@ void ToJsonValue(const std::string& value, const TUnit::type unit, /// Does pretty-printing if 'value' is numeric, and type is not NONE, otherwise constructs /// a json object containing 'value' as a literal. template -ENABLE_IF_ARITHMETIC(T, void) -ToJsonValue(const T& value, const TUnit::type unit, rapidjson::Document* document, - rapidjson::Value* out_val) { +typename boost::enable_if_c::value, void>::type ToJsonValue( + const T& value, const TUnit::type unit, rapidjson::Document* document, + rapidjson::Value* out_val) { if (unit != TUnit::NONE) { const std::string& s = PrettyPrinter::print(value, unit); ToJsonValue(s, unit, document, out_val); diff --git a/be/src/util/mem_info.h b/be/src/util/mem_info.h index f02d2ab104..04aa17d07e 100644 --- a/be/src/util/mem_info.h +++ b/be/src/util/mem_info.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_COMMON_UTIL_MEM_INFO_H #define DORIS_BE_SRC_COMMON_UTIL_MEM_INFO_H -#include #include #include "common/logging.h" diff --git a/be/src/util/mutex.cpp b/be/src/util/mutex.cpp index d4efe0f096..a6bf2755c1 100644 --- a/be/src/util/mutex.cpp +++ b/be/src/util/mutex.cpp @@ -96,7 +96,7 @@ namespace doris { } while (0) Mutex::Mutex() { - PTHREAD_MUTEX_INIT_WITH_LOG(&_lock, NULL); + PTHREAD_MUTEX_INIT_WITH_LOG(&_lock, nullptr); } Mutex::~Mutex() { diff --git a/be/src/util/mysql_row_buffer.cpp b/be/src/util/mysql_row_buffer.cpp index e993141e63..8b72bb43e9 100644 --- a/be/src/util/mysql_row_buffer.cpp +++ b/be/src/util/mysql_row_buffer.cpp @@ -22,16 +22,16 @@ #include #include "common/logging.h" +#include "date_func.h" #include "gutil/strings/numbers.h" #include "runtime/large_int_value.h" #include "util/mysql_global.h" -#include "date_func.h" namespace doris { // the first byte: // <= 250: length -// = 251: NULL +// = 251: nullptr // = 252: the next two byte is length // = 253: the next three byte is length // = 254: the next eighth byte is length @@ -41,7 +41,7 @@ static char* pack_vlen(char* packet, uint64_t length) { return packet + 1; } - /* 251 is reserved for NULL */ + /* 251 is reserved for nullptr */ if (length < 65536ULL) { *packet++ = 252; int2store(packet, length); @@ -105,7 +105,7 @@ int MysqlRowBuffer::reserve(int size) { int alloc_size = std::max(need_size, _buf_size * 2); char* new_buf = new (std::nothrow) char[alloc_size]; - if (NULL == new_buf) { + if (nullptr == new_buf) { LOG(ERROR) << "alloc memory failed. size = " << alloc_size; return -1; } @@ -172,7 +172,8 @@ static char* add_datetime(const DateTimeValue& data, char* pos, bool dynamic_mod return pos + length; } -static char* add_decimal(const DecimalV2Value& data, int round_scale, char* pos, bool dynamic_mode) { +static char* add_decimal(const DecimalV2Value& data, int round_scale, char* pos, + bool dynamic_mode) { int length = data.to_buffer(pos + !dynamic_mode, round_scale); if (!dynamic_mode) { int1store(pos++, length); @@ -325,8 +326,8 @@ int MysqlRowBuffer::push_decimal(const DecimalV2Value& data, int round_scale) { int MysqlRowBuffer::push_string(const char* str, int length) { // 9 for length pack max, 1 for sign, other for digits - if (NULL == str) { - LOG(ERROR) << "input string is NULL."; + if (nullptr == str) { + LOG(ERROR) << "input string is nullptr."; return -1; } @@ -368,7 +369,7 @@ char* MysqlRowBuffer::reserved(int size) { if (0 != ret) { LOG(ERROR) << "mysql row buffer reserve failed."; - return NULL; + return nullptr; } char* old_buf = _pos; diff --git a/be/src/util/mysql_row_buffer.h b/be/src/util/mysql_row_buffer.h index 837caa6f9a..73380a79a1 100644 --- a/be/src/util/mysql_row_buffer.h +++ b/be/src/util/mysql_row_buffer.h @@ -31,7 +31,7 @@ namespace doris { * <--------A column--------><--------A column--------><-.....-> * * The flag means value's length or null value: - * If value is NULL, flag is 251 + * If value is nullptr, flag is 251 * If value's length < 251, flag is the value's length * If 251 <= value's length < 65536, flag is 252 and the next two byte is length * If 65536 <= value's length < 16777216 , flag is 253 and the next three byte is length diff --git a/be/src/util/network_util.cpp b/be/src/util/network_util.cpp index ba858c1c4c..74236b9ce1 100644 --- a/be/src/util/network_util.cpp +++ b/be/src/util/network_util.cpp @@ -72,7 +72,7 @@ Status hostname_to_ip_addrs(const std::string& name, std::vector* a struct addrinfo* addr_info; - if (getaddrinfo(name.c_str(), NULL, &hints, &addr_info) != 0) { + if (getaddrinfo(name.c_str(), nullptr, &hints, &addr_info) != 0) { std::stringstream ss; ss << "Could not find IPv4 address for: " << name; return Status::InternalError(ss.str()); @@ -80,12 +80,12 @@ Status hostname_to_ip_addrs(const std::string& name, std::vector* a addrinfo* it = addr_info; - while (it != NULL) { + while (it != nullptr) { char addr_buf[64]; const char* result = inet_ntop(AF_INET, &((sockaddr_in*)it->ai_addr)->sin_addr, addr_buf, 64); - if (result == NULL) { + if (result == nullptr) { std::stringstream ss; ss << "Could not convert IPv4 address for: " << name; freeaddrinfo(addr_info); diff --git a/be/src/util/parse_util.h b/be/src/util/parse_util.h index 20703f17eb..30f4dcd778 100644 --- a/be/src/util/parse_util.h +++ b/be/src/util/parse_util.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_COMMON_UTIL_PARSE_UTIL_H #define DORIS_BE_SRC_COMMON_UTIL_PARSE_UTIL_H -#include #include namespace doris { @@ -38,7 +37,8 @@ public: // Returns -1 if parsing failed. // if is_percent, return the percent of parent_limit. // if parent_limit is -1, use MemInfo::physical_mem() as parent limit. - static int64_t parse_mem_spec(const std::string& mem_spec_str, int64_t parent_limit, bool* is_percent); + static int64_t parse_mem_spec(const std::string& mem_spec_str, int64_t parent_limit, + bool* is_percent); }; } // namespace doris diff --git a/be/src/util/path_builder.cpp b/be/src/util/path_builder.cpp index 35f18b87aa..6cd144e259 100644 --- a/be/src/util/path_builder.cpp +++ b/be/src/util/path_builder.cpp @@ -26,7 +26,7 @@ namespace doris { const char* PathBuilder::_s_doris_home; void PathBuilder::load_doris_home() { - if (_s_doris_home != NULL) { + if (_s_doris_home != nullptr) { return; } diff --git a/be/src/util/perf_counters.cpp b/be/src/util/perf_counters.cpp index 9acd0de29c..5b7171d065 100644 --- a/be/src/util/perf_counters.cpp +++ b/be/src/util/perf_counters.cpp @@ -480,7 +480,7 @@ void PerfCounters::snapshot(const string& name) { const std::vector* PerfCounters::counters(int snapshot) const { if (snapshot < 0 || snapshot >= _snapshots.size()) { - return NULL; + return nullptr; } return &_snapshots[snapshot]; diff --git a/be/src/util/pretty_printer.h b/be/src/util/pretty_printer.h index 0986b75822..e9593452cc 100644 --- a/be/src/util/pretty_printer.h +++ b/be/src/util/pretty_printer.h @@ -24,9 +24,8 @@ #include #include "gen_cpp/RuntimeProfile_types.h" -#include "util/cpu_info.h" -#include "util/template_util.h" #include "util/binary_cast.hpp" +#include "util/cpu_info.h" /// Truncate a double to offset decimal places. #define DOUBLE_TRUNCATE(val, offset) floor(val* pow(10, offset)) / pow(10, offset) @@ -48,8 +47,8 @@ public: /// If verbose is true, this also prints the raw value (before unit conversion) for /// types where this is applicable. template - static ENABLE_IF_ARITHMETIC(T, std::string) - print(T value, TUnit::type unit, bool verbose = false) { + static typename std::enable_if::value, std::string>::type print( + T value, TUnit::type unit, bool verbose = false) { std::stringstream ss; ss.flags(std::ios::fixed); switch (unit) { @@ -158,7 +157,8 @@ public: /// TODO: There's no good is_string equivalent, so there's a needless copy for strings /// here. template - static ENABLE_IF_NOT_ARITHMETIC(T, std::string) print(const T& value, TUnit::type unit) { + static typename std::enable_if::value, std::string>::type print( + const T& value, TUnit::type unit) { std::stringstream ss; ss << std::boolalpha << value; return ss.str(); @@ -236,12 +236,14 @@ private: /// Utility to perform integer modulo if T is integral, otherwise to use fmod(). template - static ENABLE_IF_INTEGRAL(T, int64_t) mod(const T& value, const int modulus) { + static typename boost::enable_if_c::value, int64_t>::type mod( + const T& value, const int modulus) { return value % modulus; } template - static ENABLE_IF_FLOAT(T, double) mod(const T& value, int modulus) { + static typename boost::enable_if_c::value, double>::type mod( + const T& value, int modulus) { return fmod(value, 1. * modulus); } diff --git a/be/src/util/priority_thread_pool.hpp b/be/src/util/priority_thread_pool.hpp index b39145365f..95bea20d20 100644 --- a/be/src/util/priority_thread_pool.hpp +++ b/be/src/util/priority_thread_pool.hpp @@ -18,11 +18,11 @@ #ifndef DORIS_BE_SRC_COMMON_UTIL_PRIORITY_THREAD_POOL_HPP #define DORIS_BE_SRC_COMMON_UTIL_PRIORITY_THREAD_POOL_HPP -#include -#include #include +#include #include "util/blocking_priority_queue.hpp" +#include "util/thread_group.h" namespace doris { @@ -137,7 +137,7 @@ private: BlockingPriorityQueue _work_queue; // Collection of worker threads that process work from the queue. - boost::thread_group _threads; + ThreadGroup _threads; // Guards _empty_cv std::mutex _lock; diff --git a/be/src/util/progress_updater.h b/be/src/util/progress_updater.h index a47135dd3a..9e771ac1e8 100644 --- a/be/src/util/progress_updater.h +++ b/be/src/util/progress_updater.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_COMMON_UTIL_PROGRESS_UPDATER_H #define DORIS_BE_SRC_COMMON_UTIL_PROGRESS_UPDATER_H -#include #include namespace doris { diff --git a/be/src/util/runtime_profile.cpp b/be/src/util/runtime_profile.cpp index 8b55bacd72..040e617c65 100644 --- a/be/src/util/runtime_profile.cpp +++ b/be/src/util/runtime_profile.cpp @@ -17,9 +17,9 @@ #include "util/runtime_profile.h" -#include #include #include +#include #include "common/config.h" #include "common/object_pool.h" @@ -54,11 +54,10 @@ RuntimeProfile::RuntimeProfile(const std::string& name, bool is_averaged_profile _counter_map["TotalTime"] = &_counter_total_time; } -RuntimeProfile::~RuntimeProfile() { -} +RuntimeProfile::~RuntimeProfile() {} void RuntimeProfile::merge(RuntimeProfile* other) { - DCHECK(other != NULL); + DCHECK(other != nullptr); // Merge this level { @@ -106,7 +105,7 @@ void RuntimeProfile::merge(RuntimeProfile* other) { for (int i = 0; i < other->_children.size(); ++i) { RuntimeProfile* other_child = other->_children[i].first; ChildMap::iterator j = _child_map.find(other_child->_name); - RuntimeProfile* child = NULL; + RuntimeProfile* child = nullptr; if (j != _child_map.end()) { child = j->second; @@ -195,7 +194,7 @@ void RuntimeProfile::update(const std::vector& nodes, int* for (int i = 0; i < node.num_children; ++i) { const TRuntimeProfileNode& tchild = nodes[*idx]; ChildMap::iterator j = _child_map.find(tchild.name); - RuntimeProfile* child = NULL; + RuntimeProfile* child = nullptr; if (j != _child_map.end()) { child = j->second; @@ -270,7 +269,7 @@ RuntimeProfile* RuntimeProfile::create_child(const std::string& name, bool inden DCHECK(_child_map.find(name) == _child_map.end()); RuntimeProfile* child = _pool->add(new RuntimeProfile(name)); if (_children.empty()) { - add_child_unlock(child, indent, NULL); + add_child_unlock(child, indent, nullptr); } else { ChildVector::iterator pos = prepend ? _children.begin() : _children.end(); add_child_unlock(child, indent, (*pos).first); @@ -335,7 +334,7 @@ const std::string* RuntimeProfile::get_info_string(const std::string& key) { InfoStrings::const_iterator it = _info_strings.find(key); if (it == _info_strings.end()) { - return NULL; + return nullptr; } return &it->second; @@ -412,7 +411,7 @@ RuntimeProfile::DerivedCounter* RuntimeProfile::add_derived_counter( std::lock_guard l(_counter_map_lock); if (_counter_map.find(name) != _counter_map.end()) { - return NULL; + return nullptr; } DerivedCounter* counter = _pool->add(new DerivedCounter(type, counter_fn)); @@ -444,13 +443,13 @@ RuntimeProfile::Counter* RuntimeProfile::get_counter(const std::string& name) { return _counter_map[name]; } - return NULL; + return nullptr; } void RuntimeProfile::get_counters(const std::string& name, std::vector* counters) { Counter* c = get_counter(name); - if (c != NULL) { + if (c != nullptr) { counters->push_back(c); } @@ -632,7 +631,7 @@ RuntimeProfile::Counter* RuntimeProfile::add_rate_counter(const std::string& nam default: DCHECK(false) << "Unsupported src counter type: " << src_counter->type(); - return NULL; + return nullptr; } Counter* dst_counter = add_counter(name, dst_type); @@ -668,7 +667,6 @@ RuntimeProfile::EventSequence* RuntimeProfile::add_event_sequence(const std::str return timer; } - void RuntimeProfile::print_child_counters(const std::string& prefix, const std::string& counter_name, const CounterMap& counter_map, diff --git a/be/src/util/runtime_profile.h b/be/src/util/runtime_profile.h index 6ff64cb6e8..f2632d5a2c 100644 --- a/be/src/util/runtime_profile.h +++ b/be/src/util/runtime_profile.h @@ -21,11 +21,10 @@ #include #include -#include -#include #include #include #include +#include #include "common/logging.h" #include "common/object_pool.h" @@ -33,7 +32,6 @@ #include "util/binary_cast.hpp" #include "util/stopwatch.hpp" - namespace doris { // Define macros for updating counters. The macros make it very easy to disable @@ -66,14 +64,14 @@ namespace doris { /*ThreadCounterMeasurement \ MACRO_CONCAT(SCOPED_THREAD_COUNTER_MEASUREMENT, __COUNTER__)(c)*/ #else -#define ADD_COUNTER(profile, name, type) NULL -#define ADD_TIMER(profile, name) NULL +#define ADD_COUNTER(profile, name, type) nullptr +#define ADD_TIMER(profile, name) nullptr #define SCOPED_TIMER(c) #define SCOPED_RAW_TIMER(c) #define SCOPED_ATOMIC_TIMER(c) #define COUNTER_UPDATE(c, v) #define COUNTER_SET(c, v) -#define ADD_THREADCOUNTERS(profile, prefix) NULL +#define ADD_THREADCOUNTERS(profile, prefix) nullptr #define SCOPED_THREAD_COUNTER_MEASUREMENT(c) #endif @@ -112,7 +110,7 @@ public: virtual void set(double value) { DCHECK_EQ(sizeof(value), sizeof(int64_t)); - _value.store(binary_cast(value)); + _value.store(binary_cast(value)); } virtual int64_t value() const { return _value.load(); } @@ -321,7 +319,7 @@ public: // RuntimeProfile object. // If parent_counter_name is a non-empty string, the counter is added as a child of // parent_counter_name. - // Returns NULL if the counter already exists. + // Returns nullptr if the counter already exists. DerivedCounter* add_derived_counter(const std::string& name, TUnit::type type, const DerivedCounterFunction& counter_fn, const std::string& parent_counter_name); @@ -330,7 +328,7 @@ public: // that the caller can update. The counter is owned by the RuntimeProfile object. ThreadCounters* add_thread_counters(const std::string& prefix); - // Gets the counter object with 'name'. Returns NULL if there is no counter with + // Gets the counter object with 'name'. Returns nullptr if there is no counter with // that name. Counter* get_counter(const std::string& name); @@ -351,7 +349,7 @@ public: // TODO: EventSequences are not merged by Merge() EventSequence* add_event_sequence(const std::string& key); - // Returns a pointer to the info string value for 'key'. Returns NULL if + // Returns a pointer to the info string value for 'key'. Returns nullptr if // the key does not exist. const std::string* get_info_string(const std::string& key); @@ -541,7 +539,7 @@ private: class ScopedCounter { public: ScopedCounter(RuntimeProfile::Counter* counter, int64_t val) : _val(val), _counter(counter) { - if (counter == NULL) { + if (counter == nullptr) { return; } @@ -550,7 +548,7 @@ public: // Increment the counter when object is destroyed ~ScopedCounter() { - if (_counter != NULL) { + if (_counter != nullptr) { _counter->update(_val); } } @@ -572,7 +570,7 @@ class ScopedTimer { public: ScopedTimer(RuntimeProfile::Counter* counter, const bool* is_cancelled = nullptr) : _counter(counter), _is_cancelled(is_cancelled) { - if (counter == NULL) { + if (counter == nullptr) { return; } DCHECK(counter->type() == TUnit::TIME_NS); @@ -586,7 +584,7 @@ public: bool is_cancelled() { return _is_cancelled != nullptr && *_is_cancelled; } void UpdateCounter() { - if (_counter != NULL && !is_cancelled()) { + if (_counter != nullptr && !is_cancelled()) { _counter->update(_sw.elapsed_time()); } } diff --git a/be/src/util/stopwatch.hpp b/be/src/util/stopwatch.hpp index 53905a54ee..1e2840e98d 100644 --- a/be/src/util/stopwatch.hpp +++ b/be/src/util/stopwatch.hpp @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_COMMON_UTIL_STOPWATCH_HPP #define DORIS_BE_SRC_COMMON_UTIL_STOPWATCH_HPP -#include #include namespace doris { @@ -91,6 +90,6 @@ using MonotonicStopWatch = CustomStopWatch; // Stop watch for reporting elapsed nanosec based on CLOCK_THREAD_CPUTIME_ID. using ThreadCpuStopWatch = CustomStopWatch; -} +} // namespace doris #endif diff --git a/be/src/util/streaming_sampler.h b/be/src/util/streaming_sampler.h index c475f5211a..a9578bcd7d 100644 --- a/be/src/util/streaming_sampler.h +++ b/be/src/util/streaming_sampler.h @@ -89,8 +89,8 @@ public: /// the period they were collected at. /// If lock is non-null, the lock will be taken before returning. The caller /// must unlock it. - const T* GetSamples(int* num_samples, int* period, SpinLock** lock = NULL) const { - if (lock != NULL) { + const T* GetSamples(int* num_samples, int* period, SpinLock** lock = nullptr) const { + if (lock != nullptr) { lock_.lock(); *lock = &lock_; } diff --git a/be/src/util/symbols_util.cpp b/be/src/util/symbols_util.cpp index a8d6d3c687..428917ec18 100644 --- a/be/src/util/symbols_util.cpp +++ b/be/src/util/symbols_util.cpp @@ -19,13 +19,9 @@ #include -#include -#include +#include #include -using boost::algorithm::split_regex; -using boost::regex; - namespace doris { // For the rules about gcc-compatible name mangling, see: // http://mentorembedded.github.io/cxx-abi/abi.html#mangling @@ -59,7 +55,7 @@ bool SymbolsUtil::is_mangled(const std::string& symbol) { std::string SymbolsUtil::demangle(const std::string& name) { int status = 0; - char* demangled_name = abi::__cxa_demangle(name.c_str(), NULL, NULL, &status); + char* demangled_name = abi::__cxa_demangle(name.c_str(), nullptr, nullptr, &status); if (status != 0) { return name; } @@ -174,8 +170,9 @@ std::string SymbolsUtil::mangle_user_function(const std::string& fn_name, const std::vector& arg_types, bool has_var_args, TypeDescriptor* ret_arg_type) { // We need to split fn_name by :: to separate scoping from tokens - std::vector name_tokens; - split_regex(name_tokens, fn_name, regex("::")); + const std::regex re("::"); + std::sregex_token_iterator it {fn_name.begin(), fn_name.end(), re, -1}; + std::vector name_tokens {it, {}}; // Mangled names use substitution as a builtin compression. The first time a token // is seen, we output the raw token string and store the index ("seq_id"). The @@ -236,7 +233,7 @@ std::string SymbolsUtil::mangle_user_function(const std::string& fn_name, } // Output return argument. - if (ret_arg_type != NULL) { + if (ret_arg_type != nullptr) { int repeated_symbol_idx = -1; if (argument_map.find(ret_arg_type->type) != argument_map.end()) { repeated_symbol_idx = argument_map[ret_arg_type->type]; @@ -256,8 +253,9 @@ std::string SymbolsUtil::mangle_user_function(const std::string& fn_name, std::string SymbolsUtil::mangle_prepare_or_close_function(const std::string& fn_name) { // We need to split fn_name by :: to separate scoping from tokens - std::vector name_tokens; - split_regex(name_tokens, fn_name, regex("::")); + const std::regex re("::"); + std::sregex_token_iterator it {fn_name.begin(), fn_name.end(), re, -1}; + std::vector name_tokens {it, {}}; // Mangled names use substitution as a builtin compression. The first time a token // is seen, we output the raw token string and store the index ("seq_id"). The diff --git a/be/src/util/template_util.h b/be/src/util/template_util.h deleted file mode 100644 index a420ef4815..0000000000 --- a/be/src/util/template_util.h +++ /dev/null @@ -1,52 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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. - -#ifndef IMPALA_UTIL_TEMPLATE_UTIL_H -#define IMPALA_UTIL_TEMPLATE_UTIL_H - -#include -#include -#include -#include - -/// The ENABLE_IF_* macros are used to 'enable' - i.e. to make available to the compiler - -/// a method only if a type parameter belongs to the set described by each macro. Each -/// macro takes the return type of the method as an argument (a requirement of the -/// underlying type trait template that implements this logic). -/// Usage: -/// template ENABLE_IF_ARITHMETIC(T, T) foo(T arg) { return arg + 2; } - -/// Enables a method only if 'type_param' is arithmetic, that is an integral type or a -/// floating-point type -#define ENABLE_IF_ARITHMETIC(type_param, return_type) \ - typename boost::enable_if_c::value, return_type>::type - -/// Enables a method only if 'type_param' is not arithmetic, that is neither an integral -/// type or a floating-point type -#define ENABLE_IF_NOT_ARITHMETIC(type_param, return_type) \ - typename boost::enable_if_c::value, return_type>::type - -/// Enables a method only if 'type_param' is integral, i.e. some variant of int or long -#define ENABLE_IF_INTEGRAL(type_param, return_type) \ - typename boost::enable_if_c::value, return_type>::type - -/// Enables a method only if 'type_param' is a floating point type, i.e. some variant of -/// float or double. -#define ENABLE_IF_FLOAT(type_param, return_type) \ - typename boost::enable_if_c::value, return_type>::type - -#endif diff --git a/be/src/util/thread.cpp b/be/src/util/thread.cpp index 60cb267516..9e35cc134b 100644 --- a/be/src/util/thread.cpp +++ b/be/src/util/thread.cpp @@ -46,7 +46,7 @@ namespace doris { class ThreadMgr; -__thread Thread* Thread::_tls = NULL; +__thread Thread* Thread::_tls = nullptr; // Singleton instance of ThreadMgr. Only visible in this file, used only by Thread. // // The Thread class adds a reference to thread_manager while it is supervising a thread so @@ -353,7 +353,7 @@ Status Thread::start_thread(const std::string& category, const std::string& name t->Release(); }); - int ret = pthread_create(&t->_thread, NULL, &Thread::supervise_thread, t.get()); + int ret = pthread_create(&t->_thread, nullptr, &Thread::supervise_thread, t.get()); if (ret) { return Status::RuntimeError("Could not create thread", ret, strerror(ret)); } @@ -402,7 +402,7 @@ void* Thread::supervise_thread(void* arg) { t->_functor(); pthread_cleanup_pop(true); - return NULL; + return nullptr; } void Thread::finish_thread(void* arg) { @@ -487,7 +487,7 @@ Status ThreadJoiner::join() { // Unconditionally join before returning, to guarantee that any TLS // has been destroyed (pthread_key_create() destructors only run // after a pthread's user method has returned). - int ret = pthread_join(_thread->_thread, NULL); + int ret = pthread_join(_thread->_thread, nullptr); CHECK_EQ(ret, 0); _thread->_joinable = false; return Status::OK(); diff --git a/be/src/util/thread.h b/be/src/util/thread.h index 184f51893f..c890f81286 100644 --- a/be/src/util/thread.h +++ b/be/src/util/thread.h @@ -105,7 +105,7 @@ public: const std::string& category() const; std::string to_string() const; - // The current thread of execution, or NULL if the current thread isn't a doris::Thread. + // The current thread of execution, or nullptr if the current thread isn't a doris::Thread. // This call is signal-safe. static Thread* current_thread(); @@ -181,7 +181,7 @@ private: bool _joinable; - // Thread local pointer to the current thread of execution. Will be NULL if the current + // Thread local pointer to the current thread of execution. Will be nullptr if the current // thread is not a Thread. static __thread Thread* _tls; @@ -200,7 +200,7 @@ private: // with the Thread as its only argument. Executes _functor, but before // doing so registers with the global ThreadMgr and reads the thread's // system ID. After _functor terminates, unregisters with the ThreadMgr. - // Always returns NULL. + // Always returns nullptr. // // supervise_thread() notifies start_thread() when thread initialisation is // completed via the _tid, which is set to the new thread's system ID. diff --git a/be/src/util/thread_group.h b/be/src/util/thread_group.h new file mode 100644 index 0000000000..f401a9c896 --- /dev/null +++ b/be/src/util/thread_group.h @@ -0,0 +1,116 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include + +#include "common/status.h" + +namespace doris { +class ThreadGroup { +public: + ThreadGroup() {} + ~ThreadGroup() { + for (auto thrd : _threads) { + delete thrd; + } + } + + bool is_this_thread_in() const { + std::thread::id id = std::this_thread::get_id(); + std::shared_lock guard(_mutex); + for (auto const& thrd : _threads) { + if (thrd->get_id() == id) { + return true; + } + } + return false; + } + + bool is_thread_in(std::thread* thrd) const { + if (thrd) { + std::thread::id id = thrd->get_id(); + std::shared_lock guard(_mutex); + for (auto const& th : _threads) { + if (th->get_id() == id) { + return true; + } + } + return false; + } else { + return false; + } + } + + template + std::thread* create_thread(F threadfunc) { + std::lock_guard guard(_mutex); + std::unique_ptr new_thread = std::make_unique(threadfunc); + _threads.push_back(new_thread.get()); + return new_thread.release(); + } + + Status add_thread(std::thread* thrd) { + if (thrd) { + if (!is_thread_in(thrd)) { + std::lock_guard guard(_mutex); + _threads.push_back(thrd); + return Status::OK(); + } else { + return Status::InvalidArgument("trying to add a duplicated thread"); + } + } else { + return Status::InvalidArgument("trying to add a nullptr as thread"); + } + } + + void remove_thread(std::thread* thrd) { + std::lock_guard guard(_mutex); + std::list::const_iterator it = + std::find(_threads.begin(), _threads.end(), thrd); + if (it != _threads.end()) { + _threads.erase(it); + } + } + + Status join_all() { + if (is_this_thread_in()) { + return Status::RuntimeError("trying joining itself"); + } + std::shared_lock guard(_mutex); + + for (auto thrd : _threads) { + if (thrd->joinable()) { + thrd->join(); + } + } + return Status::OK(); + } + + size_t size() const { + std::shared_lock guard(_mutex); + return _threads.size(); + } + +private: + std::list _threads; + mutable std::shared_mutex _mutex; +}; +} // namespace doris diff --git a/be/src/util/thrift_client.cpp b/be/src/util/thrift_client.cpp index 1ebaca1727..76e780c581 100644 --- a/be/src/util/thrift_client.cpp +++ b/be/src/util/thrift_client.cpp @@ -17,7 +17,6 @@ #include "util/thrift_client.h" -#include #include #include @@ -35,8 +34,8 @@ Status ThriftClientImpl::open() { try { _transport->close(); } catch (const apache::thrift::transport::TTransportException& e) { - VLOG_CRITICAL << "Error closing socket to: " << ipaddress() << ":" << port() << ", ignoring (" - << e.what() << ")"; + VLOG_CRITICAL << "Error closing socket to: " << ipaddress() << ":" << port() + << ", ignoring (" << e.what() << ")"; } // In certain cases in which the remote host is overloaded, this failure can // happen quite frequently. Let's print this error message without the stack diff --git a/be/src/util/thrift_rpc_helper.cpp b/be/src/util/thrift_rpc_helper.cpp index 63918ab608..8e456c2596 100644 --- a/be/src/util/thrift_rpc_helper.cpp +++ b/be/src/util/thrift_rpc_helper.cpp @@ -17,8 +17,8 @@ #include "util/thrift_rpc_helper.h" -#include #include +#include #include "common/status.h" #include "gen_cpp/FrontendService.h" diff --git a/be/src/util/thrift_server.cpp b/be/src/util/thrift_server.cpp index ca048f77ab..c3dc639eb8 100644 --- a/be/src/util/thrift_server.cpp +++ b/be/src/util/thrift_server.cpp @@ -28,10 +28,10 @@ #include #include -#include #include #include #include +#include #include "util/doris_metrics.h" @@ -110,7 +110,7 @@ Status ThriftServer::ThriftServerEventProcessor::start_and_wait_for_server() { _thrift_server->_started = false; _thrift_server->_server_thread.reset( - new boost::thread(&ThriftServer::ThriftServerEventProcessor::supervise, this)); + new std::thread(&ThriftServer::ThriftServerEventProcessor::supervise, this)); // Loop protects against spurious wakeup. Locks provide necessary fences to ensure // visibility. diff --git a/be/src/util/thrift_server.h b/be/src/util/thrift_server.h index 197b0b01d0..6b05bcc83a 100644 --- a/be/src/util/thrift_server.h +++ b/be/src/util/thrift_server.h @@ -21,8 +21,8 @@ #include #include -#include #include +#include #include #include "common/status.h" @@ -118,13 +118,13 @@ private: const std::string _name; // Thread that runs the TNonblockingServer::serve loop - std::unique_ptr _server_thread; + std::unique_ptr _server_thread; // Thrift housekeeping std::unique_ptr _server; std::shared_ptr _processor; - // If not NULL, called when session events happen. Not owned by us. + // If not nullptr, called when session events happen. Not owned by us. SessionHandlerIf* _session_handler; // Protects _session_keys diff --git a/be/src/util/thrift_util.h b/be/src/util/thrift_util.h index ab69ae8283..51268ac6fe 100644 --- a/be/src/util/thrift_util.h +++ b/be/src/util/thrift_util.h @@ -48,7 +48,7 @@ public: template Status serialize(T* obj, std::vector* result) { uint32_t len = 0; - uint8_t* buffer = NULL; + uint8_t* buffer = nullptr; RETURN_IF_ERROR(serialize(obj, &len, &buffer)); result->resize(len); memcpy(&((*result)[0]), buffer, len); diff --git a/be/src/util/tuple_row_compare.h b/be/src/util/tuple_row_compare.h index bb976037f1..4575cfeb3a 100644 --- a/be/src/util/tuple_row_compare.h +++ b/be/src/util/tuple_row_compare.h @@ -94,13 +94,13 @@ public: void* rhs_value = _key_expr_ctxs_rhs[i]->get_value(rhs); // The sort order of NULLs is independent of asc/desc. - if (lhs_value == NULL && rhs_value == NULL) { + if (lhs_value == nullptr && rhs_value == nullptr) { continue; } - if (lhs_value == NULL && rhs_value != NULL) { + if (lhs_value == nullptr && rhs_value != nullptr) { return _nulls_first[i]; } - if (lhs_value != NULL && rhs_value == NULL) { + if (lhs_value != nullptr && rhs_value == nullptr) { return -_nulls_first[i]; } diff --git a/be/src/util/uid_util.cpp b/be/src/util/uid_util.cpp index c51d104797..550a5057ad 100644 --- a/be/src/util/uid_util.cpp +++ b/be/src/util/uid_util.cpp @@ -37,20 +37,20 @@ std::string print_id(const PUniqueId& id) { } bool parse_id(const std::string& s, TUniqueId* id) { - DCHECK(id != NULL); + DCHECK(id != nullptr); const char* hi_part = s.c_str(); char* colon = const_cast(strchr(hi_part, '-')); - if (colon == NULL) { + if (colon == nullptr) { return false; } const char* lo_part = colon + 1; *colon = '\0'; - char* error_hi = NULL; - char* error_lo = NULL; + char* error_hi = nullptr; + char* error_lo = nullptr; id->hi = strtoul(hi_part, &error_hi, 16); id->lo = strtoul(lo_part, &error_lo, 16); diff --git a/be/src/util/url_coding.cpp b/be/src/util/url_coding.cpp index 6aea9287a3..862c34ccd7 100644 --- a/be/src/util/url_coding.cpp +++ b/be/src/util/url_coding.cpp @@ -162,7 +162,7 @@ static int mod_table[] = {0, 2, 1}; size_t base64_encode(const unsigned char* data, size_t length, unsigned char* encoded_data) { size_t output_length = (size_t)(4.0 * ceil((double)length / 3.0)); - if (encoded_data == NULL) { + if (encoded_data == nullptr) { return 0; } diff --git a/be/src/util/url_coding.h b/be/src/util/url_coding.h index 075e5b63be..695228240e 100644 --- a/be/src/util/url_coding.h +++ b/be/src/util/url_coding.h @@ -18,7 +18,7 @@ #ifndef DORIS_BE_SRC_COMMON_UTIL_URL_CODING_H #define DORIS_BE_SRC_COMMON_UTIL_URL_CODING_H -#include +#include #include #include diff --git a/be/src/util/url_parser.cpp b/be/src/util/url_parser.cpp index 338632b5bb..0ce0913f3d 100644 --- a/be/src/util/url_parser.cpp +++ b/be/src/util/url_parser.cpp @@ -44,7 +44,7 @@ const StringSearch UrlParser::_s_question_search(&_s_question); const StringSearch UrlParser::_s_hash_search(&_s_hash); bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* result) { - result->ptr = NULL; + result->ptr = nullptr; result->len = 0; // Remove leading and trailing spaces. StringValue trimmed_url = url.trim(); diff --git a/be/test/agent/cgroups_mgr_test.cpp b/be/test/agent/cgroups_mgr_test.cpp index 66b80124ee..5e51747bfd 100644 --- a/be/test/agent/cgroups_mgr_test.cpp +++ b/be/test/agent/cgroups_mgr_test.cpp @@ -75,7 +75,7 @@ public: }; std::string CgroupsMgrTest::_s_cgroup_path = "./doris_cgroup_testxxxx123"; -CgroupsMgr CgroupsMgrTest::_s_cgroups_mgr(NULL, CgroupsMgrTest::_s_cgroup_path); +CgroupsMgr CgroupsMgrTest::_s_cgroups_mgr(nullptr, CgroupsMgrTest::_s_cgroup_path); TEST_F(CgroupsMgrTest, TestIsDirectory) { // test folder exist diff --git a/be/test/exec/csv_scan_node_test.cpp b/be/test/exec/csv_scan_node_test.cpp index dd72730af9..b2feef0d65 100644 --- a/be/test/exec/csv_scan_node_test.cpp +++ b/be/test/exec/csv_scan_node_test.cpp @@ -19,7 +19,6 @@ #include -#include #include #include "gen_cpp/PlanNodes_types.h" @@ -65,7 +64,7 @@ private: TDescriptorTable _t_desc_table; DescriptorTbl* _desc_tbl; TPlanNode _tnode; - boost::scoped_ptr _env; + std::unique_ptr _env; RuntimeState* _state; }; // end class CsvScanNodeTest diff --git a/be/test/exec/hash_table_test.cpp b/be/test/exec/hash_table_test.cpp index fae151b60a..cc0d22c38f 100644 --- a/be/test/exec/hash_table_test.cpp +++ b/be/test/exec/hash_table_test.cpp @@ -106,7 +106,7 @@ protected: EXPECT_LT(val, max); if (all_unique) { - EXPECT_TRUE(results[val] == NULL); + EXPECT_TRUE(results[val] == nullptr); } EXPECT_EQ(row->get_tuple(0), expected[val]->get_tuple(0)); @@ -364,7 +364,7 @@ TEST_F(HashTableTest, GrowTableTest2) { HashTable hash_table(_build_expr, _probe_expr, 1, false, is_null_safe, initial_seed, mem_tracker, num_buckets); - LOG(INFO) << time(NULL); + LOG(INFO) << time(nullptr); // constexpr const int test_size = 5 * 1024 * 1024; constexpr const int test_size = 5 * 1024 * 100; @@ -374,7 +374,7 @@ TEST_F(HashTableTest, GrowTableTest2) { expected_size += num_to_add; } - LOG(INFO) << time(NULL); + LOG(INFO) << time(nullptr); // Validate that we can find the entries for (int i = 0; i < test_size; ++i) { @@ -382,7 +382,7 @@ TEST_F(HashTableTest, GrowTableTest2) { hash_table.find(probe_row); } - LOG(INFO) << time(NULL); + LOG(INFO) << time(nullptr); size_t counter = 0; auto func = [&](TupleRow* row) { counter++; }; diff --git a/be/test/exec/mysql_scan_node_test.cpp b/be/test/exec/mysql_scan_node_test.cpp index 186adcb911..9f553d7ed2 100644 --- a/be/test/exec/mysql_scan_node_test.cpp +++ b/be/test/exec/mysql_scan_node_test.cpp @@ -210,7 +210,7 @@ TEST_F(MysqlScanNodeTest, Prepare_fail_1) { TEST_F(MysqlScanNodeTest, Prepare_fail_2) { MysqlScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl); TableDescriptor* old = _desc_tbl->_tuple_desc_map[(TupleId)0]->_table_desc; - _desc_tbl->_tuple_desc_map[(TupleId)0]->_table_desc = NULL; + _desc_tbl->_tuple_desc_map[(TupleId)0]->_table_desc = nullptr; Status status = scan_node.prepare(&_runtim_state); ASSERT_FALSE(status.ok()); _desc_tbl->_tuple_desc_map[(TupleId)0]->_table_desc = old; @@ -242,19 +242,19 @@ TEST_F(MysqlScanNodeTest, open_fail_2) { } TEST_F(MysqlScanNodeTest, invalid_input) { MysqlScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl); - Status status = scan_node.prepare(NULL); + Status status = scan_node.prepare(nullptr); ASSERT_FALSE(status.ok()); status = scan_node.prepare(&_runtim_state); ASSERT_TRUE(status.ok()); status = scan_node.prepare(&_runtim_state); ASSERT_TRUE(status.ok()); - status = scan_node.open(NULL); + status = scan_node.open(nullptr); ASSERT_FALSE(status.ok()); status = scan_node.open(&_runtim_state); ASSERT_TRUE(status.ok()); RowBatch row_batch(scan_node._row_descriptor, 100); bool eos = false; - status = scan_node.get_next(NULL, &row_batch, &eos); + status = scan_node.get_next(nullptr, &row_batch, &eos); ASSERT_FALSE(status.ok()); while (!eos) { diff --git a/be/test/exec/mysql_scanner_test.cpp b/be/test/exec/mysql_scanner_test.cpp index 8cc2679484..56a9054263 100644 --- a/be/test/exec/mysql_scanner_test.cpp +++ b/be/test/exec/mysql_scanner_test.cpp @@ -55,7 +55,7 @@ TEST_F(MysqlScannerTest, normal_use) { bool eos = false; char** buf; unsigned long* length; - status = scanner.get_next_row(NULL, &length, &eos); + status = scanner.get_next_row(nullptr, &length, &eos); ASSERT_FALSE(status.ok()); while (!eos) { diff --git a/be/test/exec/olap_scanner_test.cpp b/be/test/exec/olap_scanner_test.cpp index 7d086c2485..d6d75440c2 100644 --- a/be/test/exec/olap_scanner_test.cpp +++ b/be/test/exec/olap_scanner_test.cpp @@ -36,7 +36,7 @@ namespace doris { static const int RES_BUF_SIZE = 100 * 1024 * 1024; static char res_buf[RES_BUF_SIZE]; -boost::shared_ptr construct_scan_ranges() { +std::shared_ptr construct_scan_ranges() { TPaloScanRange doris_scan_range; TNetworkAddress host; host.__set_hostname("host"); @@ -53,7 +53,7 @@ boost::shared_ptr construct_scan_ranges() { key_range.__set_end_key(65535); key_range.__set_column_name("UserId"); doris_scan_range.partition_column_ranges.push_back(key_range); - boost::shared_ptr scan_range(new DorisScanRange(doris_scan_range)); + std::shared_ptr scan_range(new DorisScanRange(doris_scan_range)); return scan_range; } diff --git a/be/test/exec/schema_scan_node_test.cpp b/be/test/exec/schema_scan_node_test.cpp index beeda89377..25c49d9d17 100644 --- a/be/test/exec/schema_scan_node_test.cpp +++ b/be/test/exec/schema_scan_node_test.cpp @@ -145,7 +145,7 @@ TEST_F(SchemaScanNodeTest, normal_use) { TEST_F(SchemaScanNodeTest, Prepare_fail_1) { SchemaScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl); TableDescriptor* old = _desc_tbl->_tuple_desc_map[(TupleId)0]->_table_desc; - _desc_tbl->_tuple_desc_map[(TupleId)0]->_table_desc = NULL; + _desc_tbl->_tuple_desc_map[(TupleId)0]->_table_desc = nullptr; Status status = scan_node.prepare(&runtime_state); ASSERT_FALSE(status.ok()); _desc_tbl->_tuple_desc_map[(TupleId)0]->_table_desc = old; @@ -201,17 +201,17 @@ TEST_F(SchemaScanNodeTest, get_dest_desc_fail) { } TEST_F(SchemaScanNodeTest, invalid_param) { SchemaScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl); - Status status = scan_node.prepare(NULL); + Status status = scan_node.prepare(nullptr); ASSERT_FALSE(status.ok()); status = scan_node.prepare(&runtime_state); ASSERT_TRUE(status.ok()); - status = scan_node.open(NULL); + status = scan_node.open(nullptr); ASSERT_FALSE(status.ok()); status = scan_node.open(&runtime_state); ASSERT_TRUE(status.ok()); RowBatch row_batch(scan_node._row_descriptor, 100); bool eos; - status = scan_node.get_next(NULL, &row_batch, &eos); + status = scan_node.get_next(nullptr, &row_batch, &eos); ASSERT_FALSE(status.ok()); } diff --git a/be/test/exec/schema_scanner/schema_authors_scanner_test.cpp b/be/test/exec/schema_scanner/schema_authors_scanner_test.cpp index 4f610fe838..91d761845d 100644 --- a/be/test/exec/schema_scanner/schema_authors_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_authors_scanner_test.cpp @@ -52,7 +52,7 @@ TEST_F(SchemaAuthorScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -70,7 +70,7 @@ TEST_F(SchemaAuthorScannerTest, normal_use) { TEST_F(SchemaAuthorScannerTest, use_with_no_init) { SchemaAuthorsScanner scanner; const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL == tuple_desc); + ASSERT_TRUE(nullptr == tuple_desc); Status status = scanner.start((RuntimeState*)1); ASSERT_FALSE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -81,17 +81,17 @@ TEST_F(SchemaAuthorScannerTest, use_with_no_init) { TEST_F(SchemaAuthorScannerTest, invalid_param) { SchemaAuthorsScanner scanner; - Status status = scanner.init(&_param, NULL); + Status status = scanner.init(&_param, nullptr); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; bool eos = false; - status = scanner.get_next_row(tuple, NULL, &eos); + status = scanner.get_next_row(tuple, nullptr, &eos); ASSERT_FALSE(status.ok()); } diff --git a/be/test/exec/schema_scanner/schema_charsets_scanner_test.cpp b/be/test/exec/schema_scanner/schema_charsets_scanner_test.cpp index e254b4f3e2..eb2cae61d6 100644 --- a/be/test/exec/schema_scanner/schema_charsets_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_charsets_scanner_test.cpp @@ -53,7 +53,7 @@ TEST_F(SchemaCharsetsScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -70,7 +70,7 @@ TEST_F(SchemaCharsetsScannerTest, normal_use) { TEST_F(SchemaCharsetsScannerTest, use_with_no_init) { SchemaCharsetsScanner scanner; const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL == tuple_desc); + ASSERT_TRUE(nullptr == tuple_desc); Status status = scanner.start((RuntimeState*)1); ASSERT_FALSE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -81,17 +81,17 @@ TEST_F(SchemaCharsetsScannerTest, use_with_no_init) { TEST_F(SchemaCharsetsScannerTest, invalid_param) { SchemaCharsetsScanner scanner; - Status status = scanner.init(&_param, NULL); + Status status = scanner.init(&_param, nullptr); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; bool eos = false; - status = scanner.get_next_row(tuple, NULL, &eos); + status = scanner.get_next_row(tuple, nullptr, &eos); ASSERT_FALSE(status.ok()); } diff --git a/be/test/exec/schema_scanner/schema_collations_scanner_test.cpp b/be/test/exec/schema_scanner/schema_collations_scanner_test.cpp index 9e8c5319fd..9f7a523a0c 100644 --- a/be/test/exec/schema_scanner/schema_collations_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_collations_scanner_test.cpp @@ -53,7 +53,7 @@ TEST_F(SchemaCollationsScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -70,7 +70,7 @@ TEST_F(SchemaCollationsScannerTest, normal_use) { TEST_F(SchemaCollationsScannerTest, use_with_no_init) { SchemaCollationsScanner scanner; const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL == tuple_desc); + ASSERT_TRUE(nullptr == tuple_desc); Status status = scanner.start((RuntimeState*)1); ASSERT_FALSE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -81,17 +81,17 @@ TEST_F(SchemaCollationsScannerTest, use_with_no_init) { TEST_F(SchemaCollationsScannerTest, invalid_param) { SchemaCollationsScanner scanner; - Status status = scanner.init(&_param, NULL); + Status status = scanner.init(&_param, nullptr); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; bool eos = false; - status = scanner.get_next_row(tuple, NULL, &eos); + status = scanner.get_next_row(tuple, nullptr, &eos); ASSERT_FALSE(status.ok()); } diff --git a/be/test/exec/schema_scanner/schema_columns_scanner_test.cpp b/be/test/exec/schema_scanner/schema_columns_scanner_test.cpp index 26f64d7222..3da4075903 100644 --- a/be/test/exec/schema_scanner/schema_columns_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_columns_scanner_test.cpp @@ -98,7 +98,7 @@ TEST_F(SchemaColumnsScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -115,7 +115,7 @@ TEST_F(SchemaColumnsScannerTest, one_column) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -144,14 +144,14 @@ TEST_F(SchemaColumnsScannerTest, input_fail) { db_num = 1; desc_num = 1; SchemaColumnsScanner scanner; - Status status = scanner.init(NULL, &_obj_pool); + Status status = scanner.init(nullptr, &_obj_pool); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); bool eos = false; - status = scanner.get_next_row(NULL, &_mem_pool, &eos); + status = scanner.get_next_row(nullptr, &_mem_pool, &eos); ASSERT_FALSE(status.ok()); } TEST_F(SchemaColumnsScannerTest, table_fail) { @@ -162,7 +162,7 @@ TEST_F(SchemaColumnsScannerTest, table_fail) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -179,7 +179,7 @@ TEST_F(SchemaColumnsScannerTest, desc_fail) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; diff --git a/be/test/exec/schema_scanner/schema_create_table_scanner_test.cpp b/be/test/exec/schema_scanner/schema_create_table_scanner_test.cpp index 835f9525c5..8520b81262 100644 --- a/be/test/exec/schema_scanner/schema_create_table_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_create_table_scanner_test.cpp @@ -98,7 +98,7 @@ TEST_F(SchemaCreateTableScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -115,7 +115,7 @@ TEST_F(SchemaCreateTableScannerTest, one_column) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -144,14 +144,14 @@ TEST_F(SchemaCreateTableScannerTest, input_fail) { db_num = 1; desc_num = 1; SchemaCreateTableScanner scanner; - Status status = scanner.init(NULL, &_obj_pool); + Status status = scanner.init(nullptr, &_obj_pool); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); bool eos = false; - status = scanner.get_next_row(NULL, &_mem_pool, &eos); + status = scanner.get_next_row(nullptr, &_mem_pool, &eos); ASSERT_FALSE(status.ok()); } TEST_F(SchemaCreateTableScannerTest, table_fail) { @@ -162,7 +162,7 @@ TEST_F(SchemaCreateTableScannerTest, table_fail) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -179,7 +179,7 @@ TEST_F(SchemaCreateTableScannerTest, desc_fail) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; diff --git a/be/test/exec/schema_scanner/schema_engines_scanner_test.cpp b/be/test/exec/schema_scanner/schema_engines_scanner_test.cpp index fc54af4c49..44d6375f1e 100644 --- a/be/test/exec/schema_scanner/schema_engines_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_engines_scanner_test.cpp @@ -53,7 +53,7 @@ TEST_F(SchemaEnginesScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -70,7 +70,7 @@ TEST_F(SchemaEnginesScannerTest, normal_use) { TEST_F(SchemaEnginesScannerTest, use_with_no_init) { SchemaEnginesScanner scanner; const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL == tuple_desc); + ASSERT_TRUE(nullptr == tuple_desc); Status status = scanner.start((RuntimeState*)1); ASSERT_FALSE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -81,17 +81,17 @@ TEST_F(SchemaEnginesScannerTest, use_with_no_init) { TEST_F(SchemaEnginesScannerTest, invalid_param) { SchemaEnginesScanner scanner; - Status status = scanner.init(&_param, NULL); + Status status = scanner.init(&_param, nullptr); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; bool eos = false; - status = scanner.get_next_row(tuple, NULL, &eos); + status = scanner.get_next_row(tuple, nullptr, &eos); ASSERT_FALSE(status.ok()); } diff --git a/be/test/exec/schema_scanner/schema_open_tables_scanner_test.cpp b/be/test/exec/schema_scanner/schema_open_tables_scanner_test.cpp index 112b4f960e..488e00eba2 100644 --- a/be/test/exec/schema_scanner/schema_open_tables_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_open_tables_scanner_test.cpp @@ -98,7 +98,7 @@ TEST_F(SchemaOpenTablesScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -115,7 +115,7 @@ TEST_F(SchemaOpenTablesScannerTest, one_column) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -144,14 +144,14 @@ TEST_F(SchemaOpenTablesScannerTest, input_fail) { db_num = 1; desc_num = 1; SchemaOpenTablesScanner scanner; - Status status = scanner.init(NULL, &_obj_pool); + Status status = scanner.init(nullptr, &_obj_pool); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); bool eos = false; - status = scanner.get_next_row(NULL, &_mem_pool, &eos); + status = scanner.get_next_row(nullptr, &_mem_pool, &eos); ASSERT_FALSE(status.ok()); } TEST_F(SchemaOpenTablesScannerTest, table_fail) { @@ -162,7 +162,7 @@ TEST_F(SchemaOpenTablesScannerTest, table_fail) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -179,7 +179,7 @@ TEST_F(SchemaOpenTablesScannerTest, desc_fail) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; diff --git a/be/test/exec/schema_scanner/schema_schemata_scanner_test.cpp b/be/test/exec/schema_scanner/schema_schemata_scanner_test.cpp index 4077a7fe85..ea823bed5d 100644 --- a/be/test/exec/schema_scanner/schema_schemata_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_schemata_scanner_test.cpp @@ -98,7 +98,7 @@ TEST_F(SchemaSchemataScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -115,7 +115,7 @@ TEST_F(SchemaSchemataScannerTest, one_column) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -144,14 +144,14 @@ TEST_F(SchemaSchemataScannerTest, input_fail) { db_num = 1; desc_num = 1; SchemaSchemataScanner scanner; - Status status = scanner.init(NULL, &_obj_pool); + Status status = scanner.init(nullptr, &_obj_pool); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); bool eos = false; - status = scanner.get_next_row(NULL, &_mem_pool, &eos); + status = scanner.get_next_row(nullptr, &_mem_pool, &eos); ASSERT_FALSE(status.ok()); } diff --git a/be/test/exec/schema_scanner/schema_table_names_scanner_test.cpp b/be/test/exec/schema_scanner/schema_table_names_scanner_test.cpp index 26574c1267..6e8ce962e4 100644 --- a/be/test/exec/schema_scanner/schema_table_names_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_table_names_scanner_test.cpp @@ -98,7 +98,7 @@ TEST_F(SchemaTableNamesScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -115,7 +115,7 @@ TEST_F(SchemaTableNamesScannerTest, one_column) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -144,14 +144,14 @@ TEST_F(SchemaTableNamesScannerTest, input_fail) { db_num = 1; desc_num = 1; SchemaTableNamesScanner scanner; - Status status = scanner.init(NULL, &_obj_pool); + Status status = scanner.init(nullptr, &_obj_pool); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); bool eos = false; - status = scanner.get_next_row(NULL, &_mem_pool, &eos); + status = scanner.get_next_row(nullptr, &_mem_pool, &eos); ASSERT_FALSE(status.ok()); } TEST_F(SchemaTableNamesScannerTest, table_fail) { @@ -162,7 +162,7 @@ TEST_F(SchemaTableNamesScannerTest, table_fail) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; diff --git a/be/test/exec/schema_scanner/schema_tables_scanner_test.cpp b/be/test/exec/schema_scanner/schema_tables_scanner_test.cpp index 7ec8aeaf5c..7ccc12cb11 100644 --- a/be/test/exec/schema_scanner/schema_tables_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_tables_scanner_test.cpp @@ -98,7 +98,7 @@ TEST_F(SchemaTablesScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -115,7 +115,7 @@ TEST_F(SchemaTablesScannerTest, one_column) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -144,14 +144,14 @@ TEST_F(SchemaTablesScannerTest, input_fail) { db_num = 1; desc_num = 1; SchemaTablesScanner scanner; - Status status = scanner.init(NULL, &_obj_pool); + Status status = scanner.init(nullptr, &_obj_pool); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); bool eos = false; - status = scanner.get_next_row(NULL, &_mem_pool, &eos); + status = scanner.get_next_row(nullptr, &_mem_pool, &eos); ASSERT_FALSE(status.ok()); } TEST_F(SchemaTablesScannerTest, table_fail) { @@ -162,7 +162,7 @@ TEST_F(SchemaTablesScannerTest, table_fail) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -179,7 +179,7 @@ TEST_F(SchemaTablesScannerTest, desc_fail) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; diff --git a/be/test/exec/schema_scanner/schema_variables_scanner_test.cpp b/be/test/exec/schema_scanner/schema_variables_scanner_test.cpp index e97e22a1ea..01f7f9614c 100644 --- a/be/test/exec/schema_scanner/schema_variables_scanner_test.cpp +++ b/be/test/exec/schema_scanner/schema_variables_scanner_test.cpp @@ -54,7 +54,7 @@ TEST_F(SchemaVariablesScannerTest, normal_use) { Status status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -71,7 +71,7 @@ TEST_F(SchemaVariablesScannerTest, normal_use) { TEST_F(SchemaVariablesScannerTest, use_with_no_init) { SchemaVariablesScanner scanner; const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL == tuple_desc); + ASSERT_TRUE(nullptr == tuple_desc); Status status = scanner.start((RuntimeState*)1); ASSERT_FALSE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; @@ -82,17 +82,17 @@ TEST_F(SchemaVariablesScannerTest, use_with_no_init) { TEST_F(SchemaVariablesScannerTest, invalid_param) { SchemaVariablesScanner scanner; - Status status = scanner.init(&_param, NULL); + Status status = scanner.init(&_param, nullptr); ASSERT_FALSE(status.ok()); status = scanner.init(&_param, &_obj_pool); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); Tuple* tuple = (Tuple*)g_tuple_buf; bool eos = false; - status = scanner.get_next_row(tuple, NULL, &eos); + status = scanner.get_next_row(tuple, nullptr, &eos); ASSERT_FALSE(status.ok()); } diff --git a/be/test/exec/schema_scanner_test.cpp b/be/test/exec/schema_scanner_test.cpp index 939b035963..2f8c36a17b 100644 --- a/be/test/exec/schema_scanner_test.cpp +++ b/be/test/exec/schema_scanner_test.cpp @@ -65,7 +65,7 @@ TEST_F(SchemaScannerTest, normal_use) { status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); ASSERT_EQ(65, tuple_desc->byte_size()); Tuple* tuple = (Tuple*)g_tuple_buf; bool eos; @@ -82,14 +82,14 @@ TEST_F(SchemaScannerTest, input_fail) { status = scanner.start((RuntimeState*)1); ASSERT_TRUE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL != tuple_desc); + ASSERT_TRUE(nullptr != tuple_desc); ASSERT_EQ(65, tuple_desc->byte_size()); bool eos; - status = scanner.get_next_row(NULL, &_mem_pool, &eos); + status = scanner.get_next_row(nullptr, &_mem_pool, &eos); ASSERT_FALSE(status.ok()); } TEST_F(SchemaScannerTest, invalid_param) { - SchemaScanner scanner(NULL, sizeof(s_test_columns) / sizeof(SchemaScanner::ColumnDesc)); + SchemaScanner scanner(nullptr, sizeof(s_test_columns) / sizeof(SchemaScanner::ColumnDesc)); Status status = scanner.init(&_param, &_obj_pool); ASSERT_FALSE(status.ok()); } @@ -99,7 +99,7 @@ TEST_F(SchemaScannerTest, no_init_use) { Status status = scanner.start((RuntimeState*)1); ASSERT_FALSE(status.ok()); const TupleDescriptor* tuple_desc = scanner.tuple_desc(); - ASSERT_TRUE(NULL == tuple_desc); + ASSERT_TRUE(nullptr == tuple_desc); Tuple* tuple = (Tuple*)g_tuple_buf; bool eos; status = scanner.get_next_row(tuple, &_mem_pool, &eos); diff --git a/be/test/exprs/binary_predicate_test.cpp b/be/test/exprs/binary_predicate_test.cpp index 43342b94bf..6a05e23ebd 100644 --- a/be/test/exprs/binary_predicate_test.cpp +++ b/be/test/exprs/binary_predicate_test.cpp @@ -59,9 +59,9 @@ public: slot_desc.__set_isMaterialized(true); ttbl.slotDescriptors.push_back(slot_desc); - DescriptorTbl* desc_tbl = NULL; + DescriptorTbl* desc_tbl = nullptr; ASSERT_TRUE(DescriptorTbl::create(_object_pool, ttbl, &desc_tbl).ok()); - ASSERT_TRUE(desc_tbl != NULL); + ASSERT_TRUE(desc_tbl != nullptr); _runtime_state->set_desc_tbl(desc_tbl); std::vector row_tuples; @@ -78,9 +78,9 @@ public: _schema.push_back(field); } virtual void TearDown() { - if (_object_pool != NULL) { + if (_object_pool != nullptr) { delete _object_pool; - _object_pool = NULL; + _object_pool = nullptr; } } @@ -128,12 +128,12 @@ public: expr_node.__set_int_literal(int_literal); exprs.nodes.push_back(expr_node); } - Expr* root_expr = NULL; + Expr* root_expr = nullptr; if (Expr::create_expr_tree(_object_pool, exprs, &root_expr).ok()) { return root_expr; } else { - return NULL; + return nullptr; } } @@ -151,13 +151,13 @@ private: TEST_F(BinaryOpTest, PrepareTest) { Expr* expr = create_expr(); - ASSERT_TRUE(expr != NULL); + ASSERT_TRUE(expr != nullptr); ASSERT_TRUE(expr->prepare(runtime_state(), *row_desc()).ok()); } TEST_F(BinaryOpTest, NormalTest) { Expr* expr = create_expr(); - ASSERT_TRUE(expr != NULL); + ASSERT_TRUE(expr != nullptr); ASSERT_TRUE(expr->prepare(runtime_state(), *row_desc()).ok()); int capacity = 256; VectorizedRowBatch* vec_row_batch = @@ -187,11 +187,11 @@ TEST_F(BinaryOpTest, SimplePerformanceTest) { ASSERT_EQ(1, _row_desc->tuple_descriptors().size()); for (int capacity = 128; capacity <= 1024 * 128; capacity *= 2) { Expr* expr = create_expr(); - ASSERT_TRUE(expr != NULL); + ASSERT_TRUE(expr != nullptr); ASSERT_TRUE(expr->prepare(runtime_state(), *row_desc()).ok()); int size = 1024 * 1024 / capacity; VectorizedRowBatch* vec_row_batches[size]; - srand(time(NULL)); + srand(time(nullptr)); for (int i = 0; i < size; ++i) { vec_row_batches[i] = object_pool()->add(new VectorizedRowBatch(_schema, capacity)); diff --git a/be/test/exprs/in_op_test.cpp b/be/test/exprs/in_op_test.cpp index ebdb0d75e4..3de904beb9 100644 --- a/be/test/exprs/in_op_test.cpp +++ b/be/test/exprs/in_op_test.cpp @@ -33,7 +33,7 @@ namespace doris { class InOpTest : public ::testing::Test { public: ~InOpTest() {} - InOpTest() : _object_pool(NULL), _runtime_state(NULL), _row_desc(NULL) {} + InOpTest() : _object_pool(nullptr), _runtime_state(nullptr), _row_desc(nullptr) {} virtual void SetUp() { _object_pool = new ObjectPool(); _runtime_state = _object_pool->add(new RuntimeState("")); @@ -58,9 +58,9 @@ public: slot_desc.__set_isMaterialized(true); ttbl.slotDescriptors.push_back(slot_desc); - DescriptorTbl* desc_tbl = NULL; + DescriptorTbl* desc_tbl = nullptr; ASSERT_TRUE(DescriptorTbl::create(_object_pool, ttbl, &desc_tbl).ok()); - ASSERT_TRUE(desc_tbl != NULL); + ASSERT_TRUE(desc_tbl != nullptr); _runtime_state->set_desc_tbl(desc_tbl); std::vector row_tuples; @@ -70,9 +70,9 @@ public: _row_desc = _object_pool->add(new RowDescriptor(*desc_tbl, row_tuples, nullable_tuples)); } virtual void TearDown() { - if (_object_pool != NULL) { + if (_object_pool != nullptr) { delete _object_pool; - _object_pool = NULL; + _object_pool = nullptr; } } @@ -125,12 +125,12 @@ public: exprs.nodes.push_back(expr_node); } - Expr* root_expr = NULL; + Expr* root_expr = nullptr; if (Expr::create_expr_tree(_object_pool, exprs, &root_expr).ok()) { return root_expr; } else { - return NULL; + return nullptr; } } @@ -142,13 +142,13 @@ private: TEST_F(InOpTest, PrepareTest) { Expr* expr = create_expr(); - ASSERT_TRUE(expr != NULL); + ASSERT_TRUE(expr != nullptr); ASSERT_TRUE(expr->prepare(_runtime_state, *_row_desc).ok()); } TEST_F(InOpTest, NormalTest) { Expr* expr = create_expr(); - ASSERT_TRUE(expr != NULL); + ASSERT_TRUE(expr != nullptr); ASSERT_TRUE(expr->prepare(_runtime_state, *_row_desc).ok()); int capacity = 256; VectorizedRowBatch* vec_row_batch = _object_pool->add( @@ -176,11 +176,11 @@ TEST_F(InOpTest, NormalTest) { TEST_F(InOpTest, SimplePerformanceTest) { for (int capacity = 128; capacity <= 1024 * 128; capacity *= 2) { Expr* expr = create_expr(); - ASSERT_TRUE(expr != NULL); + ASSERT_TRUE(expr != nullptr); ASSERT_TRUE(expr->prepare(_runtime_state, *_row_desc).ok()); int size = 1024 * 1024 / capacity; VectorizedRowBatch* vec_row_batches[size]; - srand(time(NULL)); + srand(time(nullptr)); for (int i = 0; i < size; ++i) { vec_row_batches[i] = _object_pool->add(new VectorizedRowBatch( diff --git a/be/test/exprs/timestamp_functions_test.cpp b/be/test/exprs/timestamp_functions_test.cpp index 7e2f042574..766365af43 100644 --- a/be/test/exprs/timestamp_functions_test.cpp +++ b/be/test/exprs/timestamp_functions_test.cpp @@ -19,8 +19,6 @@ #include -#include - #include "runtime/exec_env.h" #include "runtime/runtime_state.h" #include "runtime/test_env.h" diff --git a/be/test/olap/bit_field_test.cpp b/be/test/olap/bit_field_test.cpp index 21dfb19cd2..bedf8c4f16 100644 --- a/be/test/olap/bit_field_test.cpp +++ b/be/test/olap/bit_field_test.cpp @@ -35,10 +35,10 @@ public: void SetUp() { system("mkdir -p ./ut_dir/"); system("rm ./ut_dir/tmp_file"); - _out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, NULL); - ASSERT_TRUE(_out_stream != NULL); + _out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, nullptr); + ASSERT_TRUE(_out_stream != nullptr); _writer = new (std::nothrow) BitFieldWriter(_out_stream); - ASSERT_TRUE(_writer != NULL); + ASSERT_TRUE(_writer != nullptr); _writer->init(); } @@ -62,15 +62,15 @@ public: _shared_buffer = StorageByteBuffer::create(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE + sizeof(StreamHead)); - ASSERT_TRUE(_shared_buffer != NULL); + ASSERT_TRUE(_shared_buffer != nullptr); _stream = new (std::nothrow) - ReadOnlyFileStream(&_helper, &_shared_buffer, 0, _helper.length(), NULL, + ReadOnlyFileStream(&_helper, &_shared_buffer, 0, _helper.length(), nullptr, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, &_stats); ASSERT_EQ(OLAP_SUCCESS, _stream->init()); _reader = new (std::nothrow) BitFieldReader(_stream); - ASSERT_TRUE(_reader != NULL); + ASSERT_TRUE(_reader != nullptr); _reader->init(); } diff --git a/be/test/olap/bloom_filter_column_predicate_test.cpp b/be/test/olap/bloom_filter_column_predicate_test.cpp index a62db5483e..d4c4f1a1f6 100644 --- a/be/test/olap/bloom_filter_column_predicate_test.cpp +++ b/be/test/olap/bloom_filter_column_predicate_test.cpp @@ -32,13 +32,13 @@ namespace doris { class TestBloomFilterColumnPredicate : public testing::Test { public: - TestBloomFilterColumnPredicate() : _vectorized_batch(NULL), _row_block(nullptr) { + TestBloomFilterColumnPredicate() : _vectorized_batch(nullptr), _row_block(nullptr) { _mem_tracker.reset(new MemTracker(-1)); _mem_pool.reset(new MemPool(_mem_tracker.get())); } ~TestBloomFilterColumnPredicate() { - if (_vectorized_batch != NULL) { + if (_vectorized_batch != nullptr) { delete _vectorized_batch; } } diff --git a/be/test/olap/bloom_filter_index_test.cpp b/be/test/olap/bloom_filter_index_test.cpp index 18df4f1cc5..93d1b63b94 100644 --- a/be/test/olap/bloom_filter_index_test.cpp +++ b/be/test/olap/bloom_filter_index_test.cpp @@ -44,7 +44,7 @@ TEST_F(TestBloomFilterIndex, normal_read_and_write) { BloomFilter* bf_0 = new (std::nothrow) BloomFilter(); bf_0->init(1024); bytes = "hello"; - bf_0->add_bytes(NULL, 0); + bf_0->add_bytes(nullptr, 0); bf_0->add_bytes(bytes.c_str(), bytes.size()); writer.add_bloom_filter(bf_0); @@ -67,7 +67,7 @@ TEST_F(TestBloomFilterIndex, normal_read_and_write) { bytes = "hello"; const BloomFilter& bf__0 = reader.entry(0); - ASSERT_TRUE(bf__0.test_bytes(NULL, 0)); + ASSERT_TRUE(bf__0.test_bytes(nullptr, 0)); ASSERT_TRUE(bf__0.test_bytes(bytes.c_str(), bytes.size())); bytes = "doris"; @@ -79,8 +79,8 @@ TEST_F(TestBloomFilterIndex, normal_read_and_write) { TEST_F(TestBloomFilterIndex, abnormal_write) { char buffer[24]; BloomFilterIndexWriter writer; - ASSERT_EQ(OLAP_ERR_INPUT_PARAMETER_ERROR, writer.write_to_buffer(NULL)); - ASSERT_EQ(OLAP_ERR_INPUT_PARAMETER_ERROR, writer.write_to_buffer(NULL, 0)); + ASSERT_EQ(OLAP_ERR_INPUT_PARAMETER_ERROR, writer.write_to_buffer(nullptr)); + ASSERT_EQ(OLAP_ERR_INPUT_PARAMETER_ERROR, writer.write_to_buffer(nullptr, 0)); ASSERT_EQ(OLAP_ERR_INPUT_PARAMETER_ERROR, writer.write_to_buffer(buffer, 0)); ASSERT_EQ(sizeof(BloomFilterIndexHeader), writer.estimate_buffered_memory()); } diff --git a/be/test/olap/bloom_filter_test.cpp b/be/test/olap/bloom_filter_test.cpp index abaa8434a8..2275503206 100644 --- a/be/test/olap/bloom_filter_test.cpp +++ b/be/test/olap/bloom_filter_test.cpp @@ -74,7 +74,7 @@ TEST_F(TestBloomFilter, init_bloom_filter) { bf.reset(); ASSERT_EQ(0, bf.bit_num()); ASSERT_EQ(0, bf.hash_function_num()); - ASSERT_EQ(NULL, bf.bit_set_data()); + ASSERT_EQ(nullptr, bf.bit_set_data()); delete[] data; } } diff --git a/be/test/olap/byte_buffer_test.cpp b/be/test/olap/byte_buffer_test.cpp index 4d2b5d1cf5..340e7ddea9 100644 --- a/be/test/olap/byte_buffer_test.cpp +++ b/be/test/olap/byte_buffer_test.cpp @@ -17,10 +17,11 @@ #include "olap/byte_buffer.h" -#include #include #include +#include + #include "common/configbase.h" #include "olap/file_helper.h" #include "util/logging.h" @@ -40,10 +41,10 @@ public: // 测试基本的读写功能 TEST_F(TestByteBuffer, TestReadWrite) { - StorageByteBuffer* buf1 = NULL; + StorageByteBuffer* buf1 = nullptr; buf1 = StorageByteBuffer::create(100); - ASSERT_TRUE(buf1 != NULL); + ASSERT_TRUE(buf1 != nullptr); char in[10] = {'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j'}; for (int i = 0; i < 5; i++) { @@ -110,10 +111,10 @@ TEST_F(TestByteBuffer, TestReadWrite) { // 测试ByteBuffer对内存的引用, 尤其是智能指针的引用传递 // 使用valgrind进行内存泄露检查 TEST_F(TestByteBuffer, TestRef) { - StorageByteBuffer* buf1 = NULL; + StorageByteBuffer* buf1 = nullptr; buf1 = StorageByteBuffer::create(1000); - ASSERT_TRUE(buf1 != NULL); + ASSERT_TRUE(buf1 != nullptr); for (int i = 0; i < 256; i++) { ASSERT_EQ(OLAP_SUCCESS, buf1->put(i)); @@ -123,7 +124,7 @@ TEST_F(TestByteBuffer, TestRef) { StorageByteBuffer buf4(*buf1); ASSERT_EQ(buf2.array(), buf1->array()); - StorageByteBuffer* buf3 = NULL; + StorageByteBuffer* buf3 = nullptr; buf3 = StorageByteBuffer::reference_buffer(buf1, 10, 90); ASSERT_EQ(90u, buf3->capacity()); @@ -158,11 +159,11 @@ TEST_F(TestByteBuffer, TestMmap) { res = file_handle.open(file_name, O_RDWR); ASSERT_EQ(OLAP_SUCCESS, res); - StorageByteBuffer* buf1 = StorageByteBuffer::mmap(NULL, 80, PROT_READ | PROT_WRITE, MAP_SHARED, - file_handle.fd(), 0); + StorageByteBuffer* buf1 = StorageByteBuffer::mmap(nullptr, 80, PROT_READ | PROT_WRITE, + MAP_SHARED, file_handle.fd(), 0); // mmap完成后就可以关闭原fd file_handle.close(); - ASSERT_TRUE(buf1 != NULL); + ASSERT_TRUE(buf1 != nullptr); for (int i = 0; i < 80; i++) { char byte; diff --git a/be/test/olap/column_reader_test.cpp b/be/test/olap/column_reader_test.cpp index ee4c84ff7e..ed0d13a0e5 100644 --- a/be/test/olap/column_reader_test.cpp +++ b/be/test/olap/column_reader_test.cpp @@ -38,7 +38,7 @@ namespace doris { class TestColumn : public testing::Test { public: - TestColumn() : _column_writer(NULL), _column_reader(NULL), _stream_factory(NULL) { + TestColumn() : _column_writer(nullptr), _column_reader(nullptr), _stream_factory(nullptr) { _offsets.clear(); _mem_tracker.reset(new MemTracker(-1)); _mem_pool.reset(new MemPool(_mem_tracker.get())); @@ -55,7 +55,7 @@ public: _stream_factory = new (std::nothrow) OutStreamFactory(COMPRESS_LZ4, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE); - ASSERT_TRUE(_stream_factory != NULL); + ASSERT_TRUE(_stream_factory != nullptr); config::column_dictionary_key_ratio_threshold = 30; config::column_dictionary_key_size_threshold = 1000; } @@ -82,7 +82,7 @@ public: _column_writer = ColumnWriter::create(0, tablet_schema, _stream_factory, 1024, BLOOM_FILTER_DEFAULT_FPP); - ASSERT_TRUE(_column_writer != NULL); + ASSERT_TRUE(_column_writer != nullptr); ASSERT_EQ(_column_writer->init(), OLAP_SUCCESS); } @@ -103,7 +103,7 @@ public: _column_reader = ColumnReader::create(0, tablet_schema, included, segment_included, encodings); - ASSERT_TRUE(_column_reader != NULL); + ASSERT_TRUE(_column_reader != nullptr); system("mkdir -p ./ut_dir"); system("rm ./ut_dir/tmp_file"); @@ -142,7 +142,7 @@ public: ASSERT_TRUE(false); } - ASSERT_TRUE(buffers != NULL); + ASSERT_TRUE(buffers != nullptr); off.push_back(helper.tell()); out_stream->write_to_file(&helper, 0); length.push_back(out_stream->get_stream_length()); @@ -156,7 +156,7 @@ public: _shared_buffer = StorageByteBuffer::create(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE + sizeof(StreamHead)); - ASSERT_TRUE(_shared_buffer != NULL); + ASSERT_TRUE(_shared_buffer != nullptr); for (int i = 0; i < off.size(); ++i) { ReadOnlyFileStream* in_stream = new (std::nothrow) @@ -1013,7 +1013,7 @@ TEST_F(TestColumn, VectorizedIntColumnMassWithoutPresent) { _col_vector.reset(new ColumnVector()); - char* data = NULL; + char* data = nullptr; for (int32_t i = 0; i < 10000; ++i) { if (i % 1000 == 0) { ASSERT_EQ(_column_reader->next_vector(_col_vector.get(), 1000, _mem_pool.get()), diff --git a/be/test/olap/comparison_predicate_test.cpp b/be/test/olap/comparison_predicate_test.cpp index 0729740400..5aee788744 100644 --- a/be/test/olap/comparison_predicate_test.cpp +++ b/be/test/olap/comparison_predicate_test.cpp @@ -89,12 +89,12 @@ static std::string to_datetime_string(uint64_t& datetime_value) { #define TEST_PREDICATE_DEFINITION(CLASS_NAME) \ class CLASS_NAME : public testing::Test { \ public: \ - CLASS_NAME() : _vectorized_batch(NULL) { \ + CLASS_NAME() : _vectorized_batch(nullptr) { \ _mem_tracker.reset(new MemTracker(-1)); \ _mem_pool.reset(new MemPool(_mem_tracker.get())); \ } \ ~CLASS_NAME() { \ - if (_vectorized_batch != NULL) { \ + if (_vectorized_batch != nullptr) { \ delete _vectorized_batch; \ } \ } \ @@ -803,8 +803,7 @@ TEST_LESS_PREDICATE(int128_t, LARGEINT, "LARGEINT") TEST_F(TestLessPredicate, FLOAT_COLUMN) { TabletSchema tablet_schema; - SetTabletSchema(std::string("FLOAT_COLUMN"), "FLOAT", "REPLACE", 1, true, true, - &tablet_schema); + SetTabletSchema(std::string("FLOAT_COLUMN"), "FLOAT", "REPLACE", 1, true, true, &tablet_schema); int size = 10; std::vector return_columns; for (int i = 0; i < tablet_schema.num_columns(); ++i) { diff --git a/be/test/olap/delete_handler_test.cpp b/be/test/olap/delete_handler_test.cpp index 98823cc594..22d7fe57fb 100644 --- a/be/test/olap/delete_handler_test.cpp +++ b/be/test/olap/delete_handler_test.cpp @@ -259,7 +259,7 @@ protected: ASSERT_EQ(OLAP_SUCCESS, res); tablet = k_engine->tablet_manager()->get_tablet(_create_tablet.tablet_id, _create_tablet.tablet_schema.schema_hash); - ASSERT_TRUE(tablet.get() != NULL); + ASSERT_TRUE(tablet.get() != nullptr); _tablet_path = tablet->tablet_path(); set_create_duplicate_tablet_request(&_create_dup_tablet); @@ -267,7 +267,7 @@ protected: ASSERT_EQ(OLAP_SUCCESS, res); dup_tablet = k_engine->tablet_manager()->get_tablet( _create_dup_tablet.tablet_id, _create_dup_tablet.tablet_schema.schema_hash); - ASSERT_TRUE(dup_tablet.get() != NULL); + ASSERT_TRUE(dup_tablet.get() != nullptr); _dup_tablet_path = tablet->tablet_path(); } @@ -430,7 +430,7 @@ protected: ASSERT_EQ(OLAP_SUCCESS, res); tablet = k_engine->tablet_manager()->get_tablet(_create_tablet.tablet_id, _create_tablet.tablet_schema.schema_hash); - ASSERT_TRUE(tablet.get() != NULL); + ASSERT_TRUE(tablet.get() != nullptr); _tablet_path = tablet->tablet_path(); } diff --git a/be/test/olap/in_list_predicate_test.cpp b/be/test/olap/in_list_predicate_test.cpp index 0dd55000cc..a00cea3d61 100644 --- a/be/test/olap/in_list_predicate_test.cpp +++ b/be/test/olap/in_list_predicate_test.cpp @@ -87,13 +87,13 @@ static std::string to_datetime_string(uint64_t& datetime_value) { class TestInListPredicate : public testing::Test { public: - TestInListPredicate() : _vectorized_batch(NULL), _row_block(nullptr) { + TestInListPredicate() : _vectorized_batch(nullptr), _row_block(nullptr) { _mem_tracker.reset(new MemTracker(-1)); _mem_pool.reset(new MemPool(_mem_tracker.get())); } ~TestInListPredicate() { - if (_vectorized_batch != NULL) { + if (_vectorized_batch != nullptr) { delete _vectorized_batch; } } diff --git a/be/test/olap/lru_cache_test.cpp b/be/test/olap/lru_cache_test.cpp index 1456365cf1..e99343968a 100644 --- a/be/test/olap/lru_cache_test.cpp +++ b/be/test/olap/lru_cache_test.cpp @@ -21,8 +21,8 @@ #include -#include "util/logging.h" #include "test_util/test_util.h" +#include "util/logging.h" using namespace doris; using namespace std; @@ -83,9 +83,9 @@ public: int Lookup(int key) { std::string result; Cache::Handle* handle = _cache->lookup(EncodeKey(&result, key)); - const int r = (handle == NULL) ? -1 : DecodeValue(_cache->value(handle)); + const int r = (handle == nullptr) ? -1 : DecodeValue(_cache->value(handle)); - if (handle != NULL) { + if (handle != nullptr) { _cache->release(handle); } diff --git a/be/test/olap/memory/column_test.cpp b/be/test/olap/memory/column_test.cpp index 0033fecc37..ab2d98d021 100644 --- a/be/test/olap/memory/column_test.cpp +++ b/be/test/olap/memory/column_test.cpp @@ -70,7 +70,7 @@ struct ColumnTest { values[i] = (CppType)rand(); if (is_null(values[i])) { // set to null - EXPECT_TRUE(writer->insert((uint32_t)i, NULL).ok()); + EXPECT_TRUE(writer->insert((uint32_t)i, nullptr).ok()); } else { EXPECT_TRUE(writer->insert((uint32_t)i, &values[i]).ok()); } @@ -150,7 +150,7 @@ struct ColumnTest { values[i] = (CppType)rand(); if (is_null(values[i])) { // set to null - EXPECT_TRUE(writer->insert((uint32_t)i, NULL).ok()); + EXPECT_TRUE(writer->insert((uint32_t)i, nullptr).ok()); } else { EXPECT_TRUE(writer->insert((uint32_t)i, &values[i]).ok()); } @@ -167,7 +167,7 @@ struct ColumnTest { //CppType oldv = values[idx]; values[idx] = (CppType)rand(); if (is_null(values[idx])) { - EXPECT_TRUE(writer->update(idx, NULL).ok()); + EXPECT_TRUE(writer->update(idx, nullptr).ok()); } else { EXPECT_TRUE(writer->update(idx, &values[idx]).ok()); } diff --git a/be/test/olap/run_length_byte_test.cpp b/be/test/olap/run_length_byte_test.cpp index f27d19ef32..af12a441d5 100644 --- a/be/test/olap/run_length_byte_test.cpp +++ b/be/test/olap/run_length_byte_test.cpp @@ -35,9 +35,9 @@ using namespace testing; TEST(TestStream, UncompressOutStream) { // write data OutStream* out_stream = - new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, NULL); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor == NULL); + new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, nullptr); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor == nullptr); out_stream->write(0x5a); out_stream->flush(); @@ -63,9 +63,9 @@ TEST(TestStream, UncompressOutStream) { TEST(TestStream, UncompressOutStream2) { // write data OutStream* out_stream = - new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, NULL); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor == NULL); + new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, nullptr); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor == nullptr); for (int32_t i = 0; i < OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE; i++) { out_stream->write(0x5a); @@ -87,7 +87,7 @@ TEST(TestStream, UncompressOutStream2) { offsets.push_back(0); offsets.push_back(sizeof(StreamHead) + OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE); InStream* in_stream = - new (std::nothrow) InStream(&inputs, offsets, out_stream->get_stream_length(), NULL, + new (std::nothrow) InStream(&inputs, offsets, out_stream->get_stream_length(), nullptr, out_stream->get_total_buffer_size()); char data; @@ -112,9 +112,9 @@ TEST(TestStream, UncompressOutStream2) { TEST(TestStream, UncompressOutStream3) { // write data OutStream* out_stream = - new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, NULL); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor == NULL); + new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, nullptr); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor == nullptr); for (int32_t i = 0; i < OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE; i++) { out_stream->write(0x5a); @@ -137,7 +137,7 @@ TEST(TestStream, UncompressOutStream3) { offsets.push_back(0); offsets.push_back(sizeof(StreamHead) + OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE); InStream* in_stream = - new (std::nothrow) InStream(&inputs, offsets, out_stream->get_stream_length(), NULL, + new (std::nothrow) InStream(&inputs, offsets, out_stream->get_stream_length(), nullptr, out_stream->get_total_buffer_size()); char data; @@ -163,9 +163,9 @@ TEST(TestStream, UncompressOutStream3) { TEST(TestStream, UncompressInStream) { // write data OutStream* out_stream = - new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, NULL); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor == NULL); + new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, nullptr); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor == nullptr); out_stream->write(0x5a); out_stream->flush(); @@ -179,7 +179,7 @@ TEST(TestStream, UncompressInStream) { std::vector offsets; offsets.assign(inputs.size(), 0); InStream* in_stream = - new (std::nothrow) InStream(&inputs, offsets, out_stream->get_stream_length(), NULL, + new (std::nothrow) InStream(&inputs, offsets, out_stream->get_stream_length(), nullptr, out_stream->get_total_buffer_size()); SAFE_DELETE(out_stream); @@ -199,8 +199,8 @@ TEST(TestStream, CompressOutStream) { // write data OutStream* out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, lz4_compress); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor != NULL); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor != nullptr); char* write_data = new char[OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE]; memset(write_data, 0x5a, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE); @@ -228,8 +228,8 @@ TEST(TestStream, CompressOutStream2) { // write data OutStream* out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, lz4_compress); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor != NULL); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor != nullptr); for (int32_t i = 0; i < OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE; i++) { out_stream->write(0x5a); @@ -270,8 +270,8 @@ TEST(TestStream, CompressOutStream3) { // write data OutStream* out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, lz4_compress); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor != NULL); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor != nullptr); for (int32_t i = 0; i < OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE; i++) { out_stream->write(0x5a); @@ -318,8 +318,8 @@ TEST(TestStream, CompressOutStream3) { TEST(TestStream, CompressOutStream4) { // write data OutStream* out_stream = new (std::nothrow) OutStream(18, lz4_compress); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor != NULL); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor != nullptr); for (int32_t i = 0; i < 15; i++) { out_stream->write(0x5a); @@ -374,8 +374,8 @@ TEST(TestStream, CompressOutStream4) { TEST(TestStream, CompressMassOutStream) { // write data OutStream* out_stream = new (std::nothrow) OutStream(100, lz4_compress); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor != NULL); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor != nullptr); for (int32_t i = 0; i < 100; i++) { out_stream->write(0x5a); @@ -422,8 +422,8 @@ TEST(TestStream, CompressInStream) { // write data OutStream* out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, lz4_compress); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor != NULL); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor != nullptr); char* write_data = new char[OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE]; memset(write_data, 0x5a, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE); @@ -465,9 +465,9 @@ TEST(TestStream, CompressInStream) { TEST(TestStream, SeekUncompress) { // write data OutStream* out_stream = - new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, NULL); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor == NULL); + new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, nullptr); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor == nullptr); out_stream->write(0x5a); @@ -490,7 +490,7 @@ TEST(TestStream, SeekUncompress) { std::vector offsets; offsets.assign(inputs.size(), 0); InStream* in_stream = - new (std::nothrow) InStream(&inputs, offsets, out_stream->get_stream_length(), NULL, + new (std::nothrow) InStream(&inputs, offsets, out_stream->get_stream_length(), nullptr, out_stream->get_total_buffer_size()); ASSERT_EQ(in_stream->available(), 2); @@ -522,9 +522,9 @@ TEST(TestStream, SeekUncompress) { TEST(TestStream, SkipUncompress) { // write data OutStream* out_stream = - new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, NULL); - ASSERT_TRUE(out_stream != NULL); - ASSERT_TRUE(out_stream->_compressor == NULL); + new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, nullptr); + ASSERT_TRUE(out_stream != nullptr); + ASSERT_TRUE(out_stream->_compressor == nullptr); char write_data[] = {0x5a, 0x5b, 0x5c, 0x5d}; for (int32_t i = 0; i < sizeof(write_data); i++) { @@ -544,7 +544,7 @@ TEST(TestStream, SkipUncompress) { std::vector offsets; offsets.assign(inputs.size(), 0); InStream* in_stream = - new (std::nothrow) InStream(&inputs, offsets, out_stream->get_stream_length(), NULL, + new (std::nothrow) InStream(&inputs, offsets, out_stream->get_stream_length(), nullptr, out_stream->get_total_buffer_size()); ASSERT_EQ(in_stream->available(), sizeof(write_data) + 1); in_stream->skip(sizeof(write_data) - 1); @@ -562,7 +562,7 @@ TEST(TestStream, SeekCompress) { // write data OutStream* out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, lz4_compress); - ASSERT_TRUE(out_stream != NULL); + ASSERT_TRUE(out_stream != nullptr); for (int32_t i = 0; i < 10; i++) { out_stream->write(0x5a); @@ -615,7 +615,7 @@ TEST(TestStream, SkipCompress) { // write data OutStream* out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, lz4_compress); - ASSERT_TRUE(out_stream != NULL); + ASSERT_TRUE(out_stream != nullptr); for (int32_t i = 0; i < 10; i++) { out_stream->write(0x5a); @@ -658,10 +658,10 @@ public: virtual void SetUp() { system("mkdir -p ./ut_dir"); system("rm -rf ./ut_dir/tmp_file"); - _out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, NULL); - ASSERT_TRUE(_out_stream != NULL); + _out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, nullptr); + ASSERT_TRUE(_out_stream != nullptr); _writer = new (std::nothrow) RunLengthByteWriter(_out_stream); - ASSERT_TRUE(_writer != NULL); + ASSERT_TRUE(_writer != nullptr); } virtual void TearDown() { @@ -684,15 +684,15 @@ public: _shared_buffer = StorageByteBuffer::create(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE + sizeof(StreamHead)); - ASSERT_TRUE(_shared_buffer != NULL); + ASSERT_TRUE(_shared_buffer != nullptr); _stream = new (std::nothrow) - ReadOnlyFileStream(&helper, &_shared_buffer, 0, helper.length(), NULL, + ReadOnlyFileStream(&helper, &_shared_buffer, 0, helper.length(), nullptr, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, &_stats); ASSERT_EQ(OLAP_SUCCESS, _stream->init()); _reader = new (std::nothrow) RunLengthByteReader(_stream); - ASSERT_TRUE(_reader != NULL); + ASSERT_TRUE(_reader != nullptr); } RunLengthByteReader* _reader; diff --git a/be/test/olap/run_length_integer_test.cpp b/be/test/olap/run_length_integer_test.cpp index 098001e201..0c8f7123db 100644 --- a/be/test/olap/run_length_integer_test.cpp +++ b/be/test/olap/run_length_integer_test.cpp @@ -37,10 +37,10 @@ public: virtual void SetUp() { system("mkdir -p ./ut_dir"); system("rm -rf ./ut_dir/tmp_file"); - _out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, NULL); - ASSERT_TRUE(_out_stream != NULL); + _out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, nullptr); + ASSERT_TRUE(_out_stream != nullptr); _writer = new (std::nothrow) RunLengthIntegerWriter(_out_stream, false); - ASSERT_TRUE(_writer != NULL); + ASSERT_TRUE(_writer != nullptr); } virtual void TearDown() { @@ -63,15 +63,15 @@ public: _shared_buffer = StorageByteBuffer::create(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE + sizeof(StreamHead)); - ASSERT_TRUE(_shared_buffer != NULL); + ASSERT_TRUE(_shared_buffer != nullptr); _stream = new (std::nothrow) - ReadOnlyFileStream(&helper, &_shared_buffer, 0, helper.length(), NULL, + ReadOnlyFileStream(&helper, &_shared_buffer, 0, helper.length(), nullptr, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, &_stats); ASSERT_EQ(OLAP_SUCCESS, _stream->init()); _reader = new (std::nothrow) RunLengthIntegerReader(_stream, false); - ASSERT_TRUE(_reader != NULL); + ASSERT_TRUE(_reader != nullptr); } RunLengthIntegerReader* _reader; @@ -342,10 +342,10 @@ public: virtual void SetUp() { system("mkdir -p ./ut_dir"); system("rm ./ut_dir/tmp_file"); - _out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, NULL); - ASSERT_TRUE(_out_stream != NULL); + _out_stream = new (std::nothrow) OutStream(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, nullptr); + ASSERT_TRUE(_out_stream != nullptr); _writer = new (std::nothrow) RunLengthIntegerWriter(_out_stream, true); - ASSERT_TRUE(_writer != NULL); + ASSERT_TRUE(_writer != nullptr); } virtual void TearDown() { @@ -368,15 +368,15 @@ public: _shared_buffer = StorageByteBuffer::create(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE + sizeof(StreamHead)); - ASSERT_TRUE(_shared_buffer != NULL); + ASSERT_TRUE(_shared_buffer != nullptr); _stream = new (std::nothrow) - ReadOnlyFileStream(&helper, &_shared_buffer, 0, helper.length(), NULL, + ReadOnlyFileStream(&helper, &_shared_buffer, 0, helper.length(), nullptr, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE, &_stats); ASSERT_EQ(OLAP_SUCCESS, _stream->init()); _reader = new (std::nothrow) RunLengthIntegerReader(_stream, true); - ASSERT_TRUE(_reader != NULL); + ASSERT_TRUE(_reader != nullptr); } RunLengthIntegerReader* _reader; diff --git a/be/test/olap/schema_change_test.cpp b/be/test/olap/schema_change_test.cpp index 3129b5bc3e..ee3df975da 100644 --- a/be/test/olap/schema_change_test.cpp +++ b/be/test/olap/schema_change_test.cpp @@ -37,7 +37,7 @@ namespace doris { class TestColumn : public testing::Test { public: - TestColumn() : _column_writer(NULL), _column_reader(NULL), _stream_factory(NULL) { + TestColumn() : _column_writer(nullptr), _column_reader(nullptr), _stream_factory(nullptr) { _offsets.clear(); _map_in_streams.clear(); _present_buffers.clear(); @@ -59,7 +59,7 @@ public: _offsets.push_back(0); _stream_factory = new (std::nothrow) OutStreamFactory(COMPRESS_LZ4, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE); - ASSERT_TRUE(_stream_factory != NULL); + ASSERT_TRUE(_stream_factory != nullptr); config::column_dictionary_key_ratio_threshold = 30; config::column_dictionary_key_size_threshold = 1000; } @@ -85,7 +85,7 @@ public: void create_columnWriter(const TabletSchema& tablet_schema) { _column_writer = ColumnWriter::create(0, tablet_schema, _stream_factory, 1024, BLOOM_FILTER_DEFAULT_FPP); - ASSERT_TRUE(_column_writer != NULL); + ASSERT_TRUE(_column_writer != nullptr); ASSERT_EQ(_column_writer->init(), OLAP_SUCCESS); } @@ -107,7 +107,7 @@ public: _column_reader = ColumnReader::create(0, tablet_schema, included, segment_included, encodings); - ASSERT_TRUE(_column_reader != NULL); + ASSERT_TRUE(_column_reader != nullptr); system("mkdir -p ./ut_dir"); system("rm ./ut_dir/tmp_file"); @@ -145,7 +145,7 @@ public: ASSERT_TRUE(false); } - ASSERT_TRUE(buffers != NULL); + ASSERT_TRUE(buffers != nullptr); off.push_back(helper.tell()); out_stream->write_to_file(&helper, 0); length.push_back(out_stream->get_stream_length()); @@ -160,7 +160,7 @@ public: SAFE_DELETE(_shared_buffer); _shared_buffer = StorageByteBuffer::create(OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE + sizeof(StreamHead)); - ASSERT_TRUE(_shared_buffer != NULL); + ASSERT_TRUE(_shared_buffer != nullptr); for (auto in_stream : _map_in_streams) { delete in_stream.second; diff --git a/be/test/olap/skiplist_test.cpp b/be/test/olap/skiplist_test.cpp index 0077a3e40b..eb5831a006 100644 --- a/be/test/olap/skiplist_test.cpp +++ b/be/test/olap/skiplist_test.cpp @@ -25,12 +25,12 @@ #include "olap/schema.h" #include "runtime/mem_pool.h" #include "runtime/mem_tracker.h" +#include "test_util/test_util.h" #include "util/condition_variable.h" #include "util/hash_util.hpp" #include "util/mutex.h" #include "util/priority_thread_pool.hpp" #include "util/random.h" -#include "test_util/test_util.h" namespace doris { @@ -358,7 +358,7 @@ public: enum ReaderState { STARTING, RUNNING, DONE }; - explicit TestState(int s) : _seed(s), _quit_flag(NULL), _state(STARTING), _cv_state(&_mu) {} + explicit TestState(int s) : _seed(s), _quit_flag(false), _state(STARTING), _cv_state(&_mu) {} void wait(ReaderState s) { _mu.lock(); @@ -409,7 +409,7 @@ static void run_concurrent(int run) { for (int i = 0; i < kSize; i++) { state._t.write_step(&rnd); } - state._quit_flag.store(true, std::memory_order_release); // Any non-NULL arg will do + state._quit_flag.store(true, std::memory_order_release); // Any non-nullptr arg will do state.wait(TestState::DONE); } } diff --git a/be/test/olap/stream_index_test.cpp b/be/test/olap/stream_index_test.cpp index bf5fdab652..5c55226383 100644 --- a/be/test/olap/stream_index_test.cpp +++ b/be/test/olap/stream_index_test.cpp @@ -335,7 +335,7 @@ TEST_F(TestStreamIndex, statistic) { ASSERT_EQ(OLAP_SUCCESS, stat.init(OLAP_FIELD_TYPE_INT, true)); WrapperField* field = WrapperField::create_by_type(OLAP_FIELD_TYPE_INT); - ASSERT_TRUE(NULL != field); + ASSERT_TRUE(nullptr != field); char string_buffer[256]; static const uint32_t loop = 10; diff --git a/be/test/runtime/buffer_control_block_test.cpp b/be/test/runtime/buffer_control_block_test.cpp index 2da1f837eb..b997daddc1 100644 --- a/be/test/runtime/buffer_control_block_test.cpp +++ b/be/test/runtime/buffer_control_block_test.cpp @@ -83,7 +83,7 @@ void* cancel_thread(void* param) { BufferControlBlock* control_block = static_cast(param); sleep(1); control_block->cancel(); - return NULL; + return nullptr; } TEST_F(BufferControlBlockTest, add_then_cancel) { @@ -92,7 +92,7 @@ TEST_F(BufferControlBlockTest, add_then_cancel) { ASSERT_TRUE(control_block.init().ok()); pthread_t id; - pthread_create(&id, NULL, cancel_thread, &control_block); + pthread_create(&id, nullptr, cancel_thread, &control_block); { TFetchDataResult* add_result = new TFetchDataResult(); @@ -111,7 +111,7 @@ TEST_F(BufferControlBlockTest, add_then_cancel) { TFetchDataResult get_result; ASSERT_FALSE(control_block.get_batch(&get_result).ok()); - pthread_join(id, NULL); + pthread_join(id, nullptr); } TEST_F(BufferControlBlockTest, get_then_cancel) { @@ -119,13 +119,13 @@ TEST_F(BufferControlBlockTest, get_then_cancel) { ASSERT_TRUE(control_block.init().ok()); pthread_t id; - pthread_create(&id, NULL, cancel_thread, &control_block); + pthread_create(&id, nullptr, cancel_thread, &control_block); // get block until cancel TFetchDataResult get_result; ASSERT_FALSE(control_block.get_batch(&get_result).ok()); - pthread_join(id, NULL); + pthread_join(id, nullptr); } void* add_thread(void* param) { @@ -137,7 +137,7 @@ void* add_thread(void* param) { add_result->result_batch.rows.push_back("hello test2"); control_block->add_batch(add_result); } - return NULL; + return nullptr; } TEST_F(BufferControlBlockTest, get_then_add) { @@ -145,7 +145,7 @@ TEST_F(BufferControlBlockTest, get_then_add) { ASSERT_TRUE(control_block.init().ok()); pthread_t id; - pthread_create(&id, NULL, add_thread, &control_block); + pthread_create(&id, nullptr, add_thread, &control_block); // get block until a batch add TFetchDataResult get_result; @@ -155,14 +155,14 @@ TEST_F(BufferControlBlockTest, get_then_add) { ASSERT_STREQ("hello test1", get_result.result_batch.rows[0].c_str()); ASSERT_STREQ("hello test2", get_result.result_batch.rows[1].c_str()); - pthread_join(id, NULL); + pthread_join(id, nullptr); } void* close_thread(void* param) { BufferControlBlock* control_block = static_cast(param); sleep(1); control_block->close(Status::OK()); - return NULL; + return nullptr; } TEST_F(BufferControlBlockTest, get_then_close) { @@ -170,7 +170,7 @@ TEST_F(BufferControlBlockTest, get_then_close) { ASSERT_TRUE(control_block.init().ok()); pthread_t id; - pthread_create(&id, NULL, close_thread, &control_block); + pthread_create(&id, nullptr, close_thread, &control_block); // get block until a batch add TFetchDataResult get_result; @@ -178,7 +178,7 @@ TEST_F(BufferControlBlockTest, get_then_close) { ASSERT_TRUE(get_result.eos); ASSERT_EQ(0U, get_result.result_batch.rows.size()); - pthread_join(id, NULL); + pthread_join(id, nullptr); } } // namespace doris diff --git a/be/test/runtime/buffered_block_mgr2_test.cpp b/be/test/runtime/buffered_block_mgr2_test.cpp index bf2cfd4452..95d60f7e6f 100644 --- a/be/test/runtime/buffered_block_mgr2_test.cpp +++ b/be/test/runtime/buffered_block_mgr2_test.cpp @@ -20,11 +20,9 @@ #include #include -#include -#include -#include -#include #include +#include +#include #include "runtime/disk_io_mgr.h" #include "runtime/exec_env.h" @@ -36,12 +34,13 @@ #include "util/filesystem_util.h" #include "util/logging.h" #include "util/monotime.h" +#include "util/thread_group.h" using std::filesystem::directory_iterator; using std::filesystem::remove; -using boost::scoped_ptr; +using std::unique_ptr; using std::unordered_map; -using boost::thread; +using std::thread; using std::string; using std::stringstream; @@ -138,10 +137,10 @@ protected: /// Helper to create a simple block manager. BufferedBlockMgr2* CreateMgr(int64_t query_id, int max_buffers, int block_size, - RuntimeState** query_state = NULL) { - RuntimeState* state = NULL; + RuntimeState** query_state = nullptr) { + RuntimeState* state = nullptr; EXPECT_TRUE(_test_env->create_query_state(query_id, max_buffers, block_size, &state).ok()); - if (query_state != NULL) { + if (query_state != nullptr) { *query_state = state; } return state->block_mgr2(); @@ -151,10 +150,10 @@ protected: int reserved_blocks, const std::shared_ptr& tracker, BufferedBlockMgr2::Client** client) { - RuntimeState* state = NULL; + RuntimeState* state = nullptr; BufferedBlockMgr2* mgr = CreateMgr(query_id, max_buffers, block_size, &state); EXPECT_TRUE(mgr->register_client(reserved_blocks, tracker, state, client).ok()); - EXPECT_TRUE(client != NULL); + EXPECT_TRUE(client != nullptr); return mgr; } @@ -182,13 +181,13 @@ protected: void AllocateBlocks(BufferedBlockMgr2* block_mgr, BufferedBlockMgr2::Client* client, int num_blocks, std::vector* blocks) { - int32_t* data = NULL; + int32_t* data = nullptr; Status status; BufferedBlockMgr2::Block* new_block; for (int i = 0; i < num_blocks; ++i) { - status = block_mgr->get_new_block(client, NULL, &new_block); + status = block_mgr->get_new_block(client, nullptr, &new_block); ASSERT_TRUE(status.ok()); - ASSERT_TRUE(new_block != NULL); + ASSERT_TRUE(new_block != nullptr); data = new_block->allocate(sizeof(int32_t)); *data = blocks->size(); blocks->push_back(new_block); @@ -263,13 +262,13 @@ protected: return blocks[i]; } } - return NULL; + return nullptr; } void TestGetNewBlockImpl(int block_size) { Status status; int max_num_blocks = 5; - BufferedBlockMgr2* block_mgr = NULL; + BufferedBlockMgr2* block_mgr = nullptr; BufferedBlockMgr2::Client* client; block_mgr = CreateMgrAndClient(0, max_num_blocks, block_size, 0, _client_tracker, &client); EXPECT_EQ(_test_env->block_mgr_parent_tracker()->consumption(), 0); @@ -277,32 +276,32 @@ protected: // Allocate blocks until max_num_blocks, they should all succeed and memory // usage should go up. BufferedBlockMgr2::Block* new_block; - BufferedBlockMgr2::Block* first_block = NULL; + BufferedBlockMgr2::Block* first_block = nullptr; for (int i = 0; i < max_num_blocks; ++i) { - status = block_mgr->get_new_block(client, NULL, &new_block); - EXPECT_TRUE(new_block != NULL); + status = block_mgr->get_new_block(client, nullptr, &new_block); + EXPECT_TRUE(new_block != nullptr); EXPECT_EQ(block_mgr->bytes_allocated(), (i + 1) * block_size); - if (first_block == NULL) { + if (first_block == nullptr) { first_block = new_block; } } // Trying to allocate a new one should fail. - status = block_mgr->get_new_block(client, NULL, &new_block); - EXPECT_TRUE(new_block == NULL); + status = block_mgr->get_new_block(client, nullptr, &new_block); + EXPECT_TRUE(new_block == nullptr); EXPECT_EQ(block_mgr->bytes_allocated(), max_num_blocks * block_size); // We can allocate a new block by transferring an already allocated one. uint8_t* old_buffer = first_block->buffer(); status = block_mgr->get_new_block(client, first_block, &new_block); - EXPECT_TRUE(new_block != NULL); + EXPECT_TRUE(new_block != nullptr); EXPECT_TRUE(old_buffer == new_block->buffer()); EXPECT_EQ(block_mgr->bytes_allocated(), max_num_blocks * block_size); EXPECT_TRUE(!first_block->is_pinned()); // Trying to allocate a new one should still fail. - status = block_mgr->get_new_block(client, NULL, &new_block); - EXPECT_TRUE(new_block == NULL); + status = block_mgr->get_new_block(client, nullptr, &new_block); + EXPECT_TRUE(new_block == nullptr); EXPECT_EQ(block_mgr->bytes_allocated(), max_num_blocks * block_size); EXPECT_EQ(block_mgr->writes_issued(), 1); @@ -313,8 +312,8 @@ protected: Status status; DCHECK_GT(block_size, 0); int max_num_buffers = 5; - BufferedBlockMgr2* block_mgr = NULL; - BufferedBlockMgr2::Client* client = NULL; + BufferedBlockMgr2* block_mgr = nullptr; + BufferedBlockMgr2::Client* client = nullptr; block_mgr = CreateMgrAndClient(0, max_num_buffers, block_size, 0, _client_tracker, &client); // Check counters. @@ -384,7 +383,7 @@ protected: static const int SINGLE_THREADED_TID = -1; void TestRandomInternalImpl(RuntimeState* state, BufferedBlockMgr2* block_mgr, int num_buffers, int tid) { - DCHECK(block_mgr != NULL); + DCHECK(block_mgr != nullptr); const int num_iterations = 100000; const int iters_before_close = num_iterations - 5000; bool close_called = false; @@ -399,7 +398,7 @@ protected: BufferedBlockMgr2::Client* client; Status status = block_mgr->register_client(0, _client_tracker, state, &client); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(client != NULL); + EXPECT_TRUE(client != nullptr); pinned_blocks.reserve(num_buffers); BufferedBlockMgr2::Block* new_block; @@ -427,18 +426,18 @@ protected: std::pair block_data; int rand_pick = 0; - int32_t* data = NULL; + int32_t* data = nullptr; bool pinned = false; switch (api_function) { case New: - status = block_mgr->get_new_block(client, NULL, &new_block); + status = block_mgr->get_new_block(client, nullptr, &new_block); if (close_called || (tid != SINGLE_THREADED_TID && status.is_cancelled())) { - EXPECT_TRUE(new_block == NULL); + EXPECT_TRUE(new_block == nullptr); EXPECT_TRUE(status.is_cancelled()); continue; } EXPECT_TRUE(status.ok()); - EXPECT_TRUE(new_block != NULL); + EXPECT_TRUE(new_block != nullptr); data = MakeRandomSizeData(new_block); block_data = std::make_pair(new_block, *data); @@ -509,9 +508,9 @@ protected: // Single-threaded execution of the TestRandomInternalImpl. void TestRandomInternalSingle(int block_size) { DCHECK_GT(block_size, 0); - DCHECK(_test_env.get() != NULL); + DCHECK(_test_env.get() != nullptr); const int max_num_buffers = 100; - RuntimeState* state = NULL; + RuntimeState* state = nullptr; BufferedBlockMgr2* block_mgr = CreateMgr(0, max_num_buffers, block_size, &state); TestRandomInternalImpl(state, block_mgr, max_num_buffers, SINGLE_THREADED_TID); TearDownMgrs(); @@ -521,16 +520,16 @@ protected: void TestRandomInternalMulti(int num_threads, int block_size) { DCHECK_GT(num_threads, 0); DCHECK_GT(block_size, 0); - DCHECK(_test_env.get() != NULL); + DCHECK(_test_env.get() != nullptr); const int max_num_buffers = 100; - RuntimeState* state = NULL; + RuntimeState* state = nullptr; BufferedBlockMgr2* block_mgr = CreateMgr(0, num_threads * max_num_buffers, block_size, &state); - boost::thread_group workers; + ThreadGroup workers; for (int i = 0; i < num_threads; ++i) { - thread* t = new boost::thread(std::bind(&BufferedBlockMgrTest::TestRandomInternalImpl, - this, state, block_mgr, max_num_buffers, i)); + thread* t = new thread(std::bind(&BufferedBlockMgrTest::TestRandomInternalImpl, this, + state, block_mgr, max_num_buffers, i)); workers.add_thread(t); } workers.join_all(); @@ -543,7 +542,7 @@ protected: const int iters = 100; for (int i = 0; i < iters; ++i) { LOG(WARNING) << "CreateDestroyThread thread " << index << " begin " << i << std::endl; - boost::shared_ptr mgr; + std::shared_ptr mgr; Status status = BufferedBlockMgr2::create( state, _test_env->block_mgr_parent_tracker(), state->runtime_profile(), _test_env->tmp_file_mgr(), _block_size * num_buffers, _block_size, &mgr); @@ -555,19 +554,19 @@ protected: // BufferedBlockMgr2::~BufferedBlockMgr2(). void CreateDestroyMulti() { const int num_threads = 4; - boost::thread_group workers; + ThreadGroup workers; // Create a shared RuntimeState with no BufferedBlockMgr2. RuntimeState* shared_state = new RuntimeState(TUniqueId(), TQueryOptions(), TQueryGlobals(), _test_env->exec_env()); for (int i = 0; i < num_threads; ++i) { - thread* t = new boost::thread( + thread* t = new thread( std::bind(&BufferedBlockMgrTest::CreateDestroyThread, this, i, shared_state)); workers.add_thread(t); } workers.join_all(); } - boost::scoped_ptr _test_env; + std::unique_ptr _test_env; std::shared_ptr _client_tracker; std::vector _created_tmp_dirs; }; @@ -590,37 +589,37 @@ TEST_F(BufferedBlockMgrTest, GetNewBlockSmallBlocks) { std::vector blocks; // Allocate a small block. - BufferedBlockMgr2::Block* new_block = NULL; - EXPECT_TRUE(block_mgr->get_new_block(client, NULL, &new_block, 128).ok()); - EXPECT_TRUE(new_block != NULL); + BufferedBlockMgr2::Block* new_block = nullptr; + EXPECT_TRUE(block_mgr->get_new_block(client, nullptr, &new_block, 128).ok()); + EXPECT_TRUE(new_block != nullptr); EXPECT_EQ(block_mgr->bytes_allocated(), 0); EXPECT_EQ(_test_env->block_mgr_parent_tracker()->consumption(), 0); EXPECT_EQ(_client_tracker->consumption(), 128); EXPECT_TRUE(new_block->is_pinned()); EXPECT_EQ(new_block->bytes_remaining(), 128); - EXPECT_TRUE(new_block->buffer() != NULL); + EXPECT_TRUE(new_block->buffer() != nullptr); blocks.push_back(new_block); // Allocate a normal block - EXPECT_TRUE(block_mgr->get_new_block(client, NULL, &new_block).ok()); - EXPECT_TRUE(new_block != NULL); + EXPECT_TRUE(block_mgr->get_new_block(client, nullptr, &new_block).ok()); + EXPECT_TRUE(new_block != nullptr); EXPECT_EQ(block_mgr->bytes_allocated(), block_mgr->max_block_size()); EXPECT_EQ(_test_env->block_mgr_parent_tracker()->consumption(), block_mgr->max_block_size()); EXPECT_EQ(_client_tracker->consumption(), 128 + block_mgr->max_block_size()); EXPECT_TRUE(new_block->is_pinned()); EXPECT_EQ(new_block->bytes_remaining(), block_mgr->max_block_size()); - EXPECT_TRUE(new_block->buffer() != NULL); + EXPECT_TRUE(new_block->buffer() != nullptr); blocks.push_back(new_block); // Allocate another small block. - EXPECT_TRUE(block_mgr->get_new_block(client, NULL, &new_block, 512).ok()); - EXPECT_TRUE(new_block != NULL); + EXPECT_TRUE(block_mgr->get_new_block(client, nullptr, &new_block, 512).ok()); + EXPECT_TRUE(new_block != nullptr); EXPECT_EQ(block_mgr->bytes_allocated(), block_mgr->max_block_size()); EXPECT_EQ(_test_env->block_mgr_parent_tracker()->consumption(), block_mgr->max_block_size()); EXPECT_EQ(_client_tracker->consumption(), 128 + 512 + block_mgr->max_block_size()); EXPECT_TRUE(new_block->is_pinned()); EXPECT_EQ(new_block->bytes_remaining(), 512); - EXPECT_TRUE(new_block->buffer() != NULL); + EXPECT_TRUE(new_block->buffer() != nullptr); blocks.push_back(new_block); // Should be able to unpin and pin the middle block @@ -729,8 +728,8 @@ TEST_F(BufferedBlockMgrTest, DeleteSingleBlocks) { // Pinned I/O block. BufferedBlockMgr2::Block* new_block; - EXPECT_TRUE(block_mgr->get_new_block(client, NULL, &new_block).ok()); - EXPECT_TRUE(new_block != NULL); + EXPECT_TRUE(block_mgr->get_new_block(client, nullptr, &new_block).ok()); + EXPECT_TRUE(new_block != nullptr); EXPECT_TRUE(new_block->is_pinned()); EXPECT_TRUE(new_block->is_max_size()); new_block->del(); @@ -738,16 +737,16 @@ TEST_F(BufferedBlockMgrTest, DeleteSingleBlocks) { // Pinned non-I/O block. int small_block_size = 128; - EXPECT_TRUE(block_mgr->get_new_block(client, NULL, &new_block, small_block_size).ok()); - EXPECT_TRUE(new_block != NULL); + EXPECT_TRUE(block_mgr->get_new_block(client, nullptr, &new_block, small_block_size).ok()); + EXPECT_TRUE(new_block != nullptr); EXPECT_TRUE(new_block->is_pinned()); EXPECT_EQ(small_block_size, _client_tracker->consumption()); new_block->del(); EXPECT_EQ(0, _client_tracker->consumption()); // Unpinned I/O block - delete after written to disk. - EXPECT_TRUE(block_mgr->get_new_block(client, NULL, &new_block).ok()); - EXPECT_TRUE(new_block != NULL); + EXPECT_TRUE(block_mgr->get_new_block(client, nullptr, &new_block).ok()); + EXPECT_TRUE(new_block != nullptr); EXPECT_TRUE(new_block->is_pinned()); EXPECT_TRUE(new_block->is_max_size()); new_block->unpin(); @@ -757,8 +756,8 @@ TEST_F(BufferedBlockMgrTest, DeleteSingleBlocks) { EXPECT_TRUE(_client_tracker->consumption() == 0); // Unpinned I/O block - delete before written to disk. - EXPECT_TRUE(block_mgr->get_new_block(client, NULL, &new_block).ok()); - EXPECT_TRUE(new_block != NULL); + EXPECT_TRUE(block_mgr->get_new_block(client, nullptr, &new_block).ok()); + EXPECT_TRUE(new_block != nullptr); EXPECT_TRUE(new_block->is_pinned()); EXPECT_TRUE(new_block->is_max_size()); new_block->unpin(); @@ -784,9 +783,9 @@ TEST_F(BufferedBlockMgrTest, Close) { block_mgr->cancel(); BufferedBlockMgr2::Block* new_block; - Status status = block_mgr->get_new_block(client, NULL, &new_block); + Status status = block_mgr->get_new_block(client, nullptr, &new_block); EXPECT_TRUE(status.is_cancelled()); - EXPECT_TRUE(new_block == NULL); + EXPECT_TRUE(new_block == nullptr); status = blocks[0]->unpin(); EXPECT_TRUE(status.is_cancelled()); bool pinned; @@ -847,9 +846,9 @@ TEST_F(BufferedBlockMgrTest, WriteError) { blocks[i]->del(); } BufferedBlockMgr2::Block* new_block; - status = block_mgr->get_new_block(client, NULL, &new_block); + status = block_mgr->get_new_block(client, nullptr, &new_block); EXPECT_TRUE(status.is_cancelled()); - EXPECT_TRUE(new_block == NULL); + EXPECT_TRUE(new_block == nullptr); TearDownMgrs(); } @@ -919,7 +918,7 @@ TEST_F(BufferedBlockMgrTest, DISABLED_WriteErrorBlacklist) { const string& good_dir = tmp_dirs[1]; // Delete one file from first scratch dir for first block manager. BufferedBlockMgr2::Block* error_block = FindBlockForDir(blocks[error_mgr], error_dir); - ASSERT_TRUE(error_block != NULL) << "Expected a tmp file in dir " << error_dir; + ASSERT_TRUE(error_block != nullptr) << "Expected a tmp file in dir " << error_dir; PinBlocks(all_blocks); DeleteBackingFile(error_block); UnpinBlocks(all_blocks); // Should succeed since tmp file space was already allocated. @@ -940,8 +939,8 @@ TEST_F(BufferedBlockMgrTest, DISABLED_WriteErrorBlacklist) { // the intended behaviour. PinBlocks(blocks[no_error_mgr]); UnpinBlocks(blocks[no_error_mgr]); - EXPECT_TRUE(FindBlockForDir(blocks[no_error_mgr], good_dir) != NULL); - EXPECT_TRUE(FindBlockForDir(blocks[no_error_mgr], error_dir) != NULL); + EXPECT_TRUE(FindBlockForDir(blocks[no_error_mgr], good_dir) != nullptr); + EXPECT_TRUE(FindBlockForDir(blocks[no_error_mgr], error_dir) != nullptr); // The second block manager should avoid using bad directory for new blocks. std::vector no_error_new_blocks; AllocateBlocks(block_mgrs[no_error_mgr], clients[no_error_mgr], blocks_per_mgr, @@ -1039,14 +1038,14 @@ TEST_F(BufferedBlockMgrTest, MultipleClients) { RuntimeState* runtime_state; BufferedBlockMgr2* block_mgr = CreateMgr(0, max_num_buffers, block_size, &runtime_state); - BufferedBlockMgr2::Client* client1 = NULL; - BufferedBlockMgr2::Client* client2 = NULL; + BufferedBlockMgr2::Client* client1 = nullptr; + BufferedBlockMgr2::Client* client2 = nullptr; status = block_mgr->register_client(client1_buffers, _client_tracker, runtime_state, &client1); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(client1 != NULL); + EXPECT_TRUE(client1 != nullptr); status = block_mgr->register_client(client2_buffers, _client_tracker, runtime_state, &client2); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(client2 != NULL); + EXPECT_TRUE(client2 != nullptr); // Reserve client 1's and 2's buffers. They should succeed. bool reserved = block_mgr->try_acquire_tmp_reservation(client1, 1); @@ -1060,9 +1059,9 @@ TEST_F(BufferedBlockMgrTest, MultipleClients) { // Try allocating one more, that should fail. BufferedBlockMgr2::Block* block; - status = block_mgr->get_new_block(client1, NULL, &block); + status = block_mgr->get_new_block(client1, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block == NULL); + EXPECT_TRUE(block == nullptr); // Trying to reserve should also fail. reserved = block_mgr->try_acquire_tmp_reservation(client1, 1); @@ -1073,27 +1072,27 @@ TEST_F(BufferedBlockMgrTest, MultipleClients) { AllocateBlocks(block_mgr, client2, client2_buffers, &client2_blocks); // Try allocating one more from client 2, that should fail. - status = block_mgr->get_new_block(client2, NULL, &block); + status = block_mgr->get_new_block(client2, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block == NULL); + EXPECT_TRUE(block == nullptr); // Unpin one block from client 1. status = client1_blocks[0]->unpin(); EXPECT_TRUE(status.ok()); // Client 2 should still not be able to allocate. - status = block_mgr->get_new_block(client2, NULL, &block); + status = block_mgr->get_new_block(client2, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block == NULL); + EXPECT_TRUE(block == nullptr); // Client 2 should still not be able to reserve. reserved = block_mgr->try_acquire_tmp_reservation(client2, 1); EXPECT_FALSE(reserved); // Client 1 should be able to though. - status = block_mgr->get_new_block(client1, NULL, &block); + status = block_mgr->get_new_block(client1, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block != NULL); + EXPECT_TRUE(block != nullptr); // Unpin two of client 1's blocks (client 1 should have 3 unpinned blocks now). status = client1_blocks[1]->unpin(); @@ -1119,14 +1118,14 @@ TEST_F(BufferedBlockMgrTest, MultipleClients) { EXPECT_FALSE(pinned); // Client 2 can pick up the one reserved buffer - status = block_mgr->get_new_block(client2, NULL, &block); + status = block_mgr->get_new_block(client2, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block != NULL); + EXPECT_TRUE(block != nullptr); // But not a second BufferedBlockMgr2::Block* block2; - status = block_mgr->get_new_block(client2, NULL, &block2); + status = block_mgr->get_new_block(client2, nullptr, &block2); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block2 == NULL); + EXPECT_TRUE(block2 == nullptr); // Unpin client 2's block it got from the reservation. Sine this is a tmp // reservation, client 1 can pick it up again (it is not longer reserved). @@ -1149,15 +1148,15 @@ TEST_F(BufferedBlockMgrTest, MultipleClientsExtraBuffers) { RuntimeState* runtime_state; BufferedBlockMgr2* block_mgr = CreateMgr(0, max_num_buffers, block_size, &runtime_state); - BufferedBlockMgr2::Client* client1 = NULL; - BufferedBlockMgr2::Client* client2 = NULL; - BufferedBlockMgr2::Block* block = NULL; + BufferedBlockMgr2::Client* client1 = nullptr; + BufferedBlockMgr2::Client* client2 = nullptr; + BufferedBlockMgr2::Block* block = nullptr; status = block_mgr->register_client(client1_buffers, _client_tracker, runtime_state, &client1); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(client1 != NULL); + EXPECT_TRUE(client1 != nullptr); status = block_mgr->register_client(client2_buffers, _client_tracker, runtime_state, &client2); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(client2 != NULL); + EXPECT_TRUE(client2 != nullptr); std::vector client1_blocks; // Allocate all of client1's reserved blocks, they should all succeed. @@ -1168,20 +1167,20 @@ TEST_F(BufferedBlockMgrTest, MultipleClientsExtraBuffers) { AllocateBlocks(block_mgr, client2, client2_buffers, &client2_blocks); // We have two spare buffers now. Each client should be able to allocate it. - status = block_mgr->get_new_block(client1, NULL, &block); + status = block_mgr->get_new_block(client1, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block != NULL); - status = block_mgr->get_new_block(client2, NULL, &block); + EXPECT_TRUE(block != nullptr); + status = block_mgr->get_new_block(client2, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block != NULL); + EXPECT_TRUE(block != nullptr); // Now we are completely full, no one should be able to allocate a new block. - status = block_mgr->get_new_block(client1, NULL, &block); + status = block_mgr->get_new_block(client1, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block == NULL); - status = block_mgr->get_new_block(client2, NULL, &block); + EXPECT_TRUE(block == nullptr); + status = block_mgr->get_new_block(client2, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block == NULL); + EXPECT_TRUE(block == nullptr); TearDownMgrs(); } @@ -1196,35 +1195,35 @@ TEST_F(BufferedBlockMgrTest, ClientOversubscription) { RuntimeState* runtime_state; BufferedBlockMgr2* block_mgr = CreateMgr(0, max_num_buffers, block_size, &runtime_state); - BufferedBlockMgr2::Client* client1 = NULL; - BufferedBlockMgr2::Client* client2 = NULL; - BufferedBlockMgr2::Block* block = NULL; + BufferedBlockMgr2::Client* client1 = nullptr; + BufferedBlockMgr2::Client* client2 = nullptr; + BufferedBlockMgr2::Block* block = nullptr; status = block_mgr->register_client(client1_buffers, _client_tracker, runtime_state, &client1); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(client1 != NULL); + EXPECT_TRUE(client1 != nullptr); status = block_mgr->register_client(client2_buffers, _client_tracker, runtime_state, &client2); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(client2 != NULL); + EXPECT_TRUE(client2 != nullptr); // Client one allocates first block, should work. - status = block_mgr->get_new_block(client1, NULL, &block); + status = block_mgr->get_new_block(client1, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block != NULL); + EXPECT_TRUE(block != nullptr); // Client two allocates first block, should work. - status = block_mgr->get_new_block(client2, NULL, &block); + status = block_mgr->get_new_block(client2, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block != NULL); + EXPECT_TRUE(block != nullptr); // At this point we've used both buffers. Client one reserved one so subsequent // calls should fail with no error (but returns no block). - status = block_mgr->get_new_block(client1, NULL, &block); + status = block_mgr->get_new_block(client1, nullptr, &block); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(block == NULL); + EXPECT_TRUE(block == nullptr); // Allocate with client two. Since client two reserved 2 buffers, this should fail // with MEM_LIMIT_EXCEEDED. - status = block_mgr->get_new_block(client2, NULL, &block); + status = block_mgr->get_new_block(client2, nullptr, &block); EXPECT_TRUE(status.is_mem_limit_exceeded()); TearDownMgrs(); diff --git a/be/test/runtime/buffered_tuple_stream2_test.cpp b/be/test/runtime/buffered_tuple_stream2_test.cpp index dd8d2e56a6..e31d935fc1 100644 --- a/be/test/runtime/buffered_tuple_stream2_test.cpp +++ b/be/test/runtime/buffered_tuple_stream2_test.cpp @@ -17,9 +17,8 @@ #include -#include #include -#include +#include #include // for std::numeric_limits::max() #include @@ -38,7 +37,7 @@ using std::vector; -using boost::scoped_ptr; +using std::unique_ptr; static const int BATCH_SIZE = 250; static const uint32_t PRIME = 479001599; @@ -88,8 +87,8 @@ protected: } virtual void TearDown() { - _runtime_state = NULL; - _client = NULL; + _runtime_state = nullptr; + _client = nullptr; _pool.clear(); _mem_pool->free_all(); _test_env.reset(); @@ -137,7 +136,7 @@ protected: if (!gen_null || GenBoolValue(idx)) { row->set_tuple(j, int_tuple); } else { - row->set_tuple(j, NULL); + row->set_tuple(j, nullptr); } } batch->commit_last_row(); @@ -159,7 +158,7 @@ protected: if (!gen_null || GenBoolValue(idx)) { row->set_tuple(j, reinterpret_cast(tuple_mem)); } else { - row->set_tuple(j, NULL); + row->set_tuple(j, nullptr); } } batch->commit_last_row(); @@ -169,7 +168,7 @@ protected: } void AppendRowTuples(TupleRow* row, std::vector* results) { - DCHECK(row != NULL); + DCHECK(row != nullptr); const int int_tuples = _int_desc->tuple_descriptors().size(); for (int i = 0; i < int_tuples; ++i) { AppendValue(row->get_tuple(i), results); @@ -177,7 +176,7 @@ protected: } void AppendRowTuples(TupleRow* row, std::vector* results) { - DCHECK(row != NULL); + DCHECK(row != nullptr); const int string_tuples = _string_desc->tuple_descriptors().size(); for (int i = 0; i < string_tuples; ++i) { AppendValue(row->get_tuple(i), results); @@ -185,7 +184,7 @@ protected: } void AppendValue(Tuple* t, std::vector* results) { - if (t == NULL) { + if (t == nullptr) { // For the tests indicate null-ability using the max int value results->push_back(std::numeric_limits::max()); } else { @@ -194,7 +193,7 @@ protected: } void AppendValue(Tuple* t, std::vector* results) { - if (t == NULL) { + if (t == nullptr) { results->push_back(StringValue()); } else { uint8_t* mem = reinterpret_cast(t); @@ -256,7 +255,7 @@ protected: << " gen_null=" << gen_null; } else { ASSERT_TRUE(results[idx] == StringValue()) - << "results[" << idx << "] " << results[idx] << " not NULL"; + << "results[" << idx << "] " << results[idx] << " not nullptr"; } } } @@ -267,7 +266,7 @@ protected: void TestValues(int num_batches, RowDescriptor* desc, bool gen_null) { BufferedTupleStream2 stream(_runtime_state, *desc, _runtime_state->block_mgr2(), _client, true, false); - Status status = stream.init(-1, NULL, true); + Status status = stream.init(-1, nullptr, true); ASSERT_TRUE(status.ok()) << status.get_error_msg(); status = stream.unpin_stream(); ASSERT_TRUE(status.ok()); @@ -275,7 +274,7 @@ protected: // Add rows to the stream int offset = 0; for (int i = 0; i < num_batches; ++i) { - RowBatch* batch = NULL; + RowBatch* batch = nullptr; if (sizeof(T) == sizeof(int)) { batch = CreateIntBatch(offset, BATCH_SIZE, gen_null); } else if (sizeof(T) == sizeof(StringValue)) { @@ -320,7 +319,7 @@ protected: BufferedTupleStream2 stream(_runtime_state, *_int_desc, _runtime_state->block_mgr2(), _client, small_buffers == 0, // initial small buffers true); // read_write - Status status = stream.init(-1, NULL, true); + Status status = stream.init(-1, nullptr, true); ASSERT_TRUE(status.ok()); status = stream.prepare_for_read(true); ASSERT_TRUE(status.ok()); @@ -351,7 +350,7 @@ protected: } } - boost::scoped_ptr _test_env; + std::unique_ptr _test_env; RuntimeState* _runtime_state; BufferedBlockMgr2::Client* _client; @@ -359,7 +358,7 @@ protected: ObjectPool _pool; RowDescriptor* _int_desc; RowDescriptor* _string_desc; - boost::scoped_ptr _mem_pool; + std::unique_ptr _mem_pool; }; // Tests with a non-NULLable tuple per row. @@ -523,7 +522,7 @@ TEST_F(SimpleTupleStreamTest, UnpinPin) { BufferedTupleStream2 stream(_runtime_state, *_int_desc, _runtime_state->block_mgr2(), _client, true, false); - Status status = stream.init(-1, NULL, true); + Status status = stream.init(-1, nullptr, true); ASSERT_TRUE(status.ok()); int offset = 0; @@ -580,7 +579,7 @@ TEST_F(SimpleTupleStreamTest, SmallBuffers) { BufferedTupleStream2 stream(_runtime_state, *_int_desc, _runtime_state->block_mgr2(), _client, true, false); - Status status = stream.init(-1, NULL, false); + Status status = stream.init(-1, nullptr, false); ASSERT_TRUE(status.ok()); // Initial buffer should be small. @@ -735,11 +734,11 @@ TEST_F(ArrayTupleStreamTest, TestArrayDeepCopy) { // malloc(tuple_descs[0]->byte_size()))); // gscoped_ptr tuple1(reinterpret_cast( // malloc(tuple_descs[1]->byte_size()))); - boost::scoped_ptr row(reinterpret_cast( + std::unique_ptr row(reinterpret_cast( malloc(tuple_descs.size() * sizeof(Tuple*)))); - boost::scoped_ptr tuple0(reinterpret_cast( + std::unique_ptr tuple0(reinterpret_cast( malloc(tuple_descs[0]->byte_size()))); - boost::scoped_ptr tuple1(reinterpret_cast( + std::unique_ptr tuple1(reinterpret_cast( malloc(tuple_descs[1]->byte_size()))); memset(tuple0.get(), 0, tuple_descs[0]->byte_size()); memset(tuple1.get(), 0, tuple_descs[1]->byte_size()); @@ -754,7 +753,7 @@ TEST_F(ArrayTupleStreamTest, TestArrayDeepCopy) { int array_len = array_lens[array_len_index++ % num_array_lens]; CollectionValue* cv = tuple0->GetCollectionSlot(array_slot_desc->tuple_offset()); - cv->ptr = NULL; + cv->ptr = nullptr; cv->num_tuples = 0; CollectionValueBuilder builder(cv, *item_desc, _mem_pool.get(), array_len); Tuple* array_data; @@ -793,8 +792,8 @@ TEST_F(ArrayTupleStreamTest, TestArrayDeepCopy) { TupleRow* row = batch.GetRow(i); Tuple* tuple0 = row->get_tuple(0); Tuple* tuple1 = row->get_tuple(1); - ASSERT_TRUE(tuple0 != NULL); - ASSERT_TRUE(tuple1 != NULL); + ASSERT_TRUE(tuple0 != nullptr); + ASSERT_TRUE(tuple1 != nullptr); const SlotDescriptor* array_slot_desc = tuple_descs[0]->slots()[0]; ASSERT_FALSE(tuple0->IsNull(array_slot_desc->null_indicator_offset())); ASSERT_TRUE(tuple0->IsNull(tuple_descs[0]->slots()[1]->null_indicator_offset())); diff --git a/be/test/runtime/cache/partition_cache_test.cpp b/be/test/runtime/cache/partition_cache_test.cpp index 999909014d..ebd295301b 100644 --- a/be/test/runtime/cache/partition_cache_test.cpp +++ b/be/test/runtime/cache/partition_cache_test.cpp @@ -17,15 +17,13 @@ #include -#include - #include "gen_cpp/PaloInternalService_types.h" #include "gen_cpp/internal_service.pb.h" #include "runtime/buffer_control_block.h" #include "runtime/cache/result_cache.h" +#include "test_util/test_util.h" #include "util/cpu_info.h" #include "util/logging.h" -#include "test_util/test_util.h" namespace doris { @@ -84,10 +82,11 @@ void set_sql_key(PUniqueId* sql_key, int64 hi, int64 lo) { sql_key->set_lo(lo); } -PCacheStatus PartitionCacheTest::init_batch_data(int sql_num, int part_begin, int part_num, CacheType cache_type) { +PCacheStatus PartitionCacheTest::init_batch_data(int sql_num, int part_begin, int part_num, + CacheType cache_type) { LOG(WARNING) << "init data, sql_num:" << sql_num << ",part_num:" << part_num; - PUpdateCacheRequest* up_req = NULL; - PCacheResponse* up_res = NULL; + PUpdateCacheRequest* up_req = nullptr; + PCacheResponse* up_res = nullptr; PCacheStatus st = PCacheStatus::DEFAULT; for (int i = 1; i < sql_num + 1; i++) { LOG(WARNING) << "Sql:" << i; @@ -123,7 +122,8 @@ TEST_F(PartitionCacheTest, update_data) { TEST_F(PartitionCacheTest, update_over_partition) { init_default(); - PCacheStatus st = init_batch_data(1, 1, config::query_cache_max_partition_count + 1, CacheType::PARTITION_CACHE); + PCacheStatus st = init_batch_data(1, 1, config::query_cache_max_partition_count + 1, + CacheType::PARTITION_CACHE); ASSERT_TRUE(st == PCacheStatus::PARAM_ERROR); clear(); } @@ -277,7 +277,8 @@ TEST_F(PartitionCacheTest, fetch_data_overdue) { TEST_F(PartitionCacheTest, prune_data) { init(1, 1); - init_batch_data(LOOP_LESS_OR_MORE(10, 129), 1, 1024, CacheType::PARTITION_CACHE); // 16*1024*128=2M + init_batch_data(LOOP_LESS_OR_MORE(10, 129), 1, 1024, + CacheType::PARTITION_CACHE); // 16*1024*128=2M ASSERT_LE(_cache->get_cache_size(), 1 * 1024 * 1024); //cache_size <= 1M clear(); } diff --git a/be/test/runtime/data_stream_test.cpp b/be/test/runtime/data_stream_test.cpp index d312e43b9a..e6d4a7af5d 100644 --- a/be/test/runtime/data_stream_test.cpp +++ b/be/test/runtime/data_stream_test.cpp @@ -17,8 +17,8 @@ #include -#include #include +#include #include "common/status.h" #include "exprs/slot_ref.h" @@ -44,8 +44,8 @@ using std::string; using std::vector; using std::multiset; -using boost::scoped_ptr; -using boost::thread; +using std::unique_ptr; +using std::thread; namespace doris { @@ -186,8 +186,8 @@ protected: } virtual void TearDown() { - _lhs_slot_ctx->close(NULL); - _rhs_slot_ctx->close(NULL); + _lhs_slot_ctx->close(nullptr); + _rhs_slot_ctx->close(nullptr); _exec_env.client_cache()->test_shutdown(); stop_backend(); } @@ -220,7 +220,7 @@ protected: string _stmt; // RowBatch generation - boost::scoped_ptr _batch; + std::unique_ptr _batch; int _next_val; int64_t* _tuple_mem; @@ -239,7 +239,7 @@ protected: Status status; int num_bytes_sent; - SenderInfo() : thread_handle(NULL), num_bytes_sent(0) {} + SenderInfo() : thread_handle(nullptr), num_bytes_sent(0) {} }; std::vector _sender_info; @@ -249,7 +249,7 @@ protected: int receiver_num; thread* thread_handle; - boost::shared_ptr stream_recvr; + std::shared_ptr stream_recvr; Status status; int num_rows_received; multiset data_values; @@ -258,8 +258,8 @@ protected: : stream_type(stream_type), num_senders(num_senders), receiver_num(receiver_num), - thread_handle(NULL), - stream_recvr(NULL), + thread_handle(nullptr), + stream_recvr(nullptr), num_rows_received(0) {} ~ReceiverInfo() { @@ -335,8 +335,8 @@ protected: _lhs_slot_ctx->prepare(&_runtime_state, *_row_desc, _tracker.get()); _rhs_slot_ctx->prepare(&_runtime_state, *_row_desc, _tracker.get()); - _lhs_slot_ctx->open(NULL); - _rhs_slot_ctx->open(NULL); + _lhs_slot_ctx->open(nullptr); + _rhs_slot_ctx->open(nullptr); SortExecExprs* sort_exprs = _obj_pool.add(new SortExecExprs()); sort_exprs->init(vector(1, _lhs_slot_ctx), std::vector(1, _rhs_slot_ctx)); @@ -372,7 +372,7 @@ protected: // Start receiver (expecting given number of senders) in separate thread. void start_receiver(TPartitionType::type stream_type, int num_senders, int receiver_num, - int buffer_size, bool is_merging, TUniqueId* out_id = NULL) { + int buffer_size, bool is_merging, TUniqueId* out_id = nullptr) { VLOG_QUERY << "start receiver"; RuntimeProfile* profile = _obj_pool.add(new RuntimeProfile("TestReceiver")); TUniqueId instance_id; @@ -389,7 +389,7 @@ protected: new thread(&DataStreamTest::read_stream_merging, this, &info, profile); } - if (out_id != NULL) { + if (out_id != nullptr) { *out_id = instance_id; } } @@ -405,12 +405,12 @@ protected: // Deplete stream and print batches void read_stream(ReceiverInfo* info) { - RowBatch* batch = NULL; + RowBatch* batch = nullptr; VLOG_QUERY << "start reading"; while (!(info->status = info->stream_recvr->get_batch(&batch)).is_cancelled() && - (batch != NULL)) { - VLOG_QUERY << "read batch #rows=" << (batch != NULL ? batch->num_rows() : 0); + (batch != nullptr)) { + VLOG_QUERY << "read batch #rows=" << (batch != nullptr ? batch->num_rows() : 0); for (int i = 0; i < batch->num_rows(); ++i) { TupleRow* row = batch->get_row(i); @@ -514,10 +514,9 @@ protected: // Start backend in separate thread. void start_backend() { - boost::shared_ptr handler(new DorisTestBackend(_stream_mgr)); - boost::shared_ptr processor( - new BackendServiceProcessor(handler)); - _server = new ThriftServer("DataStreamTest backend", processor, config::port, NULL); + std::shared_ptr handler(new DorisTestBackend(_stream_mgr)); + std::shared_ptr processor(new BackendServiceProcessor(handler)); + _server = new ThriftServer("DataStreamTest backend", processor, config::port, nullptr); _server->start(); } @@ -562,7 +561,7 @@ protected: EXPECT_TRUE(sender.prepare(&state).ok()); EXPECT_TRUE(sender.open(&state).ok()); - boost::scoped_ptr batch(create_row_batch()); + std::unique_ptr batch(create_row_batch()); SenderInfo& info = _sender_info[sender_num]; int next_val = 0; diff --git a/be/test/runtime/disk_io_mgr_test.cpp b/be/test/runtime/disk_io_mgr_test.cpp index f1a64e3e2b..794ed24502 100644 --- a/be/test/runtime/disk_io_mgr_test.cpp +++ b/be/test/runtime/disk_io_mgr_test.cpp @@ -21,12 +21,13 @@ #include #include -#include -#include +#include +#include #include "util/cpu_info.h" #include "util/disk_info.h" #include "util/logging.h" +#include "util/thread_group.h" using std::string; using std::stringstream; @@ -38,9 +39,8 @@ using std::unique_lock; using std::mutex; using std::mem_fn; using std::condition_variable; -using boost::scoped_ptr; -using boost::thread; -using boost::thread_group; +using std::unique_ptr; +using std::thread; namespace doris { @@ -60,7 +60,7 @@ public: } if (status.ok()) { DiskIoMgr::ScanRange* scan_range = _pool->add(new DiskIoMgr::ScanRange()); - scan_range->reset(NULL, (*written_range)->file(), (*written_range)->len(), + scan_range->reset(nullptr, (*written_range)->file(), (*written_range)->len(), (*written_range)->offset(), 0, false, false, DiskIoMgr::ScanRange::NEVER_CACHE); validate_sync_read(io_mgr, reader, scan_range, reinterpret_cast(data), @@ -90,14 +90,14 @@ public: protected: void CreateTempFile(const char* filename, const char* data) { FILE* file = fopen(filename, "w"); - EXPECT_TRUE(file != NULL); + EXPECT_TRUE(file != nullptr); fwrite(data, 1, strlen(data), file); fclose(file); } int CreateTempFile(const char* filename, int file_size) { FILE* file = fopen(filename, "w"); - EXPECT_TRUE(file != NULL); + EXPECT_TRUE(file != nullptr); int success = fclose(file); if (success != 0) { LOG(ERROR) << "Error closing file " << filename; @@ -121,7 +121,7 @@ protected: DiskIoMgr::BufferDescriptor* buffer; Status status = io_mgr->read(reader, range, &buffer); ASSERT_TRUE(status.ok()); - ASSERT_TRUE(buffer != NULL); + ASSERT_TRUE(buffer != nullptr); EXPECT_EQ(buffer->len(), range->len()); if (expected_len < 0) { expected_len = strlen(expected); @@ -137,11 +137,11 @@ protected: memset(result, 0, expected_len + 1); while (true) { - DiskIoMgr::BufferDescriptor* buffer = NULL; + DiskIoMgr::BufferDescriptor* buffer = nullptr; Status status = range->get_next(&buffer); ASSERT_TRUE(status.ok() || status.code() == expected_status.code()); - if (buffer == NULL || !status.ok()) { - if (buffer != NULL) buffer->return_buffer(); + if (buffer == nullptr || !status.ok()) { + if (buffer != nullptr) buffer->return_buffer(); break; } ASSERT_LE(buffer->len(), expected_len); @@ -163,7 +163,7 @@ protected: DiskIoMgr::ScanRange* range; Status status = io_mgr->get_next_range(reader, &range); ASSERT_TRUE(status.ok() || status.code() == expected_status.code()); - if (range == NULL) break; + if (range == nullptr) break; validate_scan_range(range, expected_result, expected_len, expected_status); ++(*num_ranges_processed); ++num_ranges; @@ -171,15 +171,15 @@ protected: } DiskIoMgr::ScanRange* init_range(int num_buffers, const char* file_path, int offset, int len, - int disk_id, int64_t mtime, void* meta_data = NULL, + int disk_id, int64_t mtime, void* meta_data = nullptr, bool is_cached = false) { DiskIoMgr::ScanRange* range = _pool->add(new DiskIoMgr::ScanRange(num_buffers)); - range->reset(NULL, file_path, len, offset, disk_id, is_cached, true, mtime, meta_data); + range->reset(nullptr, file_path, len, offset, disk_id, is_cached, true, mtime, meta_data); EXPECT_EQ(mtime, range->mtime()); return range; } - boost::scoped_ptr _pool; + std::unique_ptr _pool; mutex _written_mutex; condition_variable _writes_done; @@ -203,7 +203,7 @@ TEST_F(DiskIoMgrTest, SingleWriter) { EXPECT_TRUE(false); } - boost::scoped_ptr read_io_mgr(new DiskIoMgr(1, 1, 1, 10)); + std::unique_ptr read_io_mgr(new DiskIoMgr(1, 1, 1, 10)); std::shared_ptr reader_mem_tracker(new MemTracker(LARGE_MEM_LIMIT)); Status status = read_io_mgr->init(reader_mem_tracker); ASSERT_TRUE(status.ok()); @@ -265,9 +265,10 @@ TEST_F(DiskIoMgrTest, InvalidWrite) { // Write to a non-existent file. DiskIoMgr::WriteRange** new_range = _pool->add(new DiskIoMgr::WriteRange*); - DiskIoMgr::WriteRange::WriteDoneCallback callback = bind( - mem_fn(&DiskIoMgrTest::write_validate_callback), this, 2, new_range, (DiskIoMgr*)NULL, - (DiskIoMgr::RequestContext*)NULL, data, Status::InternalError("Test Failure"), _1); + DiskIoMgr::WriteRange::WriteDoneCallback callback = + bind(mem_fn(&DiskIoMgrTest::write_validate_callback), this, 2, new_range, + (DiskIoMgr*)nullptr, (DiskIoMgr::RequestContext*)nullptr, data, + Status::InternalError("Test Failure"), _1); *new_range = _pool->add(new DiskIoMgr::WriteRange(tmp_file, rand(), 0, callback)); (*new_range)->set_data(reinterpret_cast(data), sizeof(int32_t)); @@ -284,7 +285,7 @@ TEST_F(DiskIoMgrTest, InvalidWrite) { new_range = _pool->add(new DiskIoMgr::WriteRange*); callback = bind(mem_fn(&DiskIoMgrTest::write_validate_callback), this, 2, new_range, - (DiskIoMgr*)NULL, (DiskIoMgr::RequestContext*)NULL, data, + (DiskIoMgr*)nullptr, (DiskIoMgr::RequestContext*)nullptr, data, Status::InternalError("Test Failure"), _1); *new_range = _pool->add(new DiskIoMgr::WriteRange(tmp_file, -1, 0, callback)); @@ -319,7 +320,7 @@ TEST_F(DiskIoMgrTest, SingleWriterCancel) { EXPECT_TRUE(false); } - boost::scoped_ptr read_io_mgr(new DiskIoMgr(1, 1, 1, 10)); + std::unique_ptr read_io_mgr(new DiskIoMgr(1, 1, 1, 10)); std::shared_ptr reader_mem_tracker(new MemTracker(LARGE_MEM_LIMIT)); Status status = read_io_mgr->init(reader_mem_tracker); ASSERT_TRUE(status.ok()); @@ -414,7 +415,7 @@ TEST_F(DiskIoMgrTest, SingleReader) { ASSERT_TRUE(status.ok()); AtomicInt num_ranges_processed; - thread_group threads; + ThreadGroup threads; for (int i = 0; i < num_read_threads; ++i) { threads.add_thread(new thread(scan_range_thread, &io_mgr, reader, data, len, Status::OK(), 0, &num_ranges_processed)); @@ -488,7 +489,7 @@ TEST_F(DiskIoMgrTest, AddScanRangeTest) { ASSERT_TRUE(status.ok()); // Start up some threads and then cancel - thread_group threads; + ThreadGroup threads; for (int i = 0; i < 3; ++i) { threads.add_thread(new thread(scan_range_thread, &io_mgr, reader, data, strlen(data), Status::Cancelled(""), 0, @@ -557,7 +558,7 @@ TEST_F(DiskIoMgrTest, SyncReadTest) { ASSERT_TRUE(status.ok()); AtomicInt num_ranges_processed; - thread_group threads; + ThreadGroup threads; for (int i = 0; i < 5; ++i) { threads.add_thread(new thread(scan_range_thread, &io_mgr, reader, data, strlen(data), Status::OK(), 0, @@ -633,7 +634,7 @@ TEST_F(DiskIoMgrTest, SingleReaderCancel) { EXPECT_EQ(num_ranges_processed, num_succesful_ranges); // Start up some threads and then cancel - thread_group threads; + ThreadGroup threads; for (int i = 0; i < 3; ++i) { threads.add_thread(new thread(scan_range_thread, &io_mgr, reader, data, strlen(data), Status::Cancelled(""), 0, @@ -705,16 +706,16 @@ TEST_F(DiskIoMgrTest, MemTrackers) { // to go over the limit eventually. while (true) { memset(result, 0, strlen(data) + 1); - DiskIoMgr::ScanRange* range = NULL; + DiskIoMgr::ScanRange* range = nullptr; status = io_mgr.get_next_range(reader, &range); ASSERT_TRUE(status.ok() || status.is_mem_limit_exceeded()); - if (range == NULL) break; + if (range == nullptr) break; while (true) { - DiskIoMgr::BufferDescriptor* buffer = NULL; + DiskIoMgr::BufferDescriptor* buffer = nullptr; Status status = range->get_next(&buffer); ASSERT_TRUE(status.ok() || status.is_mem_limit_exceeded()); - if (buffer == NULL) break; + if (buffer == nullptr) break; memcpy(result + range->offset() + buffer->scan_range_offset(), buffer->buffer(), buffer->len()); buffers.push_back(buffer); @@ -764,7 +765,7 @@ TEST_F(DiskIoMgrTest, CachedReads) { ASSERT_TRUE(status.ok()); DiskIoMgr::ScanRange* complete_range = - init_range(1, tmp_file, 0, strlen(data), 0, stat_val.st_mtime, NULL, true); + init_range(1, tmp_file, 0, strlen(data), 0, stat_val.st_mtime, nullptr, true); // Issue some reads before the async ones are issued validate_sync_read(&io_mgr, reader, complete_range, data); @@ -774,13 +775,13 @@ TEST_F(DiskIoMgrTest, CachedReads) { for (int i = 0; i < len; ++i) { int disk_id = i % num_disks; ranges.push_back(init_range(num_buffers, tmp_file, 0, len, disk_id, - stat_val.st_mtime, NULL, true)); + stat_val.st_mtime, nullptr, true)); } status = io_mgr.add_scan_ranges(reader, ranges); ASSERT_TRUE(status.ok()); AtomicInt num_ranges_processed; - thread_group threads; + ThreadGroup threads; for (int i = 0; i < 5; ++i) { threads.add_thread(new thread(scan_range_thread, &io_mgr, reader, data, strlen(data), Status::OK(), 0, &num_ranges_processed)); @@ -846,7 +847,7 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) { LOG(ERROR) << "Starting iteration " << iters; } AtomicInt num_ranges_processed; - thread_group threads; + ThreadGroup threads; std::vector ranges; int num_scan_ranges = std::min(num_reads_queued, write_offset - read_offset); @@ -956,7 +957,7 @@ TEST_F(DiskIoMgrTest, MultipleReader) { ASSERT_TRUE(status.ok()); for (int i = 0; i < NUM_READERS; ++i) { - status = io_mgr.register_context(&readers[i], NULL); + status = io_mgr.register_context(&readers[i], nullptr); ASSERT_TRUE(status.ok()); std::vector ranges; @@ -970,7 +971,7 @@ TEST_F(DiskIoMgrTest, MultipleReader) { } AtomicInt num_ranges_processed; - thread_group threads; + ThreadGroup threads; for (int i = 0; i < NUM_READERS; ++i) { for (int j = 0; j < NUM_THREADS_PER_READER; ++j) { threads.add_thread(new thread(scan_range_thread, &io_mgr, readers[i], @@ -1070,7 +1071,7 @@ TEST_F(DiskIoMgrTest, PartialRead) { stat(tmp_file, &stat_val); _pool.reset(new ObjectPool); - boost::scoped_ptr io_mgr(new DiskIoMgr(1, 1, read_len, read_len)); + std::unique_ptr io_mgr(new DiskIoMgr(1, 1, read_len, read_len)); Status status = io_mgr->init(mem_tracker); ASSERT_TRUE(status.ok()); diff --git a/be/test/runtime/free_list_test.cpp b/be/test/runtime/free_list_test.cpp index 43a7c325e5..e67aa533e6 100644 --- a/be/test/runtime/free_list_test.cpp +++ b/be/test/runtime/free_list_test.cpp @@ -37,7 +37,7 @@ TEST(FreeListTest, Basic) { EXPECT_EQ(allocated_size, 0); uint8_t* mem = pool.allocate(FreeList::min_size()); - EXPECT_TRUE(mem != NULL); + EXPECT_TRUE(mem != nullptr); list.add(mem, FreeList::min_size()); free_list_mem = list.allocate(FreeList::min_size(), &allocated_size); @@ -52,7 +52,7 @@ TEST(FreeListTest, Basic) { // Get them all back from the free list, scribbling to the // returned memory in between. // Attempt a 4th allocation from the free list and make sure - // we get NULL. + // we get nullptr. // Repeat with the same memory blocks. uint8_t* free_list_mem1 = nullptr; uint8_t* free_list_mem2 = nullptr; @@ -66,22 +66,22 @@ TEST(FreeListTest, Basic) { list.add(mem, FreeList::min_size()); free_list_mem1 = list.allocate(FreeList::min_size(), &allocated_size); - EXPECT_TRUE(free_list_mem1 != NULL); + EXPECT_TRUE(free_list_mem1 != nullptr); EXPECT_EQ(allocated_size, FreeList::min_size()); bzero(free_list_mem1, FreeList::min_size()); free_list_mem2 = list.allocate(FreeList::min_size(), &allocated_size); - EXPECT_TRUE(free_list_mem2 != NULL); + EXPECT_TRUE(free_list_mem2 != nullptr); EXPECT_EQ(allocated_size, FreeList::min_size()); bzero(free_list_mem2, FreeList::min_size()); free_list_mem3 = list.allocate(FreeList::min_size(), &allocated_size); - EXPECT_TRUE(free_list_mem3 != NULL); + EXPECT_TRUE(free_list_mem3 != nullptr); EXPECT_EQ(allocated_size, FreeList::min_size()); bzero(free_list_mem3, FreeList::min_size()); free_list_mem = list.allocate(FreeList::min_size(), &allocated_size); - EXPECT_EQ(NULL, free_list_mem); + EXPECT_EQ(nullptr, free_list_mem); EXPECT_EQ(allocated_size, 0); list.add(free_list_mem1, FreeList::min_size()); @@ -89,22 +89,22 @@ TEST(FreeListTest, Basic) { list.add(free_list_mem3, FreeList::min_size()); free_list_mem1 = list.allocate(FreeList::min_size(), &allocated_size); - EXPECT_TRUE(free_list_mem1 != NULL); + EXPECT_TRUE(free_list_mem1 != nullptr); EXPECT_EQ(allocated_size, FreeList::min_size()); bzero(free_list_mem1, FreeList::min_size()); free_list_mem2 = list.allocate(FreeList::min_size(), &allocated_size); - EXPECT_TRUE(free_list_mem2 != NULL); + EXPECT_TRUE(free_list_mem2 != nullptr); EXPECT_EQ(allocated_size, FreeList::min_size()); bzero(free_list_mem2, FreeList::min_size()); free_list_mem3 = list.allocate(FreeList::min_size(), &allocated_size); - EXPECT_TRUE(free_list_mem3 != NULL); + EXPECT_TRUE(free_list_mem3 != nullptr); EXPECT_EQ(allocated_size, FreeList::min_size()); bzero(free_list_mem3, FreeList::min_size()); free_list_mem = list.allocate(FreeList::min_size(), &allocated_size); - EXPECT_EQ(NULL, free_list_mem); + EXPECT_EQ(nullptr, free_list_mem); EXPECT_EQ(allocated_size, 0); // Try some allocations with different sizes @@ -118,16 +118,16 @@ TEST(FreeListTest, Basic) { list.add(mem2, size2); free_list_mem = list.allocate(size4, &allocated_size); - EXPECT_EQ(NULL, free_list_mem); + EXPECT_EQ(nullptr, free_list_mem); EXPECT_EQ(allocated_size, 0); free_list_mem = list.allocate(size1, &allocated_size); - EXPECT_TRUE(free_list_mem != NULL); + EXPECT_TRUE(free_list_mem != nullptr); EXPECT_EQ(allocated_size, size2); bzero(free_list_mem, size1); free_list_mem = list.allocate(size1, &allocated_size); - EXPECT_EQ(NULL, free_list_mem); + EXPECT_EQ(nullptr, free_list_mem); EXPECT_EQ(allocated_size, 0); list.add(mem2, size2); diff --git a/be/test/runtime/mem_pool_test.cpp b/be/test/runtime/mem_pool_test.cpp index f3a1061278..088a604802 100644 --- a/be/test/runtime/mem_pool_test.cpp +++ b/be/test/runtime/mem_pool_test.cpp @@ -137,7 +137,7 @@ TEST(MemPoolTest, MaxAllocation) { MemTracker tracker(-1); MemPool p1(&tracker); uint8_t* ptr = p1.allocate(LARGE_ALLOC_SIZE); - EXPECT_TRUE(ptr != NULL); + EXPECT_TRUE(ptr != nullptr); EXPECT_EQ(int_max_rounded, p1.total_reserved_bytes()); EXPECT_EQ(int_max_rounded, p1.total_allocated_bytes()); p1.free_all(); @@ -148,7 +148,7 @@ TEST(MemPoolTest, MaxAllocation) { EXPECT_EQ(p2.total_reserved_bytes(), 4096); EXPECT_EQ(p2.total_allocated_bytes(), 8); ptr = p2.allocate(LARGE_ALLOC_SIZE); - EXPECT_TRUE(ptr != NULL); + EXPECT_TRUE(ptr != nullptr); EXPECT_EQ(p2.total_reserved_bytes(), 4096LL + int_max_rounded); EXPECT_EQ(p2.total_allocated_bytes(), 8LL + int_max_rounded); p2.free_all(); @@ -160,12 +160,12 @@ TEST(MemPoolTest, MaxAllocation) { // Allocates new int_max_rounded * 2 chunk // NOTE: exceed MAX_CHUNK_SIZE limit, will not *2 ptr = p3.allocate(LARGE_ALLOC_SIZE); - EXPECT_TRUE(ptr != NULL); + EXPECT_TRUE(ptr != nullptr); EXPECT_EQ(int_max_rounded * 2, p3.total_reserved_bytes()); EXPECT_EQ(int_max_rounded * 2, p3.total_allocated_bytes()); // Uses existing int_max_rounded * 2 chunk ptr = p3.allocate(LARGE_ALLOC_SIZE); - EXPECT_TRUE(ptr != NULL); + EXPECT_TRUE(ptr != nullptr); EXPECT_EQ(int_max_rounded * 3, p3.total_reserved_bytes()); EXPECT_EQ(int_max_rounded * 3, p3.total_allocated_bytes()); @@ -173,12 +173,12 @@ TEST(MemPoolTest, MaxAllocation) { // NOTE: exceed MAX_CHUNK_SIZE limit, will not *2 #if !defined(ADDRESS_SANITIZER) || (__clang_major__ >= 3 && __clang_minor__ >= 7) ptr = p3.allocate(8); - EXPECT_TRUE(ptr != NULL); + EXPECT_TRUE(ptr != nullptr); EXPECT_EQ(int_max_rounded * 3 + 512 * 1024, p3.total_reserved_bytes()); EXPECT_EQ(int_max_rounded * 3 + 8, p3.total_allocated_bytes()); // Uses existing int_max_rounded * 4 chunk ptr = p3.allocate(LARGE_ALLOC_SIZE); - EXPECT_TRUE(ptr != NULL); + EXPECT_TRUE(ptr != nullptr); EXPECT_EQ(int_max_rounded * 4 + 512 * 1024, p3.total_reserved_bytes()); EXPECT_EQ(int_max_rounded * 4 + 8, p3.total_allocated_bytes()); #endif diff --git a/be/test/runtime/result_buffer_mgr_test.cpp b/be/test/runtime/result_buffer_mgr_test.cpp index 83d7557385..574dcc9088 100644 --- a/be/test/runtime/result_buffer_mgr_test.cpp +++ b/be/test/runtime/result_buffer_mgr_test.cpp @@ -19,8 +19,6 @@ #include -#include - #include "gen_cpp/PaloInternalService_types.h" #include "runtime/buffer_control_block.h" #include "util/cpu_info.h" @@ -44,7 +42,7 @@ TEST_F(ResultBufferMgrTest, create_normal) { query_id.lo = 10; query_id.hi = 100; - boost::shared_ptr control_block1; + std::shared_ptr control_block1; ASSERT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok()); } @@ -54,9 +52,9 @@ TEST_F(ResultBufferMgrTest, create_same_buffer) { query_id.lo = 10; query_id.hi = 100; - boost::shared_ptr control_block1; + std::shared_ptr control_block1; ASSERT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok()); - boost::shared_ptr control_block2; + std::shared_ptr control_block2; ASSERT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block2).ok()); ASSERT_EQ(control_block1.get(), control_block1.get()); @@ -68,7 +66,7 @@ TEST_F(ResultBufferMgrTest, fetch_data_normal) { query_id.lo = 10; query_id.hi = 100; - boost::shared_ptr control_block1; + std::shared_ptr control_block1; ASSERT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok()); TFetchDataResult* result = new TFetchDataResult(); @@ -86,7 +84,7 @@ TEST_F(ResultBufferMgrTest, fetch_data_no_block) { query_id.lo = 10; query_id.hi = 100; - boost::shared_ptr control_block1; + std::shared_ptr control_block1; ASSERT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok()); TFetchDataResult* result = new TFetchDataResult(); @@ -102,7 +100,7 @@ TEST_F(ResultBufferMgrTest, normal_cancel) { query_id.lo = 10; query_id.hi = 100; - boost::shared_ptr control_block1; + std::shared_ptr control_block1; ASSERT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok()); ASSERT_TRUE(buffer_mgr.cancel(query_id).ok()); diff --git a/be/test/runtime/sorter_test.cpp b/be/test/runtime/sorter_test.cpp index d610a0a0e8..7fda5e13d1 100644 --- a/be/test/runtime/sorter_test.cpp +++ b/be/test/runtime/sorter_test.cpp @@ -176,7 +176,7 @@ TEST_F(SorterTest, init_sort_exec_exprs) { // empty sort_tuple_slot_expr { SortExecExprs exec_exprs; - Status status = exec_exprs.init(_ordering_exprs, NULL, get_object_pool()); + Status status = exec_exprs.init(_ordering_exprs, nullptr, get_object_pool()); ASSERT_TRUE(status.ok()); } // full sort_tuple_slot_expr @@ -190,7 +190,7 @@ TEST_F(SorterTest, init_sort_exec_exprs) { TEST_F(SorterTest, prepare_sort_exec_exprs) { { SortExecExprs exec_exprs; - Status status = exec_exprs.init(_ordering_exprs, NULL, get_object_pool()); + Status status = exec_exprs.init(_ordering_exprs, nullptr, get_object_pool()); ASSERT_TRUE(status.ok()); status = exec_exprs.prepare(_runtime_state, *_child_row_desc, *_output_row_desc); ASSERT_TRUE(status.ok()); diff --git a/be/test/runtime/test_env.cc b/be/test/runtime/test_env.cc index e0ed0c7444..eafaaed4ba 100644 --- a/be/test/runtime/test_env.cc +++ b/be/test/runtime/test_env.cc @@ -88,7 +88,7 @@ Status TestEnv::create_query_state(int64_t query_id, int max_buffers, int block_ return Status::InternalError("Unexpected error creating RuntimeState"); } - boost::shared_ptr mgr; + std::shared_ptr mgr; RETURN_IF_ERROR(BufferedBlockMgr2::create( *runtime_state, _block_mgr_parent_tracker, (*runtime_state)->runtime_profile(), _tmp_file_mgr.get(), calculate_mem_tracker(max_buffers, block_size), block_size, &mgr)); diff --git a/be/test/runtime/thread_resource_mgr_test.cpp b/be/test/runtime/thread_resource_mgr_test.cpp index 39b9834b96..ea006b839d 100644 --- a/be/test/runtime/thread_resource_mgr_test.cpp +++ b/be/test/runtime/thread_resource_mgr_test.cpp @@ -19,7 +19,7 @@ #include -#include +#include #include #include "util/cpu_info.h" diff --git a/be/test/runtime/tmp_file_mgr_test.cpp b/be/test/runtime/tmp_file_mgr_test.cpp index 43fb7fb080..cc83240aea 100644 --- a/be/test/runtime/tmp_file_mgr_test.cpp +++ b/be/test/runtime/tmp_file_mgr_test.cpp @@ -19,9 +19,8 @@ #include -#include -#include #include +#include #include "gen_cpp/Types_types.h" // for TUniqueId #include "util/disk_info.h" @@ -63,7 +62,7 @@ TEST_F(TmpFileMgrTest, TestFileAllocation) { TmpFileMgr::File* file; Status status = tmp_file_mgr.get_file(tmp_devices[0], id, &file); EXPECT_TRUE(status.ok()); - EXPECT_TRUE(file != NULL); + EXPECT_TRUE(file != nullptr); // Apply writes of variable sizes and check space was allocated correctly. int64_t write_sizes[] = {1, 10, 1024, 4, 1024 * 1024 * 8, 1024 * 1024 * 8, 16, 10}; int num_write_sizes = sizeof(write_sizes) / sizeof(write_sizes[0]); @@ -176,7 +175,7 @@ TEST_F(TmpFileMgrTest, TestReportError) { // The good device should still be usable. TmpFileMgr::File* good_file; EXPECT_TRUE(tmp_file_mgr.get_file(devices[good_device], id, &good_file).ok()); - EXPECT_TRUE(good_file != NULL); + EXPECT_TRUE(good_file != nullptr); EXPECT_TRUE(good_file->allocate_space(128, &offset).ok()); // Attempts to allocate new files on bad device should succeed. EXPECT_TRUE(tmp_file_mgr.get_file(devices[bad_device], id, &bad_file).ok()); diff --git a/be/test/udf/uda_test.cpp b/be/test/udf/uda_test.cpp index 9613306898..dcd7ac3524 100644 --- a/be/test/udf/uda_test.cpp +++ b/be/test/udf/uda_test.cpp @@ -97,7 +97,7 @@ struct MinState { // Initialize the MinState scratch space void MinInit(FunctionContext* context, BufferVal* val) { MinState* state = reinterpret_cast(*val); - state->value = NULL; + state->value = nullptr; state->buffer_len = 0; } @@ -109,7 +109,7 @@ void MinUpdate(FunctionContext* context, const StringVal& input, BufferVal* val) MinState* state = reinterpret_cast(*val); - if (state->value == NULL) { + if (state->value == nullptr) { state->set(context, input); return; } @@ -130,7 +130,7 @@ const BufferVal MinSerialize(FunctionContext* context, const BufferVal& intermed void MinMerge(FunctionContext* context, const BufferVal& src, BufferVal* dst) { const MinState* src_state = reinterpret_cast(src); - if (src_state->value == NULL) { + if (src_state->value == nullptr) { return; } @@ -141,7 +141,7 @@ void MinMerge(FunctionContext* context, const BufferVal& src, BufferVal* dst) { StringVal MinFinalize(FunctionContext* context, const BufferVal& val) { const MinState* state = reinterpret_cast(val); - if (state->value == NULL) { + if (state->value == nullptr) { return StringVal::null(); } @@ -164,7 +164,7 @@ void XorInit(FunctionContext* context, BigIntVal* val) { void XorUpdate(FunctionContext* context, const double* input, BigIntVal* val) { // BigIntVal is the same ptr as what was passed to CountInit - if (input == NULL) { + if (input == nullptr) { return; } @@ -197,7 +197,7 @@ void DistinctEstimateInit(FunctionContext* context, StringVal* val) { } void DistinctEstimatUpdate(FunctionContext* context, const int64_t* input, StringVal* val) { - if (input == NULL) { + if (input == nullptr) { return; } @@ -211,12 +211,12 @@ void DistinctEstimatUpdate(FunctionContext* context, const int64_t* input, Strin StringVal DistinctEstimatSerialize(FunctionContext* context, const StringVal& intermediate) { int compressed_size = 0; - uint8_t* result = NULL; // SnappyCompress(intermediate.ptr, intermediate.len); + uint8_t* result = nullptr; // SnappyCompress(intermediate.ptr, intermediate.len); return StringVal(result, compressed_size); } void DistinctEstimateMerge(FunctionContext* context, const StringVal& src, StringVal* dst) { - uint8_t* src_uncompressed = NULL; // SnappyUncompress(src.ptr, src.len); + uint8_t* src_uncompressed = nullptr; // SnappyUncompress(src.ptr, src.len); for (int i = 0; i < 256; ++i) { dst->ptr[i] ^= src_uncompressed[i]; @@ -231,7 +231,7 @@ BigIntVal DistinctEstimateFinalize(FunctionContext* context, const StringVal& va } TEST(CountTest, Basic) { - UdaTestHarness test(CountInit, CountUpdate, CountMerge, NULL, + UdaTestHarness test(CountInit, CountUpdate, CountMerge, nullptr, CountFinalize); std::vector no_nulls; no_nulls.resize(1000); @@ -246,16 +246,16 @@ TEST(CountMultiArgTest, Basic) { no_nulls.resize(num); UdaTestHarness2 test2(CountInit, Count2Update, CountMerge, - NULL, CountFinalize); + nullptr, CountFinalize); EXPECT_TRUE(test2.execute(no_nulls, no_nulls, BigIntVal(2 * num))); EXPECT_FALSE(test2.execute(no_nulls, no_nulls, BigIntVal(100))); UdaTestHarness3 test3( - CountInit, Count3Update, CountMerge, NULL, CountFinalize); + CountInit, Count3Update, CountMerge, nullptr, CountFinalize); EXPECT_TRUE(test3.execute(no_nulls, no_nulls, no_nulls, BigIntVal(3 * num))); UdaTestHarness4 test4( - CountInit, Count4Update, CountMerge, NULL, CountFinalize); + CountInit, Count4Update, CountMerge, nullptr, CountFinalize); EXPECT_TRUE(test4.execute(no_nulls, no_nulls, no_nulls, no_nulls, BigIntVal(4 * num))); } @@ -272,7 +272,7 @@ bool FuzzyCompare(const BigIntVal& r1, const BigIntVal& r2) { } TEST(CountTest, FuzzyEquals) { - UdaTestHarness test(CountInit, CountUpdate, CountMerge, NULL, + UdaTestHarness test(CountInit, CountUpdate, CountMerge, nullptr, CountFinalize); std::vector no_nulls; no_nulls.resize(1000); diff --git a/be/test/udf/udf_test.cpp b/be/test/udf/udf_test.cpp index 87cac9e90e..c3b15012b9 100644 --- a/be/test/udf/udf_test.cpp +++ b/be/test/udf/udf_test.cpp @@ -17,7 +17,6 @@ #include -#include #include #include "common/logging.h" @@ -31,7 +30,7 @@ DoubleVal zero_udf(FunctionContext* context) { } StringVal log_udf(FunctionContext* context, const StringVal& arg1) { - std::cerr << (arg1.is_null ? "NULL" : std::string((char*)arg1.ptr, arg1.len)) << std::endl; + std::cerr << (arg1.is_null ? "nullptr" : std::string((char*)arg1.ptr, arg1.len)) << std::endl; return arg1; } @@ -124,13 +123,13 @@ IntVal num_var_args(FunctionContext*, const BigIntVal& dummy, int n, const IntVa IntVal validat_udf(FunctionContext* context) { EXPECT_EQ(context->version(), FunctionContext::V2_0); EXPECT_FALSE(context->has_error()); - EXPECT_TRUE(context->error_msg() == NULL); + EXPECT_TRUE(context->error_msg() == nullptr); return IntVal::null(); } IntVal validate_fail(FunctionContext* context) { EXPECT_FALSE(context->has_error()); - EXPECT_TRUE(context->error_msg() == NULL); + EXPECT_TRUE(context->error_msg() == nullptr); context->set_error("Fail"); EXPECT_TRUE(context->has_error()); EXPECT_TRUE(strcmp(context->error_msg(), "Fail") == 0); @@ -138,26 +137,14 @@ IntVal validate_fail(FunctionContext* context) { } IntVal validate_mem(FunctionContext* context) { - EXPECT_TRUE(context->allocate(0) == NULL); + EXPECT_TRUE(context->allocate(0) == nullptr); uint8_t* buffer = context->allocate(10); - EXPECT_TRUE(buffer != NULL); + EXPECT_TRUE(buffer != nullptr); memset(buffer, 0, 10); context->free(buffer); return IntVal::null(); } -StringVal time_to_string(FunctionContext* context, const TimestampVal& time) { - boost::posix_time::ptime t(*(boost::gregorian::date*)&time.date); - t += boost::posix_time::nanoseconds(time.time_of_day); - std::stringstream ss; - ss << boost::posix_time::to_iso_extended_string(t) << " " - << boost::posix_time::to_simple_string(t.time_of_day()); - std::string s = ss.str(); - StringVal result(context, s.size()); - memcpy(result.ptr, s.data(), result.len); - return result; -} - TEST(UdfTest, TestFunctionContext) { EXPECT_TRUE(UdfTestHarness::validat_udf(validat_udf, IntVal::null())); EXPECT_FALSE(UdfTestHarness::validat_udf(validate_fail, IntVal::null())); @@ -182,16 +169,16 @@ TEST(UdfTest, TestValidate) { min3, FloatVal::null(), FloatVal::null(), FloatVal::null(), FloatVal::null()))); } -TEST(UdfTest, TestTimestampVal) { - boost::gregorian::date d(2003, 3, 15); - TimestampVal t1(*(int32_t*)&d); - EXPECT_TRUE((UdfTestHarness::validat_udf(time_to_string, t1, - "2003-03-15 00:00:00"))); +// TEST(UdfTest, TestTimestampVal) { +// boost::gregorian::date d(2003, 3, 15); +// TimestampVal t1(*(int32_t*)&d); +// EXPECT_TRUE((UdfTestHarness::validat_udf(time_to_string, t1, +// "2003-03-15 00:00:00"))); - TimestampVal t2(*(int32_t*)&d, 1000L * 1000L * 5000L); - EXPECT_TRUE((UdfTestHarness::validat_udf(time_to_string, t2, - "2003-03-15 00:00:05"))); -} +// TimestampVal t2(*(int32_t*)&d, 1000L * 1000L * 5000L); +// EXPECT_TRUE((UdfTestHarness::validat_udf(time_to_string, t2, +// "2003-03-15 00:00:05"))); +// } TEST(UdfTest, TestVarArgs) { std::vector input; diff --git a/be/test/util/aes_util_test.cpp b/be/test/util/aes_util_test.cpp index d76f01326f..93517e2e58 100644 --- a/be/test/util/aes_util_test.cpp +++ b/be/test/util/aes_util_test.cpp @@ -39,13 +39,13 @@ void do_aes_test(const std::string& source, const std::string& key) { std::unique_ptr dest(new unsigned char[cipher_len]); int ret_code = AesUtil::encrypt(AES_128_ECB, (unsigned char*)source.c_str(), source.length(), - (unsigned char*)key.c_str(), key.length(), NULL, true, dest.get()); + (unsigned char*)key.c_str(), key.length(), nullptr, true, dest.get()); ASSERT_TRUE(ret_code > 0); int encrypted_length = ret_code; std::unique_ptr decrypted(new char[cipher_len]); ret_code = AesUtil::decrypt(AES_128_ECB, dest.get(), encrypted_length, (unsigned char*)key.c_str(), - key.length(), NULL, true, (unsigned char*)decrypted.get()); + key.length(), nullptr, true, (unsigned char*)decrypted.get()); ASSERT_TRUE(ret_code > 0); std::string decrypted_content(decrypted.get(), ret_code); ASSERT_EQ(source, decrypted_content); @@ -68,8 +68,8 @@ TEST_F(AesUtilTest, aes_test_by_case) { int length_1 = base64_decode(case_1.c_str(), case_1.length(), encrypt_1.get()); std::unique_ptr decrypted_1(new char[case_1.length()]); int ret_code = AesUtil::decrypt(AES_128_ECB, (unsigned char*)encrypt_1.get(), length_1, - (unsigned char*)_aes_key.c_str(), _aes_key.length(), NULL, true, - (unsigned char*)decrypted_1.get()); + (unsigned char*)_aes_key.c_str(), _aes_key.length(), nullptr, + true, (unsigned char*)decrypted_1.get()); ASSERT_TRUE(ret_code > 0); std::string decrypted_content_1(decrypted_1.get(), ret_code); ASSERT_EQ(source_1, decrypted_content_1); @@ -78,7 +78,7 @@ TEST_F(AesUtilTest, aes_test_by_case) { int length_2 = base64_decode(case_2.c_str(), case_2.length(), encrypt_2.get()); std::unique_ptr decrypted_2(new char[case_2.length()]); ret_code = AesUtil::decrypt(AES_128_ECB, (unsigned char*)encrypt_2.get(), length_2, - (unsigned char*)_aes_key.c_str(), _aes_key.length(), NULL, true, + (unsigned char*)_aes_key.c_str(), _aes_key.length(), nullptr, true, (unsigned char*)decrypted_2.get()); ASSERT_TRUE(ret_code > 0); std::string decrypted_content_2(decrypted_2.get(), ret_code); diff --git a/be/test/util/arrow/arrow_work_flow_test.cpp b/be/test/util/arrow/arrow_work_flow_test.cpp index 6fd4e626e2..658a5ac3a6 100644 --- a/be/test/util/arrow/arrow_work_flow_test.cpp +++ b/be/test/util/arrow/arrow_work_flow_test.cpp @@ -21,7 +21,6 @@ #include #include -#include #include #include "common/logging.h" diff --git a/be/test/util/blocking_queue_test.cpp b/be/test/util/blocking_queue_test.cpp index 94dcd8cbf4..eb87eb02e5 100644 --- a/be/test/util/blocking_queue_test.cpp +++ b/be/test/util/blocking_queue_test.cpp @@ -21,8 +21,8 @@ #include #include -#include #include +#include namespace doris { @@ -91,16 +91,16 @@ public: void Run() { for (int i = 0; i < _nthreads; ++i) { - _threads.push_back(boost::shared_ptr( - new boost::thread(std::bind(&MultiThreadTest::inserter_thread, this, i)))); - _threads.push_back(boost::shared_ptr( - new boost::thread(std::bind(&MultiThreadTest::RemoverThread, this)))); + _threads.push_back(std::shared_ptr( + new std::thread(std::bind(&MultiThreadTest::inserter_thread, this, i)))); + _threads.push_back(std::shared_ptr( + new std::thread(std::bind(&MultiThreadTest::RemoverThread, this)))); } // We add an extra thread to ensure that there aren't enough elements in // the queue to go around. This way, we test removal after shutdown. - _threads.push_back(boost::shared_ptr( - new boost::thread(std::bind(&MultiThreadTest::RemoverThread, this)))); + _threads.push_back(std::shared_ptr( + new std::thread(std::bind(&MultiThreadTest::RemoverThread, this)))); for (int i = 0; i < _threads.size(); ++i) { _threads[i]->join(); @@ -120,7 +120,7 @@ public: } private: - typedef std::vector> ThreadVector; + typedef std::vector> ThreadVector; int _iterations; int _nthreads; diff --git a/be/test/util/decompress_test.cpp b/be/test/util/decompress_test.cpp index a5ae437e0d..a9612e94a8 100644 --- a/be/test/util/decompress_test.cpp +++ b/be/test/util/decompress_test.cpp @@ -27,7 +27,6 @@ #include "util/compress.h" using namespace std; -using namespace boost; namespace doris { @@ -49,19 +48,19 @@ protected: } void RunTest(THdfsCompression::type format) { - boost::scoped_ptr compressor; - boost::scoped_ptr decompressor; + std::unique_ptr compressor; + std::unique_ptr decompressor; MemPool* mem_pool = new MemPool; - EXPECT_TRUE(Codec::create_compressor(NULL, mem_pool, true, format, &compressor).ok()); - EXPECT_TRUE(Codec::create_compressor(NULL, mem_pool, true, format, &decompressor).ok()); + EXPECT_TRUE(Codec::create_compressor(nullptr, mem_pool, true, format, &compressor).ok()); + EXPECT_TRUE(Codec::create_compressor(nullptr, mem_pool, true, format, &decompressor).ok()); - uint8_t* compressed = NULL; + uint8_t* compressed = nullptr; int compressed_length = 0; EXPECT_TRUE( compressor->process_block(sizeof(_input), _input, &compressed_length, &compressed) .ok()); - uint8_t* output = NULL; + uint8_t* output = nullptr; int out_len = 0; EXPECT_TRUE( decompressor->process_block(compressed_length, compressed, &out_len, &output).ok()); diff --git a/be/test/util/internal_queue_test.cpp b/be/test/util/internal_queue_test.cpp index 8a237394ef..9d08847615 100644 --- a/be/test/util/internal_queue_test.cpp +++ b/be/test/util/internal_queue_test.cpp @@ -20,16 +20,16 @@ #include #include -#include #include +#include #include "common/configbase.h" #include "test_util/test_util.h" #include "util/logging.h" +#include "util/thread_group.h" using std::vector; -using boost::thread; -using boost::thread_group; +using std::thread; namespace doris { @@ -49,14 +49,14 @@ TEST(InternalQueue, TestBasic) { InternalQueue list; ASSERT_TRUE(list.empty()); ASSERT_EQ(list.size(), 0); - ASSERT_TRUE(list.dequeue() == NULL); + ASSERT_TRUE(list.dequeue() == nullptr); ASSERT_TRUE(list.validate()); list.enqueue(&one); ASSERT_TRUE(!list.empty()); ASSERT_EQ(list.size(), 1); IntNode* i = list.dequeue(); - ASSERT_TRUE(i != NULL); + ASSERT_TRUE(i != nullptr); ASSERT_TRUE(list.empty()); ASSERT_EQ(list.size(), 0); ASSERT_EQ(i->value, 1); @@ -70,22 +70,22 @@ TEST(InternalQueue, TestBasic) { ASSERT_TRUE(list.validate()); i = list.dequeue(); - ASSERT_TRUE(i != NULL); + ASSERT_TRUE(i != nullptr); ASSERT_EQ(i->value, 1); ASSERT_TRUE(list.validate()); i = list.dequeue(); - ASSERT_TRUE(i != NULL); + ASSERT_TRUE(i != nullptr); ASSERT_EQ(i->value, 2); ASSERT_TRUE(list.validate()); i = list.dequeue(); - ASSERT_TRUE(i != NULL); + ASSERT_TRUE(i != nullptr); ASSERT_EQ(i->value, 3); ASSERT_TRUE(list.validate()); i = list.dequeue(); - ASSERT_TRUE(i != NULL); + ASSERT_TRUE(i != nullptr); ASSERT_EQ(i->value, 4); ASSERT_TRUE(list.validate()); @@ -96,7 +96,7 @@ TEST(InternalQueue, TestBasic) { IntNode* node = list.head(); int val = 1; - while (node != NULL) { + while (node != nullptr) { ASSERT_EQ(node->value, val); node = node->next(); ++val; @@ -104,7 +104,7 @@ TEST(InternalQueue, TestBasic) { node = list.tail(); val = 4; - while (node != NULL) { + while (node != nullptr) { ASSERT_EQ(node->value, val); node = node->prev(); --val; @@ -112,11 +112,11 @@ TEST(InternalQueue, TestBasic) { for (int i = 0; i < 4; ++i) { node = list.pop_back(); - ASSERT_TRUE(node != NULL); + ASSERT_TRUE(node != nullptr); ASSERT_EQ(node->value, 4 - i); ASSERT_TRUE(list.validate()); } - ASSERT_TRUE(list.pop_back() == NULL); + ASSERT_TRUE(list.pop_back() == nullptr); ASSERT_EQ(list.size(), 0); ASSERT_TRUE(list.empty()); } @@ -149,7 +149,7 @@ TEST(InternalQueue, TestRemove) { ASSERT_EQ(queue.size(), nodes.size() / 2); for (int i = 0; i < nodes.size() / 2; ++i) { IntNode* node = queue.dequeue(); - ASSERT_TRUE(node != NULL); + ASSERT_TRUE(node != nullptr); ASSERT_EQ(node->value, i * 2 + 1); } } @@ -178,7 +178,7 @@ void ConsumerThread(InternalQueue* queue, int num_consumes, int delta, int previous_value = -1; for (int i = 0; i < num_consumes && !*failed;) { IntNode* node = queue->dequeue(); - if (node == NULL) { + if (node == nullptr) { continue; } ++i; @@ -277,8 +277,8 @@ TEST(InternalQueue, TestMultiProducerMultiConsumer) { } InternalQueue queue; - thread_group consumers; - thread_group producers; + ThreadGroup consumers; + ThreadGroup producers; for (int i = 0; i < num_producers; ++i) { producers.add_thread(new thread(ProducerThread, &queue, num_per_producer, &nodes, diff --git a/be/test/util/rle_encoding_test.cpp b/be/test/util/rle_encoding_test.cpp index f4495e2091..9534ea371d 100644 --- a/be/test/util/rle_encoding_test.cpp +++ b/be/test/util/rle_encoding_test.cpp @@ -30,13 +30,13 @@ #include +#include "test_util/test_util.h" #include "util/bit_stream_utils.h" #include "util/bit_stream_utils.inline.h" #include "util/bit_util.h" #include "util/debug_util.h" #include "util/faststring.h" #include "util/rle_encoding.h" -#include "test_util/test_util.h" using std::string; using std::vector; @@ -47,7 +47,7 @@ const int kMaxWidth = 64; class TestRle : public testing::Test {}; // Validates encoding of values by encoding and decoding them. If -// expected_encoding != NULL, also validates that the encoded buffer is +// expected_encoding != nullptr, also validates that the encoded buffer is // exactly 'expected_encoding'. // if expected_len is not -1, it will validate the encoded size is correct. template diff --git a/be/test/util/runtime_profile_test.cpp b/be/test/util/runtime_profile_test.cpp index 1002c3324e..76c7f28474 100644 --- a/be/test/util/runtime_profile_test.cpp +++ b/be/test/util/runtime_profile_test.cpp @@ -21,14 +21,13 @@ #include #include -#include +#include #include #include "common/object_pool.h" #include "util/cpu_info.h" using namespace std; -using namespace boost; namespace impala { @@ -54,7 +53,7 @@ TEST(CountersTest, Basic) { // Updating/setting counter counter_a = profile_a.AddCounter("A", TCounterType::UNIT); - EXPECT_TRUE(counter_a != NULL); + EXPECT_TRUE(counter_a != nullptr); counter_a->Update(10); counter_a->Update(-5); EXPECT_EQ(counter_a->value(), 5); @@ -62,14 +61,14 @@ TEST(CountersTest, Basic) { EXPECT_EQ(counter_a->value(), 1); counter_b = profile_a2.AddCounter("B", TCounterType::BYTES); - EXPECT_TRUE(counter_b != NULL); + EXPECT_TRUE(counter_b != nullptr); // Serialize/deserialize profile_a.ToThrift(&thrift_profile.nodes); RuntimeProfile* from_thrift = RuntimeProfile::CreateFromThrift(&pool, thrift_profile); counter_merged = from_thrift->GetCounter("A"); EXPECT_EQ(counter_merged->value(), 1); - EXPECT_TRUE(from_thrift->GetCounter("Not there") == NULL); + EXPECT_TRUE(from_thrift->GetCounter("Not there") == nullptr); // Merge RuntimeProfile merged_profile(&pool, "Merged"); @@ -96,7 +95,7 @@ TEST(CountersTest, Basic) { void ValidateCounter(RuntimeProfile* profile, const string& name, int64_t value) { RuntimeProfile::Counter* counter = profile->GetCounter(name); - EXPECT_TRUE(counter != NULL); + EXPECT_TRUE(counter != nullptr); EXPECT_EQ(counter->value(), value); } @@ -243,11 +242,11 @@ TEST(CountersTest, DerivedCounters) { TEST(CountersTest, InfoStringTest) { ObjectPool pool; RuntimeProfile profile(&pool, "Profile"); - EXPECT_TRUE(profile.GetInfoString("Key") == NULL); + EXPECT_TRUE(profile.GetInfoString("Key") == nullptr); profile.AddInfoString("Key", "Value"); const string* value = profile.GetInfoString("Key"); - EXPECT_TRUE(value != NULL); + EXPECT_TRUE(value != nullptr); EXPECT_EQ(*value, "Value"); // Convert it to thrift @@ -257,14 +256,14 @@ TEST(CountersTest, InfoStringTest) { // Convert it back RuntimeProfile* from_thrift = RuntimeProfile::CreateFromThrift(&pool, tprofile); value = from_thrift->GetInfoString("Key"); - EXPECT_TRUE(value != NULL); + EXPECT_TRUE(value != nullptr); EXPECT_EQ(*value, "Value"); // Test update. RuntimeProfile update_dst_profile(&pool, "Profile2"); update_dst_profile.Update(tprofile); value = update_dst_profile.GetInfoString("Key"); - EXPECT_TRUE(value != NULL); + EXPECT_TRUE(value != nullptr); EXPECT_EQ(*value, "Value"); // Update the original profile, convert it to thrift and update from the dst diff --git a/be/test/util/string_parser_test.cpp b/be/test/util/string_parser_test.cpp index 01f2d06d63..4503ecf61c 100644 --- a/be/test/util/string_parser_test.cpp +++ b/be/test/util/string_parser_test.cpp @@ -97,7 +97,7 @@ void test_float_value(const std::string& s, StringParser::ParseResult exp_result EXPECT_EQ(exp_result, result); if (exp_result == StringParser::PARSE_SUCCESS && result == exp_result) { - T exp_val = strtod(s.c_str(), NULL); + T exp_val = strtod(s.c_str(), nullptr); EXPECT_EQ(exp_val, val); } } diff --git a/be/test/util/thread_group_test.cpp b/be/test/util/thread_group_test.cpp new file mode 100644 index 0000000000..dbc8902947 --- /dev/null +++ b/be/test/util/thread_group_test.cpp @@ -0,0 +1,81 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "util/thread_group.h" + +#include +#include +#include + +#include +#include +#include + +#include "common/logging.h" +#include "common/status.h" +#include "gutil/basictypes.h" +#include "gutil/ref_counted.h" +#include "util/countdown_latch.h" +#include "util/runtime_profile.h" + +using std::string; + +namespace doris { + +class ThreadGroupTest : public ::testing::Test { +public: + virtual void SetUp() {} + virtual void TearDown() {} + void increment_count() { + std::unique_lock lock(mutex); + ++count; + } + int count = 0; + std::mutex mutex; +}; + +TEST_F(ThreadGroupTest, TestJoinALL) { + ThreadGroup threads; + for (int i = 0; i < 10; ++i) { + threads.create_thread(&increment_count); + } + threads.join_all(); + EXPECT_EQ(10, count); +} + +TEST_F(ThreadGroupTest, TestThreadIn) { + ThreadGroup threads; + std::thread* th = new std::thread(&increment_count); + threads.add_thread(th); + ASSERT_FALSE(threads.is_this_thread_in()); + threads.join_all(); + ThreadGroup threads2; + std::thread* th2 = new std::thread(&increment_count); + threads2.add_thread(th2); + ASSERT_TRUE(threads2.is_thread_in(th2)); + threads2.remove_thread(th2); + ASSERT_FALSE(threads2.is_thread_in(th2)); + th2->join(); + delete th2; +} + +} // namespace doris + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/util/thread_pool_test.cpp b/be/test/util/thread_pool_test.cpp index fbdbaa698d..0ca9348c45 100644 --- a/be/test/util/thread_pool_test.cpp +++ b/be/test/util/thread_pool_test.cpp @@ -21,8 +21,8 @@ #include #include -#include #include +#include #include "util/logging.h" diff --git a/contrib/udf/src/udaf_orthogonal_bitmap/bitmap_value.h b/contrib/udf/src/udaf_orthogonal_bitmap/bitmap_value.h index c5e3e8e187..fc1389803d 100644 --- a/contrib/udf/src/udaf_orthogonal_bitmap/bitmap_value.h +++ b/contrib/udf/src/udaf_orthogonal_bitmap/bitmap_value.h @@ -178,7 +178,7 @@ public: /** * Construct a roaring object from the C struct. * - * Passing a NULL point is unsafe. + * Passing a nullptr point is unsafe. */ Roaring64Map(roaring_bitmap_t* s) { emplaceOrInsert(0, s); } @@ -413,14 +413,14 @@ public: // we put std::numeric_limits<>::max/min in parenthesis // to avoid a clash with the Windows.h header under Windows return roarings.size() == ((size_t)(std::numeric_limits::max)()) + 1 - ? std::all_of(roarings.cbegin(), roarings.cend(), - [](const std::pair& roaring_map_entry) { - // roarings within map are saturated if cardinality - // is uint32_t max + 1 - return roaring_map_entry.second.cardinality() == - ((uint64_t)(std::numeric_limits::max)()) + - 1; - }) + ? std::all_of( + roarings.cbegin(), roarings.cend(), + [](const std::pair& roaring_map_entry) { + // roarings within map are saturated if cardinality + // is uint32_t max + 1 + return roaring_map_entry.second.cardinality() == + ((uint64_t)(std::numeric_limits::max)()) + 1; + }) : false; } @@ -591,7 +591,7 @@ public: /** * Iterate over the bitmap elements. The function iterator is called once - * for all the values with ptr (can be NULL) as the second parameter of each + * for all the values with ptr (can be nullptr) as the second parameter of each * call. * * roaring_iterator is simply a pointer to a function that returns bool @@ -729,11 +729,12 @@ public: } // start with type code, map size and size of keys for each map entry size_t init = 1 + varint_length(roarings.size()) + roarings.size() * sizeof(uint32_t); - return std::accumulate(roarings.cbegin(), roarings.cend(), init, - [=](size_t previous, const std::pair& map_entry) { - // add in bytes used by each Roaring - return previous + map_entry.second.getSizeInBytes(); - }); + return std::accumulate( + roarings.cbegin(), roarings.cend(), init, + [=](size_t previous, const std::pair& map_entry) { + // add in bytes used by each Roaring + return previous + map_entry.second.getSizeInBytes(); + }); } /** @@ -803,16 +804,17 @@ public: return true; }, (void*)&outer_iter_data); - std::for_each( - ++map_iter, roarings.cend(), [](const std::pair& map_entry) { - map_entry.second.iterate( - [](uint32_t low_bits, void* high_bits) -> bool { - std::printf(",%llu", (long long unsigned)uniteBytes( - *(uint32_t*)high_bits, low_bits)); - return true; - }, - (void*)&map_entry.first); - }); + std::for_each(++map_iter, roarings.cend(), + [](const std::pair& map_entry) { + map_entry.second.iterate( + [](uint32_t low_bits, void* high_bits) -> bool { + std::printf(",%llu", + (long long unsigned)uniteBytes( + *(uint32_t*)high_bits, low_bits)); + return true; + }, + (void*)&map_entry.first); + }); } else std::printf("{"); std::printf("}\n"); diff --git a/contrib/udf/src/udaf_orthogonal_bitmap/string_value.h b/contrib/udf/src/udaf_orthogonal_bitmap/string_value.h index 860ccadb29..1443c223fa 100644 --- a/contrib/udf/src/udaf_orthogonal_bitmap/string_value.h +++ b/contrib/udf/src/udaf_orthogonal_bitmap/string_value.h @@ -19,6 +19,7 @@ #define DORIS_CONTRIB_UDF_SRC_UDAF_BITMAP_STRING_VALUE_H #include + #include "udf.h" namespace doris_udf { @@ -37,14 +38,12 @@ struct StringValue { char* ptr; size_t len; - StringValue(char* ptr, int len): ptr(ptr), len(len) {} - StringValue(): ptr(NULL), len(0) {} + StringValue(char* ptr, int len) : ptr(ptr), len(len) {} + StringValue() : ptr(nullptr), len(0) {} /// Construct a StringValue from 's'. 's' must be valid for as long as /// this object is valid. - explicit StringValue(const std::string& s) : - ptr(const_cast(s.c_str())), len(s.size()) { - } + explicit StringValue(const std::string& s) : ptr(const_cast(s.c_str())), len(s.size()) {} void replace(char* ptr, int len) { this->ptr = ptr; @@ -84,57 +83,31 @@ struct StringValue { return string_compare(this->ptr, this->len, other.ptr, other.len, this->len) == 0; } - bool operator==(const StringValue& other) const { - return eq(other); - } + bool operator==(const StringValue& other) const { return eq(other); } // != - bool ne(const StringValue& other) const { - return !eq(other); - } + bool ne(const StringValue& other) const { return !eq(other); } // <= - bool le(const StringValue& other) const { - return compare(other) <= 0; - } + bool le(const StringValue& other) const { return compare(other) <= 0; } // >= - bool ge(const StringValue& other) const { - return compare(other) >= 0; - } + bool ge(const StringValue& other) const { return compare(other) >= 0; } // < - bool lt(const StringValue& other) const { - return compare(other) < 0; - } + bool lt(const StringValue& other) const { return compare(other) < 0; } // > - bool gt(const StringValue& other) const { - return compare(other) > 0; - } + bool gt(const StringValue& other) const { return compare(other) > 0; } - bool operator!=(const StringValue& other) const { - return ne(other); - } + bool operator!=(const StringValue& other) const { return ne(other); } - bool operator<=(const StringValue& other) const { - return le(other); - } + bool operator<=(const StringValue& other) const { return le(other); } - bool operator>=(const StringValue& other) const { - return ge(other); - } + bool operator>=(const StringValue& other) const { return ge(other); } - bool operator<(const StringValue& other) const { - return lt(other); - } + bool operator<(const StringValue& other) const { return lt(other); } - bool operator>(const StringValue& other) const { - return gt(other); - } + bool operator>(const StringValue& other) const { return gt(other); } - std::string debug_string() const { - return std::string(ptr, len); - } + std::string debug_string() const { return std::string(ptr, len); } - std::string to_string() const { - return std::string(ptr, len); - } + std::string to_string() const { return std::string(ptr, len); } // Returns the substring starting at start_pos until the end of string. StringValue substring(int start_pos) const { @@ -170,6 +143,6 @@ struct StringValue { } }; -} +} // namespace doris_udf #endif diff --git a/samples/connect/cpp/doris_client.cpp b/samples/connect/cpp/doris_client.cpp index e3306da13c..677b565693 100644 --- a/samples/connect/cpp/doris_client.cpp +++ b/samples/connect/cpp/doris_client.cpp @@ -37,25 +37,25 @@ using std::string; DorisClient::DorisClient() { //init connection - _client = mysql_init(NULL); - if (_client == NULL) { + _client = mysql_init(nullptr); + if (_client == nullptr) { std::cout << "Error:" << mysql_error(_client); } } DorisClient::~DorisClient() { //close connection - if (_client != NULL) { + if (_client != nullptr) { mysql_close(_client); } } bool DorisClient::init(const string& host, const string& user, const string& passwd, - const string& db_name, int port, const string& sock) { + const string& db_name, int port, const string& sock) { // create connection _client = mysql_real_connect(_client, host.c_str(), user.c_str(), passwd.c_str(), - db_name.c_str(), port, sock.c_str(), 0); - if (_client == NULL) { + db_name.c_str(), port, sock.c_str(), 0); + if (_client == nullptr) { std::cout << "Error: " << mysql_error(_client); return false; } @@ -129,9 +129,10 @@ int main() { std::cout << "init new DorisClient" << std::endl; // create doris table - string sql_create_table = "CREATE TABLE cpp_doris_table(siteid INT,citycode SMALLINT,pv BIGINT SUM) "\ - "AGGREGATE KEY(siteid, citycode) DISTRIBUTED BY HASH(siteid) BUCKETS 10 "\ - "PROPERTIES(\"replication_num\" = \"1\")"; + string sql_create_table = + "CREATE TABLE cpp_doris_table(siteid INT,citycode SMALLINT,pv BIGINT SUM) " + "AGGREGATE KEY(siteid, citycode) DISTRIBUTED BY HASH(siteid) BUCKETS 10 " + "PROPERTIES(\"replication_num\" = \"1\")"; std::cout << sql_create_table << std::endl; client_new.exec(sql_create_table); diff --git a/thirdparty/download-thirdparty.sh b/thirdparty/download-thirdparty.sh index bfe20ac37d..97485806d0 100755 --- a/thirdparty/download-thirdparty.sh +++ b/thirdparty/download-thirdparty.sh @@ -267,6 +267,7 @@ if [ ! -f $PATCHED_MARK ]; then touch $PATCHED_MARK fi cd - +echo "Finished patching $HDFS3_SOURCE" # aws-c-cal patch to fix compile error # This bug has been fixed in new version of aws-c-cal @@ -278,6 +279,5 @@ if [ $AWS_C_CAL_SOURCE == "aws-c-cal-0.4.5" ]; then fi cd - fi - -echo "Finished patching $HDFS3_SOURCE" +echo "Finished patching $AWS_C_CAL_SOURCE" diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh index ea34e57b9d..48baabf080 100644 --- a/thirdparty/vars.sh +++ b/thirdparty/vars.sh @@ -431,4 +431,3 @@ LIBDIVIDE PDQSORT BENCHMARK" -