You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by ya...@apache.org on 2021/11/17 02:18:48 UTC

[incubator-doris] branch master updated: [refactor] replace boost smart ptr with stl (#6856)

This is an automated email from the ASF dual-hosted git repository.

yangzhg pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 6c63809  [refactor] replace boost smart ptr with stl  (#6856)
6c63809 is described below

commit 6c6380969b40526600c6fd49db86213b166f5fd6
Author: Zhengguo Yang <ya...@gmail.com>
AuthorDate: Wed Nov 17 10:18:35 2021 +0800

    [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<T[]>
    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 +++--
 be/src/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 +-
 .../exec/schema_scanner/schema_columns_scanner.cpp |  32 +--
 be/src/exec/schema_scanner/schema_helper.cpp       |   2 +-
 .../schema_schema_privileges_scanner.cpp           |  20 +-
 .../schema_scanner/schema_schemata_scanner.cpp     |  18 +-
 .../schema_table_privileges_scanner.cpp            |  20 +-
 .../exec/schema_scanner/schema_tables_scanner.cpp  |  34 ++--
 .../schema_user_privileges_scanner.cpp             |  20 +-
 .../exec/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               |  48 ++---
 be/src/olap/rowset/segment_reader.cpp              |  50 ++---
 be/src/olap/rowset/segment_v2/column_reader.cpp    |   2 +-
 be/src/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 ++-
 be/src/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 ++--
 be/src/runtime/bufferpool/buffer_pool_internal.h   |   8 +-
 be/src/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_scanner/schema_authors_scanner_test.cpp |  10 +-
 .../schema_charsets_scanner_test.cpp               |  10 +-
 .../schema_collations_scanner_test.cpp             |  10 +-
 .../schema_scanner/schema_columns_scanner_test.cpp |  12 +-
 .../schema_create_table_scanner_test.cpp           |  12 +-
 .../schema_scanner/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_scanner/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 +-
 .../olap/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 ++++++++++-----------
 be/test/runtime/buffered_tuple_stream2_test.cpp    |  49 +++--
 be/test/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 +-
 .../udf/src/udaf_orthogonal_bitmap/bitmap_value.h  |  52 ++---
 .../udf/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, 3442 insertions(+), 3395 deletions(-)

diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index 9c3631c..d88c9b5 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 5c178a7..764b08a 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 4fba068..f122d67 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<TTabletInfo>* 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<TTabletInfo>* 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 1fb5bb3..e02668c 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<TTaskType::type, set<int64_t>> 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 6048c99..2f5ed7f 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 <boost/thread.hpp>
 #include <map>
 #include <shared_mutex>
+#include <thread>
 
 #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 4ad4de2..df81d26 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 49684ac..cb2d748 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 d021573..0b8eac7 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 df3bc53..3cadea4 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<TupleRow*>(&_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 464d364..a45236e 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 <boost/scoped_ptr.hpp>
 #include <functional>
 
 #include "exec/exec_node.h"
@@ -64,14 +63,14 @@ public:
     virtual void push_down_predicate(RuntimeState* state, std::list<ExprContext*>* expr_ctxs);
 
 private:
-    boost::scoped_ptr<HashTable> _hash_tbl;
+    std::unique_ptr<HashTable> _hash_tbl;
     HashTable::Iterator _output_iterator;
 
     std::vector<AggFnEvaluator*> _aggregate_evaluators;
 
     /// FunctionContext for each agg fn and backing pool.
     std::vector<doris_udf::FunctionContext*> _agg_fn_ctxs;
-    boost::scoped_ptr<MemPool> _agg_fn_pool;
+    std::unique_ptr<MemPool> _agg_fn_pool;
 
     // Exprs used to evaluate input rows
     std::vector<ExprContext*> _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<MemPool> _tuple_pool;
+    std::unique_ptr<MemPool> _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 0d17441..c105760 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 65867b5..df1b4ce 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<TTupleId> 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<bool>(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 8c6d7ad..0c3863e 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<MemPool> _curr_tuple_pool;
-    boost::scoped_ptr<MemPool> _prev_tuple_pool;
+    std::unique_ptr<MemPool> _curr_tuple_pool;
+    std::unique_ptr<MemPool> _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<RowBatch> _prev_child_batch;
-    boost::scoped_ptr<RowBatch> _curr_child_batch;
+    std::unique_ptr<RowBatch> _prev_child_batch;
+    std::unique_ptr<RowBatch> _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<BufferedTupleStream2> _input_stream;
+    std::unique_ptr<BufferedTupleStream2> _input_stream;
 
     // Pool used for O(1) allocations that live until close.
-    boost::scoped_ptr<MemPool> _mem_pool;
+    std::unique_ptr<MemPool> _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 a82e5f7..da97701 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>* status) {
+void BlockingJoinNode::build_side_thread(RuntimeState* state, std::promise<Status>* 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<Status> build_side_status;
+    std::promise<Status> 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<uint8_t*>(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 f0406d3..f5ae609 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 <boost/scoped_ptr.hpp>
-#include <boost/thread.hpp>
+#include <future>
 #include <string>
+#include <thread>
 
 #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<MemPool> _build_pool; // holds everything referenced from build side
+    bool _eos;                            // if true, nothing left to return in get_next()
+    std::unique_ptr<MemPool> _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<RowBatch> _left_batch;
+    std::unique_ptr<RowBatch> _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>* status);
+    void build_side_thread(RuntimeState* state, std::promise<Status>* status);
 };
 
 } // namespace doris
diff --git a/be/src/exec/broker_scan_node.h b/be/src/exec/broker_scan_node.h
index 6c4f40e..a05e7c5 100644
--- a/be/src/exec/broker_scan_node.h
+++ b/be/src/exec/broker_scan_node.h
@@ -19,6 +19,7 @@
 
 #include <atomic>
 #include <condition_variable>
+#include <deque>
 #include <map>
 #include <mutex>
 #include <string>
@@ -84,8 +85,7 @@ private:
     // Scan one range
     Status scanner_scan(const TBrokerScanRange& scan_range,
                         const std::vector<ExprContext*>& pre_filter_ctxs,
-                        const std::vector<ExprContext*>& conjunct_ctxs,
-                        ScannerCounter* counter);
+                        const std::vector<ExprContext*>& conjunct_ctxs, ScannerCounter* counter);
 
     std::unique_ptr<BaseScanner> create_scanner(const TBrokerScanRange& scan_range,
                                                 const std::vector<ExprContext*>& pre_filter_ctxs,
diff --git a/be/src/exec/broker_scanner.cpp b/be/src/exec/broker_scanner.cpp
index e99bf7e..b265eb0 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 76bca88..6200e05 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 <boost/scoped_ptr.hpp>
-#include <boost/thread.hpp>
 #include <string>
+#include <thread>
 #include <unordered_set>
 
 #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<ObjectPool> _build_batch_pool;
+    std::unique_ptr<ObjectPool> _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 5f830e8..fe92247 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 <boost/scoped_ptr.hpp>
 #include <fstream>
 #include <sstream>
 
@@ -110,11 +109,11 @@ private:
     int _slot_num;
 
     // Pool for allocating tuple data, including all varying-length slots.
-    boost::scoped_ptr<MemPool> _tuple_pool;
+    std::unique_ptr<MemPool> _tuple_pool;
     // Util class for doing real file reading
-    boost::scoped_ptr<CsvScanner> _csv_scanner;
+    std::unique_ptr<CsvScanner> _csv_scanner;
     // Helper class for converting text to other types;
-    boost::scoped_ptr<TextConverter> _text_converter;
+    std::unique_ptr<TextConverter> _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 7b2e336..aefa087 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<TExpr>& output_exprs,
                                   const TPlanFragmentExecParams& params,
-                                  const RowDescriptor& row_desc,
-                                  bool is_vec,
-                                  boost::scoped_ptr<DataSink>* sink,
-                                  DescriptorTbl& desc_tbl) {
-    DataSink* tmp_sink = NULL;
+                                  const RowDescriptor& row_desc, bool is_vec,
+                                  std::unique_ptr<DataSink>* 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 8d1bb64..30f5580 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 <boost/scoped_ptr.hpp>
 #include <vector>
 
 #include "common/status.h"
@@ -74,10 +73,8 @@ public:
     static Status create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink,
                                    const std::vector<TExpr>& output_exprs,
                                    const TPlanFragmentExecParams& params,
-                                   const RowDescriptor& row_desc,
-                                   bool is_vec,
-                                   boost::scoped_ptr<DataSink>* sink,
-                                   DescriptorTbl& desc_tbl);
+                                   const RowDescriptor& row_desc, bool is_vec,
+                                   std::unique_ptr<DataSink>* 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 fe805a9..2f215f4 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 170c0e9..63add75 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<ExtLiteral> query_conditions;
             query_conditions.emplace_back(literal);
             std::vector<ExtColumnDesc> 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 44ca493..396c086 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<char*>(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<StringValue*>(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<char*>(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<StringValue*>(slot)->ptr = buffer;
diff --git a/be/src/exec/es_http_scan_node.cpp b/be/src/exec/es_http_scan_node.cpp
index 7a78787..012110d 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<bool*>(value) == false) {
+            void* value = _conjunct_ctxs[conj_idx]->get_value(nullptr);
+            if (value == nullptr || *reinterpret_cast<bool*>(value) == false) {
                 _eos = true;
             }
         }
diff --git a/be/src/exec/es_scan_node.cpp b/be/src/exec/es_scan_node.cpp
index b40ad09..fad2669 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<char*>(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<StringValue*>(slot)->ptr = buffer;
diff --git a/be/src/exec/exchange_node.cpp b/be/src/exec/exchange_node.cpp
index 14386e8..14299c6 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 <boost/scoped_ptr.hpp>
-
 #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<bool>(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 7551d88..6077dd4 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 <boost/scoped_ptr.hpp>
-
 #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<DataStreamRecvr> _stream_recvr;
+    std::shared_ptr<DataStreamRecvr> _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<RowBatch> _input_batch;
+    // std::unique_ptr<RowBatch> _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 18b5af1..2e02498 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 fd6a7a1..63f048e 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<RuntimeProfile> _runtime_profile;
+    std::unique_ptr<RuntimeProfile> _runtime_profile;
 
     /// Account for peak memory used by this node
     std::shared_ptr<MemTracker> _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<MemPool> _expr_mem_pool;
+    std::unique_ptr<MemPool> _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 447b3f5..bcb7280 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<uint8_t[]>* buf, int64_t* length) = 0;
diff --git a/be/src/exec/hash_join_node.cpp b/be/src/exec/hash_join_node.cpp
index b1b9356..551c4b2 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<TEqJoinCondition>& 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>* status) {
+void HashJoinNode::build_side_thread(RuntimeState* state, std::promise<Status>* 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<Status> thread_status;
+    std::promise<Status> 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<uint8_t*>(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 f097a2c..8c914fa 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 <boost/scoped_ptr.hpp>
-#include <boost/thread.hpp>
+#include <future>
 #include <string>
+#include <thread>
 #include <unordered_set>
 
 #include "exec/exec_node.h"
@@ -65,7 +65,7 @@ protected:
 private:
     friend class IRuntimeFilter;
 
-    boost::scoped_ptr<HashTable> _hash_tbl;
+    std::unique_ptr<HashTable> _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<MemPool> _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<MemPool> _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<RowBatch> _probe_batch;
+    std::unique_ptr<RowBatch> _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>* status);
+    void build_side_thread(RuntimeState* state, std::promise<Status>* 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 b512f30..b31f796 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 867f513..9275b18 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<ExprContext*>& 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<ExprContext*>& 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<ExprContext*>& 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 ee74d8f..d150bb5 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 <boost/cstdint.hpp>
 #include <vector>
 
 #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 <bool check_match>
         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<ExprContext*>& exprs);
 
diff --git a/be/src/exec/merge_join_node.cpp b/be/src/exec/merge_join_node.cpp
index f972ab5..d83e872 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<TEqJoinCondition>& 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 5b15eef..d8b294e 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 <boost/scoped_ptr.hpp>
-#include <boost/thread.hpp>
 #include <string>
+#include <thread>
 #include <unordered_set>
 
 #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<ChildReaderContext> _left_child_ctx;
-    boost::scoped_ptr<ChildReaderContext> _right_child_ctx;
+    std::unique_ptr<ChildReaderContext> _left_child_ctx;
+    std::unique_ptr<ChildReaderContext> _right_child_ctx;
     // _build_tuple_idx[i] is the tuple index of child(1)'s tuple[i] in the output row
     std::vector<int> _right_tuple_idx;
     int _right_tuple_size;
diff --git a/be/src/exec/merge_node.cpp b/be/src/exec/merge_node.cpp
index 13e7d2a..513284e 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<ExprContext*>& 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 7344dab..a81ece1 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 <boost/scoped_ptr.hpp>
-
 #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<RowBatch> _child_row_batch;
+    std::unique_ptr<RowBatch> _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 5a78747..634f47c 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<const MySQLTableDescriptor*>(_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 26718f1..a6be664 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 7b50a61..bc2d787 100644
--- a/be/src/exec/odbc_connector.cpp
+++ b/be/src/exec/odbc_connector.cpp
@@ -19,7 +19,6 @@
 
 #include <sqlext.h>
 
-#include <boost/algorithm/string.hpp>
 #include <codecvt>
 
 #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<SQLCHAR*>(message),
                          (SQLSMALLINT)(sizeof(message) / sizeof(WCHAR)),
-                         (SQLSMALLINT*)NULL) == SQL_SUCCESS) {
+                         (SQLSMALLINT*)nullptr) == SQL_SUCCESS) {
         // Hide data truncated..
         if (wcsncmp(reinterpret_cast<const wchar_t*>(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 478c3f0..c5d9016 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 <fmt/format.h>
 #include <sql.h>
 
 #include <boost/format.hpp>
-#include <boost/ptr_container/ptr_vector.hpp>
 #include <cstdlib>
-#include <fmt/format.h>
 #include <string>
 #include <vector>
 
-#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<DataBinding> _columns_data;
+    std::vector<std::unique_ptr<DataBinding>> _columns_data;
 };
 
 } // namespace doris
diff --git a/be/src/exec/odbc_scan_node.cpp b/be/src/exec/odbc_scan_node.cpp
index 0921134..6bdc620 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 9157e1b..1cb610d 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<std::mutex> 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<bool*>(value) == false) {
+            void* value = _conjunct_ctxs[conj_idx]->get_value(nullptr);
+            if (value == nullptr || *reinterpret_cast<bool*>(value) == false) {
                 _eos = true;
                 break;
             }
@@ -666,7 +666,7 @@ Status OlapScanNode::get_hints(const TPaloScanRange& scan_range, int block_row_c
                                std::vector<std::unique_ptr<OlapScanRange>>* 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<bool, void*> 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<void*>(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<void*>(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<std::mutex> 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 a510a17..129bd2c 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 0715d02..689fe0a 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 b2d9db1..54081e7 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<false>(&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<StringValue*>(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<char*>(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<NewAggFnEvaluator*>
     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*>(tuple_data);
@@ -946,7 +946,7 @@ Tuple* PartitionedAggregationNode::ConstructIntermediateTuple(
 Tuple* PartitionedAggregationNode::ConstructIntermediateTuple(
         const vector<NewAggFnEvaluator*>& 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<NewAggFnEvaluator*>&
             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<NewAggFnEvaluator*>&
 
 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<NewAggFnEvaluator*>& 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 vector<NewAggFnEvaluator
             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;
@@ -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 eb83580..ff0d823 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 <boost/scoped_ptr.hpp>
 #include <deque>
 
 #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<NewAggFnEvaluator*> agg_fn_evals_;
-    boost::scoped_ptr<MemPool> agg_fn_pool_;
+    std::unique_ptr<MemPool> agg_fn_pool_;
 
     /// Exprs used to evaluate input rows
     std::vector<Expr*> grouping_exprs_;
@@ -227,16 +226,16 @@ private:
 
     RuntimeState* state_;
     /// Allocator for hash table memory.
-    boost::scoped_ptr<Suballocator> ht_allocator_;
+    std::unique_ptr<Suballocator> 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<MemPool> mem_pool_;
+    std::unique_ptr<MemPool> mem_pool_;
 
     // MemPool for allocations made by copying expr results
-    boost::scoped_ptr<MemPool> expr_results_pool_;
+    std::unique_ptr<MemPool> 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<RowBatch> child_batch_;
+    std::unique_ptr<RowBatch> 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<PartitionedHashTableCtx> ht_ctx_;
+    std::unique_ptr<PartitionedHashTableCtx> ht_ctx_;
 
     /// Object pool that holds the Partition objects in hash_partitions_.
-    boost::scoped_ptr<ObjectPool> partition_pool_;
+    std::unique_ptr<ObjectPool> 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<PartitionedHashTable> hash_tbl;
+        std::unique_ptr<PartitionedHashTable> 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<NewAggFnEvaluator*> agg_fn_evals;
-        boost::scoped_ptr<MemPool> agg_fn_pool;
+        std::unique_ptr<MemPool> 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<BufferedTupleStream3> aggregated_row_stream;
+        std::unique_ptr<BufferedTupleStream3> 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<BufferedTupleStream3> unaggregated_row_stream;
+        std::unique_ptr<BufferedTupleStream3> 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<BufferedTupleStream3> serialize_stream_;
+    std::unique_ptr<BufferedTupleStream3> 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<NewAggFnEvaluator*>& 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 6d45a2c..ae62c64 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<false>(hash);
+            if (LIKELY(hash_tbl != nullptr)) hash_tbl->PrefetchBucket<false>(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<AGGREGATED_ROWS>(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 b6801a4..0f6092e 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<Expr*>& build
           finds_some_nulls_(std::accumulate(finds_nulls_.begin(), finds_nulls_.end(), false,
                                             std::logical_or<bool>())),
           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<TupleRow*>(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<bool>& finds_nulls, int32_t initial_seed, int max_levels,
         int num_build_tuples, MemPool* mem_pool, MemPool* expr_results_pool,
         const std::shared_ptr<MemTracker>& tracker, const RowDescriptor& row_desc,
-        const RowDescriptor& row_desc_probe, boost::scoped_ptr<PartitionedHashTableCtx>* ht_ctx) {
+        const RowDescriptor& row_desc_probe, std::unique_ptr<PartitionedHashTableCtx>* 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<ExprContext*>&
     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<void*>(&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<false>(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<MemTracker>& 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<Suballocation> 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<true>(new_buckets, num_buckets, NULL, bucket_to_copy->hash, &found);
+                Probe<true>(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<TupleRow*>(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 8cfa4a5..7dd4dc4 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 <boost/cstdint.hpp>
-#include <boost/scoped_ptr.hpp>
 #include <memory>
 #include <vector>
 
@@ -117,7 +115,7 @@ public:
                          int num_build_tuples, MemPool* mem_pool, MemPool* expr_results_pool,
                          const std::shared_ptr<MemTracker>& tracker, const RowDescriptor& row_desc,
                          const RowDescriptor& row_desc_probe,
-                         boost::scoped_ptr<PartitionedHashTableCtx>* ht_ctx);
+                         std::unique_ptr<PartitionedHashTableCtx>* 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<bool>(*(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<uint8_t> expr_values_array_;
+        std::unique_ptr<uint8_t[]> 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<uint8_t> expr_values_null_array_;
+        std::unique_ptr<uint8_t[]> expr_values_null_array_;
 
         /// Array for caching up to 'capacity_' number of rows worth of hashed values.
-        boost::scoped_array<uint32_t> expr_values_hash_array_;
+        std::unique_ptr<uint32_t[]> 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<ExprContext*>& 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 0432ccb..32a5562 100644
--- a/be/src/exec/partitioned_hash_table.inline.h
+++ b/be/src/exec/partitioned_hash_table.inline.h
@@ -53,7 +53,7 @@ template <bool FORCE_NULL_EQUALITY>
 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<FORCE_NULL_EQUALITY>(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<false>(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<true>(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 <const bool READ>
@@ -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 fef20d5..4b1b0ae 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 c13092f..c52fede 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 <boost/cstdint.hpp>
 #include <sstream>
 
 #include "common/logging.h"
diff --git a/be/src/exec/repeat_node.cpp b/be/src/exec/repeat_node.cpp
index 9b07340..78d937e 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 706d813..c5503c4 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 1676ba5..0de0b09 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<uint8_t[]>* buf, int64_t* length) override;
diff --git a/be/src/exec/scanner_ir.cpp b/be/src/exec/scanner_ir.cpp
index 6aea5f6..e14e574 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 1a6352c..4a85c0f 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<char*>(_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<const SchemaTableDescriptor*>(_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<Tuple*>(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 3bbc804..2052de0 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 <boost/scoped_ptr.hpp>
-
 #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<MemPool> _tuple_pool;
+    std::unique_ptr<MemPool> _tuple_pool;
     // Jni helper for scanning an schema table.
-    boost::scoped_ptr<SchemaScanner> _schema_scanner;
+    std::unique_ptr<SchemaScanner> _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 ef9ded8..20d796f 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 74d0a92..c6311dd 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 2b8ce49..390a0af 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 b0f9c08..f59cc00 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 <boost/thread/thread.hpp>
 #include <sstream>
+#include <thread>
 
 #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 d0e665e..f675691 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<StringValue*>(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 5f9ae4e..d4a69b3 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<StringValue*>(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<StringValue*>(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 993c1bb..132114b 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<StringValue*>(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 96e95a5..078ac3f 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 9436bc9..1d931c8 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<StringValue*>(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 f626584..52d7148 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 1c8585d..2505768 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 1d6c468..91c723a 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 <boost/scoped_ptr.hpp>
-
 #include "exec/exec_node.h"
 #include "runtime/mem_pool.h"
 
@@ -41,7 +39,7 @@ public:
 
 private:
     // current row batch of child
-    boost::scoped_ptr<RowBatch> _child_row_batch;
+    std::unique_ptr<RowBatch> _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 523efe0..0ca6dd5 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 35f8e63..7eceb18 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<TExpr>& ordering_exprs,
                            const std::vector<TExpr>* 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 03a6ea1..898fd0b 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<TExpr>& ordering_exprs,
                 const std::vector<TExpr>* 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 3003058..ef527a1 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 c33c70e..954df09 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<SpillSorter> _sorter;
+    std::unique_ptr<SpillSorter> _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 9b04f06..7d470bf 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 b47ab63..7e98e1d 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<false>(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<false>(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 74a7dd0..5c29147 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 <boost/scoped_ptr.hpp>
 #include <queue>
 
 #include "exec/exec_node.h"
@@ -75,7 +74,7 @@ private:
     TupleDescriptor* _materialized_tuple_desc;
 
     // Comparator for _priority_queue.
-    boost::scoped_ptr<TupleRowComparator> _tuple_row_less_than;
+    std::unique_ptr<TupleRowComparator> _tuple_row_less_than;
 
     // After computing the TopN in the priority_queue, pop them and put them in this vector
     std::vector<Tuple*> _sorted_top_n;
@@ -86,7 +85,7 @@ private:
     Tuple* _tmp_tuple;
 
     // Stores everything referenced in _priority_queue
-    boost::scoped_ptr<MemPool> _tuple_pool;
+    std::unique_ptr<MemPool> _tuple_pool;
 
     // Iterator over elements in _sorted_top_n.
     std::vector<Tuple*>::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<SortingHeap<Tuple*, std::vector<Tuple*>, 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 756ff87..5b49f71 100644
--- a/be/src/exec/union_node.h
+++ b/be/src/exec/union_node.h
@@ -17,8 +17,6 @@
 
 #pragma once
 
-#include <boost/scoped_ptr.hpp>
-
 #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<RowBatch> _child_batch;
+    std::unique_ptr<RowBatch> _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<UnionMaterializeBatchFn> _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 4c2a9c4..aa15a67 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 f8a3d60..bcb7e35 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<MemTracker>& 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<FunctionContext::TypeDesc> 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<uint64_t>(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<InitFn>(_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<AggFnEvaluator*>& exprs) {
diff --git a/be/src/exprs/agg_fn_evaluator.h b/be/src/exprs/agg_fn_evaluator.h
index 2ed950d..e1af158 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 <boost/scoped_array.hpp>
-#include <boost/scoped_ptr.hpp>
 #include <string>
 #include <vector>
 
@@ -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<AggFnEvaluator*>& 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<HybridMap> _hybrid_map;
+    std::unique_ptr<HybridMap> _hybrid_map;
     bool _is_multi_distinct;
     std::vector<ExprContext*> _input_exprs_ctxs;
-    boost::scoped_array<char> _string_buffer; //for count distinct
+    std::unique_ptr<char[]> _string_buffer;   //for count distinct
     int _string_buffer_len;                   //for count distinct
     std::shared_ptr<MemTracker> _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<doris_udf::FunctionContext> _ctx;
+    // std::unique_ptr<doris_udf::FunctionContext> _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 09295a0..231a0fd 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 <typename T>
 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<PercentileApproxState*>(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<PercentileApproxState*>(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<AvgState*>(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<DecimalV2AvgState*>(dst->ptr);
 
@@ -415,11 +416,11 @@ StringVal AggregateFunctions::decimalv2_avg_serialize(FunctionContext* ctx, cons
 template <typename T>
 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<AvgState*>(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<DecimalV2AvgState*>(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<const AvgState*>(src.ptr);
-    DCHECK(dst->ptr != NULL);
+    DCHECK(dst->ptr != nullptr);
     DCHECK_EQ(sizeof(AvgState), dst->len);
     AvgState* dst_struct = reinterpret_cast<AvgState*>(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<DecimalV2AvgState*>(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<AvgState*>(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<AvgState*>(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<DecimalV2KnuthVarianceState*>(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<KnuthVarianceState*>(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<DecimalV2KnuthVarianceState*>(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<DecimalV2KnuthVarianceState*>(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<DecimalV2KnuthVarianceState*>(state_sv.ptr);
@@ -2395,17 +2396,23 @@ template void AggregateFunctions::sum<FloatVal, DoubleVal>(FunctionContext*, con
 template void AggregateFunctions::sum<DoubleVal, DoubleVal>(FunctionContext*, const DoubleVal& src,
                                                             DoubleVal* dst);
 
-template void AggregateFunctions::min_init<BooleanVal>(doris_udf::FunctionContext *, BooleanVal* dst);
-template void AggregateFunctions::min_init<TinyIntVal>(doris_udf::FunctionContext *, TinyIntVal* dst);
-template void AggregateFunctions::min_init<SmallIntVal>(doris_udf::FunctionContext *, SmallIntVal* dst);
-template void AggregateFunctions::min_init<IntVal>(doris_udf::FunctionContext *, IntVal* dst);
-template void AggregateFunctions::min_init<BigIntVal>(doris_udf::FunctionContext *, BigIntVal* dst);
-template void AggregateFunctions::min_init<LargeIntVal>(doris_udf::FunctionContext *, LargeIntVal* dst);
-template void AggregateFunctions::min_init<FloatVal>(doris_udf::FunctionContext *, FloatVal* dst);
-template void AggregateFunctions::min_init<DoubleVal>(doris_udf::FunctionContext *, DoubleVal* dst);
-template void AggregateFunctions::min_init<DateTimeVal>(doris_udf::FunctionContext *, DateTimeVal* dst);
-template void AggregateFunctions::min_init<DecimalV2Val>(doris_udf::FunctionContext *, DecimalV2Val* dst);
-template void AggregateFunctions::min_init<StringVal>(doris_udf::FunctionContext *, StringVal* dst);
+template void AggregateFunctions::min_init<BooleanVal>(doris_udf::FunctionContext*,
+                                                       BooleanVal* dst);
+template void AggregateFunctions::min_init<TinyIntVal>(doris_udf::FunctionContext*,
+                                                       TinyIntVal* dst);
+template void AggregateFunctions::min_init<SmallIntVal>(doris_udf::FunctionContext*,
+                                                        SmallIntVal* dst);
+template void AggregateFunctions::min_init<IntVal>(doris_udf::FunctionContext*, IntVal* dst);
+template void AggregateFunctions::min_init<BigIntVal>(doris_udf::FunctionContext*, BigIntVal* dst);
+template void AggregateFunctions::min_init<LargeIntVal>(doris_udf::FunctionContext*,
+                                                        LargeIntVal* dst);
+template void AggregateFunctions::min_init<FloatVal>(doris_udf::FunctionContext*, FloatVal* dst);
+template void AggregateFunctions::min_init<DoubleVal>(doris_udf::FunctionContext*, DoubleVal* dst);
+template void AggregateFunctions::min_init<DateTimeVal>(doris_udf::FunctionContext*,
+                                                        DateTimeVal* dst);
+template void AggregateFunctions::min_init<DecimalV2Val>(doris_udf::FunctionContext*,
+                                                         DecimalV2Val* dst);
+template void AggregateFunctions::min_init<StringVal>(doris_udf::FunctionContext*, StringVal* dst);
 
 template void AggregateFunctions::min<BooleanVal>(FunctionContext*, const BooleanVal& src,
                                                   BooleanVal* dst);
@@ -2439,17 +2446,23 @@ template void AggregateFunctions::avg_remove<doris_udf::SmallIntVal>(doris_udf::
                                                                      doris_udf::SmallIntVal const&,
                                                                      doris_udf::StringVal*);
 
-template void AggregateFunctions::max_init<BooleanVal>(doris_udf::FunctionContext *, BooleanVal* dst);
-template void AggregateFunctions::max_init<TinyIntVal>(doris_udf::FunctionContext *, TinyIntVal* dst);
-template void AggregateFunctions::max_init<SmallIntVal>(doris_udf::FunctionContext *, SmallIntVal* dst);
-template void AggregateFunctions::max_init<IntVal>(doris_udf::FunctionContext *, IntVal* dst);
-template void AggregateFunctions::max_init<BigIntVal>(doris_udf::FunctionContext *, BigIntVal* dst);
-template void AggregateFunctions::max_init<LargeIntVal>(doris_udf::FunctionContext *, LargeIntVal* dst);
-template void AggregateFunctions::max_init<FloatVal>(doris_udf::FunctionContext *, FloatVal* dst);
-template void AggregateFunctions::max_init<DoubleVal>(doris_udf::FunctionContext *, DoubleVal* dst);
-template void AggregateFunctions::max_init<DateTimeVal>(doris_udf::FunctionContext *, DateTimeVal* dst);
-template void AggregateFunctions::max_init<DecimalV2Val>(doris_udf::FunctionContext *, DecimalV2Val* dst);
-template void AggregateFunctions::max_init<StringVal>(doris_udf::FunctionContext *, StringVal* dst);
+template void AggregateFunctions::max_init<BooleanVal>(doris_udf::FunctionContext*,
+                                                       BooleanVal* dst);
+template void AggregateFunctions::max_init<TinyIntVal>(doris_udf::FunctionContext*,
+                                                       TinyIntVal* dst);
+template void AggregateFunctions::max_init<SmallIntVal>(doris_udf::FunctionContext*,
+                                                        SmallIntVal* dst);
+template void AggregateFunctions::max_init<IntVal>(doris_udf::FunctionContext*, IntVal* dst);
+template void AggregateFunctions::max_init<BigIntVal>(doris_udf::FunctionContext*, BigIntVal* dst);
+template void AggregateFunctions::max_init<LargeIntVal>(doris_udf::FunctionContext*,
+                                                        LargeIntVal* dst);
+template void AggregateFunctions::max_init<FloatVal>(doris_udf::FunctionContext*, FloatVal* dst);
+template void AggregateFunctions::max_init<DoubleVal>(doris_udf::FunctionContext*, DoubleVal* dst);
+template void AggregateFunctions::max_init<DateTimeVal>(doris_udf::FunctionContext*,
+                                                        DateTimeVal* dst);
+template void AggregateFunctions::max_init<DecimalV2Val>(doris_udf::FunctionContext*,
+                                                         DecimalV2Val* dst);
+template void AggregateFunctions::max_init<StringVal>(doris_udf::FunctionContext*, StringVal* dst);
 
 template void AggregateFunctions::max<BooleanVal>(FunctionContext*, const BooleanVal& src,
                                                   BooleanVal* dst);
@@ -2769,8 +2782,9 @@ template void AggregateFunctions::offset_fn_update<DecimalV2Val>(FunctionContext
                                                                  const DecimalV2Val&,
                                                                  DecimalV2Val* dst);
 
-template void AggregateFunctions::percentile_update<BigIntVal>(
-        FunctionContext* ctx, const BigIntVal&, const DoubleVal&, StringVal*);
+template void AggregateFunctions::percentile_update<BigIntVal>(FunctionContext* ctx,
+                                                               const BigIntVal&, const DoubleVal&,
+                                                               StringVal*);
 
 template void AggregateFunctions::percentile_approx_update<doris_udf::DoubleVal>(
         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 1668305..2f1e67b 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 <typename T>
     static void init_zero_not_null(doris_udf::FunctionContext*, T* dst);
 
-
     template <typename SRC_VAL, typename DST_VAL>
     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 <typename T>
-    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 18cecad..c5fd416 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<AnyVal*>(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<void*>(*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 3eb6cf6..90d2f59 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 <typename T>
     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 <typename T>
     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 90cce5a..06546c2 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 6bc7cd6..ef772cb 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 2cba685..36e24d5 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 <typename T>
     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 bd1ca43..4eda394 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 4bc7584..5d52d85 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 79fa34c..e1290fc 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<CaseExprState*>(                      \
                 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 3e907bd..e22a3cf 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 c579fb9..42312a3 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 14412e0..f36ac2d 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 6aa3140..8ba3709 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 <boost/smart_ptr.hpp>
-
 #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<char> p;
+    std::unique_ptr<char[]> 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<char> p;
+    std::unique_ptr<char[]> 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<char> p;
+    std::unique_ptr<char[]> 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<char> p;
+    std::unique_ptr<char[]> 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 544af35..97352da 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::vector<TExprNo
         return Status::InternalError("Failed to reconstruct expression tree from thrift.");
     }
     int num_children = nodes[*node_idx].num_children;
-    Expr* expr = NULL;
+    Expr* expr = nullptr;
     RETURN_IF_ERROR(create_expr(pool, nodes[*node_idx], &expr));
-    DCHECK(expr != NULL);
-    if (parent != NULL) {
+    DCHECK(expr != nullptr);
+    if (parent != nullptr) {
         parent->add_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<Expr*>& exprs, std::vector<in
     for (int i = 0; i < exprs.size(); ++i) {
         data[i].expr_idx = i;
 
-        if (exprs[i]->type().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<ExprContext*>& ctxs, RuntimeState* state,
                                  std::vector<ExprContext*>* 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 5bc3c50..795dca6 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 <boost/scoped_ptr.hpp>
-//
+
 #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<MemTracker>& tracker);
@@ -207,7 +206,7 @@ public:
     /// Convenience function for opening multiple expr trees.
     static Status open(const std::vector<ExprContext*>& 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<int>* 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 fcc67c1..ac2694a 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<MemTracker>& 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<char*>(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 5c7f4ec..45896a2 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 f55a6d6..62c6be8 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 9db94ce..53143d7 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 b90c1b9..35a8f0b 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 <boost/shared_ptr.hpp>
 #include <string>
 #include <unordered_set>
 
@@ -68,7 +67,7 @@ private:
     const bool _is_not_in;
     bool _is_prepare;
     bool _null_in_set;
-    boost::shared_ptr<HybridSetBase> _hybrid_set;
+    std::shared_ptr<HybridSetBase> _hybrid_set;
 };
 
 } // namespace doris
diff --git a/be/src/exprs/info_func.cpp b/be/src/exprs/info_func.cpp
index 2aab808..5f4d1ae 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 ed04f00..e85f214 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<JsonPath>& 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 933d9e0..b76d46d 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<StringVal*>(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<StringVal*>(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 53ac852..f2f5aad 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<std::mt19937*>(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<int64_t*>(e->children()[i]->get_value(row));
-        if (arg == NULL) {
-            return NULL;
+        if (arg == nullptr) {
+            return nullptr;
         }
 
         if (*arg > *reinterpret_cast<int64_t*>(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<double*>(e->children()[i]->get_value(row));
-        if (arg == NULL) {
-            return NULL;
+        if (arg == nullptr) {
+            return nullptr;
         }
 
         if (*arg > *reinterpret_cast<double*>(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<StringValue*>(e->children()[i]->get_value(row));
-        if (arg == NULL) {
-            return NULL;
+        if (arg == nullptr) {
+            return nullptr;
         }
         if (*arg > *reinterpret_cast<StringValue*>(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<DateTimeValue*>(e->children()[i]->get_value(row));
-        if (arg == NULL) {
-            return NULL;
+        if (arg == nullptr) {
+            return nullptr;
         }
         if (*arg > *reinterpret_cast<DateTimeValue*>(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<int64_t*>(e->children()[i]->get_value(row));
-        if (arg == NULL) {
-            return NULL;
+        if (arg == nullptr) {
+            return nullptr;
         }
 
         if (*arg < *reinterpret_cast<int64_t*>(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<double*>(e->children()[i]->get_value(row));
-        if (arg == NULL) {
-            return NULL;
+        if (arg == nullptr) {
+            return nullptr;
         }
 
         if (*arg < *reinterpret_cast<double*>(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<DecimalV2Value*>(e->children()[i]->get_value(row));
-        if (arg == NULL) {
-            return NULL;
+        if (arg == nullptr) {
+            return nullptr;
         }
         if (*arg < *reinterpret_cast<DecimalV2Value*>(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<StringValue*>(e->children()[i]->get_value(row));
-        if (arg == NULL) {
-            return NULL;
+        if (arg == nullptr) {
+            return nullptr;
         }
         if (*arg < *reinterpret_cast<StringValue*>(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<DateTimeValue*>(e->children()[i]->get_value(row));
-        if (arg == NULL) {
-            return NULL;
+        if (arg == nullptr) {
+            return nullptr;
         }
         if (*arg < *reinterpret_cast<DateTimeValue*>(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 b7f92d5..7a2209b 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 a03626e..822e8fc 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 <boost/scoped_ptr.hpp>
-#include <boost/shared_array.hpp>
 #include <string>
 
 #include "codegen/doris_ir.h"
@@ -180,7 +178,8 @@ public:
                        Tuple* dst);
     static void Serialize(const std::vector<NewAggFnEvaluator*>& evals, Tuple* dst);
     static void GetValue(const std::vector<NewAggFnEvaluator*>& evals, Tuple* src, Tuple* dst);
-    static void Finalize(const std::vector<NewAggFnEvaluator*>& evals, Tuple* src, Tuple* dst, bool add_null = false);
+    static void Finalize(const std::vector<NewAggFnEvaluator*>& 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<FunctionContext> agg_fn_ctx_;
+    std::unique_ptr<FunctionContext> 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 c713b82..b0a6390 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<SetType>* state = reinterpret_cast<SetLookupState<SetType>*>(
                 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 <typename T, typename SetType>
 BooleanVal InPredicate::set_lookup(SetLookupState<SetType>* state, const T& v) {
-    DCHECK(state != NULL);
+    DCHECK(state != nullptr);
     SetType val = get_val<T, SetType>(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 e71917d..6238894 100644
--- a/be/src/exprs/new_in_predicate.h
+++ b/be/src/exprs/new_in_predicate.h
@@ -308,10 +308,10 @@ private:
 
     template <typename SetType>
     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<SetType> val_set;
diff --git a/be/src/exprs/operators.cpp b/be/src/exprs/operators.cpp
index 2380bbc..aaffc0b 100644
--- a/be/src/exprs/operators.cpp
+++ b/be/src/exprs/operators.cpp
@@ -17,8 +17,6 @@
 
 #include "exprs/operators.h"
 
-#include <boost/cstdint.hpp>
-
 #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 17e7f3b..407e50b 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 c887073..2a7fc82 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<AnyVal*>* 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 <typename RETURN_TYPE>
 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<AnyVal*>* 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<BooleanVal>(context, row);
     }
     BooleanWrapper fn = reinterpret_cast<BooleanWrapper>(_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<TinyIntVal>(context, row);
     }
     TinyIntWrapper fn = reinterpret_cast<TinyIntWrapper>(_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<SmallIntVal>(context, row);
     }
     SmallIntWrapper fn = reinterpret_cast<SmallIntWrapper>(_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<IntVal>(context, row);
     }
     IntWrapper fn = reinterpret_cast<IntWrapper>(_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<BigIntVal>(context, row);
     }
     BigIntWrapper fn = reinterpret_cast<BigIntWrapper>(_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<LargeIntVal>(context, row);
     }
     LargeIntWrapper fn = reinterpret_cast<LargeIntWrapper>(_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<FloatVal>(context, row);
     }
     FloatWrapper fn = reinterpret_cast<FloatWrapper>(_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<DoubleVal>(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<StringVal>(context, row);
     }
     StringWrapper fn = reinterpret_cast<StringWrapper>(_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<DateTimeVal>(context, row);
     }
     DatetimeWrapper fn = reinterpret_cast<DatetimeWrapper>(_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<DecimalV2Val>(context, row);
     }
     DecimalV2Wrapper fn = reinterpret_cast<DecimalV2Wrapper>(_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<CollectionVal>(context, row);
     }
 
diff --git a/be/src/exprs/scalar_fn_call.h b/be/src/exprs/scalar_fn_call.h
index 33ae72f..e94e9d0 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 bc106d2..0068c03 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<bool*>(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<int16_t*>(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<int32_t*>(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<int64_t*>(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<PackedInt128*>(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<float*>(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<double*>(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<DateTimeValue*>(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 cbeb4b6..110140e 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 1c95f2f..aedf633 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 <re2/re2.h>
 
@@ -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<char*>(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<re2::RE2*>(
             context->get_function_state(FunctionContext::FRAGMENT_LOCAL));
     std::unique_ptr<re2::RE2> 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<re2::RE2*>(
             context->get_function_state(FunctionContext::FRAGMENT_LOCAL));
-    std::unique_ptr<re2::RE2> scoped_re; // destroys re if state->re is NULL
-    if (re == NULL) {
+    std::unique_ptr<re2::RE2> 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<UrlParser::UrlPart*>(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<UrlParser::UrlPart*>(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<int64_t, 26>(context, rounded.int_value(), abs(rounded.frac_value() / 10000000));
+    return do_money_format<int64_t, 26>(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 610187a..34eeecf 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 155837c..ff1112b 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 <boost/date_time/gregorian/gregorian.hpp>
-#include <boost/date_time/local_time/local_time.hpp>
-#include <boost/date_time/posix_time/posix_time.hpp>
-#include <boost/date_time/time_zone_base.hpp>
-#include <boost/thread/thread.hpp>
+#include <thread>
 
 #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 &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 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 31ca79a..c0b0f2f 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 f6630c2..87b9c96 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 8e342e1..302a83b 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 <boost/scoped_ptr.hpp>
+#include <cstdint>
 
 #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 da3a775..cf518d3 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 <boost/scoped_ptr.hpp>
-
 #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 f96bfb4..182531f 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 <boost/scoped_ptr.hpp>
 #include <map>
 #include <mutex>
 
diff --git a/be/src/http/action/snapshot_action.h b/be/src/http/action/snapshot_action.h
index c606c39..665a5db 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 <boost/scoped_ptr.hpp>
+#include <cstdint>
+#include <string>
 
 #include "http/http_handler.h"
 
diff --git a/be/src/olap/aggregate_func.h b/be/src/olap/aggregate_func.h
index 621441c..a64ed14 100644
--- a/be/src/olap/aggregate_func.h
+++ b/be/src/olap/aggregate_func.h
@@ -425,7 +425,7 @@ struct AggregateFuncTraits<OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL, field_typ
     static void update(RowCursorCell* dst, const RowCursorCell& src, MemPool* mem_pool) {
         bool src_null = src.is_null();
         if (src_null) {
-            // Ignore it if src is NULL
+            // Ignore it if src is nullptr
             return;
         }
 
@@ -440,7 +440,7 @@ struct AggregateFuncTraits<OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL, OLAP_FIEL
     static void update(RowCursorCell* dst, const RowCursorCell& src, MemPool* mem_pool) {
         bool src_null = src.is_null();
         if (src_null) {
-            // Ignore it if src is NULL
+            // Ignore it if src is nullptr
             return;
         }
 
diff --git a/be/src/olap/base_compaction.cpp b/be/src/olap/base_compaction.cpp
index 5e4869a..cc7c358 100644
--- a/be/src/olap/base_compaction.cpp
+++ b/be/src/olap/base_compaction.cpp
@@ -137,7 +137,7 @@ OLAPStatus BaseCompaction::pick_rowsets_to_compact() {
     // 3. the interval since last base compaction reaches the threshold
     int64_t base_creation_time = _input_rowsets[0]->creation_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 105d669..b69c9d0 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 f2e99bf..0cc877e 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<char>(memory, BufDeleter());
+    if (buf != nullptr && memory != nullptr) {
+        buf->_buf = std::shared_ptr<char>(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<char>(memory, deleter);
+    buf->_buf = std::shared_ptr<char>(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<char>(memory, deleter);
+    buf->_buf = std::shared_ptr<char>(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 02574ff..0ac4e41 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 <boost/shared_ptr.hpp>
-
 #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<char> _buf; // 托管的内存
+    std::shared_ptr<char> _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 e48392a..9000ce2 100644
--- a/be/src/olap/data_dir.cpp
+++ b/be/src/olap/data_dir.cpp
@@ -28,8 +28,8 @@
 #include <boost/algorithm/string/predicate.hpp>
 #include <boost/algorithm/string/split.hpp>
 #include <boost/algorithm/string/trim.hpp>
-#include <filesystem>
 #include <boost/interprocess/sync/file_lock.hpp>
+#include <filesystem>
 #include <fstream>
 #include <set>
 #include <sstream>
@@ -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<std::mutex> 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<std::string> 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 dcd0fab..bcab67b 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 63ef199..f90eae4 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 2ffd8f1..be1b43a 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<FileDescriptor*>(_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 e89cbd3..d74892f 100644
--- a/be/src/olap/file_helper.h
+++ b/be/src/olap/file_helper.h
@@ -240,7 +240,7 @@ private:
 template <typename MessageType, typename ExtraType, typename FileHandlerType>
 OLAPStatus FileHeader<MessageType, ExtraType, FileHandlerType>::prepare(
         FileHandlerType* file_handler) {
-    if (NULL == file_handler) {
+    if (nullptr == file_handler) {
         return OLAP_ERR_INPUT_PARAMETER_ERROR;
     }
 
@@ -274,7 +274,7 @@ OLAPStatus FileHeader<MessageType, ExtraType, FileHandlerType>::prepare(
 template <typename MessageType, typename ExtraType, typename FileHandlerType>
 OLAPStatus FileHeader<MessageType, ExtraType, FileHandlerType>::serialize(
         FileHandlerType* file_handler) {
-    if (NULL == file_handler) {
+    if (nullptr == file_handler) {
         return OLAP_ERR_INPUT_PARAMETER_ERROR;
     }
 
@@ -310,7 +310,7 @@ OLAPStatus FileHeader<MessageType, ExtraType, FileHandlerType>::serialize(
 template <typename MessageType, typename ExtraType, typename FileHandlerType>
 OLAPStatus FileHeader<MessageType, ExtraType, FileHandlerType>::unserialize(
         FileHandlerType* file_handler) {
-    if (NULL == file_handler) {
+    if (nullptr == file_handler) {
         return OLAP_ERR_INPUT_PARAMETER_ERROR;
     }
 
@@ -347,10 +347,10 @@ OLAPStatus FileHeader<MessageType, ExtraType, FileHandlerType>::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<MessageType, ExtraType, FileHandlerType>::unserialize(
 
     std::unique_ptr<char[]> 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 1108e79..ad1614f 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 323e9a8..d33454a 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 55d80de..75ab135 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<MemTracker> 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 a50517c..12fb912 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 <algorithm>
 #include <map>
 
 #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 b597523..0809b27 100644
--- a/be/src/olap/in_stream.cpp
+++ b/be/src/olap/in_stream.cpp
@@ -31,8 +31,8 @@ InStream::InStream(std::vector<StorageByteBuffer*>* inputs, const std::vector<ui
           _compress_buffer_size(compress_buffer_size),
           _current_offset(0),
           _current_range(0),
-          _compressed(NULL),
-          _uncompressed(NULL) {}
+          _compressed(nullptr),
+          _uncompressed(nullptr) {}
 
 InStream::~InStream() {
     SAFE_DELETE(_compressed);
@@ -42,13 +42,13 @@ InStream::~InStream() {
 OLAPStatus InStream::_slice(uint64_t chunk_size, StorageByteBuffer** out_slice) {
     uint64_t len = chunk_size;
     uint64_t old_offset = _current_offset;
-    StorageByteBuffer* slice = NULL;
+    StorageByteBuffer* slice = nullptr;
 
     //如果buffer够读,拿出一个chunksize,并设置position
     if (OLAP_LIKELY(_compressed->remaining() >= 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 b2f602e..492097c 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<char*>(_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 0a2ce0f..3ae16db 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<MemTracker> 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<MemTracker> 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<MemTracker> parent_tracker) {
+Cache* new_lru_cache(const std::string& name, size_t capacity,
+                     std::shared_ptr<MemTracker> 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<MemTracker> parent_tracker) {
+Cache* new_typed_lru_cache(const std::string& name, size_t capacity, LRUCacheType type,
+                           std::shared_ptr<MemTracker> 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 94e262d..b9f71f1 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<MemTracker> parent_tracekr = nullptr);
+                            std::shared_ptr<MemTracker> parent_tracekr = nullptr);
 
 extern Cache* new_typed_lru_cache(const std::string& name, size_t capacity, LRUCacheType type,
-        std::shared_ptr<MemTracker> parent_tracekr = nullptr);
+                                  std::shared_ptr<MemTracker> 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<MemTracker> parent);
+    explicit ShardedLRUCache(const std::string& name, size_t total_capacity, LRUCacheType type,
+                             std::shared_ptr<MemTracker> 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 16f3bf4..9503f12 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 4bc6b51..fec9c8a 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<WrapperField*, WrapperField*>& 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<WrapperField*, WrapperField*> &statistic) const {
+bool CondColumn::eval(const std::pair<WrapperField*, WrapperField*>& 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 fd3a794..ea34452 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 de71f51..14ae7e4 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<SegmentMetaInfo>::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<TabletColumn>* 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 08f6f6c..1b9c704 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 4a64568..d9ce6c4 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<int64_t>(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 b6900c6..2b7000e 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 2376758..533f6a1 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 d907aea..747568c 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<TabletVars>& tablet_vars,
                                     std::vector<TTabletInfo>* 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 e1fb787..d782ee4 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 97cbdd1..20ed4ae 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 e2f8b5a..b97dc0a 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<TabletColumn>& schema,
     return _init(columns);
 }
 
-OLAPStatus RowCursor::_init_scan_key(const TabletSchema& schema, const std::vector<std::string>& scan_keys) {
+OLAPStatus RowCursor::_init_scan_key(const TabletSchema& schema,
+                                     const std::vector<std::string>& 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 60f5224..f809914 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 3e01f8e..9970137 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 33452a2..d2b44c8 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 25fc9e0..e870a3c 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<StreamName, ReadOnlyFileStream*>* 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<StreamName, ReadOnlyFileStream*>*
     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<StreamName, ReadOnlyFileStream*>* 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<StreamName, ReadOnlyFileStrea
     ReadOnlyFileStream* length_stream =
             extract_stream(_column_unique_id, StreamInfoMessage::LENGTH, streams);
 
-    if (NULL == length_stream) {
+    if (nullptr == length_stream) {
         OLAP_LOG_WARNING("specified stream not found. [unique_id = %u]", _column_unique_id);
         return OLAP_ERR_COLUMN_STREAM_NOT_EXIST;
     }
 
     _length_reader = new (std::nothrow) RunLengthIntegerReader(length_stream, false);
 
-    if (NULL == _length_reader) {
+    if (nullptr == _length_reader) {
         OLAP_LOG_WARNING("fail to malloc RunLengthIntegerReader");
         return OLAP_ERR_MALLOC_ERROR;
     }
@@ -270,12 +270,12 @@ StringColumnDictionaryReader::StringColumnDictionaryReader(uint32_t column_uniqu
         : _eof(false),
           _dictionary_size(dictionary_size),
           _column_unique_id(column_unique_id),
-          _values(NULL),
+          _values(nullptr),
           //_dictionary_size(0),
-          //_offset_dictionary(NULL),
-          //_dictionary_data_buffer(NULL),
-          _read_buffer(NULL),
-          _data_reader(NULL) {}
+          //_offset_dictionary(nullptr),
+          //_dictionary_data_buffer(nullptr),
+          _read_buffer(nullptr),
+          _data_reader(nullptr) {}
 
 StringColumnDictionaryReader::~StringColumnDictionaryReader() {
     //SAFE_DELETE_ARRAY(_offset_dictionary);
... 14251 lines suppressed ...

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org