You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by li...@apache.org on 2023/01/22 08:39:32 UTC

[doris] branch master updated: [Refactor]Merged string_value into string_ref (#15925)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 199d7d3be8 [Refactor]Merged string_value into string_ref (#15925)
199d7d3be8 is described below

commit 199d7d3be8dba15d607d7dbdb0d9defd1774ba0c
Author: ZhaoChangle <zh...@selectdb.com>
AuthorDate: Sun Jan 22 16:39:23 2023 +0800

    [Refactor]Merged string_value into string_ref (#15925)
---
 be/src/exec/arrow/parquet_reader.cpp               |  15 +-
 be/src/exec/base_scanner.cpp                       |   8 +-
 be/src/exec/es/es_predicate.cpp                    |   4 +-
 be/src/exec/es/es_scroll_parser.cpp                |  22 +-
 be/src/exec/es/es_scroll_parser.h                  |   2 +-
 be/src/exec/hash_table.cpp                         |   6 +-
 be/src/exec/olap_utils.h                           |   2 +-
 be/src/exec/schema_scanner.cpp                     |   2 +-
 .../schema_scanner/schema_backends_scanner.cpp     |  12 +-
 .../schema_scanner/schema_charsets_scanner.cpp     |  38 ++--
 .../schema_scanner/schema_collations_scanner.cpp   |  50 ++---
 .../exec/schema_scanner/schema_columns_scanner.cpp | 154 +++++++-------
 .../exec/schema_scanner/schema_files_scanner.cpp   |  58 ++---
 .../schema_scanner/schema_partitions_scanner.cpp   |  28 +--
 .../exec/schema_scanner/schema_rowsets_scanner.cpp |  12 +-
 .../schema_schema_privileges_scanner.cpp           |  24 +--
 .../schema_scanner/schema_schemata_scanner.cpp     |  50 ++---
 .../schema_scanner/schema_statistics_scanner.cpp   |  24 +--
 .../schema_table_privileges_scanner.cpp            |  26 +--
 .../exec/schema_scanner/schema_tables_scanner.cpp  | 100 ++++-----
 .../schema_user_privileges_scanner.cpp             |  22 +-
 .../schema_scanner/schema_variables_scanner.cpp    |  26 +--
 .../exec/schema_scanner/schema_views_scanner.cpp   | 104 ++++-----
 be/src/exec/text_converter.cpp                     |  10 +-
 be/src/exec/text_converter.h                       |   4 +-
 be/src/exec/text_converter.hpp                     |  17 +-
 be/src/exprs/agg_fn_evaluator.cpp                  |  16 +-
 be/src/exprs/aggregate_functions.cpp               |  24 +--
 be/src/exprs/anyval_util.h                         |  16 +-
 be/src/exprs/binary_predicate.cpp                  |   6 +-
 be/src/exprs/bitmap_function.cpp                   |  34 +--
 be/src/exprs/bloomfilter_predicate.h               |  16 +-
 be/src/exprs/create_predicate_function.h           |   4 +-
 be/src/exprs/expr.h                                |   5 +-
 be/src/exprs/expr_context.cpp                      |   4 +-
 be/src/exprs/expr_value.h                          |  16 +-
 be/src/exprs/hybrid_set.h                          |  48 ++---
 be/src/exprs/info_func.cpp                         |   6 +-
 be/src/exprs/like_predicate.cpp                    |  22 +-
 be/src/exprs/like_predicate.h                      |   6 +-
 be/src/exprs/literal.cpp                           |   6 +-
 be/src/exprs/match_predicate.cpp                   |   6 +-
 be/src/exprs/math_functions.cpp                    |   4 +-
 be/src/exprs/minmax_predicate.h                    |  14 +-
 be/src/exprs/new_agg_fn_evaluator.cc               |  10 +-
 be/src/exprs/new_in_predicate.cpp                  |   6 +-
 be/src/exprs/operators.cpp                         |   4 +-
 be/src/exprs/rpc_fn.cpp                            |  12 +-
 be/src/exprs/runtime_filter.cpp                    |  34 +--
 be/src/exprs/slot_ref.cpp                          |   2 +-
 be/src/exprs/string_functions.cpp                  |  31 ++-
 be/src/exprs/string_functions.h                    |   2 +-
 be/src/olap/aggregate_func.h                       |   2 +-
 be/src/olap/comparison_predicate.h                 |  20 +-
 be/src/olap/hll.cpp                                |   2 +-
 be/src/olap/in_list_predicate.h                    |  31 ++-
 be/src/olap/like_column_predicate.cpp              |  18 +-
 be/src/olap/like_column_predicate.h                |  25 ++-
 be/src/olap/null_predicate.cpp                     |   2 +-
 be/src/olap/predicate_creator.h                    |   6 +-
 be/src/olap/rowset/segment_v2/bloom_filter.h       |   2 +-
 be/src/olap/rowset/segment_v2/column_reader.cpp    |   8 +-
 .../rowset/segment_v2/inverted_index_reader.cpp    |  10 +-
 be/src/runtime/CMakeLists.txt                      |   1 -
 be/src/runtime/collection_value.cpp                |  34 +--
 be/src/runtime/fold_constant_executor.cpp          |   2 +-
 be/src/runtime/primitive_type.cpp                  |   4 +-
 be/src/runtime/primitive_type.h                    |  10 +-
 be/src/runtime/raw_value.cpp                       |  54 ++---
 be/src/runtime/raw_value.h                         |  38 ++--
 be/src/runtime/string_buffer.hpp                   |  40 ++--
 be/src/runtime/string_search.hpp                   |  31 ++-
 be/src/runtime/string_value.cpp                    |  57 -----
 be/src/runtime/string_value.h                      | 211 -------------------
 be/src/runtime/string_value.hpp                    |  59 ------
 be/src/runtime/tuple.cpp                           |  66 +++---
 be/src/runtime/tuple.h                             |  12 +-
 be/src/runtime/type_limit.h                        |   8 +-
 be/src/udf/udf.h                                   |   6 +-
 be/src/util/bitmap_intersect.h                     |  22 +-
 be/src/util/simd/vstring_function.h                |   2 +-
 be/src/util/static_asserts.cpp                     |   8 +-
 be/src/util/url_parser.cpp                         | 102 ++++-----
 be/src/util/url_parser.h                           |  42 ++--
 be/src/vec/CMakeLists.txt                          |   1 +
 .../aggregate_function_orthogonal_bitmap.cpp       |   2 +-
 be/src/vec/columns/column_dictionary.h             |  56 ++---
 be/src/vec/columns/predicate_column.h              |  34 +--
 be/src/vec/common/hash_table/hash.h                |   3 +-
 be/src/vec/common/hash_table/hash_table.h          |   2 +-
 .../vec/common/hash_table/hash_table_key_holder.h  |   6 +-
 be/src/vec/common/hash_table/string_hash_map.h     |  26 +--
 be/src/vec/common/hash_table/string_hash_table.h   |  27 +--
 be/src/vec/common/string_ref.cpp                   |  81 +++++++
 be/src/vec/common/string_ref.h                     | 233 +++++++++++++++------
 be/src/vec/core/block.cpp                          |  38 ++--
 be/src/vec/core/types.h                            |   4 +-
 .../vec/exec/data_gen_functions/vnumbers_tvf.cpp   |   2 +-
 be/src/vec/exec/format/orc/vorc_reader.cpp         |   4 +-
 be/src/vec/exec/format/parquet/parquet_pred_cmp.h  |   8 +-
 be/src/vec/exec/format/table/iceberg_reader.cpp    |   2 +-
 be/src/vec/exec/scan/vscan_node.cpp                |  10 +-
 be/src/vec/exec/vbroker_scan_node.cpp              |   2 +-
 be/src/vec/exec/vbroker_scanner.cpp                |   6 +-
 be/src/vec/exec/vdata_gen_scan_node.cpp            |   1 -
 be/src/vec/exec/vjson_scanner.cpp                  |  12 +-
 be/src/vec/exec/vmysql_scan_node.cpp               |   2 +-
 be/src/vec/exec/vschema_scan_node.cpp              |   8 +-
 .../vec/exprs/table_function/vexplode_json_array.h |   4 +-
 be/src/vec/exprs/table_function/vexplode_split.h   |   2 +-
 be/src/vec/exprs/vbloom_predicate.cpp              |   2 +-
 be/src/vec/exprs/vdirect_in_predicate.h            |   2 +-
 be/src/vec/functions/function_regexp.cpp           |   2 +-
 be/src/vec/functions/function_string.cpp           |   6 +-
 be/src/vec/functions/function_string.h             |  33 +--
 be/src/vec/functions/like.cpp                      | 126 +++++------
 be/src/vec/functions/like.h                        |  77 ++++---
 be/src/vec/io/io_helper.h                          |   4 +-
 be/src/vec/olap/vertical_merge_iterator.cpp        |   7 +-
 be/src/vec/olap/vertical_merge_iterator.h          |   4 +-
 be/src/vec/runtime/vfile_result_writer.cpp         |   2 +-
 be/src/vec/runtime/vorc_writer.cpp                 |  26 +--
 be/test/exprs/bitmap_function_test.cpp             |   3 +-
 be/test/exprs/bloom_filter_predicate_test.cpp      |  24 +--
 be/test/exprs/hybrid_set_test.cpp                  |  34 +--
 be/test/olap/block_column_predicate_test.cpp       |   2 +-
 be/test/runtime/string_buffer_test.cpp             |   2 +-
 be/test/runtime/string_value_test.cpp              |  11 +-
 be/test/util/array_parser_test.cpp                 |   4 +-
 be/test/vec/core/block_test.cpp                    |   2 +-
 be/test/vec/exec/parquet/parquet_thrift_test.cpp   |  18 +-
 gensrc/script/gen_functions.py                     |   2 +-
 gensrc/script/gen_vector_functions.py              |   2 +-
 133 files changed, 1479 insertions(+), 1601 deletions(-)

diff --git a/be/src/exec/arrow/parquet_reader.cpp b/be/src/exec/arrow/parquet_reader.cpp
index c19e8e4269..ba2fe9098f 100644
--- a/be/src/exec/arrow/parquet_reader.cpp
+++ b/be/src/exec/arrow/parquet_reader.cpp
@@ -31,9 +31,9 @@
 #include "io/file_reader.h"
 #include "runtime/descriptors.h"
 #include "runtime/mem_pool.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple.h"
 #include "util/string_util.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -118,15 +118,18 @@ Status ParquetReaderWrap::size(int64_t* size) {
     }
 }
 
+// TODO: NEED TO REWRITE COMPLETELY. the way writing now is WRONG.
+// StringRef shouldn't managing exclusive memory cause it will break RAII.
+// besides, accessing object which is essentially const by non-const object
+// is UB!
 inline void ParquetReaderWrap::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool,
                                          const uint8_t* value, int32_t len) {
     tuple->set_not_null(slot_desc->null_indicator_offset());
     void* slot = tuple->get_slot(slot_desc->tuple_offset());
-    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
-    memcpy(str_slot->ptr, value, len);
-    str_slot->len = len;
-    return;
+    StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+    str_slot->data = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(const_cast<char*>(str_slot->data), value, len); // !
+    str_slot->size = len;
 }
 
 inline Status ParquetReaderWrap::set_field_null(Tuple* tuple, const SlotDescriptor* slot_desc) {
diff --git a/be/src/exec/base_scanner.cpp b/be/src/exec/base_scanner.cpp
index 7e6887c9b5..193860f286 100644
--- a/be/src/exec/base_scanner.cpp
+++ b/be/src/exec/base_scanner.cpp
@@ -222,7 +222,7 @@ Status BaseScanner::_fill_dest_tuple(Tuple* dest_tuple, MemPool* mem_pool) {
                         },
                         [&]() -> std::string {
                             // Type of the slot is must be Varchar in _src_tuple.
-                            StringValue* raw_value = _src_tuple->get_string_slot(
+                            StringRef* raw_value = _src_tuple->get_string_slot(
                                     _src_slot_descs_order_by_dest[dest_index]->tuple_offset());
                             std::string raw_string;
                             if (raw_value != nullptr) { //is not null then get raw value
@@ -418,10 +418,10 @@ void BaseScanner::fill_slots_of_columns_from_path(
         auto slot_desc = _src_slot_descs.at(i + start);
         _src_tuple->set_not_null(slot_desc->null_indicator_offset());
         void* slot = _src_tuple->get_slot(slot_desc->tuple_offset());
-        auto* str_slot = reinterpret_cast<StringValue*>(slot);
+        auto* str_slot = reinterpret_cast<StringRef*>(slot);
         const std::string& column_from_path = columns_from_path[i];
-        str_slot->ptr = const_cast<char*>(column_from_path.c_str());
-        str_slot->len = column_from_path.size();
+        str_slot->data = column_from_path.c_str();
+        str_slot->size = column_from_path.size();
     }
 }
 
diff --git a/be/src/exec/es/es_predicate.cpp b/be/src/exec/es/es_predicate.cpp
index 9773cc32c6..f3080ea86a 100644
--- a/be/src/exec/es/es_predicate.cpp
+++ b/be/src/exec/es/es_predicate.cpp
@@ -31,8 +31,8 @@
 #include "exprs/in_predicate.h"
 #include "runtime/datetime_value.h"
 #include "runtime/large_int_value.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple_row.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -124,7 +124,7 @@ double ExtLiteral::get_double() {
 
 std::string ExtLiteral::get_string() {
     DCHECK(_type == TYPE_VARCHAR || _type == TYPE_CHAR || _type == TYPE_STRING);
-    return (reinterpret_cast<StringValue*>(_value))->to_string();
+    return (reinterpret_cast<StringRef*>(_value))->to_string();
 }
 
 std::string ExtLiteral::get_date_string() {
diff --git a/be/src/exec/es/es_scroll_parser.cpp b/be/src/exec/es/es_scroll_parser.cpp
index 7170057ac2..46eced5d08 100644
--- a/be/src/exec/es/es_scroll_parser.cpp
+++ b/be/src/exec/es/es_scroll_parser.cpp
@@ -29,8 +29,8 @@
 #include "rapidjson/writer.h"
 #include "runtime/mem_pool.h"
 #include "runtime/memory/mem_tracker.h"
-#include "runtime/string_value.h"
 #include "util/string_parser.hpp"
+#include "vec/common/string_ref.h"
 #include "vec/runtime/vdatetime_value.h"
 
 namespace doris {
@@ -256,9 +256,9 @@ static Status insert_int_value(const rapidjson::Value& col, PrimitiveType type,
     return Status::OK();
 }
 
-ScrollParser::ScrollParser(bool doc_value_mode) : _scroll_id(""), _size(0), _line_index(0) {}
+ScrollParser::ScrollParser(bool doc_value_mode) : _size(0), _line_index(0) {}
 
-ScrollParser::~ScrollParser() {}
+ScrollParser::~ScrollParser() = default;
 
 Status ScrollParser::parse(const std::string& scroll_result, bool exactly_once) {
     // rely on `_size !=0 ` to determine whether scroll ends
@@ -294,7 +294,7 @@ Status ScrollParser::parse(const std::string& scroll_result, bool exactly_once)
     return Status::OK();
 }
 
-int ScrollParser::get_size() {
+int ScrollParser::get_size() const {
     return _size;
 }
 
@@ -325,7 +325,7 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, Tuple* tuple,
         if (!slot_desc->is_materialized()) {
             continue;
         }
-        // _id field must exists in every document, this is guaranteed by ES
+        // _id field must exist in every document, this is guaranteed by ES
         // if _id was found in tuple, we would get `_id` value from inner-hit node
         // json-format response would like below:
         //    "hits": {
@@ -355,8 +355,8 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, Tuple* tuple,
                 RETURN_LIMIT_EXCEEDED(nullptr, details, len);
             }
             memcpy(buffer, _id.data(), len);
-            reinterpret_cast<StringValue*>(slot)->ptr = buffer;
-            reinterpret_cast<StringValue*>(slot)->len = len;
+            reinterpret_cast<StringRef*>(slot)->data = buffer;
+            reinterpret_cast<StringRef*>(slot)->size = len;
             continue;
         }
 
@@ -389,7 +389,7 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, Tuple* tuple,
         case TYPE_STRING: {
             // sometimes elasticsearch user post some not-string value to Elasticsearch Index.
             // because of reading value from _source, we can not process all json type and then just transfer the value to original string representation
-            // this may be a tricky, but we can workaround this issue
+            // this may be a tricky, but we can work around this issue
             std::string val;
             if (pure_doc_value) {
                 if (!col[0].IsString()) {
@@ -413,8 +413,8 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, Tuple* tuple,
                 RETURN_LIMIT_EXCEEDED(nullptr, details, val_size);
             }
             memcpy(buffer, val.data(), val_size);
-            reinterpret_cast<StringValue*>(slot)->ptr = buffer;
-            reinterpret_cast<StringValue*>(slot)->len = val_size;
+            reinterpret_cast<StringRef*>(slot)->data = buffer;
+            reinterpret_cast<StringRef*>(slot)->size = val_size;
             break;
         }
 
@@ -638,7 +638,7 @@ Status ScrollParser::fill_columns(const TupleDescriptor* tuple_desc,
         case TYPE_STRING: {
             // sometimes elasticsearch user post some not-string value to Elasticsearch Index.
             // because of reading value from _source, we can not process all json type and then just transfer the value to original string representation
-            // this may be a tricky, but we can workaround this issue
+            // this may be a tricky, but we can work around this issue
             std::string val;
             if (pure_doc_value) {
                 if (!col[0].IsString()) {
diff --git a/be/src/exec/es/es_scroll_parser.h b/be/src/exec/es/es_scroll_parser.h
index 561150ac04..f3c7af70fc 100644
--- a/be/src/exec/es/es_scroll_parser.h
+++ b/be/src/exec/es/es_scroll_parser.h
@@ -41,7 +41,7 @@ public:
                         bool* line_eof, const std::map<std::string, std::string>& docvalue_context);
 
     const std::string& get_scroll_id();
-    int get_size();
+    int get_size() const;
 
 private:
     // helper method for processing date/datetime cols with rapidjson::Value
diff --git a/be/src/exec/hash_table.cpp b/be/src/exec/hash_table.cpp
index 4bb795954a..6db56ba2ff 100644
--- a/be/src/exec/hash_table.cpp
+++ b/be/src/exec/hash_table.cpp
@@ -132,11 +132,11 @@ uint32_t HashTable::hash_variable_len_row() {
 
             if (_expr_value_null_bits[i]) {
                 // Hash the null random seed values at 'loc'
-                hash = HashUtil::hash(loc, sizeof(StringValue), hash);
+                hash = HashUtil::hash(loc, sizeof(StringRef), hash);
             } else {
                 // Hash the string
-                StringValue* str = reinterpret_cast<StringValue*>(loc);
-                hash = HashUtil::hash(str->ptr, str->len, hash);
+                StringRef* str = reinterpret_cast<StringRef*>(loc);
+                hash = HashUtil::hash(str->data, str->size, hash);
             }
         }
     }
diff --git a/be/src/exec/olap_utils.h b/be/src/exec/olap_utils.h
index 6f1dca46e2..e28dcf0a2b 100644
--- a/be/src/exec/olap_utils.h
+++ b/be/src/exec/olap_utils.h
@@ -70,7 +70,7 @@ inline CompareLargeFunc get_compare_func(PrimitiveType type) {
     case TYPE_CHAR:
     case TYPE_VARCHAR:
     case TYPE_STRING:
-        return compare_large<StringValue>;
+        return compare_large<StringRef>;
 
     default:
         DCHECK(false) << "Unsupported Compare type";
diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp
index a452077f6c..4f2d029df7 100644
--- a/be/src/exec/schema_scanner.cpp
+++ b/be/src/exec/schema_scanner.cpp
@@ -33,7 +33,7 @@
 #include "exec/schema_scanner/schema_variables_scanner.h"
 #include "exec/schema_scanner/schema_views_scanner.h"
 #include "runtime/define_primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
diff --git a/be/src/exec/schema_scanner/schema_backends_scanner.cpp b/be/src/exec/schema_scanner/schema_backends_scanner.cpp
index 50684ebbcb..9c1b0c015a 100644
--- a/be/src/exec/schema_scanner/schema_backends_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_backends_scanner.cpp
@@ -27,8 +27,8 @@
 #include "runtime/define_primitive_type.h"
 #include "runtime/exec_env.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
 #include "util/thrift_rpc_helper.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -87,12 +87,12 @@ Status SchemaBackendsScanner::_fill_one_col(Tuple* tuple, MemPool* pool, size_t
         *(reinterpret_cast<int32_t*>(slot)) = _batch_data[_row_idx].column_value[col_idx].intVal;
     } else if (it->second == TYPE_VARCHAR) {
         void* slot = tuple->get_slot(_tuple_desc->slots()[col_idx]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-        str_slot->ptr =
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+        str_slot->data =
                 (char*)pool->allocate(_batch_data[_row_idx].column_value[col_idx].stringVal.size());
-        str_slot->len = _batch_data[_row_idx].column_value[col_idx].stringVal.size();
-        memcpy(str_slot->ptr, _batch_data[_row_idx].column_value[col_idx].stringVal.c_str(),
-               str_slot->len);
+        str_slot->size = _batch_data[_row_idx].column_value[col_idx].stringVal.size();
+        memcpy(const_cast<char*>(str_slot->data),
+               _batch_data[_row_idx].column_value[col_idx].stringVal.c_str(), str_slot->size);
     } else if (it->second == TYPE_DOUBLE) {
         void* slot = tuple->get_slot(_tuple_desc->slots()[col_idx]->tuple_offset());
         *(reinterpret_cast<double_t*>(slot)) =
diff --git a/be/src/exec/schema_scanner/schema_charsets_scanner.cpp b/be/src/exec/schema_scanner/schema_charsets_scanner.cpp
index 27772c85bf..0a96db94c4 100644
--- a/be/src/exec/schema_scanner/schema_charsets_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_charsets_scanner.cpp
@@ -17,15 +17,15 @@
 
 #include "exec/schema_scanner/schema_charsets_scanner.h"
 
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaCharsetsScanner::_s_css_columns[] = {
         //   name,       type,          size
-        {"CHARACTER_SET_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"DEFAULT_COLLATE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"DESCRIPTION", TYPE_VARCHAR, sizeof(StringValue), false},
+        {"CHARACTER_SET_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"DEFAULT_COLLATE_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"DESCRIPTION", TYPE_VARCHAR, sizeof(StringRef), false},
         {"MAXLEN", TYPE_BIGINT, sizeof(int64_t), false},
 };
 
@@ -44,38 +44,38 @@ Status SchemaCharsetsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
     // variables names
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[0]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         int len = strlen(_s_charsets[_index].charset);
-        str_slot->ptr = (char*)pool->allocate(len + 1);
-        if (nullptr == str_slot->ptr) {
+        str_slot->data = (char*)pool->allocate(len + 1);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("No Memory.");
         }
-        memcpy(str_slot->ptr, _s_charsets[_index].charset, len + 1);
-        str_slot->len = len;
+        memcpy(const_cast<char*>(str_slot->data), _s_charsets[_index].charset, len + 1);
+        str_slot->size = len;
     }
     // DEFAULT_COLLATE_NAME
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         int len = strlen(_s_charsets[_index].default_collation);
-        str_slot->ptr = (char*)pool->allocate(len + 1);
-        if (nullptr == str_slot->ptr) {
+        str_slot->data = (char*)pool->allocate(len + 1);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("No Memory.");
         }
-        memcpy(str_slot->ptr, _s_charsets[_index].default_collation, len + 1);
-        str_slot->len = len;
+        memcpy(const_cast<char*>(str_slot->data), _s_charsets[_index].default_collation, len + 1);
+        str_slot->size = len;
     }
     // DESCRIPTION
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[2]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         int len = strlen(_s_charsets[_index].description);
-        str_slot->ptr = (char*)pool->allocate(len + 1);
-        if (nullptr == str_slot->ptr) {
+        str_slot->data = (char*)pool->allocate(len + 1);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("No Memory.");
         }
-        memcpy(str_slot->ptr, _s_charsets[_index].description, len + 1);
-        str_slot->len = len;
+        memcpy(const_cast<char*>(str_slot->data), _s_charsets[_index].description, len + 1);
+        str_slot->size = len;
     }
     // maxlen
     {
diff --git a/be/src/exec/schema_scanner/schema_collations_scanner.cpp b/be/src/exec/schema_scanner/schema_collations_scanner.cpp
index 304eec82b1..3201deb68b 100644
--- a/be/src/exec/schema_scanner/schema_collations_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_collations_scanner.cpp
@@ -18,17 +18,17 @@
 #include "exec/schema_scanner/schema_collations_scanner.h"
 
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaCollationsScanner::_s_cols_columns[] = {
         //   name,       type,          size
-        {"COLLATION_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"CHARACTER_SET_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
+        {"COLLATION_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"CHARACTER_SET_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
         {"ID", TYPE_BIGINT, sizeof(int64_t), false},
-        {"IS_DEFAULT", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"IS_COMPILED", TYPE_VARCHAR, sizeof(StringValue), false},
+        {"IS_DEFAULT", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"IS_COMPILED", TYPE_VARCHAR, sizeof(StringRef), false},
         {"SORTLEN", TYPE_BIGINT, sizeof(int64_t), false},
 };
 
@@ -48,26 +48,26 @@ Status SchemaCollationsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
     // COLLATION_NAME
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[0]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         int len = strlen(_s_collations[_index].name);
-        str_slot->ptr = (char*)pool->allocate(len + 1);
-        if (nullptr == str_slot->ptr) {
+        str_slot->data = (char*)pool->allocate(len + 1);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("No Memory.");
         }
-        memcpy(str_slot->ptr, _s_collations[_index].name, len + 1);
-        str_slot->len = len;
+        memcpy(const_cast<char*>(str_slot->data), _s_collations[_index].name, len + 1);
+        str_slot->size = len;
     }
     // charset
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         int len = strlen(_s_collations[_index].charset);
-        str_slot->ptr = (char*)pool->allocate(len + 1);
-        if (nullptr == str_slot->ptr) {
+        str_slot->data = (char*)pool->allocate(len + 1);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("No Memory.");
         }
-        memcpy(str_slot->ptr, _s_collations[_index].charset, len + 1);
-        str_slot->len = len;
+        memcpy(const_cast<char*>(str_slot->data), _s_collations[_index].charset, len + 1);
+        str_slot->size = len;
     }
     // id
     {
@@ -77,26 +77,26 @@ Status SchemaCollationsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
     // is_default
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[3]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         int len = strlen(_s_collations[_index].is_default);
-        str_slot->ptr = (char*)pool->allocate(len + 1);
-        if (nullptr == str_slot->ptr) {
+        str_slot->data = (char*)pool->allocate(len + 1);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("No Memory.");
         }
-        memcpy(str_slot->ptr, _s_collations[_index].is_default, len + 1);
-        str_slot->len = len;
+        memcpy(const_cast<char*>(str_slot->data), _s_collations[_index].is_default, len + 1);
+        str_slot->size = len;
     }
     // IS_COMPILED
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[4]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         int len = strlen(_s_collations[_index].is_compile);
-        str_slot->ptr = (char*)pool->allocate(len + 1);
-        if (nullptr == str_slot->ptr) {
+        str_slot->data = (char*)pool->allocate(len + 1);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("No Memory.");
         }
-        memcpy(str_slot->ptr, _s_collations[_index].is_compile, len + 1);
-        str_slot->len = len;
+        memcpy(const_cast<char*>(str_slot->data), _s_collations[_index].is_compile, len + 1);
+        str_slot->size = len;
     }
     // sortlen
     {
diff --git a/be/src/exec/schema_scanner/schema_columns_scanner.cpp b/be/src/exec/schema_scanner/schema_columns_scanner.cpp
index fcf78542bf..60a133b6d0 100644
--- a/be/src/exec/schema_scanner/schema_columns_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_columns_scanner.cpp
@@ -21,35 +21,35 @@
 
 #include "exec/schema_scanner/schema_helper.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaColumnsScanner::_s_col_columns[] = {
         //   name,       type,          size,                     is_null
-        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"COLUMN_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
+        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"COLUMN_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
         {"ORDINAL_POSITION", TYPE_BIGINT, sizeof(int64_t), false},
-        {"COLUMN_DEFAULT", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"IS_NULLABLE", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"DATA_TYPE", TYPE_VARCHAR, sizeof(StringValue), false},
+        {"COLUMN_DEFAULT", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"IS_NULLABLE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"DATA_TYPE", TYPE_VARCHAR, sizeof(StringRef), false},
         {"CHARACTER_MAXIMUM_LENGTH", TYPE_BIGINT, sizeof(int64_t), true},
         {"CHARACTER_OCTET_LENGTH", TYPE_BIGINT, sizeof(int64_t), true},
         {"NUMERIC_PRECISION", TYPE_BIGINT, sizeof(int64_t), true},
         {"NUMERIC_SCALE", TYPE_BIGINT, sizeof(int64_t), true},
         {"DATETIME_PRECISION", TYPE_BIGINT, sizeof(int64_t), true},
-        {"CHARACTER_SET_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"COLLATION_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"COLUMN_TYPE", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"COLUMN_KEY", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"EXTRA", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"PRIVILEGES", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"COLUMN_COMMENT", TYPE_VARCHAR, sizeof(StringValue), false},
+        {"CHARACTER_SET_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"COLLATION_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"COLUMN_TYPE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"COLUMN_KEY", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"EXTRA", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"PRIVILEGES", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"COLUMN_COMMENT", TYPE_VARCHAR, sizeof(StringRef), false},
         {"COLUMN_SIZE", TYPE_BIGINT, sizeof(int64_t), true},
         {"DECIMAL_DIGITS", TYPE_BIGINT, sizeof(int64_t), true},
-        {"GENERATION_EXPRESSION", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"GENERATION_EXPRESSION", TYPE_VARCHAR, sizeof(StringRef), true},
         {"SRS_ID", TYPE_BIGINT, sizeof(int64_t), true},
 };
 
@@ -59,7 +59,7 @@ SchemaColumnsScanner::SchemaColumnsScanner()
           _table_index(0),
           _column_index(0) {}
 
-SchemaColumnsScanner::~SchemaColumnsScanner() {}
+SchemaColumnsScanner::~SchemaColumnsScanner() = default;
 
 Status SchemaColumnsScanner::start(RuntimeState* state) {
     if (!_is_init) {
@@ -218,6 +218,9 @@ std::string SchemaColumnsScanner::type_to_string(TColumnDesc& desc) {
     }
 }
 
+// TODO: ALL schema algorithm like these need to be refactor to avoid UB and
+// keep StringRef semantic to be kept.
+
 //fill row in the "INFORMATION_SCHEMA COLUMNS"
 //Reference from https://dev.mysql.com/doc/refman/8.0/en/information-schema-columns-table.html
 Status SchemaColumnsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
@@ -230,39 +233,43 @@ Status SchemaColumnsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
             tuple->set_null(_tuple_desc->slots()[0]->null_indicator_offset());
         } else {
             void* slot = tuple->get_slot(_tuple_desc->slots()[0]->tuple_offset());
-            StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+            StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+            // todo: we may change all of StringRef in similar usage
+            // to Slice someday to distinguish different purposes of use.
+            // or just merge them.
             std::string catalog_name = _db_result.catalogs[_db_index - 1];
-            str_slot->ptr = (char*)pool->allocate(catalog_name.size());
-            str_slot->len = catalog_name.size();
-            memcpy(str_slot->ptr, catalog_name.c_str(), str_slot->len);
+            str_slot->data = (char*)pool->allocate(catalog_name.size());
+            str_slot->size = catalog_name.size();
+            memcpy(const_cast<char*>(str_slot->data), catalog_name.c_str(), str_slot->size);
         }
     }
     // TABLE_SCHEMA
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         std::string db_name = SchemaHelper::extract_db_name(_db_result.dbs[_db_index - 1]);
-        str_slot->ptr = (char*)pool->allocate(db_name.size());
-        str_slot->len = db_name.size();
-        memcpy(str_slot->ptr, db_name.c_str(), str_slot->len);
+        str_slot->data = (char*)pool->allocate(db_name.size());
+        str_slot->size = db_name.size();
+        memcpy(const_cast<char*>(str_slot->data), db_name.c_str(), str_slot->size);
     }
     // TABLE_NAME
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[2]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-        str_slot->ptr = (char*)pool->allocate(_table_result.tables[_table_index - 1].length());
-        str_slot->len = _table_result.tables[_table_index - 1].length();
-        memcpy(str_slot->ptr, _table_result.tables[_table_index - 1].c_str(), str_slot->len);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+        str_slot->data = (char*)pool->allocate(_table_result.tables[_table_index - 1].length());
+        str_slot->size = _table_result.tables[_table_index - 1].length();
+        memcpy(const_cast<char*>(str_slot->data), _table_result.tables[_table_index - 1].c_str(),
+               str_slot->size);
     }
     // COLUMN_NAME
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[3]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-        str_slot->ptr = (char*)pool->allocate(
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+        str_slot->data = (char*)pool->allocate(
                 _desc_result.columns[_column_index].columnDesc.columnName.length());
-        str_slot->len = _desc_result.columns[_column_index].columnDesc.columnName.length();
-        memcpy(str_slot->ptr, _desc_result.columns[_column_index].columnDesc.columnName.c_str(),
-               str_slot->len);
+        str_slot->size = _desc_result.columns[_column_index].columnDesc.columnName.length();
+        memcpy(const_cast<char*>(str_slot->data),
+               _desc_result.columns[_column_index].columnDesc.columnName.c_str(), str_slot->size);
     }
     // ORDINAL_POSITION
     {
@@ -275,33 +282,33 @@ Status SchemaColumnsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
     // IS_NULLABLE
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[6]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
 
         if (_desc_result.columns[_column_index].columnDesc.__isset.isAllowNull) {
             if (_desc_result.columns[_column_index].columnDesc.isAllowNull) {
-                str_slot->len = strlen("YES");
-                str_slot->ptr = (char*)pool->allocate(str_slot->len);
-                memcpy(str_slot->ptr, "YES", str_slot->len);
+                str_slot->size = strlen("YES");
+                str_slot->data = (char*)pool->allocate(str_slot->size);
+                memcpy(const_cast<char*>(str_slot->data), "YES", str_slot->size);
             } else {
-                str_slot->len = strlen("NO");
-                str_slot->ptr = (char*)pool->allocate(str_slot->len);
-                memcpy(str_slot->ptr, "NO", str_slot->len);
+                str_slot->size = strlen("NO");
+                str_slot->data = (char*)pool->allocate(str_slot->size);
+                memcpy(const_cast<char*>(str_slot->data), "NO", str_slot->size);
             }
         } else {
-            str_slot->len = strlen("NO");
-            str_slot->ptr = (char*)pool->allocate(str_slot->len);
-            memcpy(str_slot->ptr, "NO", str_slot->len);
+            str_slot->size = strlen("NO");
+            str_slot->data = (char*)pool->allocate(str_slot->size);
+            memcpy(const_cast<char*>(str_slot->data), "NO", str_slot->size);
         }
     }
     // DATA_TYPE
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[7]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         std::string buffer =
                 to_mysql_data_type_string(_desc_result.columns[_column_index].columnDesc);
-        str_slot->len = buffer.length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        memcpy(str_slot->ptr, buffer.c_str(), str_slot->len);
+        str_slot->size = buffer.length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        memcpy(const_cast<char*>(str_slot->data), buffer.c_str(), str_slot->size);
     }
     // CHARACTER_MAXIMUM_LENGTH
     // For string columns, the maximum length in characters.
@@ -366,51 +373,54 @@ Status SchemaColumnsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
     // COLUMN_TYPE
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[15]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         std::string buffer = type_to_string(_desc_result.columns[_column_index].columnDesc);
-        str_slot->len = buffer.length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        memcpy(str_slot->ptr, buffer.c_str(), str_slot->len);
+        str_slot->size = buffer.length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        memcpy(const_cast<char*>(str_slot->data), buffer.c_str(), str_slot->size);
     }
     // COLUMN_KEY
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[16]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         if (_desc_result.columns[_column_index].columnDesc.__isset.columnKey) {
-            str_slot->len = _desc_result.columns[_column_index].columnDesc.columnKey.length();
-            str_slot->ptr = (char*)pool->allocate(
+            str_slot->size = _desc_result.columns[_column_index].columnDesc.columnKey.length();
+            str_slot->data = (char*)pool->allocate(
                     _desc_result.columns[_column_index].columnDesc.columnKey.length());
-            memcpy(str_slot->ptr, _desc_result.columns[_column_index].columnDesc.columnKey.c_str(),
-                   str_slot->len);
+            memcpy(const_cast<char*>(str_slot->data),
+                   _desc_result.columns[_column_index].columnDesc.columnKey.c_str(),
+                   str_slot->size);
         } else {
-            str_slot->len = strlen("") + 1;
-            str_slot->ptr = (char*)pool->allocate(str_slot->len);
-            memcpy(str_slot->ptr, "", str_slot->len);
+            str_slot->size = strlen("") + 1;
+            str_slot->data = (char*)pool->allocate(str_slot->size);
+            memcpy(const_cast<char*>(str_slot->data), "", str_slot->size);
         }
     }
     // EXTRA
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[17]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-        str_slot->len = strlen("") + 1;
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        memcpy(str_slot->ptr, "", str_slot->len);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+        str_slot->size = strlen("") + 1;
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        memcpy(const_cast<char*>(str_slot->data), "", str_slot->size);
     }
     // PRIVILEGES
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[18]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-        str_slot->len = strlen("") + 1;
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        memcpy(str_slot->ptr, "", str_slot->len);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+        str_slot->size = strlen("") + 1;
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        memcpy(const_cast<char*>(str_slot->data), "", str_slot->size);
     }
     // COLUMN_COMMENT
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[19]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-        str_slot->ptr = (char*)pool->allocate(_desc_result.columns[_column_index].comment.length());
-        str_slot->len = _desc_result.columns[_column_index].comment.length();
-        memcpy(str_slot->ptr, _desc_result.columns[_column_index].comment.c_str(), str_slot->len);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+        str_slot->data =
+                (char*)pool->allocate(_desc_result.columns[_column_index].comment.length());
+        str_slot->size = _desc_result.columns[_column_index].comment.length();
+        memcpy(const_cast<char*>(str_slot->data),
+               _desc_result.columns[_column_index].comment.c_str(), str_slot->size);
     }
     // COLUMN_SIZE
     {
diff --git a/be/src/exec/schema_scanner/schema_files_scanner.cpp b/be/src/exec/schema_scanner/schema_files_scanner.cpp
index d237d3da6e..c9a6b9721d 100644
--- a/be/src/exec/schema_scanner/schema_files_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_files_scanner.cpp
@@ -19,50 +19,50 @@
 
 #include "exec/schema_scanner/schema_helper.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaFilesScanner::_s_tbls_columns[] = {
         //   name,       type,          size,     is_null
         {"FILE_ID", TYPE_BIGINT, sizeof(int64_t), true},
-        {"FILE_NAME", TYPE_STRING, sizeof(StringValue), true},
-        {"FILE_TYPE", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLESPACE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"TABLE_CATALOG", TYPE_CHAR, sizeof(StringValue), false},
-        {"TABLE_SCHEMA", TYPE_STRING, sizeof(StringValue), true},
-        {"TABLE_NAME", TYPE_STRING, sizeof(StringValue), true},
-        {"LOGFILE_GROUP_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"FILE_NAME", TYPE_STRING, sizeof(StringRef), true},
+        {"FILE_TYPE", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLESPACE_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"TABLE_CATALOG", TYPE_CHAR, sizeof(StringRef), false},
+        {"TABLE_SCHEMA", TYPE_STRING, sizeof(StringRef), true},
+        {"TABLE_NAME", TYPE_STRING, sizeof(StringRef), true},
+        {"LOGFILE_GROUP_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
         {"LOGFILE_GROUP_NUMBER", TYPE_BIGINT, sizeof(int64_t), true},
-        {"ENGINE", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"FULLTEXT_KEYS", TYPE_STRING, sizeof(StringValue), true},
-        {"DELETED_ROWS", TYPE_STRING, sizeof(StringValue), true},
-        {"UPDATE_COUNT", TYPE_STRING, sizeof(StringValue), true},
+        {"ENGINE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"FULLTEXT_KEYS", TYPE_STRING, sizeof(StringRef), true},
+        {"DELETED_ROWS", TYPE_STRING, sizeof(StringRef), true},
+        {"UPDATE_COUNT", TYPE_STRING, sizeof(StringRef), true},
         {"FREE_EXTENTS", TYPE_BIGINT, sizeof(int64_t), true},
         {"TOTAL_EXTENTS", TYPE_BIGINT, sizeof(int64_t), true},
         {"EXTENT_SIZE", TYPE_BIGINT, sizeof(int64_t), true},
         {"INITIAL_SIZE", TYPE_BIGINT, sizeof(int64_t), true},
         {"MAXIMUM_SIZE", TYPE_BIGINT, sizeof(int64_t), true},
         {"AUTOEXTEND_SIZE", TYPE_BIGINT, sizeof(int64_t), true},
-        {"CREATION_TIME", TYPE_STRING, sizeof(StringValue), true},
-        {"LAST_UPDATE_TIME", TYPE_STRING, sizeof(StringValue), true},
-        {"LAST_ACCESS_TIME", TYPE_STRING, sizeof(StringValue), true},
-        {"RECOVER_TIME", TYPE_STRING, sizeof(StringValue), true},
-        {"TRANSACTION_COUNTER", TYPE_STRING, sizeof(StringValue), true},
+        {"CREATION_TIME", TYPE_STRING, sizeof(StringRef), true},
+        {"LAST_UPDATE_TIME", TYPE_STRING, sizeof(StringRef), true},
+        {"LAST_ACCESS_TIME", TYPE_STRING, sizeof(StringRef), true},
+        {"RECOVER_TIME", TYPE_STRING, sizeof(StringRef), true},
+        {"TRANSACTION_COUNTER", TYPE_STRING, sizeof(StringRef), true},
         {"VERSION", TYPE_BIGINT, sizeof(int64_t), true},
-        {"ROW_FORMAT", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_ROWS", TYPE_STRING, sizeof(StringValue), true},
-        {"AVG_ROW_LENGTH", TYPE_STRING, sizeof(StringValue), true},
-        {"DATA_LENGTH", TYPE_STRING, sizeof(StringValue), true},
-        {"MAX_DATA_LENGTH", TYPE_STRING, sizeof(StringValue), true},
-        {"INDEX_LENGTH", TYPE_STRING, sizeof(StringValue), true},
+        {"ROW_FORMAT", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_ROWS", TYPE_STRING, sizeof(StringRef), true},
+        {"AVG_ROW_LENGTH", TYPE_STRING, sizeof(StringRef), true},
+        {"DATA_LENGTH", TYPE_STRING, sizeof(StringRef), true},
+        {"MAX_DATA_LENGTH", TYPE_STRING, sizeof(StringRef), true},
+        {"INDEX_LENGTH", TYPE_STRING, sizeof(StringRef), true},
         {"DATA_FREE", TYPE_BIGINT, sizeof(int64_t), true},
-        {"CREATE_TIME", TYPE_STRING, sizeof(StringValue), true},
-        {"UPDATE_TIME", TYPE_STRING, sizeof(StringValue), true},
-        {"CHECK_TIME", TYPE_STRING, sizeof(StringValue), true},
-        {"CHECKSUM", TYPE_STRING, sizeof(StringValue), true},
-        {"STATUS", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"EXTRA", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"CREATE_TIME", TYPE_STRING, sizeof(StringRef), true},
+        {"UPDATE_TIME", TYPE_STRING, sizeof(StringRef), true},
+        {"CHECK_TIME", TYPE_STRING, sizeof(StringRef), true},
+        {"CHECKSUM", TYPE_STRING, sizeof(StringRef), true},
+        {"STATUS", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"EXTRA", TYPE_VARCHAR, sizeof(StringRef), true},
 };
 
 SchemaFilesScanner::SchemaFilesScanner()
diff --git a/be/src/exec/schema_scanner/schema_partitions_scanner.cpp b/be/src/exec/schema_scanner/schema_partitions_scanner.cpp
index 018e86ede5..71c10ce193 100644
--- a/be/src/exec/schema_scanner/schema_partitions_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_partitions_scanner.cpp
@@ -20,24 +20,24 @@
 #include "exec/schema_scanner/schema_helper.h"
 #include "runtime/datetime_value.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaPartitionsScanner::_s_tbls_columns[] = {
         //   name,       type,          size,     is_null
-        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"PARTITION_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"SUBPARTITION_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"PARTITION_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"SUBPARTITION_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
         {"PARTITION_ORDINAL_POSITION", TYPE_INT, sizeof(int32_t), true},
         {"SUBPARTITION_ORDINAL_POSITION", TYPE_INT, sizeof(int32_t), true},
-        {"PARTITION_METHOD", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"SUBPARTITION_METHOD", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"PARTITION_EXPRESSION", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"SUBPARTITION_EXPRESSION", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"PARTITION_DESCRIPTION", TYPE_STRING, sizeof(StringValue), true},
+        {"PARTITION_METHOD", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"SUBPARTITION_METHOD", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"PARTITION_EXPRESSION", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"SUBPARTITION_EXPRESSION", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"PARTITION_DESCRIPTION", TYPE_STRING, sizeof(StringRef), true},
         {"TABLE_ROWS", TYPE_BIGINT, sizeof(int64_t), true},
         {"AVG_ROW_LENGTH", TYPE_BIGINT, sizeof(int64_t), true},
         {"DATA_LENGTH", TYPE_BIGINT, sizeof(int64_t), true},
@@ -48,9 +48,9 @@ SchemaScanner::ColumnDesc SchemaPartitionsScanner::_s_tbls_columns[] = {
         {"UPDATE_TIME", TYPE_DATETIME, sizeof(DateTimeValue), true},
         {"CHECK_TIME", TYPE_DATETIME, sizeof(DateTimeValue), true},
         {"CHECKSUM", TYPE_BIGINT, sizeof(int64_t), true},
-        {"PARTITION_COMMENT", TYPE_STRING, sizeof(StringValue), false},
-        {"NODEGROUP", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLESPACE_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"PARTITION_COMMENT", TYPE_STRING, sizeof(StringRef), false},
+        {"NODEGROUP", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLESPACE_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
 };
 
 SchemaPartitionsScanner::SchemaPartitionsScanner()
diff --git a/be/src/exec/schema_scanner/schema_rowsets_scanner.cpp b/be/src/exec/schema_scanner/schema_rowsets_scanner.cpp
index 00314b8306..800be0c1aa 100644
--- a/be/src/exec/schema_scanner/schema_rowsets_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_rowsets_scanner.cpp
@@ -29,12 +29,12 @@
 #include "olap/tablet.h"
 #include "runtime/descriptors.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 namespace doris {
 SchemaScanner::ColumnDesc SchemaRowsetsScanner::_s_tbls_columns[] = {
         //   name,       type,          size,     is_null
         {"BACKEND_ID", TYPE_BIGINT, sizeof(int64_t), true},
-        {"ROWSET_ID", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"ROWSET_ID", TYPE_VARCHAR, sizeof(StringRef), true},
         {"TABLET_ID", TYPE_BIGINT, sizeof(int64_t), true},
         {"ROWSET_NUM_ROWS", TYPE_BIGINT, sizeof(int64_t), true},
         {"TXN_ID", TYPE_BIGINT, sizeof(int64_t), true},
@@ -109,11 +109,11 @@ Status SchemaRowsetsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
     // ROWSET_ID
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         std::string rowset_id = rowset->rowset_id().to_string();
-        str_slot->ptr = (char*)pool->allocate(rowset_id.size());
-        str_slot->len = rowset_id.size();
-        memcpy(str_slot->ptr, rowset_id.c_str(), str_slot->len);
+        str_slot->data = (char*)pool->allocate(rowset_id.size());
+        str_slot->size = rowset_id.size();
+        memcpy(const_cast<char*>(str_slot->data), rowset_id.c_str(), str_slot->size);
     }
     // TABLET_ID
     {
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 196f796d36..6b082ba82a 100644
--- a/be/src/exec/schema_scanner/schema_schema_privileges_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_schema_privileges_scanner.cpp
@@ -19,17 +19,17 @@
 
 #include "exec/schema_scanner/schema_helper.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaSchemaPrivilegesScanner::_s_tbls_columns[] = {
         //   name,       type,          size,     is_null
-        {"GRANTEE", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"PRIVILEGE_TYPE", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"IS_GRANTABLE", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"GRANTEE", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"PRIVILEGE_TYPE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"IS_GRANTABLE", TYPE_VARCHAR, sizeof(StringRef), true},
 };
 
 SchemaSchemaPrivilegesScanner::SchemaSchemaPrivilegesScanner()
@@ -102,13 +102,13 @@ Status SchemaSchemaPrivilegesScanner::fill_one_col(const std::string* src, MemPo
     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 (nullptr == str_slot->ptr) {
-        return Status::InternalError("Allocate memcpy failed.");
+    StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+    str_slot->size = src->length();
+    str_slot->data = (char*)pool->allocate(str_slot->size);
+    if (nullptr == str_slot->data) {
+        return Status::InternalError("Allocate memory failed.");
     }
-    memcpy(str_slot->ptr, src->c_str(), str_slot->len);
+    memcpy(const_cast<char*>(str_slot->data), src->c_str(), str_slot->size);
     return Status::OK();
 }
 
diff --git a/be/src/exec/schema_scanner/schema_schemata_scanner.cpp b/be/src/exec/schema_scanner/schema_schemata_scanner.cpp
index 278e93ddb1..10427963f9 100644
--- a/be/src/exec/schema_scanner/schema_schemata_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_schemata_scanner.cpp
@@ -19,24 +19,24 @@
 
 #include "exec/schema_scanner/schema_helper.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaSchemataScanner::_s_columns[] = {
         //   name,       type,          size
-        {"CATALOG_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"SCHEMA_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"DEFAULT_CHARACTER_SET_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"DEFAULT_COLLATION_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"SQL_PATH", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"CATALOG_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"SCHEMA_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"DEFAULT_CHARACTER_SET_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"DEFAULT_COLLATION_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"SQL_PATH", TYPE_VARCHAR, sizeof(StringRef), true},
 };
 
 SchemaSchemataScanner::SchemaSchemataScanner()
         : SchemaScanner(_s_columns, sizeof(_s_columns) / sizeof(SchemaScanner::ColumnDesc)),
           _db_index(0) {}
 
-SchemaSchemataScanner::~SchemaSchemataScanner() {}
+SchemaSchemataScanner::~SchemaSchemataScanner() = default;
 
 Status SchemaSchemataScanner::start(RuntimeState* state) {
     if (!_is_init) {
@@ -80,43 +80,43 @@ Status SchemaSchemataScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
             tuple->set_null(_tuple_desc->slots()[0]->null_indicator_offset());
         } else {
             void* slot = tuple->get_slot(_tuple_desc->slots()[0]->tuple_offset());
-            StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+            StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
             std::string catalog_name = _db_result.catalogs[_db_index];
-            str_slot->ptr = (char*)pool->allocate(catalog_name.size());
-            str_slot->len = catalog_name.size();
-            memcpy(str_slot->ptr, catalog_name.c_str(), str_slot->len);
+            str_slot->data = (char*)pool->allocate(catalog_name.size());
+            str_slot->size = catalog_name.size();
+            memcpy(const_cast<char*>(str_slot->data), catalog_name.c_str(), str_slot->size);
         }
     }
     // schema
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         std::string db_name = SchemaHelper::extract_db_name(_db_result.dbs[_db_index]);
-        str_slot->ptr = (char*)pool->allocate(db_name.size());
-        str_slot->len = db_name.size();
-        memcpy(str_slot->ptr, db_name.c_str(), str_slot->len);
+        str_slot->data = (char*)pool->allocate(db_name.size());
+        str_slot->size = db_name.size();
+        memcpy(const_cast<char*>(str_slot->data), db_name.c_str(), str_slot->size);
     }
     // DEFAULT_CHARACTER_SET_NAME
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[2]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-        str_slot->len = strlen("utf8") + 1;
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+        str_slot->size = strlen("utf8") + 1;
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("Allocate memory failed.");
         }
-        memcpy(str_slot->ptr, "utf8", str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), "utf8", str_slot->size);
     }
     // DEFAULT_COLLATION_NAME
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[3]->tuple_offset());
-        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 (nullptr == str_slot->ptr) {
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+        str_slot->size = strlen("utf8_general_ci") + 1;
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("Allocate memory failed.");
         }
-        memcpy(str_slot->ptr, "utf8_general_ci", str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), "utf8_general_ci", str_slot->size);
     }
     // SQL_PATH
     { tuple->set_null(_tuple_desc->slots()[4]->null_indicator_offset()); }
diff --git a/be/src/exec/schema_scanner/schema_statistics_scanner.cpp b/be/src/exec/schema_scanner/schema_statistics_scanner.cpp
index 51835ed22a..366765dda6 100644
--- a/be/src/exec/schema_scanner/schema_statistics_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_statistics_scanner.cpp
@@ -18,27 +18,27 @@
 #include "exec/schema_scanner/schema_statistics_scanner.h"
 
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaStatisticsScanner::_s_cols_statistics[] = {
         //   name,       type,          size,                     is_null
-        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
+        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
         {"NON_UNIQUE", TYPE_BIGINT, sizeof(int64_t), false},
-        {"INDEX_SCHEMA", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"INDEX_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
+        {"INDEX_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"INDEX_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
         {"SEQ_IN_INDEX", TYPE_BIGINT, sizeof(int64_t), false},
-        {"COLUMN_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"COLLATION", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"COLUMN_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"COLLATION", TYPE_VARCHAR, sizeof(StringRef), true},
         {"CARDINALITY", TYPE_BIGINT, sizeof(int64_t), true},
         {"SUB_PART", TYPE_BIGINT, sizeof(int64_t), true},
-        {"PACKED", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"NULLABLE", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"INDEX_TYPE", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"COMMENT", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"PACKED", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"NULLABLE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"INDEX_TYPE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"COMMENT", TYPE_VARCHAR, sizeof(StringRef), true},
 };
 
 SchemaStatisticsScanner::SchemaStatisticsScanner()
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 372e721c28..1d572bc276 100644
--- a/be/src/exec/schema_scanner/schema_table_privileges_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_table_privileges_scanner.cpp
@@ -19,18 +19,18 @@
 
 #include "exec/schema_scanner/schema_helper.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaTablePrivilegesScanner::_s_tbls_columns[] = {
         //   name,       type,          size,     is_null
-        {"GRANTEE", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"PRIVILEGE_TYPE", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"IS_GRANTABLE", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"GRANTEE", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"PRIVILEGE_TYPE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"IS_GRANTABLE", TYPE_VARCHAR, sizeof(StringRef), true},
 };
 
 SchemaTablePrivilegesScanner::SchemaTablePrivilegesScanner()
@@ -111,13 +111,13 @@ Status SchemaTablePrivilegesScanner::fill_one_col(const std::string* src, MemPoo
     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 (nullptr == str_slot->ptr) {
-        return Status::InternalError("Allocate memcpy failed.");
+    StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+    str_slot->size = src->length();
+    str_slot->data = (char*)pool->allocate(str_slot->size);
+    if (nullptr == str_slot->data) {
+        return Status::InternalError("Allocate memory failed.");
     }
-    memcpy(str_slot->ptr, src->c_str(), str_slot->len);
+    memcpy(const_cast<char*>(str_slot->data), src->c_str(), str_slot->size);
     return Status::OK();
 }
 
diff --git a/be/src/exec/schema_scanner/schema_tables_scanner.cpp b/be/src/exec/schema_scanner/schema_tables_scanner.cpp
index 8bf05b894f..ee9febe3f2 100644
--- a/be/src/exec/schema_scanner/schema_tables_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_tables_scanner.cpp
@@ -19,19 +19,19 @@
 
 #include "exec/schema_scanner/schema_helper.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaTablesScanner::_s_tbls_columns[] = {
         //   name,       type,          size,     is_null
-        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"TABLE_TYPE", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"ENGINE", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"TABLE_TYPE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"ENGINE", TYPE_VARCHAR, sizeof(StringRef), true},
         {"VERSION", TYPE_BIGINT, sizeof(int64_t), true},
-        {"ROW_FORMAT", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"ROW_FORMAT", TYPE_VARCHAR, sizeof(StringRef), true},
         {"TABLE_ROWS", TYPE_BIGINT, sizeof(int64_t), true},
         {"AVG_ROW_LENGTH", TYPE_BIGINT, sizeof(int64_t), true},
         {"DATA_LENGTH", TYPE_BIGINT, sizeof(int64_t), true},
@@ -42,10 +42,10 @@ SchemaScanner::ColumnDesc SchemaTablesScanner::_s_tbls_columns[] = {
         {"CREATE_TIME", TYPE_DATETIME, sizeof(DateTimeValue), true},
         {"UPDATE_TIME", TYPE_DATETIME, sizeof(DateTimeValue), true},
         {"CHECK_TIME", TYPE_DATETIME, sizeof(DateTimeValue), true},
-        {"TABLE_COLLATION", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"TABLE_COLLATION", TYPE_VARCHAR, sizeof(StringRef), true},
         {"CHECKSUM", TYPE_BIGINT, sizeof(int64_t), true},
-        {"CREATE_OPTIONS", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_COMMENT", TYPE_VARCHAR, sizeof(StringValue), false},
+        {"CREATE_OPTIONS", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_COMMENT", TYPE_VARCHAR, sizeof(StringRef), false},
 };
 
 SchemaTablesScanner::SchemaTablesScanner()
@@ -97,57 +97,57 @@ Status SchemaTablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
             tuple->set_null(_tuple_desc->slots()[0]->null_indicator_offset());
         } else {
             void* slot = tuple->get_slot(_tuple_desc->slots()[0]->tuple_offset());
-            StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+            StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
             std::string catalog_name = _db_result.catalogs[_db_index - 1];
-            str_slot->ptr = (char*)pool->allocate(catalog_name.size());
-            str_slot->len = catalog_name.size();
-            memcpy(str_slot->ptr, catalog_name.c_str(), str_slot->len);
+            str_slot->data = (char*)pool->allocate(catalog_name.size());
+            str_slot->size = catalog_name.size();
+            memcpy(const_cast<char*>(str_slot->data), catalog_name.c_str(), str_slot->size);
         }
     }
     // schema
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         std::string db_name = SchemaHelper::extract_db_name(_db_result.dbs[_db_index - 1]);
-        str_slot->ptr = (char*)pool->allocate(db_name.size());
-        str_slot->len = db_name.size();
-        memcpy(str_slot->ptr, db_name.c_str(), str_slot->len);
+        str_slot->data = (char*)pool->allocate(db_name.size());
+        str_slot->size = db_name.size();
+        memcpy(const_cast<char*>(str_slot->data), db_name.c_str(), str_slot->size);
     }
     // name
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[2]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         const std::string* src = &tbl_status.name;
-        str_slot->len = src->length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
-            return Status::InternalError("Allocate memcpy failed.");
+        str_slot->size = src->length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
+            return Status::InternalError("Allocate memory failed.");
         }
-        memcpy(str_slot->ptr, src->c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), src->c_str(), str_slot->size);
     }
     // type
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[3]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         const std::string* src = &tbl_status.type;
-        str_slot->len = src->length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
-            return Status::InternalError("Allocate memcpy failed.");
+        str_slot->size = src->length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
+            return Status::InternalError("Allocate memory failed.");
         }
-        memcpy(str_slot->ptr, src->c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), src->c_str(), str_slot->size);
     }
     // engine
     if (tbl_status.__isset.engine) {
         void* slot = tuple->get_slot(_tuple_desc->slots()[4]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         const std::string* src = &tbl_status.engine;
-        str_slot->len = src->length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
-            return Status::InternalError("Allocate memcpy failed.");
+        str_slot->size = src->length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
+            return Status::InternalError("Allocate memory failed.");
         }
-        memcpy(str_slot->ptr, src->c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), src->c_str(), str_slot->size);
     } else {
         tuple->set_null(_tuple_desc->slots()[4]->null_indicator_offset());
     }
@@ -222,14 +222,14 @@ Status SchemaTablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
     // collation
     if (tbl_status.__isset.collation) {
         void* slot = tuple->get_slot(_tuple_desc->slots()[17]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         const std::string* src = &tbl_status.collation;
-        str_slot->len = src->length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
-            return Status::InternalError("Allocate memcpy failed.");
+        str_slot->size = src->length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
+            return Status::InternalError("Allocate memory failed.");
         }
-        memcpy(str_slot->ptr, src->c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), src->c_str(), str_slot->size);
     } else {
         tuple->set_null(_tuple_desc->slots()[17]->null_indicator_offset());
     }
@@ -240,17 +240,17 @@ Status SchemaTablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
     // create_comment
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[20]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         const std::string* src = &tbl_status.comment;
-        str_slot->len = src->length();
-        if (str_slot->len == 0) {
-            str_slot->ptr = nullptr;
+        str_slot->size = src->length();
+        if (str_slot->size == 0) {
+            str_slot->data = nullptr;
         } else {
-            str_slot->ptr = (char*)pool->allocate(str_slot->len);
-            if (nullptr == str_slot->ptr) {
-                return Status::InternalError("Allocate memcpy failed.");
+            str_slot->data = (char*)pool->allocate(str_slot->size);
+            if (nullptr == str_slot->data) {
+                return Status::InternalError("Allocate memory failed.");
             }
-            memcpy(str_slot->ptr, src->c_str(), str_slot->len);
+            memcpy(const_cast<char*>(str_slot->data), src->c_str(), str_slot->size);
         }
     }
     _table_index++;
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 533311319e..e2a1b86eb2 100644
--- a/be/src/exec/schema_scanner/schema_user_privileges_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_user_privileges_scanner.cpp
@@ -19,16 +19,16 @@
 
 #include "exec/schema_scanner/schema_helper.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaUserPrivilegesScanner::_s_tbls_columns[] = {
         //   name,       type,          size,     is_null
-        {"GRANTEE", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"PRIVILEGE_TYPE", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"IS_GRANTABLE", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"GRANTEE", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"PRIVILEGE_TYPE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"IS_GRANTABLE", TYPE_VARCHAR, sizeof(StringRef), true},
 };
 
 SchemaUserPrivilegesScanner::SchemaUserPrivilegesScanner()
@@ -93,13 +93,13 @@ Status SchemaUserPrivilegesScanner::fill_one_col(const std::string* src, MemPool
     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 (nullptr == str_slot->ptr) {
-        return Status::InternalError("Allocate memcpy failed.");
+    StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+    str_slot->size = src->length();
+    str_slot->data = (char*)pool->allocate(str_slot->size);
+    if (nullptr == str_slot->data) {
+        return Status::InternalError("Allocate memory failed.");
     }
-    memcpy(str_slot->ptr, src->c_str(), str_slot->len);
+    memcpy(const_cast<char*>(str_slot->data), src->c_str(), str_slot->size);
     return Status::OK();
 }
 
diff --git a/be/src/exec/schema_scanner/schema_variables_scanner.cpp b/be/src/exec/schema_scanner/schema_variables_scanner.cpp
index 69842d81f0..f550e8a0dd 100644
--- a/be/src/exec/schema_scanner/schema_variables_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_variables_scanner.cpp
@@ -20,14 +20,14 @@
 #include "exec/schema_scanner/schema_helper.h"
 #include "runtime/primitive_type.h"
 #include "runtime/runtime_state.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaVariablesScanner::_s_vars_columns[] = {
         //   name,       type,          size
-        {"VARIABLE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"VARIABLE_VALUE", TYPE_VARCHAR, sizeof(StringValue), false},
+        {"VARIABLE_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"VARIABLE_VALUE", TYPE_VARCHAR, sizeof(StringRef), false},
 };
 
 SchemaVariablesScanner::SchemaVariablesScanner(TVarType::type type)
@@ -65,26 +65,26 @@ Status SchemaVariablesScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
     // variables names
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[0]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         int len = strlen(_begin->first.c_str());
-        str_slot->ptr = (char*)pool->allocate(len + 1);
-        if (nullptr == str_slot->ptr) {
+        str_slot->data = (char*)pool->allocate(len + 1);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("No Memory.");
         }
-        memcpy(str_slot->ptr, _begin->first.c_str(), len + 1);
-        str_slot->len = len;
+        memcpy(const_cast<char*>(str_slot->data), _begin->first.c_str(), len + 1);
+        str_slot->size = len;
     }
     // value
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         int len = strlen(_begin->second.c_str());
-        str_slot->ptr = (char*)pool->allocate(len + 1);
-        if (nullptr == str_slot->ptr) {
+        str_slot->data = (char*)pool->allocate(len + 1);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("No Memory.");
         }
-        memcpy(str_slot->ptr, _begin->second.c_str(), len + 1);
-        str_slot->len = len;
+        memcpy(const_cast<char*>(str_slot->data), _begin->second.c_str(), len + 1);
+        str_slot->size = len;
     }
     ++_begin;
     return Status::OK();
diff --git a/be/src/exec/schema_scanner/schema_views_scanner.cpp b/be/src/exec/schema_scanner/schema_views_scanner.cpp
index 4df4d1de31..af04de20df 100644
--- a/be/src/exec/schema_scanner/schema_views_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_views_scanner.cpp
@@ -19,22 +19,22 @@
 
 #include "exec/schema_scanner/schema_helper.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 SchemaScanner::ColumnDesc SchemaViewsScanner::_s_tbls_columns[] = {
         //   name,       type,          size,     is_null
-        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
-        {"VIEW_DEFINITION", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"CHECK_OPTION", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"IS_UPDATABLE", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"DEFINER", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"SECURITY_TYPE", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"CHARACTER_SET_CLIENT", TYPE_VARCHAR, sizeof(StringValue), true},
-        {"COLLATION_CONNECTION", TYPE_VARCHAR, sizeof(StringValue), true},
+        {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"VIEW_DEFINITION", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"CHECK_OPTION", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"IS_UPDATABLE", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"DEFINER", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"SECURITY_TYPE", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"CHARACTER_SET_CLIENT", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"COLLATION_CONNECTION", TYPE_VARCHAR, sizeof(StringRef), true},
 };
 
 SchemaViewsScanner::SchemaViewsScanner()
@@ -85,100 +85,100 @@ Status SchemaViewsScanner::fill_one_row(Tuple* tuple, MemPool* pool) {
     // schema
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         std::string db_name = SchemaHelper::extract_db_name(_db_result.dbs[_db_index - 1]);
-        str_slot->ptr = (char*)pool->allocate(db_name.size());
-        str_slot->len = db_name.size();
-        memcpy(str_slot->ptr, db_name.c_str(), str_slot->len);
+        str_slot->data = (char*)pool->allocate(db_name.size());
+        str_slot->size = db_name.size();
+        memcpy(const_cast<char*>(str_slot->data), db_name.c_str(), str_slot->size);
     }
     // name
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[2]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         const std::string* src = &tbl_status.name;
-        str_slot->len = src->length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
+        str_slot->size = src->length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("Allocate memcpy failed.");
         }
-        memcpy(str_slot->ptr, src->c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), src->c_str(), str_slot->size);
     }
     // definition
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[3]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         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 (nullptr == str_slot->ptr) {
+        str_slot->size = ddl_sql->length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("Allocate memcpy failed.");
         }
-        memcpy(str_slot->ptr, ddl_sql->c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), ddl_sql->c_str(), str_slot->size);
     }
     // check_option
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[4]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         // This is from views in mysql
         const std::string check_option = "NONE";
-        str_slot->len = check_option.length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
+        str_slot->size = check_option.length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("Allocate memcpy failed.");
         }
-        memcpy(str_slot->ptr, check_option.c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), check_option.c_str(), str_slot->size);
     }
     // is_updatable
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[5]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         // This is from views in mysql
         const std::string is_updatable = "NO";
-        str_slot->len = is_updatable.length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
+        str_slot->size = is_updatable.length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("Allocate memcpy failed.");
         }
-        memcpy(str_slot->ptr, is_updatable.c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), is_updatable.c_str(), str_slot->size);
     }
     // definer
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[6]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         // This is from views in mysql
         const std::string definer = "root@%";
-        str_slot->len = definer.length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
+        str_slot->size = definer.length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
             return Status::InternalError("Allocate memcpy failed.");
         }
-        memcpy(str_slot->ptr, definer.c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), definer.c_str(), str_slot->size);
     }
     // security_type
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[7]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         // This is from views in mysql
         const std::string security_type = "DEFINER";
-        str_slot->len = security_type.length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
-            return Status::InternalError("Allocate memcpy failed.");
+        str_slot->size = security_type.length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
+            return Status::InternalError("Allocate memory failed.");
         }
-        memcpy(str_slot->ptr, security_type.c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), security_type.c_str(), str_slot->size);
     }
     // character_set_client
     {
         void* slot = tuple->get_slot(_tuple_desc->slots()[8]->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
         // This is from views in mysql
         const std::string encoding = "utf8";
-        str_slot->len = encoding.length();
-        str_slot->ptr = (char*)pool->allocate(str_slot->len);
-        if (nullptr == str_slot->ptr) {
-            return Status::InternalError("Allocate memcpy failed.");
+        str_slot->size = encoding.length();
+        str_slot->data = (char*)pool->allocate(str_slot->size);
+        if (nullptr == str_slot->data) {
+            return Status::InternalError("Allocate memory failed.");
         }
-        memcpy(str_slot->ptr, encoding.c_str(), str_slot->len);
+        memcpy(const_cast<char*>(str_slot->data), encoding.c_str(), str_slot->size);
     }
     // collation_connection
     { tuple->set_null(_tuple_desc->slots()[9]->null_indicator_offset()); }
diff --git a/be/src/exec/text_converter.cpp b/be/src/exec/text_converter.cpp
index 5888eefc43..0839578098 100644
--- a/be/src/exec/text_converter.cpp
+++ b/be/src/exec/text_converter.cpp
@@ -22,12 +22,12 @@
 #include "runtime/decimalv2_value.h"
 #include "runtime/descriptors.h"
 #include "runtime/mem_pool.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple.h"
 #include "util/string_parser.hpp"
 #include "util/types.h"
 #include "vec/columns/column_complex.h"
 #include "vec/columns/column_nullable.h"
+#include "vec/common/string_ref.h"
 #include "vec/runtime/vdatetime_value.h"
 
 namespace doris {
@@ -188,10 +188,10 @@ bool TextConverter::write_vec_column(const SlotDescriptor* slot_desc,
     return true;
 }
 
-void TextConverter::unescape_string(StringValue* value, MemPool* pool) {
-    char* new_data = reinterpret_cast<char*>(pool->allocate(value->len));
-    unescape_string(value->ptr, new_data, &value->len);
-    value->ptr = new_data;
+void TextConverter::unescape_string(StringRef* value, MemPool* pool) {
+    char* new_data = reinterpret_cast<char*>(pool->allocate(value->size));
+    unescape_string(value->data, new_data, &value->size);
+    value->data = new_data;
 }
 
 void TextConverter::unescape_string(const char* src, char* dest, size_t* len) {
diff --git a/be/src/exec/text_converter.h b/be/src/exec/text_converter.h
index 79deac95ff..69c27e2c29 100644
--- a/be/src/exec/text_converter.h
+++ b/be/src/exec/text_converter.h
@@ -23,7 +23,7 @@ namespace doris {
 class MemPool;
 class SlotDescriptor;
 class Status;
-struct StringValue;
+struct StringRef;
 class Tuple;
 class TupleDescriptor;
 
@@ -68,7 +68,7 @@ public:
     void unescape_string_on_spot(const char* src, size_t* len);
     // Removes escape characters from 'str', allocating a new string from pool.
     // 'str' is updated with the new ptr and length.
-    void unescape_string(StringValue* str, MemPool* pool);
+    void unescape_string(StringRef* str, MemPool* pool);
 
 private:
     char _escape_char;
diff --git a/be/src/exec/text_converter.hpp b/be/src/exec/text_converter.hpp
index 55aee22d0a..b6add9183f 100644
--- a/be/src/exec/text_converter.hpp
+++ b/be/src/exec/text_converter.hpp
@@ -25,7 +25,6 @@
 #include "runtime/decimalv2_value.h"
 #include "runtime/descriptors.h"
 #include "runtime/mem_pool.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple.h"
 #include "text_converter.h"
 #include "util/binary_cast.hpp"
@@ -43,7 +42,7 @@ inline bool TextConverter::write_slot(const SlotDescriptor* slot_desc, Tuple* tu
                                       const char* data, int len, bool copy_string, bool need_escape,
                                       MemPool* pool) {
     //Small batch import only \N is considered to be NULL, there is no replace_value function for batch import
-    if (true == slot_desc->is_nullable()) {
+    if (slot_desc->is_nullable()) {
         if (len == 2 && data[0] == '\\' && data[1] == 'N') {
             tuple->set_null(slot_desc->null_indicator_offset());
             return true;
@@ -61,20 +60,20 @@ inline bool TextConverter::write_slot(const SlotDescriptor* slot_desc, Tuple* tu
     case TYPE_VARCHAR:
     case TYPE_CHAR:
     case TYPE_STRING: {
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-        str_slot->ptr = const_cast<char*>(data);
-        str_slot->len = len;
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+        str_slot->data = const_cast<char*>(data);
+        str_slot->size = len;
         if (len != 0 && (copy_string || need_escape)) {
-            DCHECK(pool != NULL);
+            DCHECK(pool != nullptr);
             char* slot_data = reinterpret_cast<char*>(pool->allocate(len));
 
             if (need_escape) {
-                unescape_string(data, slot_data, &str_slot->len);
+                unescape_string(data, slot_data, &str_slot->size);
             } else {
-                memcpy(slot_data, data, str_slot->len);
+                memcpy(slot_data, data, str_slot->size);
             }
 
-            str_slot->ptr = slot_data;
+            str_slot->data = slot_data;
         }
 
         break;
diff --git a/be/src/exprs/agg_fn_evaluator.cpp b/be/src/exprs/agg_fn_evaluator.cpp
index 5998c5c79f..e3a4b876da 100644
--- a/be/src/exprs/agg_fn_evaluator.cpp
+++ b/be/src/exprs/agg_fn_evaluator.cpp
@@ -309,8 +309,7 @@ inline void AggFnEvaluator::set_any_val(const void* slot, const TypeDescriptor&
     case TYPE_OBJECT:
     case TYPE_STRING:
     case TYPE_QUANTILE_STATE:
-        reinterpret_cast<const StringValue*>(slot)->to_string_val(
-                reinterpret_cast<StringVal*>(dst));
+        reinterpret_cast<const StringRef*>(slot)->to_string_val(reinterpret_cast<StringVal*>(dst));
         return;
 
     case TYPE_DATE:
@@ -381,8 +380,7 @@ inline void AggFnEvaluator::set_output_slot(const AnyVal* src, const SlotDescrip
     case TYPE_OBJECT:
     case TYPE_QUANTILE_STATE:
     case TYPE_STRING:
-        *reinterpret_cast<StringValue*>(slot) =
-                StringValue::from_string_val(*reinterpret_cast<const StringVal*>(src));
+        *reinterpret_cast<StringRef*>(slot) = StringRef(*reinterpret_cast<const StringVal*>(src));
         return;
 
     case TYPE_DATE:
@@ -484,11 +482,11 @@ bool AggFnEvaluator::count_distinct_data_filter(TupleRow* row, Tuple* dst) {
         _string_buffer.reset(new char[_string_buffer_len]);
     }
 
-    StringValue string_val(_string_buffer.get(), total_len);
+    StringRef string_ref(_string_buffer.get(), total_len);
     // the content of StringVal:
     //    header: the STRING_VALUE's len
     //    body:   all input parameters' content
-    char* begin = string_val.ptr;
+    char* begin = const_cast<char*>(string_ref.data);
 
     for (int i = 0; i < vec_size; i++) {
         memcpy(begin, &vec_string_len[0], int_size);
@@ -584,11 +582,11 @@ bool AggFnEvaluator::count_distinct_data_filter(TupleRow* row, Tuple* dst) {
         }
     }
 
-    DCHECK(begin == string_val.ptr + string_val.len)
+    DCHECK(begin == string_ref.data + string_ref.size)
             << "COUNT_DISTINCT: StringVal's len doesn't match";
     bool is_add_buckets = false;
-    bool is_filter = is_in_hybridmap(&string_val, dst, &is_add_buckets);
-    update_mem_trackers(is_filter, is_add_buckets, string_val.len);
+    bool is_filter = is_in_hybridmap(&string_ref, dst, &is_add_buckets);
+    update_mem_trackers(is_filter, is_add_buckets, string_ref.size);
     return is_filter;
 }
 
diff --git a/be/src/exprs/aggregate_functions.cpp b/be/src/exprs/aggregate_functions.cpp
index 202dd1852b..b84bbec8fa 100644
--- a/be/src/exprs/aggregate_functions.cpp
+++ b/be/src/exprs/aggregate_functions.cpp
@@ -32,10 +32,10 @@
 #include "olap/hll.h"
 #include "runtime/datetime_value.h"
 #include "runtime/decimalv2_value.h"
-#include "runtime/string_value.h"
 #include "udf/udf_internal.h"
 #include "util/counts.h"
 #include "util/tdigest.h"
+#include "vec/common/string_ref.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
@@ -703,7 +703,7 @@ void AggregateFunctions::min(FunctionContext* ctx, const StringVal& src, StringV
         return;
     }
 
-    if (dst->is_null || StringValue::from_string_val(src) < StringValue::from_string_val(*dst)) {
+    if (dst->is_null || StringRef(src) < StringRef(*dst)) {
         if (!dst->is_null) {
             ctx->free(dst->ptr);
         }
@@ -719,7 +719,7 @@ void AggregateFunctions::max(FunctionContext* ctx, const StringVal& src, StringV
         return;
     }
 
-    if (dst->is_null || StringValue::from_string_val(src) > StringValue::from_string_val(*dst)) {
+    if (dst->is_null || StringRef(src) > StringRef(*dst)) {
         if (!dst->is_null) {
             ctx->free(dst->ptr);
         }
@@ -1369,15 +1369,15 @@ public:
 
     static void destroy(const StringVal& dst) { delete (MultiDistinctStringCountState*)dst.ptr; }
 
-    void update(StringValue* sv) { _set.insert(sv); }
+    void update(StringRef* sv) { _set.insert(sv); }
 
     StringVal serialize(FunctionContext* ctx) {
         // calculate total serialize buffer length
         int total_serialized_set_length = 1;
         HybridSetBase::IteratorBase* iterator = _set.begin();
         while (iterator->has_next()) {
-            const StringValue* value = reinterpret_cast<const StringValue*>(iterator->get_value());
-            total_serialized_set_length += STRING_LENGTH_RECORD_LENGTH + value->len;
+            const StringRef* value = reinterpret_cast<const StringRef*>(iterator->get_value());
+            total_serialized_set_length += STRING_LENGTH_RECORD_LENGTH + value->size;
             iterator->next();
         }
         StringVal result(ctx, total_serialized_set_length);
@@ -1387,14 +1387,14 @@ public:
         writer++;
         iterator = _set.begin();
         while (iterator->has_next()) {
-            const StringValue* value = reinterpret_cast<const StringValue*>(iterator->get_value());
+            const StringRef* value = reinterpret_cast<const StringRef*>(iterator->get_value());
             // length, it is unnecessary to consider little or big endian for
             // all running in little-endian.
-            *(int*)writer = value->len;
+            *(int*)writer = value->size;
             writer += STRING_LENGTH_RECORD_LENGTH;
             // value
-            memcpy(writer, value->ptr, value->len);
-            writer += value->len;
+            memcpy(writer, value->data, value->size);
+            writer += value->size;
             iterator->next();
         }
         return result;
@@ -1410,7 +1410,7 @@ public:
         while (reader < end) {
             const int length = *(int*)reader;
             reader += STRING_LENGTH_RECORD_LENGTH;
-            StringValue value((char*)reader, length);
+            StringRef value((char*)reader, length);
             _set.insert(&value);
             reader += length;
         }
@@ -1626,7 +1626,7 @@ void AggregateFunctions::count_distinct_string_update(FunctionContext* ctx, Stri
     if (src.is_null) return;
     MultiDistinctStringCountState* state =
             reinterpret_cast<MultiDistinctStringCountState*>(dst->ptr);
-    StringValue sv = StringValue::from_string_val(src);
+    StringRef sv = StringRef(src);
     state->update(&sv);
 }
 
diff --git a/be/src/exprs/anyval_util.h b/be/src/exprs/anyval_util.h
index 007f95f0eb..69144524c5 100644
--- a/be/src/exprs/anyval_util.h
+++ b/be/src/exprs/anyval_util.h
@@ -202,11 +202,11 @@ public:
     template <typename Val>
     static Val max_val(FunctionContext* ctx) {
         if constexpr (std::is_same_v<Val, StringVal>) {
-            StringValue sv = type_limit<StringValue>::max();
+            StringRef sv = type_limit<StringRef>::max();
             StringVal max_val;
-            max_val.ptr = ctx->allocate(sv.len);
-            memcpy(max_val.ptr, sv.ptr, sv.len);
-            max_val.len = sv.len;
+            max_val.ptr = ctx->allocate(sv.size);
+            memcpy(max_val.ptr, sv.data, sv.size);
+            max_val.len = sv.size;
 
             return max_val;
         } else if constexpr (std::is_same_v<Val, DateTimeVal>) {
@@ -416,8 +416,8 @@ public:
         case TYPE_OBJECT:
         case TYPE_QUANTILE_STATE:
         case TYPE_STRING:
-            reinterpret_cast<const StringValue*>(slot)->to_string_val(
-                    reinterpret_cast<doris_udf::StringVal*>(dst));
+            reinterpret_cast<const StringRef*>(slot)->to_string_val(
+                    reinterpret_cast<StringVal*>(dst));
             return;
         case TYPE_DECIMALV2:
             reinterpret_cast<doris_udf::DecimalV2Val*>(dst)->val =
@@ -502,8 +502,8 @@ template <>
 inline bool AnyValUtil::equals_internal(const StringVal& x, const StringVal& y) {
     DCHECK(!x.is_null);
     DCHECK(!y.is_null);
-    StringValue x_sv = StringValue::from_string_val(x);
-    StringValue y_sv = StringValue::from_string_val(y);
+    StringRef x_sv = StringRef(x);
+    StringRef y_sv = StringRef(y);
     return x_sv == y_sv;
 }
 
diff --git a/be/src/exprs/binary_predicate.cpp b/be/src/exprs/binary_predicate.cpp
index 6fa54942de..7408c3307a 100644
--- a/be/src/exprs/binary_predicate.cpp
+++ b/be/src/exprs/binary_predicate.cpp
@@ -21,7 +21,7 @@
 
 #include "gen_cpp/Exprs_types.h"
 #include "runtime/decimalv2_value.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -396,8 +396,8 @@ DATETIME_BINARY_PRED_FNS()
         if (v2.is_null) {                                                \
             return BooleanVal::null();                                   \
         }                                                                \
-        StringValue pv1 = StringValue::from_string_val(v1);              \
-        StringValue pv2 = StringValue::from_string_val(v2);              \
+        StringRef pv1 = StringRef(v1);                                   \
+        StringRef pv2 = StringRef(v2);                                   \
         return BooleanVal(pv1 OP pv2);                                   \
     }
 
diff --git a/be/src/exprs/bitmap_function.cpp b/be/src/exprs/bitmap_function.cpp
index 2b40bd7322..9438b2a39e 100644
--- a/be/src/exprs/bitmap_function.cpp
+++ b/be/src/exprs/bitmap_function.cpp
@@ -260,7 +260,7 @@ StringVal BitmapFunctions::bitmap_serialize(FunctionContext* ctx, const StringVa
     return result;
 }
 
-// This is a init function for intersect_count not for bitmap_intersect, not for _orthogonal_bitmap_intersect(bitmap,t,t)
+// This is an init function for intersect_count not for bitmap_intersect, not for _orthogonal_bitmap_intersect(bitmap,t,t)
 template <typename T, typename ValType>
 void BitmapFunctions::bitmap_intersect_init(FunctionContext* ctx, StringVal* dst) {
     dst->is_null = false;
@@ -277,7 +277,7 @@ void BitmapFunctions::bitmap_intersect_init(FunctionContext* ctx, StringVal* dst
     dst->ptr = (uint8_t*)intersect;
 }
 
-// This is a update function for intersect_count/ORTHOGONAL_BITMAP_INTERSECT_COUNT/ORTHOGONAL_BITMAP_INTERSECT(bitmap,t,t)
+// This is an update function for intersect_count/ORTHOGONAL_BITMAP_INTERSECT_COUNT/ORTHOGONAL_BITMAP_INTERSECT(bitmap,t,t)
 // not for bitmap_intersect(Bitmap)
 template <typename T, typename ValType>
 void BitmapFunctions::bitmap_intersect_update(FunctionContext* ctx, const StringVal& src,
@@ -956,8 +956,8 @@ template void BitmapFunctions::bitmap_intersect_init<DateTimeValue, DateTimeVal>
         FunctionContext* ctx, StringVal* dst);
 template void BitmapFunctions::bitmap_intersect_init<DecimalV2Value, DecimalV2Val>(
         FunctionContext* ctx, StringVal* dst);
-template void BitmapFunctions::bitmap_intersect_init<StringValue, StringVal>(FunctionContext* ctx,
-                                                                             StringVal* dst);
+template void BitmapFunctions::bitmap_intersect_init<StringRef, StringVal>(FunctionContext* ctx,
+                                                                           StringVal* dst);
 
 template void BitmapFunctions::bitmap_intersect_update<int8_t, TinyIntVal>(
         FunctionContext* ctx, const StringVal& src, const TinyIntVal& key, int num_key,
@@ -986,7 +986,7 @@ template void BitmapFunctions::bitmap_intersect_update<DateTimeValue, DateTimeVa
 template void BitmapFunctions::bitmap_intersect_update<DecimalV2Value, DecimalV2Val>(
         FunctionContext* ctx, const StringVal& src, const DecimalV2Val& key, int num_key,
         const DecimalV2Val* keys, const StringVal* dst);
-template void BitmapFunctions::bitmap_intersect_update<StringValue, StringVal>(
+template void BitmapFunctions::bitmap_intersect_update<StringRef, StringVal>(
         FunctionContext* ctx, const StringVal& src, const StringVal& key, int num_key,
         const StringVal* keys, const StringVal* dst);
 
@@ -1017,9 +1017,9 @@ template void BitmapFunctions::bitmap_intersect_merge<DateTimeValue>(FunctionCon
 template void BitmapFunctions::bitmap_intersect_merge<DecimalV2Value>(FunctionContext* ctx,
                                                                       const StringVal& src,
                                                                       const StringVal* dst);
-template void BitmapFunctions::bitmap_intersect_merge<StringValue>(FunctionContext* ctx,
-                                                                   const StringVal& src,
-                                                                   const StringVal* dst);
+template void BitmapFunctions::bitmap_intersect_merge<StringRef>(FunctionContext* ctx,
+                                                                 const StringVal& src,
+                                                                 const StringVal* dst);
 
 template StringVal BitmapFunctions::bitmap_intersect_serialize<int8_t>(FunctionContext* ctx,
                                                                        const StringVal& src);
@@ -1039,8 +1039,8 @@ template StringVal BitmapFunctions::bitmap_intersect_serialize<DateTimeValue>(Fu
                                                                               const StringVal& src);
 template StringVal BitmapFunctions::bitmap_intersect_serialize<DecimalV2Value>(
         FunctionContext* ctx, const StringVal& src);
-template StringVal BitmapFunctions::bitmap_intersect_serialize<StringValue>(FunctionContext* ctx,
-                                                                            const StringVal& src);
+template StringVal BitmapFunctions::bitmap_intersect_serialize<StringRef>(FunctionContext* ctx,
+                                                                          const StringVal& src);
 
 template BigIntVal BitmapFunctions::bitmap_intersect_finalize<int8_t>(FunctionContext* ctx,
                                                                       const StringVal& src);
@@ -1060,8 +1060,8 @@ template BigIntVal BitmapFunctions::bitmap_intersect_finalize<DateTimeValue>(Fun
                                                                              const StringVal& src);
 template BigIntVal BitmapFunctions::bitmap_intersect_finalize<DecimalV2Value>(FunctionContext* ctx,
                                                                               const StringVal& src);
-template BigIntVal BitmapFunctions::bitmap_intersect_finalize<StringValue>(FunctionContext* ctx,
-                                                                           const StringVal& src);
+template BigIntVal BitmapFunctions::bitmap_intersect_finalize<StringRef>(FunctionContext* ctx,
+                                                                         const StringVal& src);
 
 template void BitmapFunctions::orthogonal_bitmap_intersect_count_init<int8_t, TinyIntVal>(
         FunctionContext* ctx, StringVal* dst);
@@ -1075,7 +1075,7 @@ template void BitmapFunctions::orthogonal_bitmap_intersect_count_init<float, Flo
         FunctionContext* ctx, StringVal* dst);
 template void BitmapFunctions::orthogonal_bitmap_intersect_count_init<double, DoubleVal>(
         FunctionContext* ctx, StringVal* dst);
-template void BitmapFunctions::orthogonal_bitmap_intersect_count_init<StringValue, StringVal>(
+template void BitmapFunctions::orthogonal_bitmap_intersect_count_init<StringRef, StringVal>(
         FunctionContext* ctx, StringVal* dst);
 
 template void BitmapFunctions::orthogonal_bitmap_intersect_init<int8_t, TinyIntVal>(
@@ -1090,7 +1090,7 @@ template void BitmapFunctions::orthogonal_bitmap_intersect_init<float, FloatVal>
         FunctionContext* ctx, StringVal* dst);
 template void BitmapFunctions::orthogonal_bitmap_intersect_init<double, DoubleVal>(
         FunctionContext* ctx, StringVal* dst);
-template void BitmapFunctions::orthogonal_bitmap_intersect_init<StringValue, StringVal>(
+template void BitmapFunctions::orthogonal_bitmap_intersect_init<StringRef, StringVal>(
         FunctionContext* ctx, StringVal* dst);
 
 template StringVal BitmapFunctions::orthogonal_bitmap_intersect_serialize<int8_t>(
@@ -1105,7 +1105,7 @@ template StringVal BitmapFunctions::orthogonal_bitmap_intersect_serialize<float>
         FunctionContext* ctx, const StringVal& src);
 template StringVal BitmapFunctions::orthogonal_bitmap_intersect_serialize<double>(
         FunctionContext* ctx, const StringVal& src);
-template StringVal BitmapFunctions::orthogonal_bitmap_intersect_serialize<StringValue>(
+template StringVal BitmapFunctions::orthogonal_bitmap_intersect_serialize<StringRef>(
         FunctionContext* ctx, const StringVal& src);
 
 template BigIntVal BitmapFunctions::orthogonal_bitmap_intersect_finalize<int8_t>(
@@ -1120,7 +1120,7 @@ template BigIntVal BitmapFunctions::orthogonal_bitmap_intersect_finalize<float>(
         FunctionContext* ctx, const StringVal& src);
 template BigIntVal BitmapFunctions::orthogonal_bitmap_intersect_finalize<double>(
         FunctionContext* ctx, const StringVal& src);
-template BigIntVal BitmapFunctions::orthogonal_bitmap_intersect_finalize<StringValue>(
+template BigIntVal BitmapFunctions::orthogonal_bitmap_intersect_finalize<StringRef>(
         FunctionContext* ctx, const StringVal& src);
 
 template StringVal BitmapFunctions::orthogonal_bitmap_intersect_count_serialize<int8_t>(
@@ -1135,6 +1135,6 @@ template StringVal BitmapFunctions::orthogonal_bitmap_intersect_count_serialize<
         FunctionContext* ctx, const StringVal& src);
 template StringVal BitmapFunctions::orthogonal_bitmap_intersect_count_serialize<double>(
         FunctionContext* ctx, const StringVal& src);
-template StringVal BitmapFunctions::orthogonal_bitmap_intersect_count_serialize<StringValue>(
+template StringVal BitmapFunctions::orthogonal_bitmap_intersect_count_serialize<StringRef>(
         FunctionContext* ctx, const StringVal& src);
 } // namespace doris
diff --git a/be/src/exprs/bloomfilter_predicate.h b/be/src/exprs/bloomfilter_predicate.h
index ec3b24434e..bf77a8fb3d 100644
--- a/be/src/exprs/bloomfilter_predicate.h
+++ b/be/src/exprs/bloomfilter_predicate.h
@@ -294,17 +294,17 @@ struct StringFindOp {
     }
 
     void insert(BloomFilterAdaptor& bloom_filter, const void* data) const {
-        const auto* value = reinterpret_cast<const StringValue*>(data);
+        const auto* value = reinterpret_cast<const StringRef*>(data);
         if (value) {
-            bloom_filter.add_bytes(value->ptr, value->len);
+            bloom_filter.add_bytes(value->data, value->size);
         }
     }
     bool find(const BloomFilterAdaptor& bloom_filter, const void* data) const {
-        const auto* value = reinterpret_cast<const StringValue*>(data);
+        const auto* value = reinterpret_cast<const StringRef*>(data);
         if (value == nullptr) {
             return false;
         }
-        return bloom_filter.test(Slice(value->ptr, value->len));
+        return bloom_filter.test(Slice(value->data, value->size));
     }
     bool find_olap_engine(const BloomFilterAdaptor& bloom_filter, const void* data) const {
         return StringFindOp::find(bloom_filter, data);
@@ -318,13 +318,13 @@ struct StringFindOp {
 // when filer used by the storage engine
 struct FixedStringFindOp : public StringFindOp {
     bool find_olap_engine(const BloomFilterAdaptor& bloom_filter, const void* input_data) const {
-        const auto* value = reinterpret_cast<const StringValue*>(input_data);
-        int64_t size = value->len;
-        char* data = value->ptr;
+        const auto* value = reinterpret_cast<const StringRef*>(input_data);
+        int64_t size = value->size;
+        const char* data = value->data;
         while (size > 0 && data[size - 1] == '\0') {
             size--;
         }
-        return bloom_filter.test(Slice(value->ptr, size));
+        return bloom_filter.test(Slice(value->data, size));
     }
 };
 
diff --git a/be/src/exprs/create_predicate_function.h b/be/src/exprs/create_predicate_function.h
index a9418a6955..3178b7a970 100644
--- a/be/src/exprs/create_predicate_function.h
+++ b/be/src/exprs/create_predicate_function.h
@@ -43,8 +43,8 @@ public:
     template <PrimitiveType type>
     static BasePtr get_function() {
         using CppType = typename PrimitiveTypeTraits<type>::CppType;
-        using Set = std::conditional_t<std::is_same_v<CppType, StringValue>, StringSet,
-                                       HybridSet<type>>;
+        using Set =
+                std::conditional_t<std::is_same_v<CppType, StringRef>, StringSet, HybridSet<type>>;
         return new Set();
     };
 };
diff --git a/be/src/exprs/expr.h b/be/src/exprs/expr.h
index 8588f37429..403021e1ce 100644
--- a/be/src/exprs/expr.h
+++ b/be/src/exprs/expr.h
@@ -29,12 +29,11 @@
 #include "gen_cpp/Opcodes_types.h"
 #include "runtime/descriptors.h"
 #include "runtime/large_int_value.h"
-#include "runtime/string_value.h"
-#include "runtime/string_value.hpp"
 #include "runtime/tuple.h"
 #include "runtime/tuple_row.h"
 #include "udf/udf.h"
 #include "util/string_parser.hpp"
+#include "vec/common/string_ref.h"
 #include "vec/data_types/data_type_decimal.h"
 #include "vec/io/io_helper.h"
 
@@ -585,7 +584,7 @@ Status create_texpr_literal_node(const void* data, TExprNode* node, int precisio
         (*node).__set_float_literal(float_literal);
         (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DOUBLE));
     } else if constexpr ((T == TYPE_STRING) || (T == TYPE_CHAR) || (T == TYPE_VARCHAR)) {
-        auto origin_value = reinterpret_cast<const StringValue*>(data);
+        auto origin_value = reinterpret_cast<const StringRef*>(data);
         (*node).__set_node_type(TExprNodeType::STRING_LITERAL);
         TStringLiteral string_literal;
         string_literal.__set_value(origin_value->to_string());
diff --git a/be/src/exprs/expr_context.cpp b/be/src/exprs/expr_context.cpp
index b019de99c0..99aec19935 100644
--- a/be/src/exprs/expr_context.cpp
+++ b/be/src/exprs/expr_context.cpp
@@ -242,8 +242,8 @@ void* ExprContext::get_value(Expr* e, TupleRow* row, int precision, int scale) {
         if (v.is_null) {
             return nullptr;
         }
-        _result.string_val.ptr = reinterpret_cast<char*>(v.ptr);
-        _result.string_val.len = v.len;
+        _result.string_val.data = reinterpret_cast<char*>(v.ptr);
+        _result.string_val.size = v.len;
         return &_result.string_val;
     }
     case TYPE_DATE:
diff --git a/be/src/exprs/expr_value.h b/be/src/exprs/expr_value.h
index d38dadd45e..a3cc441675 100644
--- a/be/src/exprs/expr_value.h
+++ b/be/src/exprs/expr_value.h
@@ -23,8 +23,8 @@
 #include "runtime/collection_value.h"
 #include "runtime/datetime_value.h"
 #include "runtime/decimalv2_value.h"
-#include "runtime/string_value.h"
 #include "runtime/types.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -43,7 +43,7 @@ struct ExprValue {
     float float_val;
     double double_val;
     std::string string_data;
-    StringValue string_val;
+    StringRef string_val;
     DateTimeValue datetime_val;
     doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> datev2_val;
     doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datetimev2_val;
@@ -76,13 +76,11 @@ struct ExprValue {
     ExprValue(int64_t i, int32_t f) : decimalv2_val(i, f) {}
 
     // c'tor for string values
-    ExprValue(const std::string& str)
-            : string_data(str),
-              string_val(const_cast<char*>(string_data.data()), string_data.size()) {}
+    ExprValue(const std::string& str) : string_data(str), string_val(string_data) {}
 
     // Set string value to copy of str
-    void set_string_val(const StringValue& str) {
-        string_data = std::string(str.ptr, str.len);
+    void set_string_val(const StringRef& str) {
+        string_data = std::string(str.data, str.size);
         sync_string_val();
     }
 
@@ -95,8 +93,8 @@ struct ExprValue {
     // string_data. If not called after mutating string_data,
     // string_val->ptr may point at garbage.
     void sync_string_val() {
-        string_val.ptr = const_cast<char*>(string_data.data());
-        string_val.len = string_data.size();
+        string_val.data = string_data.data();
+        string_val.size = string_data.size();
     }
 
     // Sets the value for type to '0' and returns a pointer to the data
diff --git a/be/src/exprs/hybrid_set.h b/be/src/exprs/hybrid_set.h
index a2ca4553da..25606b226e 100644
--- a/be/src/exprs/hybrid_set.h
+++ b/be/src/exprs/hybrid_set.h
@@ -23,7 +23,7 @@
 #include "runtime/decimalv2_value.h"
 #include "runtime/define_primitive_type.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -159,8 +159,8 @@ public:
             return;
         }
 
-        const auto* value = reinterpret_cast<const StringValue*>(data);
-        std::string str_value(value->ptr, value->len);
+        const auto* value = reinterpret_cast<const StringRef*>(data);
+        std::string str_value(value->data, value->size);
         _set.insert(str_value);
     }
 
@@ -185,8 +185,8 @@ public:
             return false;
         }
 
-        auto* value = reinterpret_cast<const StringValue*>(data);
-        std::string_view str_value(const_cast<const char*>(value->ptr), value->len);
+        auto* value = reinterpret_cast<const StringRef*>(data);
+        std::string_view str_value(const_cast<const char*>(value->data), value->size);
         auto it = _set.find(str_value);
 
         return !(it == _set.end());
@@ -206,8 +206,8 @@ public:
         ~Iterator() override = default;
         bool has_next() const override { return !(_begin == _end); }
         const void* get_value() override {
-            _value.ptr = const_cast<char*>(_begin->data());
-            _value.len = _begin->length();
+            _value.data = const_cast<char*>(_begin->data());
+            _value.size = _begin->length();
             return &_value;
         }
         void next() override { ++_begin; }
@@ -215,7 +215,7 @@ public:
     private:
         typename phmap::flat_hash_set<std::string>::iterator _begin;
         typename phmap::flat_hash_set<std::string>::iterator _end;
-        StringValue _value;
+        StringRef _value;
     };
 
     IteratorBase* begin() override {
@@ -230,8 +230,8 @@ private:
 };
 
 // note: Two difference from StringSet
-// 1 StringValue has better comparison performance than std::string
-// 2 std::string keeps its own memory, bug StringValue just keeps ptr and len, so you the caller should manage memory of StringValue
+// 1 StringRef has better comparison performance than std::string
+// 2 std::string keeps its own memory, bug StringRef just keeps ptr and len, so you the caller should manage memory of StringRef
 class StringValueSet : public HybridSetBase {
 public:
     StringValueSet() = default;
@@ -243,13 +243,13 @@ public:
             return;
         }
 
-        const auto* value = reinterpret_cast<const StringValue*>(data);
-        StringValue sv(value->ptr, value->len);
+        const auto* value = reinterpret_cast<const StringRef*>(data);
+        StringRef sv(value->data, value->size);
         _set.insert(sv);
     }
 
     void insert(void* data, size_t size) override {
-        StringValue sv(reinterpret_cast<char*>(data), size);
+        StringRef sv(reinterpret_cast<char*>(data), size);
         _set.insert(sv);
     }
 
@@ -269,7 +269,7 @@ public:
             return false;
         }
 
-        auto* value = reinterpret_cast<const StringValue*>(data);
+        auto* value = reinterpret_cast<const StringRef*>(data);
         auto it = _set.find(*value);
 
         return !(it == _set.end());
@@ -280,39 +280,39 @@ public:
             return false;
         }
 
-        StringValue sv(reinterpret_cast<const char*>(data), size);
+        StringRef sv(reinterpret_cast<const char*>(data), size);
         auto it = _set.find(sv);
         return !(it == _set.end());
     }
 
     class Iterator : public IteratorBase {
     public:
-        Iterator(phmap::flat_hash_set<StringValue>::iterator begin,
-                 phmap::flat_hash_set<StringValue>::iterator end)
+        Iterator(phmap::flat_hash_set<StringRef>::iterator begin,
+                 phmap::flat_hash_set<StringRef>::iterator end)
                 : _begin(begin), _end(end) {}
         ~Iterator() override = default;
         bool has_next() const override { return !(_begin == _end); }
         const void* get_value() override {
-            _value.ptr = const_cast<char*>(_begin->ptr);
-            _value.len = _begin->len;
+            _value.data = const_cast<char*>(_begin->data);
+            _value.size = _begin->size;
             return &_value;
         }
         void next() override { ++_begin; }
 
     private:
-        typename phmap::flat_hash_set<StringValue>::iterator _begin;
-        typename phmap::flat_hash_set<StringValue>::iterator _end;
-        StringValue _value;
+        typename phmap::flat_hash_set<StringRef>::iterator _begin;
+        typename phmap::flat_hash_set<StringRef>::iterator _end;
+        StringRef _value;
     };
 
     IteratorBase* begin() override {
         return _pool.add(new (std::nothrow) Iterator(_set.begin(), _set.end()));
     }
 
-    phmap::flat_hash_set<StringValue>* get_inner_set() { return &_set; }
+    phmap::flat_hash_set<StringRef>* get_inner_set() { return &_set; }
 
 private:
-    phmap::flat_hash_set<StringValue> _set;
+    phmap::flat_hash_set<StringRef> _set;
     ObjectPool _pool;
 };
 
diff --git a/be/src/exprs/info_func.cpp b/be/src/exprs/info_func.cpp
index 1418b04acd..2277f1cc1f 100644
--- a/be/src/exprs/info_func.cpp
+++ b/be/src/exprs/info_func.cpp
@@ -25,11 +25,7 @@ InfoFunc::InfoFunc(const TExprNode& node)
         : Expr(node), _int_value(node.info_func.int_value), _str_value(node.info_func.str_value) {}
 
 StringVal InfoFunc::get_string_val(ExprContext* context, TupleRow*) {
-    StringVal val;
-    StringValue value(_str_value);
-    value.to_string_val(&val);
-
-    return val;
+    return {_str_value.c_str(), static_cast<int64_t>(_str_value.size())};
 }
 
 BigIntVal InfoFunc::get_big_int_val(ExprContext* context, TupleRow*) {
diff --git a/be/src/exprs/like_predicate.cpp b/be/src/exprs/like_predicate.cpp
index cd4fa586c4..9ea70ffa6e 100644
--- a/be/src/exprs/like_predicate.cpp
+++ b/be/src/exprs/like_predicate.cpp
@@ -63,8 +63,8 @@ void LikePredicate::like_prepare(FunctionContext* context,
         if (pattern_val.is_null) {
             return;
         }
-        StringValue pattern = StringValue::from_string_val(pattern_val);
-        std::string pattern_str(pattern.ptr, pattern.len);
+        StringRef pattern = StringRef(pattern_val);
+        std::string pattern_str(pattern.data, pattern.size);
         std::string search_string;
         if (RE2::FullMatch(pattern_str, LIKE_ENDS_WITH_RE, &search_string)) {
             remove_escape_character(&search_string);
@@ -268,10 +268,10 @@ BooleanVal LikePredicate::constant_substring_fn(FunctionContext* context, const
     }
     LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
             context->get_function_state(FunctionContext::THREAD_LOCAL));
-    if (state->search_string_sv.len == 0) {
+    if (state->search_string_sv.size == 0) {
         return BooleanVal(true);
     }
-    StringValue pattern_value = StringValue::from_string_val(val);
+    StringRef pattern_value = StringRef(val);
     return BooleanVal(state->substring_pattern.search(&pattern_value) != -1);
 }
 
@@ -282,10 +282,10 @@ BooleanVal LikePredicate::constant_starts_with_fn(FunctionContext* context, cons
     }
     LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
             context->get_function_state(FunctionContext::THREAD_LOCAL));
-    if (val.len < state->search_string_sv.len) {
+    if (val.len < state->search_string_sv.size) {
         return BooleanVal(false);
     } else {
-        StringValue v = StringValue(reinterpret_cast<char*>(val.ptr), state->search_string_sv.len);
+        StringRef v = StringRef(reinterpret_cast<char*>(val.ptr), state->search_string_sv.size);
         return BooleanVal(state->search_string_sv.eq((v)));
     }
 }
@@ -297,12 +297,12 @@ BooleanVal LikePredicate::constant_ends_with_fn(FunctionContext* context, const
     }
     LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
             context->get_function_state(FunctionContext::THREAD_LOCAL));
-    if (val.len < state->search_string_sv.len) {
+    if (val.len < state->search_string_sv.size) {
         return BooleanVal(false);
     } else {
-        char* ptr = reinterpret_cast<char*>(val.ptr) + val.len - state->search_string_sv.len;
-        int len = state->search_string_sv.len;
-        StringValue v = StringValue(ptr, len);
+        char* ptr = reinterpret_cast<char*>(val.ptr) + val.len - state->search_string_sv.size;
+        int len = state->search_string_sv.size;
+        StringRef v = StringRef(ptr, len);
         return BooleanVal(state->search_string_sv.eq(v));
     }
 }
@@ -314,7 +314,7 @@ BooleanVal LikePredicate::constant_equals_fn(FunctionContext* context, const Str
     }
     LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
             context->get_function_state(FunctionContext::THREAD_LOCAL));
-    return BooleanVal(state->search_string_sv.eq(StringValue::from_string_val(val)));
+    return BooleanVal(state->search_string_sv.eq(StringRef(val)));
 }
 
 BooleanVal LikePredicate::constant_regex_fn_partial(FunctionContext* context, const StringVal& val,
diff --git a/be/src/exprs/like_predicate.h b/be/src/exprs/like_predicate.h
index c2e4671d89..f9edeb1dd7 100644
--- a/be/src/exprs/like_predicate.h
+++ b/be/src/exprs/like_predicate.h
@@ -44,7 +44,7 @@ struct LikePredicateState {
     /// separately.
     LikePredicateFunction function;
 
-    /// Holds the string the StringValue points to and is set any time StringValue is
+    /// Holds the string the StringRef points to and is set any time StringRef is
     /// used.
     std::string search_string;
 
@@ -52,7 +52,7 @@ struct LikePredicateState {
     /// constant string or has a constant string at the beginning or end of the pattern.
     /// This will be set in order to check for that pattern in the corresponding part of
     /// the string.
-    StringValue search_string_sv;
+    StringRef search_string_sv;
 
     /// Used for LIKE predicates if the pattern is a constant argument and has a constant
     /// string in the middle of it. This will be use in order to check for the substring
@@ -66,7 +66,7 @@ struct LikePredicateState {
 
     void set_search_string(const std::string& search_string_arg) {
         search_string = search_string_arg;
-        search_string_sv = StringValue(search_string);
+        search_string_sv = StringRef(search_string);
         substring_pattern.set_pattern(&search_string_sv);
     }
 };
diff --git a/be/src/exprs/literal.cpp b/be/src/exprs/literal.cpp
index 28211abc2a..15ff1be5fe 100644
--- a/be/src/exprs/literal.cpp
+++ b/be/src/exprs/literal.cpp
@@ -162,7 +162,7 @@ Decimal32Val Literal::get_decimal32_val(ExprContext* context, TupleRow* row) {
     DCHECK(_type.type == TYPE_DECIMAL32) << _type;
     StringParser::ParseResult result;
     auto decimal32_value = StringParser::string_to_decimal<int32_t>(
-            _value.string_val.ptr, _value.string_val.len, _type.precision, _type.scale, &result);
+            _value.string_val.data, _value.string_val.size, _type.precision, _type.scale, &result);
     if (result == StringParser::ParseResult::PARSE_SUCCESS) {
         return Decimal32Val(decimal32_value);
     } else {
@@ -174,7 +174,7 @@ Decimal64Val Literal::get_decimal64_val(ExprContext* context, TupleRow* row) {
     DCHECK(_type.type == TYPE_DECIMAL64) << _type;
     StringParser::ParseResult result;
     auto decimal_value = StringParser::string_to_decimal<int64_t>(
-            _value.string_val.ptr, _value.string_val.len, _type.precision, _type.scale, &result);
+            _value.string_val.data, _value.string_val.size, _type.precision, _type.scale, &result);
     if (result == StringParser::ParseResult::PARSE_SUCCESS) {
         return Decimal64Val(decimal_value);
     } else {
@@ -186,7 +186,7 @@ Decimal128Val Literal::get_decimal128_val(ExprContext* context, TupleRow* row) {
     DCHECK(_type.type == TYPE_DECIMAL128I) << _type;
     StringParser::ParseResult result;
     auto decimal_value = StringParser::string_to_decimal<int128_t>(
-            _value.string_val.ptr, _value.string_val.len, _type.precision, _type.scale, &result);
+            _value.string_val.data, _value.string_val.size, _type.precision, _type.scale, &result);
     if (result == StringParser::ParseResult::PARSE_SUCCESS) {
         return Decimal128Val(decimal_value);
     } else {
diff --git a/be/src/exprs/match_predicate.cpp b/be/src/exprs/match_predicate.cpp
index 972a1d6728..2eaaeb929c 100644
--- a/be/src/exprs/match_predicate.cpp
+++ b/be/src/exprs/match_predicate.cpp
@@ -25,7 +25,7 @@
 #include "exec/olap_utils.h"
 #include "exprs/string_functions.h"
 #include "olap/schema.h"
-#include "runtime/string_value.hpp"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -49,10 +49,10 @@ Status MatchPredicate::evaluate(const Schema& schema, InvertedIndexIterator* ite
     if (is_string_type(column_desc->type()) ||
         (column_desc->type() == OLAP_FIELD_TYPE_ARRAY &&
          is_string_type(column_desc->get_sub_field(0)->type_info()->type()))) {
-        StringValue match_value;
+        StringRef match_value;
         int32_t length = _value.length();
         char* buffer = const_cast<char*>(_value.c_str());
-        match_value.replace(buffer, length);
+        match_value.replace(buffer, length); //is it safe?
         s = iterator->read_from_inverted_index(column_desc->name(), &match_value,
                                                inverted_index_query_type, num_rows, &roaring);
     } else if (column_desc->type() == OLAP_FIELD_TYPE_ARRAY &&
diff --git a/be/src/exprs/math_functions.cpp b/be/src/exprs/math_functions.cpp
index bc5507eea3..6eacf5fe6e 100644
--- a/be/src/exprs/math_functions.cpp
+++ b/be/src/exprs/math_functions.cpp
@@ -680,7 +680,7 @@ LEAST_FNS();
     }
 
 #define LEAST_NONNUMERIC_FNS()                                     \
-    LEAST_NONNUMERIC_FN(string_val, StringVal, StringValue);       \
+    LEAST_NONNUMERIC_FN(string_val, StringVal, StringRef);         \
     LEAST_NONNUMERIC_FN(datetime_val, DateTimeVal, DateTimeValue); \
     LEAST_NONNUMERIC_FN(decimal_val, DecimalV2Val, DecimalV2Value);
 
@@ -723,7 +723,7 @@ GREATEST_FNS();
     }
 
 #define GREATEST_NONNUMERIC_FNS()                                     \
-    GREATEST_NONNUMERIC_FN(string_val, StringVal, StringValue);       \
+    GREATEST_NONNUMERIC_FN(string_val, StringVal, StringRef);         \
     GREATEST_NONNUMERIC_FN(datetime_val, DateTimeVal, DateTimeValue); \
     GREATEST_NONNUMERIC_FN(decimal_val, DecimalV2Val, DecimalV2Value);
 
diff --git a/be/src/exprs/minmax_predicate.h b/be/src/exprs/minmax_predicate.h
index a237f7f8b2..4394c36d69 100644
--- a/be/src/exprs/minmax_predicate.h
+++ b/be/src/exprs/minmax_predicate.h
@@ -94,20 +94,20 @@ public:
     }
 
     Status merge(MinMaxFuncBase* minmax_func, ObjectPool* pool) override {
-        if constexpr (std::is_same_v<T, StringValue>) {
+        if constexpr (std::is_same_v<T, StringRef>) {
             MinMaxNumFunc<T>* other_minmax = static_cast<MinMaxNumFunc<T>*>(minmax_func);
 
             if (other_minmax->_min < _min) {
                 auto& other_min = other_minmax->_min;
-                auto str = pool->add(new std::string(other_min.ptr, other_min.len));
-                _min.ptr = str->data();
-                _min.len = str->length();
+                auto str = pool->add(new std::string(other_min.data, other_min.size));
+                _min.data = str->data();
+                _min.size = str->length();
             }
             if (other_minmax->_max > _max) {
                 auto& other_max = other_minmax->_max;
-                auto str = pool->add(new std::string(other_max.ptr, other_max.len));
-                _max.ptr = str->data();
-                _max.len = str->length();
+                auto str = pool->add(new std::string(other_max.data, other_max.size));
+                _max.data = str->data();
+                _max.size = str->length();
             }
         } else {
             MinMaxNumFunc<T>* other_minmax = static_cast<MinMaxNumFunc<T>*>(minmax_func);
diff --git a/be/src/exprs/new_agg_fn_evaluator.cc b/be/src/exprs/new_agg_fn_evaluator.cc
index 3ce6375dde..f795522009 100644
--- a/be/src/exprs/new_agg_fn_evaluator.cc
+++ b/be/src/exprs/new_agg_fn_evaluator.cc
@@ -30,8 +30,8 @@
 #include "exprs/expr_context.h"
 #include "runtime/raw_value.h"
 #include "runtime/runtime_state.h"
-#include "runtime/string_value.h"
 #include "udf/udf_internal.h"
+#include "vec/common/string_ref.h"
 
 using namespace doris;
 using namespace doris_udf;
@@ -255,9 +255,9 @@ void NewAggFnEvaluator::SetDstSlot(const AnyVal* src, const SlotDescriptor& dst_
     case TYPE_OBJECT:
     case TYPE_QUANTILE_STATE:
     case TYPE_STRING:
-        *reinterpret_cast<StringValue*>(slot) =
-                StringValue::from_string_val(*reinterpret_cast<const StringVal*>(src));
+        *reinterpret_cast<StringRef*>(slot) = *reinterpret_cast<const StringVal*>(src);
         return;
+
     case TYPE_DATE:
     case TYPE_DATETIME:
         *reinterpret_cast<DateTimeValue*>(slot) =
@@ -267,6 +267,7 @@ void NewAggFnEvaluator::SetDstSlot(const AnyVal* src, const SlotDescriptor& dst_
     case TYPE_DECIMALV2:
         *reinterpret_cast<PackedInt128*>(slot) = reinterpret_cast<const DecimalV2Val*>(src)->val;
         return;
+
     default:
         DCHECK(false) << "NYI: " << dst_slot_desc.type();
     }
@@ -353,8 +354,7 @@ inline void NewAggFnEvaluator::set_any_val(const void* slot, const TypeDescripto
     case TYPE_OBJECT:
     case TYPE_QUANTILE_STATE:
     case TYPE_STRING:
-        reinterpret_cast<const StringValue*>(slot)->to_string_val(
-                reinterpret_cast<StringVal*>(dst));
+        reinterpret_cast<const StringRef*>(slot)->to_string_val(reinterpret_cast<StringVal*>(dst));
         return;
 
     case TYPE_DATE:
diff --git a/be/src/exprs/new_in_predicate.cpp b/be/src/exprs/new_in_predicate.cpp
index 04d739ee5f..db88cfcd24 100644
--- a/be/src/exprs/new_in_predicate.cpp
+++ b/be/src/exprs/new_in_predicate.cpp
@@ -33,9 +33,9 @@ SetType get_val(const FunctionContext::TypeDesc* type, const T& x) {
 }
 
 template <>
-StringValue get_val(const FunctionContext::TypeDesc* type, const StringVal& x) {
+StringRef get_val(const FunctionContext::TypeDesc* type, const StringVal& x) {
     DCHECK(!x.is_null);
-    return StringValue::from_string_val(x);
+    return StringRef(x);
 }
 
 template <>
@@ -173,7 +173,7 @@ IN_FUNCTIONS(IntVal, int32_t, int_val)
 IN_FUNCTIONS(BigIntVal, int64_t, big_int_val)
 IN_FUNCTIONS(FloatVal, float, float_val)
 IN_FUNCTIONS(DoubleVal, double, double_val)
-IN_FUNCTIONS(StringVal, StringValue, string_val)
+IN_FUNCTIONS(StringVal, StringRef, string_val)
 IN_FUNCTIONS(DateTimeVal, DateTimeValue, datetime_val)
 IN_FUNCTIONS(DecimalV2Val, DecimalV2Value, decimalv2_val)
 IN_FUNCTIONS(LargeIntVal, __int128, large_int_val)
diff --git a/be/src/exprs/operators.cpp b/be/src/exprs/operators.cpp
index 625a1db346..16e903deb1 100644
--- a/be/src/exprs/operators.cpp
+++ b/be/src/exprs/operators.cpp
@@ -21,7 +21,7 @@
 #include "exprs/operators.h"
 
 #include "runtime/datetime_value.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -106,7 +106,7 @@ void Operators::init() {}
     BINARY_PREDICATE_NUMERIC_FN(NAME, large_int_val, LargeIntVal, OP);                           \
     BINARY_PREDICATE_NUMERIC_FN(NAME, float_val, FloatVal, OP);                                  \
     BINARY_PREDICATE_NUMERIC_FN(NAME, double_val, DoubleVal, OP);                                \
-    BINARY_PREDICATE_NONNUMERIC_FN(NAME, string_val, string_val, StringVal, StringValue, OP);    \
+    BINARY_PREDICATE_NONNUMERIC_FN(NAME, string_val, string_val, StringVal, StringRef, OP);      \
     BINARY_PREDICATE_NONNUMERIC_FN(NAME, datetime_val, datetime_val, DateTimeVal, DateTimeValue, \
                                    OP);
 
diff --git a/be/src/exprs/rpc_fn.cpp b/be/src/exprs/rpc_fn.cpp
index bb130415ae..01363922e8 100644
--- a/be/src/exprs/rpc_fn.cpp
+++ b/be/src/exprs/rpc_fn.cpp
@@ -151,20 +151,20 @@ Status RPCFn::call_internal(ExprContext* context, TupleRow* row, PFunctionCallRe
         case TYPE_STRING:
         case TYPE_CHAR: {
             ptype->set_id(PGenericType::STRING);
-            StringValue value = *reinterpret_cast<StringValue*>(src_slot);
-            arg->add_string_value(value.ptr, value.len);
+            StringRef value = *reinterpret_cast<StringRef*>(src_slot);
+            arg->add_string_value(value.data, value.size);
             break;
         }
         case TYPE_HLL: {
             ptype->set_id(PGenericType::HLL);
-            StringValue value = *reinterpret_cast<StringValue*>(src_slot);
-            arg->add_string_value(value.ptr, value.len);
+            StringRef value = *reinterpret_cast<StringRef*>(src_slot);
+            arg->add_string_value(value.data, value.size);
             break;
         }
         case TYPE_OBJECT: {
             ptype->set_id(PGenericType::BITMAP);
-            StringValue value = *reinterpret_cast<StringValue*>(src_slot);
-            arg->add_string_value(value.ptr, value.len);
+            StringRef value = *reinterpret_cast<StringRef*>(src_slot);
+            arg->add_string_value(value.data, value.size);
             break;
         }
         case TYPE_DECIMALV2: {
diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp
index b2dc2351a6..4ba1bd271e 100644
--- a/be/src/exprs/runtime_filter.cpp
+++ b/be/src/exprs/runtime_filter.cpp
@@ -558,8 +558,8 @@ public:
         case TYPE_VARCHAR:
         case TYPE_HLL:
         case TYPE_STRING: {
-            // StringRef->StringValue
-            StringValue data = StringValue(const_cast<char*>(value.data), value.size);
+            // StringRef->StringRef
+            StringRef data = StringRef(value.data, value.size);
             insert(reinterpret_cast<const void*>(&data));
             break;
         }
@@ -861,7 +861,7 @@ public:
                                        ObjectPool* pool) {
                 auto& string_val_ref = column.stringval();
                 auto val_ptr = pool->add(new std::string(string_val_ref));
-                StringValue string_val(const_cast<char*>(val_ptr->c_str()), val_ptr->length());
+                StringRef string_val(val_ptr->c_str(), val_ptr->length());
                 set->insert(&string_val);
             });
             break;
@@ -994,8 +994,8 @@ public:
             auto& max_val_ref = minmax_filter->max_val().stringval();
             auto min_val_ptr = _pool->add(new std::string(min_val_ref));
             auto max_val_ptr = _pool->add(new std::string(max_val_ref));
-            StringValue min_val(const_cast<char*>(min_val_ptr->c_str()), min_val_ptr->length());
-            StringValue max_val(const_cast<char*>(max_val_ptr->c_str()), max_val_ptr->length());
+            StringRef min_val(min_val_ptr->c_str(), min_val_ptr->length());
+            StringRef max_val(max_val_ptr->c_str(), max_val_ptr->length());
             return _context.minmax_func->assign(&min_val, &max_val);
         }
         default:
@@ -1028,12 +1028,12 @@ public:
             case TYPE_VARCHAR:
             case TYPE_CHAR:
             case TYPE_STRING: {
-                StringValue* min_value = static_cast<StringValue*>(_context.minmax_func->get_min());
-                StringValue* max_value = static_cast<StringValue*>(_context.minmax_func->get_max());
-                auto min_val_ptr = _pool->add(new std::string(min_value->ptr));
-                auto max_val_ptr = _pool->add(new std::string(max_value->ptr));
-                StringValue min_val(const_cast<char*>(min_val_ptr->c_str()), min_val_ptr->length());
-                StringValue max_val(const_cast<char*>(max_val_ptr->c_str()), max_val_ptr->length());
+                StringRef* min_value = static_cast<StringRef*>(_context.minmax_func->get_min());
+                StringRef* max_value = static_cast<StringRef*>(_context.minmax_func->get_max());
+                auto min_val_ptr = _pool->add(new std::string(min_value->data));
+                auto max_val_ptr = _pool->add(new std::string(max_value->data));
+                StringRef min_val(min_val_ptr->c_str(), min_val_ptr->length());
+                StringRef max_val(max_val_ptr->c_str(), max_val_ptr->length());
                 _context.minmax_func->assign(&min_val, &max_val);
             }
             default:
@@ -1616,8 +1616,8 @@ void IRuntimeFilter::to_protobuf(PInFilter* filter) {
     case TYPE_CHAR:
     case TYPE_VARCHAR:
     case TYPE_STRING: {
-        batch_copy<StringValue>(filter, it, [](PColumnValue* column, const StringValue* value) {
-            column->set_stringval(std::string(value->ptr, value->len));
+        batch_copy<StringRef>(filter, it, [](PColumnValue* column, const StringRef* value) {
+            column->set_stringval(std::string(value->data, value->size));
         });
         return;
     }
@@ -1725,12 +1725,12 @@ void IRuntimeFilter::to_protobuf(PMinMaxFilter* filter) {
     case TYPE_CHAR:
     case TYPE_VARCHAR:
     case TYPE_STRING: {
-        const StringValue* min_string_value = reinterpret_cast<const StringValue*>(min_data);
+        const StringRef* min_string_value = reinterpret_cast<const StringRef*>(min_data);
         filter->mutable_min_val()->set_stringval(
-                std::string(min_string_value->ptr, min_string_value->len));
-        const StringValue* max_string_value = reinterpret_cast<const StringValue*>(max_data);
+                std::string(min_string_value->data, min_string_value->size));
+        const StringRef* max_string_value = reinterpret_cast<const StringRef*>(max_data);
         filter->mutable_max_val()->set_stringval(
-                std::string(max_string_value->ptr, max_string_value->len));
+                std::string(max_string_value->data, max_string_value->size));
         break;
     }
     default: {
diff --git a/be/src/exprs/slot_ref.cpp b/be/src/exprs/slot_ref.cpp
index ce4c3319e7..915704b66a 100644
--- a/be/src/exprs/slot_ref.cpp
+++ b/be/src/exprs/slot_ref.cpp
@@ -208,7 +208,7 @@ StringVal SlotRef::get_string_val(ExprContext* context, TupleRow* row) {
         return StringVal::null();
     }
     StringVal result;
-    StringValue* sv = reinterpret_cast<StringValue*>(t->get_slot(_slot_offset));
+    StringRef* sv = reinterpret_cast<StringRef*>(t->get_slot(_slot_offset));
     sv->to_string_val(&result);
     return result;
 }
diff --git a/be/src/exprs/string_functions.cpp b/be/src/exprs/string_functions.cpp
index 0db6741f05..0f9e3f5b19 100644
--- a/be/src/exprs/string_functions.cpp
+++ b/be/src/exprs/string_functions.cpp
@@ -421,8 +421,8 @@ IntVal StringFunctions::instr(FunctionContext* context, const StringVal& str,
     if (substr.len == 0) {
         return IntVal(1);
     }
-    StringValue str_sv = StringValue::from_string_val(str);
-    StringValue substr_sv = StringValue::from_string_val(substr);
+    StringRef str_sv = StringRef(str);
+    StringRef substr_sv = StringRef(substr);
     StringSearch search(&substr_sv);
     // Hive returns positions starting from 1.
     int loc = search.search(&str_sv);
@@ -467,17 +467,17 @@ IntVal StringFunctions::locate_pos(FunctionContext* context, const StringVal& su
     if (start_pos.val <= 0 || start_pos.val > str.len || start_pos.val > char_len) {
         return IntVal(0);
     }
-    StringValue substr_sv = StringValue::from_string_val(substr);
+    StringRef substr_sv = StringRef(substr);
     StringSearch search(&substr_sv);
     // Input start_pos.val starts from 1.
-    StringValue adjusted_str(reinterpret_cast<char*>(str.ptr) + index[start_pos.val - 1],
-                             str.len - index[start_pos.val - 1]);
+    StringRef adjusted_str(reinterpret_cast<char*>(str.ptr) + index[start_pos.val - 1],
+                           str.len - index[start_pos.val - 1]);
     int32_t match_pos = search.search(&adjusted_str);
     if (match_pos >= 0) {
         // Hive returns the position in the original string starting from 1.
         size_t char_len = 0;
         for (size_t i = 0, char_size = 0; i < match_pos; i += char_size) {
-            char_size = UTF8_BYTE_LENGTH[(unsigned char)(adjusted_str.ptr)[i]];
+            char_size = UTF8_BYTE_LENGTH[(unsigned char)(adjusted_str.data)[i]];
             ++char_len;
         }
         match_pos = char_len;
@@ -735,14 +735,14 @@ IntVal StringFunctions::find_in_set(FunctionContext* context, const StringVal& s
     int32_t token_index = 1;
     int32_t start = 0;
     int32_t end;
-    StringValue str_sv = StringValue::from_string_val(str);
+    StringRef str_sv = StringRef(str);
     do {
         end = start;
         // Position end.
         while (end < str_set.len && str_set.ptr[end] != ',') {
             ++end;
         }
-        StringValue token(reinterpret_cast<char*>(str_set.ptr) + start, end - start);
+        StringRef token(reinterpret_cast<char*>(str_set.ptr) + start, end - start);
         if (str_sv.eq(token)) {
             return IntVal(token_index);
         }
@@ -767,7 +767,7 @@ void StringFunctions::parse_url_prepare(FunctionContext* ctx,
         return;
     }
     UrlParser::UrlPart* url_part = new UrlParser::UrlPart;
-    *url_part = UrlParser::get_url_part(StringValue::from_string_val(*part));
+    *url_part = UrlParser::get_url_part(StringRef(*part));
     if (*url_part == UrlParser::INVALID) {
         std::stringstream ss;
         ss << "Invalid URL part: " << AnyValUtil::to_string(*part) << std::endl
@@ -793,11 +793,11 @@ StringVal StringFunctions::parse_url(FunctionContext* ctx, const StringVal& url,
         url_part = *reinterpret_cast<UrlParser::UrlPart*>(state);
     } else {
         DCHECK(!ctx->is_arg_constant(1));
-        url_part = UrlParser::get_url_part(StringValue::from_string_val(newPart));
+        url_part = UrlParser::get_url_part(StringRef(newPart));
     }
 
-    StringValue result;
-    if (!UrlParser::parse_url(StringValue::from_string_val(url), url_part, &result)) {
+    StringRef result;
+    if (!UrlParser::parse_url(StringRef(url), url_part, &result)) {
         // url is malformed, or url_part is invalid.
         if (url_part == UrlParser::INVALID) {
             std::stringstream ss;
@@ -836,12 +836,11 @@ StringVal StringFunctions::parse_url_key(FunctionContext* ctx, const StringVal&
         url_part = *reinterpret_cast<UrlParser::UrlPart*>(state);
     } else {
         DCHECK(!ctx->is_arg_constant(1));
-        url_part = UrlParser::get_url_part(StringValue::from_string_val(part));
+        url_part = UrlParser::get_url_part(StringRef(part));
     }
 
-    StringValue result;
-    if (!UrlParser::parse_url_key(StringValue::from_string_val(url), url_part,
-                                  StringValue::from_string_val(key), &result)) {
+    StringRef result;
+    if (!UrlParser::parse_url_key(StringRef(url), url_part, StringRef(key), &result)) {
         // url is malformed, or url_part is invalid.
         if (url_part == UrlParser::INVALID) {
             std::stringstream ss;
diff --git a/be/src/exprs/string_functions.h b/be/src/exprs/string_functions.h
index ec3e7c25b0..0ee17334cd 100644
--- a/be/src/exprs/string_functions.h
+++ b/be/src/exprs/string_functions.h
@@ -28,7 +28,7 @@
 #include <string_view>
 
 #include "gutil/strings/numbers.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
diff --git a/be/src/olap/aggregate_func.h b/be/src/olap/aggregate_func.h
index 5a397b658f..f477ddb22a 100644
--- a/be/src/olap/aggregate_func.h
+++ b/be/src/olap/aggregate_func.h
@@ -24,9 +24,9 @@
 #include "runtime/datetime_value.h"
 #include "runtime/decimalv2_value.h"
 #include "runtime/mem_pool.h"
-#include "runtime/string_value.h"
 #include "util/bitmap_value.h"
 #include "util/quantile_state.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
diff --git a/be/src/olap/comparison_predicate.h b/be/src/olap/comparison_predicate.h
index ec2be5b609..26d97a27ed 100644
--- a/be/src/olap/comparison_predicate.h
+++ b/be/src/olap/comparison_predicate.h
@@ -220,8 +220,8 @@ public:
 
     bool evaluate_and(const segment_v2::BloomFilter* bf) const override {
         if constexpr (PT == PredicateType::EQ) {
-            if constexpr (std::is_same_v<T, StringValue>) {
-                return bf->test_bytes(_value.ptr, _value.len);
+            if constexpr (std::is_same_v<T, StringRef>) {
+                return bf->test_bytes(_value.data, _value.size);
             } else if constexpr (Type == TYPE_DATE) {
                 return bf->test_bytes(const_cast<char*>(reinterpret_cast<const char*>(&_value)),
                                       sizeof(uint24_t));
@@ -254,7 +254,7 @@ public:
                                      .get_data();
 
             if (nested_column.is_column_dictionary()) {
-                if constexpr (std::is_same_v<T, StringValue>) {
+                if constexpr (std::is_same_v<T, StringRef>) {
                     auto* dict_column_ptr =
                             vectorized::check_and_get_column<vectorized::ColumnDictI32>(
                                     nested_column);
@@ -266,7 +266,7 @@ public:
                     _base_loop_vec<true, is_and>(size, flags, null_map.data(), data_array,
                                                  dict_code);
                 } else {
-                    LOG(FATAL) << "column_dictionary must use StringValue predicate.";
+                    LOG(FATAL) << "column_dictionary must use StringRef predicate.";
                 }
             } else {
                 auto* data_array = reinterpret_cast<const vectorized::PredicateColumnType<
@@ -278,7 +278,7 @@ public:
             }
         } else {
             if (column.is_column_dictionary()) {
-                if constexpr (std::is_same_v<T, StringValue>) {
+                if constexpr (std::is_same_v<T, StringRef>) {
                     auto* dict_column_ptr =
                             vectorized::check_and_get_column<vectorized::ColumnDictI32>(column);
                     auto dict_code = _is_range() ? dict_column_ptr->find_code_by_bound(
@@ -288,7 +288,7 @@ public:
 
                     _base_loop_vec<false, is_and>(size, flags, nullptr, data_array, dict_code);
                 } else {
-                    LOG(FATAL) << "column_dictionary must use StringValue predicate.";
+                    LOG(FATAL) << "column_dictionary must use StringRef predicate.";
                 }
             } else {
                 auto* data_array =
@@ -457,7 +457,7 @@ private:
     void _base_evaluate_bit(const vectorized::IColumn* column, const uint8_t* null_map,
                             const uint16_t* sel, uint16_t size, bool* flags) const {
         if (column->is_column_dictionary()) {
-            if constexpr (std::is_same_v<T, StringValue>) {
+            if constexpr (std::is_same_v<T, StringRef>) {
                 auto* dict_column_ptr =
                         vectorized::check_and_get_column<vectorized::ColumnDictI32>(column);
                 auto* data_array = dict_column_ptr->get_data().data();
@@ -467,7 +467,7 @@ private:
                 _base_loop_bit<is_nullable, is_and>(sel, size, flags, null_map, data_array,
                                                     dict_code);
             } else {
-                LOG(FATAL) << "column_dictionary must use StringValue predicate.";
+                LOG(FATAL) << "column_dictionary must use StringRef predicate.";
             }
         } else {
             auto* data_array =
@@ -503,7 +503,7 @@ private:
     uint16_t _base_evaluate(const vectorized::IColumn* column, const uint8_t* null_map,
                             uint16_t* sel, uint16_t size) const {
         if (column->is_column_dictionary()) {
-            if constexpr (std::is_same_v<T, StringValue>) {
+            if constexpr (std::is_same_v<T, StringRef>) {
                 auto* dict_column_ptr =
                         vectorized::check_and_get_column<vectorized::ColumnDictI32>(column);
                 auto* data_array = dict_column_ptr->get_data().data();
@@ -513,7 +513,7 @@ private:
 
                 return _base_loop<is_nullable>(sel, size, null_map, data_array, dict_code);
             } else {
-                LOG(FATAL) << "column_dictionary must use StringValue predicate.";
+                LOG(FATAL) << "column_dictionary must use StringRef predicate.";
                 return 0;
             }
         } else {
diff --git a/be/src/olap/hll.cpp b/be/src/olap/hll.cpp
index bc2ba3de39..7c234d1703 100644
--- a/be/src/olap/hll.cpp
+++ b/be/src/olap/hll.cpp
@@ -21,8 +21,8 @@
 #include <map>
 
 #include "common/logging.h"
-#include "runtime/string_value.h"
 #include "util/coding.h"
+#include "vec/common/string_ref.h"
 
 using std::map;
 using std::nothrow;
diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h
index e85b1280f9..5a0990cc21 100644
--- a/be/src/olap/in_list_predicate.h
+++ b/be/src/olap/in_list_predicate.h
@@ -31,21 +31,15 @@
 #include "olap/wrapper_field.h"
 #include "runtime/define_primitive_type.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
 #include "runtime/type_limit.h"
 #include "uint24.h"
 #include "vec/columns/column_dictionary.h"
+#include "vec/common/string_ref.h"
 #include "vec/core/types.h"
 
-// for string value
 template <>
-struct std::hash<doris::StringValue> {
-    uint64_t operator()(const doris::StringValue& rhs) const { return hash_value(rhs); }
-};
-
-template <>
-struct std::equal_to<doris::StringValue> {
-    bool operator()(const doris::StringValue& lhs, const doris::StringValue& rhs) const {
+struct std::equal_to<doris::StringRef> {
+    bool operator()(const doris::StringRef& lhs, const doris::StringRef& rhs) const {
         return lhs == rhs;
     }
 };
@@ -122,13 +116,14 @@ public:
             auto values = ((HybridSetType*)hybrid_set.get())->get_inner_set();
 
             if constexpr (is_string_type(Type)) {
-                for (auto& value : *values) {
-                    StringValue sv = {value.data(), int(value.size())};
+                // values' type is "phmap::flat_hash_set<std::string>"
+                for (const std::string& value : *values) {
+                    StringRef sv = value;
                     if constexpr (Type == TYPE_CHAR) {
                         _temp_datas.push_back("");
                         _temp_datas.back().resize(std::max(char_length, value.size()));
                         memcpy(_temp_datas.back().data(), value.data(), value.size());
-                        sv = {_temp_datas.back().data(), int(_temp_datas.back().size())};
+                        sv = _temp_datas.back();
                     }
                     _values->insert(sv);
                 }
@@ -346,8 +341,8 @@ public:
     bool evaluate_and(const segment_v2::BloomFilter* bf) const override {
         if constexpr (PT == PredicateType::IN_LIST) {
             for (auto value : *_values) {
-                if constexpr (std::is_same_v<T, StringValue>) {
-                    if (bf->test_bytes(value.ptr, value.len)) {
+                if constexpr (std::is_same_v<T, StringRef>) {
+                    if (bf->test_bytes(value.data, value.size)) {
                         return true;
                     }
                 } else if constexpr (Type == TYPE_DATE) {
@@ -388,7 +383,7 @@ private:
         uint16_t new_size = 0;
 
         if (column->is_column_dictionary()) {
-            if constexpr (std::is_same_v<T, StringValue>) {
+            if constexpr (std::is_same_v<T, StringRef>) {
                 auto* nested_col_ptr = vectorized::check_and_get_column<
                         vectorized::ColumnDictionary<vectorized::Int32>>(column);
                 auto& data_array = nested_col_ptr->get_data();
@@ -427,7 +422,7 @@ private:
                     }
                 }
             } else {
-                LOG(FATAL) << "column_dictionary must use StringValue predicate.";
+                LOG(FATAL) << "column_dictionary must use StringRef predicate.";
             }
         } else {
             auto* nested_col_ptr = vectorized::check_and_get_column<
@@ -467,7 +462,7 @@ private:
                             const vectorized::PaddedPODArray<vectorized::UInt8>* null_map,
                             const uint16_t* sel, uint16_t size, bool* flags) const {
         if (column->is_column_dictionary()) {
-            if constexpr (std::is_same_v<T, StringValue>) {
+            if constexpr (std::is_same_v<T, StringRef>) {
                 auto* nested_col_ptr = vectorized::check_and_get_column<
                         vectorized::ColumnDictionary<vectorized::Int32>>(column);
                 auto& data_array = nested_col_ptr->get_data();
@@ -503,7 +498,7 @@ private:
                     }
                 }
             } else {
-                LOG(FATAL) << "column_dictionary must use StringValue predicate.";
+                LOG(FATAL) << "column_dictionary must use StringRef predicate.";
             }
         } else {
             auto* nested_col_ptr = vectorized::check_and_get_column<
diff --git a/be/src/olap/like_column_predicate.cpp b/be/src/olap/like_column_predicate.cpp
index 4da3576425..cc5eb83a0b 100644
--- a/be/src/olap/like_column_predicate.cpp
+++ b/be/src/olap/like_column_predicate.cpp
@@ -18,8 +18,8 @@
 #include "olap/like_column_predicate.h"
 
 #include "olap/field.h"
-#include "runtime/string_value.hpp"
 #include "udf/udf.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -74,11 +74,11 @@ uint16_t LikeColumnPredicate<is_vectorized>::evaluate(const vectorized::IColumn&
                     for (uint16_t i = 0; i != size; i++) {
                         uint16_t idx = sel[i];
                         sel[new_size] = idx;
-                        StringValue cell_value = nested_col_ptr->get_shrink_value(data_array[idx]);
+                        StringRef cell_value = nested_col_ptr->get_shrink_value(data_array[idx]);
                         unsigned char flag = 0;
                         (_state->scalar_function)(
                                 const_cast<vectorized::LikeSearchState*>(&_like_state),
-                                StringRef(cell_value.ptr, cell_value.len), pattern, &flag);
+                                StringRef(cell_value.data, cell_value.size), pattern, &flag);
                         new_size += _opposite ^ flag;
                     }
                 } else {
@@ -90,11 +90,11 @@ uint16_t LikeColumnPredicate<is_vectorized>::evaluate(const vectorized::IColumn&
                             continue;
                         }
 
-                        StringValue cell_value = nested_col_ptr->get_shrink_value(data_array[idx]);
+                        StringRef cell_value = nested_col_ptr->get_shrink_value(data_array[idx]);
                         unsigned char flag = 0;
                         (_state->scalar_function)(
                                 const_cast<vectorized::LikeSearchState*>(&_like_state),
-                                StringRef(cell_value.ptr, cell_value.len), pattern, &flag);
+                                StringRef(cell_value.data, cell_value.size), pattern, &flag);
                         new_size += _opposite ^ flag;
                     }
                 }
@@ -120,11 +120,11 @@ uint16_t LikeColumnPredicate<is_vectorized>::evaluate(const vectorized::IColumn&
                             continue;
                         }
 
-                        StringValue cell_value = str_col->get_data()[idx];
+                        StringRef cell_value = str_col->get_data()[idx];
                         unsigned char flag = 0;
                         (_state->scalar_function)(
                                 const_cast<vectorized::LikeSearchState*>(&_like_state),
-                                StringRef(cell_value.ptr, cell_value.len), pattern, &flag);
+                                StringRef(cell_value.data, cell_value.size), pattern, &flag);
                         new_size += _opposite ^ flag;
                     }
                 }
@@ -137,11 +137,11 @@ uint16_t LikeColumnPredicate<is_vectorized>::evaluate(const vectorized::IColumn&
                 for (uint16_t i = 0; i != size; i++) {
                     uint16_t idx = sel[i];
                     sel[new_size] = idx;
-                    StringValue cell_value = nested_col_ptr->get_shrink_value(data_array[idx]);
+                    StringRef cell_value = nested_col_ptr->get_shrink_value(data_array[idx]);
                     unsigned char flag = 0;
                     (_state->scalar_function)(
                             const_cast<vectorized::LikeSearchState*>(&_like_state),
-                            StringRef(cell_value.ptr, cell_value.len), pattern, &flag);
+                            StringRef(cell_value.data, cell_value.size), pattern, &flag);
                     new_size += _opposite ^ flag;
                 }
             } else {
diff --git a/be/src/olap/like_column_predicate.h b/be/src/olap/like_column_predicate.h
index 2be7980103..00d7f0f848 100644
--- a/be/src/olap/like_column_predicate.h
+++ b/be/src/olap/like_column_predicate.h
@@ -18,9 +18,9 @@
 
 #include "exprs/like_predicate.h"
 #include "olap/column_predicate.h"
-#include "runtime/string_value.h"
 #include "udf/udf.h"
 #include "vec/columns/column_dictionary.h"
+#include "vec/common/string_ref.h"
 #include "vec/core/types.h"
 #include "vec/functions/like.h"
 
@@ -48,7 +48,12 @@ public:
                           bool* flags) const override;
 
     std::string get_search_str() const override {
-        return std::string(reinterpret_cast<char*>(pattern.ptr), pattern.len);
+        if constexpr (std::is_same_v<PatternType, StringRef>) {
+            return std::string(reinterpret_cast<const char*>(pattern.data), pattern.size);
+        } else if constexpr (std::is_same_v<PatternType, StringVal>) {
+            return std::string(reinterpret_cast<const char*>(pattern.ptr), pattern.len);
+        }
+        DCHECK(false);
     }
     bool is_opposite() const { return _opposite; }
 
@@ -86,18 +91,18 @@ private:
                             continue;
                         }
 
-                        StringValue cell_value = nested_col_ptr->get_shrink_value(data_array[i]);
+                        StringRef cell_value = nested_col_ptr->get_shrink_value(data_array[i]);
                         if constexpr (is_and) {
                             unsigned char flag = 0;
                             (_state->scalar_function)(
                                     const_cast<vectorized::LikeSearchState*>(&_like_state),
-                                    StringRef(cell_value.ptr, cell_value.len), pattern, &flag);
+                                    StringRef(cell_value.data, cell_value.size), pattern, &flag);
                             flags[i] &= _opposite ^ flag;
                         } else {
                             unsigned char flag = 0;
                             (_state->scalar_function)(
                                     const_cast<vectorized::LikeSearchState*>(&_like_state),
-                                    StringRef(cell_value.ptr, cell_value.len), pattern, &flag);
+                                    StringRef(cell_value.data, cell_value.size), pattern, &flag);
                             flags[i] = _opposite ^ flag;
                         }
                     }
@@ -110,18 +115,18 @@ private:
                             vectorized::ColumnDictionary<vectorized::Int32>>(column);
                     auto& data_array = nested_col_ptr->get_data();
                     for (uint16_t i = 0; i < size; i++) {
-                        StringValue cell_value = nested_col_ptr->get_shrink_value(data_array[i]);
+                        StringRef cell_value = nested_col_ptr->get_shrink_value(data_array[i]);
                         if constexpr (is_and) {
                             unsigned char flag = 0;
                             (_state->scalar_function)(
                                     const_cast<vectorized::LikeSearchState*>(&_like_state),
-                                    StringRef(cell_value.ptr, cell_value.len), pattern, &flag);
+                                    StringRef(cell_value.data, cell_value.size), pattern, &flag);
                             flags[i] &= _opposite ^ flag;
                         } else {
                             unsigned char flag = 0;
                             (_state->scalar_function)(
                                     const_cast<vectorized::LikeSearchState*>(&_like_state),
-                                    StringRef(cell_value.ptr, cell_value.len), pattern, &flag);
+                                    StringRef(cell_value.data, cell_value.size), pattern, &flag);
                             flags[i] = _opposite ^ flag;
                         }
                     }
@@ -138,9 +143,9 @@ private:
     }
 
     std::string _origin;
-    // life time controlled by scan node
+    // lifetime controlled by scan node
     doris_udf::FunctionContext* _fn_ctx;
-    using PatternType = std::conditional_t<is_vectorized, StringValue, StringVal>;
+    using PatternType = std::conditional_t<is_vectorized, StringRef, StringVal>;
     using StateType = std::conditional_t<is_vectorized, vectorized::LikeState, LikePredicateState>;
     PatternType pattern;
 
diff --git a/be/src/olap/null_predicate.cpp b/be/src/olap/null_predicate.cpp
index 9c8d5ccd67..c0cc938be8 100644
--- a/be/src/olap/null_predicate.cpp
+++ b/be/src/olap/null_predicate.cpp
@@ -18,8 +18,8 @@
 #include "olap/null_predicate.h"
 
 #include "olap/field.h"
-#include "runtime/string_value.hpp"
 #include "vec/columns/column_nullable.h"
+#include "vec/common/string_ref.h"
 
 using namespace doris::vectorized;
 
diff --git a/be/src/olap/predicate_creator.h b/be/src/olap/predicate_creator.h
index 5faad0268b..55e9a0a642 100644
--- a/be/src/olap/predicate_creator.h
+++ b/be/src/olap/predicate_creator.h
@@ -116,8 +116,8 @@ public:
     }
 
 private:
-    static StringValue convert(const TabletColumn& column, const std::string& condition,
-                               MemPool* pool) {
+    static StringRef convert(const TabletColumn& column, const std::string& condition,
+                             MemPool* pool) {
         size_t length = condition.length();
         if constexpr (Type == TYPE_CHAR) {
             length = std::max(static_cast<size_t>(column.length()), length);
@@ -127,7 +127,7 @@ private:
         memset(buffer, 0, length);
         memory_copy(buffer, condition.data(), condition.length());
 
-        return StringValue(buffer, length);
+        return {buffer, length};
     }
 };
 
diff --git a/be/src/olap/rowset/segment_v2/bloom_filter.h b/be/src/olap/rowset/segment_v2/bloom_filter.h
index 139ff3aea2..c235ec39b9 100644
--- a/be/src/olap/rowset/segment_v2/bloom_filter.h
+++ b/be/src/olap/rowset/segment_v2/bloom_filter.h
@@ -126,7 +126,7 @@ public:
         add_hash(code);
     }
 
-    bool test_bytes(char* buf, uint32_t size) const {
+    bool test_bytes(const char* buf, uint32_t size) const {
         if (buf == nullptr) {
             return *_has_null;
         }
diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp
index c8d82d798f..316483033f 100644
--- a/be/src/olap/rowset/segment_v2/column_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/column_reader.cpp
@@ -193,8 +193,8 @@ Status ColumnReader::next_batch_of_zone_map(size_t* n, vectorized::MutableColumn
         assert_cast<vectorized::ColumnNullable&>(*dst).insert_default();
     } else {
         if (is_string) {
-            auto sv = (StringValue*)max_value->cell_ptr();
-            dst->insert_data(sv->ptr, sv->len);
+            auto sv = (StringRef*)max_value->cell_ptr();
+            dst->insert_data(sv->data, sv->size);
         } else {
             dst->insert_many_fix_len_data(static_cast<const char*>(max_value->cell_ptr()), 1);
         }
@@ -205,8 +205,8 @@ Status ColumnReader::next_batch_of_zone_map(size_t* n, vectorized::MutableColumn
         assert_cast<vectorized::ColumnNullable&>(*dst).insert_null_elements(size);
     } else {
         if (is_string) {
-            auto sv = (StringValue*)min_value->cell_ptr();
-            dst->insert_many_data(sv->ptr, sv->len, size);
+            auto sv = (StringRef*)min_value->cell_ptr();
+            dst->insert_many_data(sv->data, sv->size, size);
         } else {
             // TODO: the work may cause performance problem, opt latter
             for (int i = 0; i < size; ++i) {
diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp
index cff316f6ac..402a88f2c5 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp
@@ -32,8 +32,8 @@
 #include "olap/rowset/segment_v2/inverted_index_desc.h"
 #include "olap/tablet_schema.h"
 #include "olap/utils.h"
-#include "runtime/string_value.h"
 #include "util/time.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 namespace segment_v2 {
@@ -110,7 +110,7 @@ Status FullTextIndexReader::query(const std::string& column_name, const void* qu
                                   InvertedIndexQueryType query_type,
                                   InvertedIndexParserType analyser_type,
                                   roaring::Roaring* bit_map) {
-    std::string search_str = reinterpret_cast<const StringValue*>(query_value)->to_string();
+    std::string search_str = reinterpret_cast<const StringRef*>(query_value)->to_string();
     VLOG_DEBUG << column_name
                << " begin to load the fulltext index from clucene, query_str=" << search_str;
     std::unique_ptr<lucene::search::Query> query;
@@ -210,9 +210,9 @@ Status StringTypeInvertedIndexReader::query(const std::string& column_name, cons
                                             InvertedIndexQueryType query_type,
                                             InvertedIndexParserType analyser_type,
                                             roaring::Roaring* bit_map) {
-    const StringValue* search_query = reinterpret_cast<const StringValue*>(query_value);
-    auto act_len = strnlen(search_query->ptr, search_query->len);
-    std::string search_str(search_query->ptr, act_len);
+    const StringRef* search_query = reinterpret_cast<const StringRef*>(query_value);
+    auto act_len = strnlen(search_query->data, search_query->size);
+    std::string search_str(search_query->data, act_len);
     VLOG_DEBUG << "begin to query the inverted index from clucene"
                << ", column_name: " << column_name << ", search_str: " << search_str;
     std::wstring column_name_ws = std::wstring(column_name.begin(), column_name.end());
diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt
index dfc375bd24..4351fe1ee5 100644
--- a/be/src/runtime/CMakeLists.txt
+++ b/be/src/runtime/CMakeLists.txt
@@ -41,7 +41,6 @@ set(RUNTIME_FILES
     runtime_state.cpp
     runtime_filter_mgr.cpp
     runtime_predicate.cpp
-    string_value.cpp
     jsonb_value.cpp
     thread_context.cpp
     thread_resource_mgr.cpp
diff --git a/be/src/runtime/collection_value.cpp b/be/src/runtime/collection_value.cpp
index 8192e35dc7..fc9fb65a24 100644
--- a/be/src/runtime/collection_value.cpp
+++ b/be/src/runtime/collection_value.cpp
@@ -23,9 +23,9 @@
 #include "common/utils.h"
 #include "runtime/mem_pool.h"
 #include "runtime/raw_value.h"
-#include "runtime/string_value.h"
 #include "runtime/types.h"
 #include "util/mem_util.hpp"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -87,19 +87,19 @@ struct CollectionValueSubTypeTrait<TYPE_DOUBLE> {
 
 template <>
 struct CollectionValueSubTypeTrait<TYPE_CHAR> {
-    using CppType = StringValue;
+    using CppType = StringRef;
     using AnyValType = StringVal;
 };
 
 template <>
 struct CollectionValueSubTypeTrait<TYPE_VARCHAR> {
-    using CppType = StringValue;
+    using CppType = StringRef;
     using AnyValType = StringVal;
 };
 
 template <>
 struct CollectionValueSubTypeTrait<TYPE_STRING> {
-    using CppType = StringValue;
+    using CppType = StringRef;
     using AnyValType = StringVal;
 };
 
@@ -186,43 +186,43 @@ struct ArrayIteratorFunctions : public GenericArrayIteratorFunctions<type> {};
 
 template <PrimitiveType type>
 struct ArrayIteratorFunctionsForString : public GenericArrayIteratorFunctions<type> {
-    using CppType = StringValue;
+    using CppType = StringRef;
     using AnyValType = StringVal;
 
     static void shallow_set(void* item, const AnyVal* value) {
         const auto* src = static_cast<const AnyValType*>(value);
         auto* dst = static_cast<CppType*>(item);
-        dst->ptr = convert_to<char*>(src->ptr);
-        dst->len = src->len;
+        dst->data = convert_to<char*>(src->ptr);
+        dst->size = src->len;
     }
     static void shallow_get(AnyVal* value, const void* item) {
         const auto* src = static_cast<const CppType*>(item);
         auto* dst = static_cast<AnyValType*>(value);
-        dst->ptr = convert_to<uint8_t*>(src->ptr);
-        dst->len = src->len;
+        dst->ptr = convert_to<uint8_t*>(src->data);
+        dst->len = src->size;
     }
     static void self_deep_copy(void* item, const TypeDescriptor&,
                                const GenMemFootprintFunc& gen_mem_footprint, bool convert_ptrs) {
         auto* string = static_cast<CppType*>(item);
-        if (!string->len) {
+        if (!string->size) {
             return;
         }
-        MemFootprint footprint = gen_mem_footprint(string->len);
+        MemFootprint footprint = gen_mem_footprint(string->size);
         int64_t offset = footprint.first;
         auto* copied_string = reinterpret_cast<char*>(footprint.second);
-        memory_copy(copied_string, string->ptr, string->len);
-        string->ptr = (convert_ptrs ? convert_to<char*>(offset) : copied_string);
+        memory_copy(copied_string, string->data, string->size);
+        string->data = (convert_ptrs ? convert_to<char*>(offset) : copied_string);
     }
     static void deserialize(void* item, const char* tuple_data, const TypeDescriptor& type_desc) {
         DCHECK((item != nullptr) && (tuple_data != nullptr)) << "item or tuple_data is nullptr";
         auto* string_value = static_cast<CppType*>(item);
-        if (string_value->len) {
-            int64_t offset = convert_to<int64_t>(string_value->ptr);
-            string_value->ptr = convert_to<char*>(tuple_data + offset);
+        if (string_value->size) {
+            int64_t offset = convert_to<int64_t>(string_value->data);
+            string_value->data = convert_to<char*>(tuple_data + offset);
         }
     }
     static size_t get_byte_size(const void* item, const TypeDescriptor&) {
-        return static_cast<const CppType*>(item)->len;
+        return static_cast<const CppType*>(item)->size;
     }
 };
 
diff --git a/be/src/runtime/fold_constant_executor.cpp b/be/src/runtime/fold_constant_executor.cpp
index a782024eb1..3fad048846 100644
--- a/be/src/runtime/fold_constant_executor.cpp
+++ b/be/src/runtime/fold_constant_executor.cpp
@@ -234,7 +234,7 @@ string FoldConstantExecutor::_get_result(void* src, size_t size, PrimitiveType s
         if constexpr (is_vec) {
             return std::string((char*)src, size);
         }
-        return (reinterpret_cast<StringValue*>(src))->to_string();
+        return (reinterpret_cast<StringRef*>(src))->to_string();
     }
     case TYPE_DATE:
     case TYPE_DATETIME: {
diff --git a/be/src/runtime/primitive_type.cpp b/be/src/runtime/primitive_type.cpp
index 779676a45d..a5af1569bc 100644
--- a/be/src/runtime/primitive_type.cpp
+++ b/be/src/runtime/primitive_type.cpp
@@ -21,7 +21,7 @@
 #include "runtime/collection_value.h"
 #include "runtime/define_primitive_type.h"
 #include "runtime/jsonb_value.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -584,7 +584,7 @@ int get_slot_size(PrimitiveType type) {
     case TYPE_OBJECT:
     case TYPE_HLL:
     case TYPE_QUANTILE_STATE:
-        return sizeof(StringValue);
+        return sizeof(StringRef);
     case TYPE_JSONB:
         return sizeof(JsonBinaryValue);
     case TYPE_ARRAY:
diff --git a/be/src/runtime/primitive_type.h b/be/src/runtime/primitive_type.h
index 5af7639290..96272edc71 100644
--- a/be/src/runtime/primitive_type.h
+++ b/be/src/runtime/primitive_type.h
@@ -32,7 +32,7 @@ class ColumnString;
 
 class DateTimeValue;
 class DecimalV2Value;
-struct StringValue;
+struct StringRef;
 struct JsonBinaryValue;
 
 PrimitiveType convert_type_to_primitive(FunctionContext::Type type);
@@ -210,24 +210,24 @@ struct PrimitiveTypeTraits<TYPE_LARGEINT> {
 };
 template <>
 struct PrimitiveTypeTraits<TYPE_CHAR> {
-    using CppType = StringValue;
+    using CppType = StringRef;
     using ColumnType = vectorized::ColumnString;
 };
 template <>
 struct PrimitiveTypeTraits<TYPE_VARCHAR> {
-    using CppType = StringValue;
+    using CppType = StringRef;
     using ColumnType = vectorized::ColumnString;
 };
 
 template <>
 struct PrimitiveTypeTraits<TYPE_STRING> {
-    using CppType = StringValue;
+    using CppType = StringRef;
     using ColumnType = vectorized::ColumnString;
 };
 
 template <>
 struct PrimitiveTypeTraits<TYPE_HLL> {
-    using CppType = StringValue;
+    using CppType = StringRef;
     using ColumnType = vectorized::ColumnString;
 };
 
diff --git a/be/src/runtime/raw_value.cpp b/be/src/runtime/raw_value.cpp
index 67e70d4be2..7f2896ee19 100644
--- a/be/src/runtime/raw_value.cpp
+++ b/be/src/runtime/raw_value.cpp
@@ -40,7 +40,7 @@ void RawValue::print_value_as_bytes(const void* value, const TypeDescriptor& typ
     }
 
     const char* chars = reinterpret_cast<const char*>(value);
-    const StringValue* string_val = nullptr;
+    const StringRef* string_val = nullptr;
 
     switch (type.type) {
     case TYPE_NULL:
@@ -77,8 +77,8 @@ void RawValue::print_value_as_bytes(const void* value, const TypeDescriptor& typ
     case TYPE_HLL:
     case TYPE_CHAR:
     case TYPE_STRING:
-        string_val = reinterpret_cast<const StringValue*>(value);
-        stream->write(static_cast<char*>(string_val->ptr), string_val->len);
+        string_val = reinterpret_cast<const StringRef*>(value);
+        stream->write(const_cast<char*>(string_val->data), string_val->size);
         return;
 
     case TYPE_DATE:
@@ -140,7 +140,7 @@ void RawValue::print_value(const void* value, const TypeDescriptor& type, int sc
     }
 
     std::string tmp;
-    const StringValue* string_val = nullptr;
+    const StringRef* string_val = nullptr;
 
     switch (type.type) {
     case TYPE_BOOLEAN: {
@@ -177,8 +177,8 @@ void RawValue::print_value(const void* value, const TypeDescriptor& type, int sc
     case TYPE_CHAR:
     case TYPE_VARCHAR:
     case TYPE_STRING:
-        string_val = reinterpret_cast<const StringValue*>(value);
-        tmp.assign(static_cast<char*>(string_val->ptr), string_val->len);
+        string_val = reinterpret_cast<const StringRef*>(value);
+        tmp.assign(const_cast<char*>(string_val->data), string_val->size);
         *stream << tmp;
         return;
 
@@ -288,7 +288,7 @@ void RawValue::print_value(const void* value, const TypeDescriptor& type, int sc
 
     std::stringstream out;
     out.precision(ASCII_PRECISION);
-    const StringValue* string_val = nullptr;
+    const StringRef* string_val = nullptr;
     std::string tmp;
     bool val = false;
 
@@ -305,12 +305,12 @@ void RawValue::print_value(const void* value, const TypeDescriptor& type, int sc
     case TYPE_HLL:
     case TYPE_QUANTILE_STATE:
     case TYPE_STRING: {
-        string_val = reinterpret_cast<const StringValue*>(value);
+        string_val = reinterpret_cast<const StringRef*>(value);
         std::stringstream ss;
-        ss << "ptr:" << (void*)string_val->ptr << " len:" << string_val->len;
+        ss << "ptr:" << (void*)string_val->data << " len:" << string_val->size;
         tmp = ss.str();
-        if (string_val->len <= 1000) {
-            tmp.assign(static_cast<char*>(string_val->ptr), string_val->len);
+        if (string_val->size <= 1000) {
+            tmp.assign(const_cast<char*>(string_val->data), string_val->size);
         }
         str->swap(tmp);
         return;
@@ -416,15 +416,15 @@ void RawValue::write(const void* value, void* dst, const TypeDescriptor& type, M
     case TYPE_VARCHAR:
     case TYPE_CHAR:
     case TYPE_STRING: {
-        const StringValue* src = reinterpret_cast<const StringValue*>(value);
-        StringValue* dest = reinterpret_cast<StringValue*>(dst);
-        dest->len = src->len;
+        const StringRef* src = reinterpret_cast<const StringRef*>(value);
+        StringRef* dest = reinterpret_cast<StringRef*>(dst);
+        dest->size = src->size;
 
         if (pool != nullptr) {
-            dest->ptr = reinterpret_cast<char*>(pool->allocate(dest->len));
-            memcpy(dest->ptr, src->ptr, dest->len);
+            dest->data = reinterpret_cast<char*>(pool->allocate(dest->size));
+            memcpy(const_cast<char*>(dest->data), src->data, dest->size);
         } else {
-            dest->ptr = src->ptr;
+            dest->data = src->data;
         }
 
         break;
@@ -508,12 +508,12 @@ void RawValue::write(const void* value, const TypeDescriptor& type, void* dst, u
     case TYPE_CHAR:
     case TYPE_STRING: {
         DCHECK(buf != nullptr);
-        const StringValue* src = reinterpret_cast<const StringValue*>(value);
-        StringValue* dest = reinterpret_cast<StringValue*>(dst);
-        dest->len = src->len;
-        dest->ptr = reinterpret_cast<char*>(*buf);
-        memcpy(dest->ptr, src->ptr, dest->len);
-        *buf += dest->len;
+        const StringRef* src = reinterpret_cast<const StringRef*>(value);
+        StringRef* dest = reinterpret_cast<StringRef*>(dst);
+        dest->size = src->size;
+        dest->data = reinterpret_cast<char*>(*buf);
+        memcpy(const_cast<char*>(dest->data), src->data, dest->size);
+        *buf += dest->size;
         break;
     }
 
@@ -550,8 +550,8 @@ void RawValue::write(const void* value, Tuple* tuple, const SlotDescriptor* slot
 }
 
 int RawValue::compare(const void* v1, const void* v2, const TypeDescriptor& type) {
-    const StringValue* string_value1;
-    const StringValue* string_value2;
+    const StringRef* string_value1;
+    const StringRef* string_value2;
     const DateTimeValue* ts_value1;
     const DateTimeValue* ts_value2;
     float f1 = 0;
@@ -610,8 +610,8 @@ int RawValue::compare(const void* v1, const void* v2, const TypeDescriptor& type
     case TYPE_VARCHAR:
     case TYPE_HLL:
     case TYPE_STRING:
-        string_value1 = reinterpret_cast<const StringValue*>(v1);
-        string_value2 = reinterpret_cast<const StringValue*>(v2);
+        string_value1 = reinterpret_cast<const StringRef*>(v1);
+        string_value2 = reinterpret_cast<const StringRef*>(v2);
         return string_value1->compare(*string_value2);
 
     case TYPE_DATE:
diff --git a/be/src/runtime/raw_value.h b/be/src/runtime/raw_value.h
index ceb722e847..3e1e05a496 100644
--- a/be/src/runtime/raw_value.h
+++ b/be/src/runtime/raw_value.h
@@ -24,10 +24,10 @@
 
 #include "common/consts.h"
 #include "common/logging.h"
-#include "runtime/string_value.h"
 #include "runtime/types.h"
 #include "util/hash_util.hpp"
 #include "util/types.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -123,8 +123,8 @@ public:
 };
 
 inline bool RawValue::lt(const void* v1, const void* v2, const TypeDescriptor& type) {
-    const StringValue* string_value1;
-    const StringValue* string_value2;
+    const StringRef* string_value1;
+    const StringRef* string_value2;
 
     switch (type.type) {
     case TYPE_BOOLEAN:
@@ -152,8 +152,8 @@ inline bool RawValue::lt(const void* v1, const void* v2, const TypeDescriptor& t
     case TYPE_VARCHAR:
     case TYPE_HLL:
     case TYPE_STRING:
-        string_value1 = reinterpret_cast<const StringValue*>(v1);
-        string_value2 = reinterpret_cast<const StringValue*>(v2);
+        string_value1 = reinterpret_cast<const StringRef*>(v1);
+        string_value2 = reinterpret_cast<const StringRef*>(v2);
         return string_value1->lt(*string_value2);
 
     case TYPE_DATE:
@@ -195,8 +195,8 @@ inline bool RawValue::lt(const void* v1, const void* v2, const TypeDescriptor& t
     };
 }
 inline bool RawValue::eq(const void* v1, const void* v2, const TypeDescriptor& type) {
-    const StringValue* string_value1;
-    const StringValue* string_value2;
+    const StringRef* string_value1;
+    const StringRef* string_value2;
 
     switch (type.type) {
     case TYPE_BOOLEAN:
@@ -224,8 +224,8 @@ inline bool RawValue::eq(const void* v1, const void* v2, const TypeDescriptor& t
     case TYPE_VARCHAR:
     case TYPE_HLL:
     case TYPE_STRING:
-        string_value1 = reinterpret_cast<const StringValue*>(v1);
-        string_value2 = reinterpret_cast<const StringValue*>(v2);
+        string_value1 = reinterpret_cast<const StringRef*>(v1);
+        string_value2 = reinterpret_cast<const StringRef*>(v2);
         return string_value1->eq(*string_value2);
 
     case TYPE_DATE:
@@ -284,8 +284,8 @@ inline uint32_t RawValue::get_hash_value(const void* v, const PrimitiveType& typ
     case TYPE_CHAR:
     case TYPE_HLL:
     case TYPE_STRING: {
-        const StringValue* string_value = reinterpret_cast<const StringValue*>(v);
-        return HashUtil::hash(string_value->ptr, string_value->len, seed);
+        const StringRef* string_value = reinterpret_cast<const StringRef*>(v);
+        return HashUtil::hash(string_value->data, string_value->size, seed);
     }
 
     case TYPE_BOOLEAN: {
@@ -353,8 +353,8 @@ inline uint32_t RawValue::get_hash_value_fvn(const void* v, const PrimitiveType&
     case TYPE_HLL:
     case TYPE_OBJECT:
     case TYPE_STRING: {
-        const StringValue* string_value = reinterpret_cast<const StringValue*>(v);
-        return HashUtil::fnv_hash(string_value->ptr, string_value->len, seed);
+        const StringRef* string_value = reinterpret_cast<const StringRef*>(v);
+        return HashUtil::fnv_hash(string_value->data, string_value->size, seed);
     }
 
     case TYPE_BOOLEAN: {
@@ -421,20 +421,20 @@ inline uint32_t RawValue::zlib_crc32(const void* v, const TypeDescriptor& type,
     case TYPE_VARCHAR:
     case TYPE_HLL:
     case TYPE_STRING: {
-        const StringValue* string_value = reinterpret_cast<const StringValue*>(v);
-        return HashUtil::zlib_crc_hash(string_value->ptr, string_value->len, seed);
+        const StringRef* string_value = reinterpret_cast<const StringRef*>(v);
+        return HashUtil::zlib_crc_hash(string_value->data, string_value->size, seed);
     }
     case TYPE_CHAR: {
         // TODO(zc): ugly, use actual value to compute hash value
-        const StringValue* string_value = reinterpret_cast<const StringValue*>(v);
+        const StringRef* string_value = reinterpret_cast<const StringRef*>(v);
         int len = 0;
-        while (len < string_value->len) {
-            if (string_value->ptr[len] == '\0') {
+        while (len < string_value->size) {
+            if (string_value->data[len] == '\0') {
                 break;
             }
             len++;
         }
-        return HashUtil::zlib_crc_hash(string_value->ptr, len, seed);
+        return HashUtil::zlib_crc_hash(string_value->data, len, seed);
     }
     case TYPE_BOOLEAN:
     case TYPE_TINYINT:
diff --git a/be/src/runtime/string_buffer.hpp b/be/src/runtime/string_buffer.hpp
index e3893138c2..e54473d4e9 100644
--- a/be/src/runtime/string_buffer.hpp
+++ b/be/src/runtime/string_buffer.hpp
@@ -18,24 +18,24 @@
 #pragma once
 
 #include "runtime/mem_pool.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
 // Dynamic-sizable string (similar to std::string) but without as many
 // copies and allocations.
-// StringBuffer wraps a StringValue object with a pool and memory buffer length.
+// StringBuffer wraps a StringRef object with a pool and memory buffer length.
 // It supports a subset of the std::string functionality but will only allocate
 // bigger string buffers as necessary.  std::string tries to be immutable and will
 // reallocate very often.  std::string should be avoided in all hot paths.
 class StringBuffer {
 public:
     // C'tor for StringBuffer.  Memory backing the string will be allocated from
-    // the pool as necessary.  Can optionally be initialized from a StringValue.
-    StringBuffer(MemPool* pool, StringValue* str) : _pool(pool), _buffer_size(0) {
+    // the pool as necessary.  Can optionally be initialized from a StringRef.
+    StringBuffer(MemPool* pool, StringRef* str) : _pool(pool), _buffer_size(0) {
         if (str != NULL) {
             _string_value = *str;
-            _buffer_size = str->len;
+            _buffer_size = str->size;
         }
     }
 
@@ -45,14 +45,14 @@ public:
 
     // append 'str' to the current string, allocating a new buffer as necessary.
     void append(const char* str, int len) {
-        int new_len = len + _string_value.len;
+        int new_len = len + _string_value.size;
 
         if (new_len > _buffer_size) {
             grow_buffer(new_len);
         }
 
-        memcpy(_string_value.ptr + _string_value.len, str, len);
-        _string_value.len = new_len;
+        memcpy(const_cast<char*>(_string_value.data) + _string_value.size, str, len);
+        _string_value.size = new_len;
     }
 
     // TODO: switch everything to uint8_t?
@@ -64,23 +64,23 @@ public:
         append(str, len);
     }
 
-    // clear the underlying StringValue.  The allocated buffer can be reused.
-    void clear() { _string_value.len = 0; }
+    // clear the underlying StringRef.  The allocated buffer can be reused.
+    void clear() { _string_value.size = 0; }
 
-    // Clears the underlying buffer and StringValue
+    // Clears the underlying buffer and StringRef
     void reset() {
-        _string_value.len = 0;
+        _string_value.size = 0;
         _buffer_size = 0;
     }
 
     // Returns whether the current string is empty
-    bool empty() const { return _string_value.len == 0; }
+    bool empty() const { return _string_value.size == 0; }
 
     // Returns the length of the current string
-    int size() const { return _string_value.len; }
+    int size() const { return _string_value.size; }
 
-    // Returns the underlying StringValue
-    const StringValue& str() const { return _string_value; }
+    // Returns the underlying StringRef
+    const StringRef& str() const { return _string_value; }
 
     // Returns the buffer size
     int buffer_size() const { return _buffer_size; }
@@ -92,16 +92,16 @@ private:
     void grow_buffer(int new_len) {
         char* new_buffer = reinterpret_cast<char*>(_pool->allocate(new_len));
 
-        if (_string_value.len > 0) {
-            memcpy(new_buffer, _string_value.ptr, _string_value.len);
+        if (_string_value.size > 0) {
+            memcpy(new_buffer, _string_value.data, _string_value.size);
         }
 
-        _string_value.ptr = new_buffer;
+        _string_value.data = new_buffer;
         _buffer_size = new_len;
     }
 
     MemPool* _pool;
-    StringValue _string_value;
+    StringRef _string_value;
     int _buffer_size;
 };
 
diff --git a/be/src/runtime/string_search.hpp b/be/src/runtime/string_search.hpp
index 6565f516b4..a7d00673aa 100644
--- a/be/src/runtime/string_search.hpp
+++ b/be/src/runtime/string_search.hpp
@@ -22,42 +22,37 @@
 #include <vector>
 
 #include "common/logging.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 #include "vec/common/volnitsky.h"
 
 namespace doris {
 
 class StringSearch {
 public:
-    virtual ~StringSearch() {}
+    virtual ~StringSearch() = default;
     StringSearch() : _pattern(nullptr) {}
 
-    StringSearch(const StringValue* pattern) { set_pattern(pattern); }
-
-    void set_pattern(const StringValue* pattern) {
-        _pattern = pattern;
-        _vol_searcher.reset(new Volnitsky(pattern->ptr, pattern->len));
-    }
+    StringSearch(const StringRef* pattern) { set_pattern(pattern); }
 
     void set_pattern(const StringRef* pattern) {
-        _pattern = reinterpret_cast<const StringValue*>(pattern);
+        _pattern = pattern;
         _vol_searcher.reset(new Volnitsky(pattern->data, pattern->size));
     }
 
     // search for this pattern in str.
     //   Returns the offset into str if the pattern exists
     //   Returns -1 if the pattern is not found
-    int search(const StringValue* str) const {
-        auto it = search(str->ptr, str->len);
-        if (it == str->ptr + str->len) {
+    int search(const StringRef* str) const {
+        auto it = search(str->data, str->size);
+        if (it == str->data + str->size) {
             return -1;
         } else {
-            return it - str->ptr;
+            return it - str->data;
         }
     }
 
     int search(const StringRef& str) const {
-        auto it = search(const_cast<char*>(str.data), str.size);
+        auto it = search(str.data, str.size);
         if (it == str.data + str.size) {
             return -1;
         } else {
@@ -68,18 +63,18 @@ public:
     // search for this pattern in str.
     //   Returns the offset into str if the pattern exists
     //   Returns str+len if the pattern is not found
-    const char* search(char* str, size_t len) const {
-        if (!str || !_pattern || _pattern->len == 0) {
+    const char* search(const char* str, size_t len) const {
+        if (!str || !_pattern || _pattern->size == 0) {
             return str + len;
         }
 
         return _vol_searcher->search(str, len);
     }
 
-    inline size_t get_pattern_length() { return _pattern ? _pattern->len : 0; }
+    inline size_t get_pattern_length() { return _pattern ? _pattern->size : 0; }
 
 private:
-    const StringValue* _pattern;
+    const StringRef* _pattern;
     std::unique_ptr<Volnitsky> _vol_searcher;
 };
 
diff --git a/be/src/runtime/string_value.cpp b/be/src/runtime/string_value.cpp
deleted file mode 100644
index aa16783545..0000000000
--- a/be/src/runtime/string_value.cpp
+++ /dev/null
@@ -1,57 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-// This file is copied from
-// https://github.com/apache/impala/blob/branch-2.9.0/be/src/runtime/string-value.cc
-// and modified by Doris
-
-#include "runtime/string_value.h"
-
-#include <cstring>
-
-#include "exec/olap_utils.h"
-
-namespace doris {
-
-std::string StringValue::debug_string() const {
-    return std::string(ptr, len);
-}
-
-std::string StringValue::to_string() const {
-    return std::string(ptr, len);
-}
-
-std::ostream& operator<<(std::ostream& os, const StringValue& string_value) {
-    return os << string_value.debug_string();
-}
-
-std::size_t operator-(const StringValue& v1, const StringValue& v2) {
-    return 0;
-}
-
-constexpr char StringValue::MIN_CHAR = 0x00;
-
-constexpr char StringValue::MAX_CHAR = 0xff;
-
-StringValue StringValue::min_string_val() {
-    return StringValue((char*)(&StringValue::MIN_CHAR), 0);
-}
-
-StringValue StringValue::max_string_val() {
-    return StringValue((char*)(&StringValue::MAX_CHAR), 1);
-}
-
-} // namespace doris
diff --git a/be/src/runtime/string_value.h b/be/src/runtime/string_value.h
deleted file mode 100644
index 4878ca31e4..0000000000
--- a/be/src/runtime/string_value.h
+++ /dev/null
@@ -1,211 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-// This file is copied from
-// https://github.com/apache/impala/blob/branch-2.9.0/be/src/runtime/string-value.h
-// and modified by Doris
-
-#pragma once
-
-#include <string.h>
-
-#include "udf/udf.h"
-#include "util/cpu_info.h"
-#include "util/hash_util.hpp"
-#include "vec/common/string_ref.h"
-#ifdef __SSE4_2__
-#include "util/sse_util.hpp"
-#endif
-
-namespace doris {
-
-// Compare two strings using sse4.2 intrinsics if they are available. This code assumes
-// that the trivial cases are already handled (i.e. one string is empty).
-// Returns:
-//   < 0 if s1 < s2
-//   0 if s1 == s2
-//   > 0 if s1 > s2
-// The SSE code path is just under 2x faster than the non-sse code path.
-//   - s1/n1: ptr/len for the first string
-//   - s2/n2: ptr/len for the second string
-//   - len: min(n1, n2) - this can be more cheaply passed in by the caller
-static inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2,
-                                 int64_t len) {
-    DCHECK_EQ(len, std::min(n1, n2));
-#ifdef __SSE4_2__
-    while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) {
-        __m128i xmm0 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s1));
-        __m128i xmm1 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s2));
-        int chars_match = _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1,
-                                       sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE);
-        if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) {
-            return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match];
-        }
-        len -= sse_util::CHARS_PER_128_BIT_REGISTER;
-        s1 += sse_util::CHARS_PER_128_BIT_REGISTER;
-        s2 += sse_util::CHARS_PER_128_BIT_REGISTER;
-    }
-#endif
-    unsigned char u1, u2;
-    while (len-- > 0) {
-        u1 = (unsigned char)*s1++;
-        u2 = (unsigned char)*s2++;
-        if (u1 != u2) return u1 - u2;
-        if (u1 == '\0') return n1 - n2;
-    }
-
-    return n1 - n2;
-}
-
-// The format of a string-typed slot.
-// The returned StringValue of all functions that return StringValue
-// shares its buffer the parent.
-struct StringValue {
-    const static char MIN_CHAR;
-    const static char MAX_CHAR;
-
-    static const int MAX_LENGTH = (1 << 30);
-    // TODO: change ptr to an offset relative to a contiguous memory block,
-    // so that we can send row batches between nodes without having to swizzle
-    // pointers
-    // NOTE: This struct should keep the same memory layout with Slice, otherwise
-    // it will lead to BE crash.
-    // TODO(zc): we should unify this struct with Slice some day.
-    char* ptr;
-    size_t len;
-
-    StringValue(char* ptr, int len) : ptr(ptr), len(len) {}
-    StringValue(const char* ptr, int len) : ptr(const_cast<char*>(ptr)), len(len) {}
-    StringValue() : ptr(nullptr), len(0) {}
-    StringValue(const StringRef& str) : ptr(const_cast<char*>(str.data)), len(str.size) {}
-
-    /// Construct a StringValue from 's'.  's' must be valid for as long as
-    /// this object is valid.
-    explicit StringValue(const std::string& s) : ptr(const_cast<char*>(s.c_str())), len(s.size()) {
-        DCHECK_LE(len, MAX_LENGTH);
-    }
-
-    void replace(char* ptr, int len) {
-        this->ptr = ptr;
-        this->len = len;
-    }
-
-    // Byte-by-byte comparison. Returns:
-    // this < other: -1
-    // this == other: 0
-    // this > other: 1
-    int compare(const StringValue& other) const {
-        int l = std::min(len, other.len);
-
-        if (l == 0) {
-            if (len == other.len) {
-                return 0;
-            } else if (len == 0) {
-                return -1;
-            } else {
-                DCHECK_EQ(other.len, 0);
-                return 1;
-            }
-        }
-
-        return string_compare(this->ptr, this->len, other.ptr, other.len, l);
-    }
-
-    // ==
-    bool eq(const StringValue& other) const {
-        if (this->len != other.len) {
-            return false;
-        }
-
-#if defined(__SSE2__)
-        return memequalSSE2Wide(this->ptr, other.ptr, this->len);
-#endif
-
-        return string_compare(this->ptr, this->len, other.ptr, other.len, this->len) == 0;
-    }
-
-    bool operator==(const StringValue& other) const { return eq(other); }
-    // !=
-    bool ne(const StringValue& other) const { return !eq(other); }
-    // <=
-    bool le(const StringValue& other) const { return compare(other) <= 0; }
-    // >=
-    bool ge(const StringValue& other) const { return compare(other) >= 0; }
-    // <
-    bool lt(const StringValue& other) const { return compare(other) < 0; }
-    // >
-    bool gt(const StringValue& other) const { return compare(other) > 0; }
-
-    bool operator!=(const StringValue& other) const { return ne(other); }
-
-    bool operator<=(const StringValue& other) const { return le(other); }
-
-    bool operator>=(const StringValue& other) const { return ge(other); }
-
-    bool operator<(const StringValue& other) const { return lt(other); }
-
-    bool operator>(const StringValue& other) const { return gt(other); }
-
-    std::string debug_string() const;
-
-    std::string to_string() const;
-
-    // Returns the substring starting at start_pos until the end of string.
-    StringValue substring(int start_pos) const;
-
-    // Returns the substring starting at start_pos with given length.
-    // If new_len < 0 then the substring from start_pos to end of string is returned.
-    StringValue substring(int start_pos, int new_len) const;
-
-    // Trims leading and trailing spaces.
-    StringValue trim() const;
-
-    // Find the first position char of appear, return -1 if not found
-    int64_t find_first_of(char c) const;
-
-    void to_string_val(doris_udf::StringVal* sv) const {
-        *sv = doris_udf::StringVal(reinterpret_cast<uint8_t*>(ptr), len);
-    }
-
-    static StringValue from_string_val(const doris_udf::StringVal& sv) {
-        return StringValue(reinterpret_cast<char*>(sv.ptr), sv.len);
-    }
-
-    static StringValue min_string_val();
-
-    static StringValue max_string_val();
-
-    struct Comparator {
-        bool operator()(const StringValue& a, const StringValue& b) const {
-            return a.compare(b) < 0;
-        }
-    };
-
-    struct HashOfStringValue {
-        size_t operator()(const StringValue& v) const { return HashUtil::hash(v.ptr, v.len, 0); }
-    };
-};
-
-// This function must be called 'hash_value' to be picked up by boost.
-inline std::size_t hash_value(const StringValue& v) {
-    return HashUtil::hash(v.ptr, v.len, 0);
-}
-
-std::ostream& operator<<(std::ostream& os, const StringValue& string_value);
-
-std::size_t operator-(const StringValue& v1, const StringValue& v2);
-
-} // namespace doris
diff --git a/be/src/runtime/string_value.hpp b/be/src/runtime/string_value.hpp
deleted file mode 100644
index fb8039e661..0000000000
--- a/be/src/runtime/string_value.hpp
+++ /dev/null
@@ -1,59 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-// This file is copied from
-// https://github.com/apache/impala/blob/branch-2.9.0/be/src/runtime/string-value.hpp
-// and modified by Doris
-
-#pragma once
-
-#include <cstring>
-
-#include "runtime/string_value.h"
-
-namespace doris {
-
-inline StringValue StringValue::substring(int start_pos) const {
-    return StringValue(ptr + start_pos, len - start_pos);
-}
-
-inline StringValue StringValue::substring(int start_pos, int new_len) const {
-    return StringValue(ptr + start_pos, (new_len < 0) ? (len - start_pos) : new_len);
-}
-
-inline StringValue StringValue::trim() const {
-    // Remove leading and trailing spaces.
-    int32_t begin = 0;
-
-    while (begin < len && ptr[begin] == ' ') {
-        ++begin;
-    }
-
-    int32_t end = len - 1;
-
-    while (end > begin && ptr[end] == ' ') {
-        --end;
-    }
-
-    return StringValue(ptr + begin, end - begin + 1);
-}
-
-inline int64_t StringValue::find_first_of(char c) const {
-    const char* p = static_cast<const char*>(memchr(ptr, c, len));
-    return p == nullptr ? -1 : p - ptr;
-}
-
-} // namespace doris
diff --git a/be/src/runtime/tuple.cpp b/be/src/runtime/tuple.cpp
index 641c13c4bb..b71a55b050 100644
--- a/be/src/runtime/tuple.cpp
+++ b/be/src/runtime/tuple.cpp
@@ -30,9 +30,9 @@
 #include "runtime/descriptors.h"
 #include "runtime/mem_pool.h"
 #include "runtime/raw_value.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple_row.h"
 #include "util/mem_util.hpp"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -57,8 +57,8 @@ int64_t Tuple::varlen_byte_size(const TupleDescriptor& desc) const {
         if (is_null((*slot)->null_indicator_offset())) {
             continue;
         }
-        const StringValue* string_val = get_string_slot((*slot)->tuple_offset());
-        result += string_val->len;
+        const StringRef* string_val = get_string_slot((*slot)->tuple_offset());
+        result += string_val->size;
     }
 
     return result;
@@ -76,17 +76,17 @@ void Tuple::deep_copy(Tuple* dst, const TupleDescriptor& desc, MemPool* pool, bo
     // allocate in the same pool and then copy all non-null string slots
     for (auto string_slot : desc.string_slots()) {
         DCHECK(string_slot->type().is_string_type());
-        StringValue* string_v = dst->get_string_slot(string_slot->tuple_offset());
+        StringRef* string_v = dst->get_string_slot(string_slot->tuple_offset());
         if (!dst->is_null(string_slot->null_indicator_offset())) {
-            if (string_v->len != 0) {
+            if (string_v->size != 0) {
                 int64_t offset = pool->total_allocated_bytes();
-                char* string_copy = (char*)(pool->allocate(string_v->len));
-                memory_copy(string_copy, string_v->ptr, string_v->len);
-                string_v->ptr = (convert_ptrs ? convert_to<char*>(offset) : string_copy);
+                char* string_copy = (char*)(pool->allocate(string_v->size));
+                memory_copy(string_copy, string_v->data, string_v->size);
+                string_v->data = (convert_ptrs ? convert_to<char*>(offset) : string_copy);
             }
         } else {
-            string_v->ptr = nullptr;
-            string_v->len = 0;
+            string_v->data = nullptr;
+            string_v->size = 0;
         }
     }
 
@@ -135,14 +135,14 @@ int64_t Tuple::dcopy_with_new(Tuple* dst, const TupleDescriptor& desc) {
         DCHECK(slot->type().is_string_type());
 
         if (!dst->is_null(slot->null_indicator_offset())) {
-            StringValue* string_v = dst->get_string_slot(slot->tuple_offset());
-            bytes += string_v->len;
-            if (string_v->len != 0) {
-                char* string_copy = new char[string_v->len];
-                memory_copy(string_copy, string_v->ptr, string_v->len);
-                string_v->ptr = string_copy;
+            StringRef* string_v = dst->get_string_slot(slot->tuple_offset());
+            bytes += string_v->size;
+            if (string_v->size != 0) {
+                char* string_copy = new char[string_v->size];
+                memory_copy(string_copy, string_v->data, string_v->size);
+                string_v->data = string_copy;
             } else {
-                string_v->ptr = nullptr;
+                string_v->data = nullptr;
             }
         }
     }
@@ -153,10 +153,10 @@ int64_t Tuple::release_string(const TupleDescriptor& desc) {
     int64_t bytes = 0;
     for (auto slot : desc.string_slots()) {
         if (!is_null(slot->null_indicator_offset())) {
-            StringValue* string_v = get_string_slot(slot->tuple_offset());
-            delete[] string_v->ptr;
-            string_v->ptr = nullptr;
-            bytes += string_v->len;
+            StringRef* string_v = get_string_slot(slot->tuple_offset());
+            delete[] string_v->data;
+            string_v->data = nullptr;
+            bytes += string_v->size;
         }
     }
     return bytes;
@@ -171,15 +171,15 @@ void Tuple::deep_copy(const TupleDescriptor& desc, char** data, int64_t* offset,
 
     for (auto slot_desc : desc.string_slots()) {
         DCHECK(slot_desc->type().is_string_type());
-        StringValue* string_v = dst->get_string_slot(slot_desc->tuple_offset());
+        StringRef* string_v = dst->get_string_slot(slot_desc->tuple_offset());
         if (!dst->is_null(slot_desc->null_indicator_offset())) {
-            memory_copy(*data, string_v->ptr, string_v->len);
-            string_v->ptr = (convert_ptrs ? convert_to<char*>(*offset) : *data);
-            *data += string_v->len;
-            *offset += string_v->len;
+            memory_copy(*data, string_v->data, string_v->size);
+            string_v->data = (convert_ptrs ? convert_to<char*>(*offset) : *data);
+            *data += string_v->size;
+            *offset += string_v->size;
         } else {
-            string_v->ptr = (convert_ptrs ? convert_to<char*>(*offset) : *data);
-            string_v->len = 0;
+            string_v->data = (convert_ptrs ? convert_to<char*>(*offset) : *data);
+            string_v->size = 0;
         }
     }
 
@@ -198,7 +198,7 @@ void Tuple::deep_copy(const TupleDescriptor& desc, char** data, int64_t* offset,
 template <bool collect_string_vals>
 void Tuple::materialize_exprs(TupleRow* row, const TupleDescriptor& desc,
                               const std::vector<ExprContext*>& materialize_expr_ctxs, MemPool* pool,
-                              std::vector<StringValue*>* non_null_var_len_values,
+                              std::vector<StringRef*>* non_null_var_len_values,
                               int* total_var_len) {
     if (collect_string_vals) {
         non_null_var_len_values->clear();
@@ -235,9 +235,9 @@ void Tuple::materialize_exprs(TupleRow* row, const TupleDescriptor& desc,
             RawValue::write(src, dst, slot_desc->type(), pool);
             if (collect_string_vals) {
                 if (slot_desc->type().is_string_type()) {
-                    StringValue* string_val = convert_to<StringValue*>(dst);
+                    StringRef* string_val = convert_to<StringRef*>(dst);
                     non_null_var_len_values->push_back(string_val);
-                    *total_var_len += string_val->len;
+                    *total_var_len += string_val->size;
                 }
             }
         } else {
@@ -252,12 +252,12 @@ void Tuple::materialize_exprs(TupleRow* row, const TupleDescriptor& desc,
 template void Tuple::materialize_exprs<false>(
         TupleRow* row, const TupleDescriptor& desc,
         const std::vector<ExprContext*>& materialize_expr_ctxs, MemPool* pool,
-        std::vector<StringValue*>* non_null_var_values, int* total_var_len);
+        std::vector<StringRef*>* non_null_var_values, int* total_var_len);
 
 template void Tuple::materialize_exprs<true>(TupleRow* row, const TupleDescriptor& desc,
                                              const std::vector<ExprContext*>& materialize_expr_ctxs,
                                              MemPool* pool,
-                                             std::vector<StringValue*>* non_null_var_values,
+                                             std::vector<StringRef*>* non_null_var_values,
                                              int* total_var_len);
 
 std::string Tuple::to_string(const TupleDescriptor& d) const {
diff --git a/be/src/runtime/tuple.h b/be/src/runtime/tuple.h
index 821f9b3eac..e19d8cc68c 100644
--- a/be/src/runtime/tuple.h
+++ b/be/src/runtime/tuple.h
@@ -28,7 +28,7 @@
 
 namespace doris {
 
-struct StringValue;
+struct StringRef;
 class CollectionValue;
 class TupleDescriptor;
 class DateTimeValue;
@@ -115,7 +115,7 @@ public:
     template <bool collect_string_vals>
     void materialize_exprs(TupleRow* row, const TupleDescriptor& desc,
                            const std::vector<ExprContext*>& materialize_expr_ctxs, MemPool* pool,
-                           std::vector<StringValue*>* non_null_var_len_values, int* total_var_len);
+                           std::vector<StringRef*>* non_null_var_len_values, int* total_var_len);
 
     // Turn null indicator bit on.
     // Turn null indicator bit on. For non-nullable slots, the mask will be 0 and
@@ -147,14 +147,14 @@ public:
         return &_data[offset];
     }
 
-    StringValue* get_string_slot(int offset) {
+    StringRef* get_string_slot(int offset) {
         DCHECK(offset != -1); // -1 offset indicates non-materialized slot
-        return reinterpret_cast<StringValue*>(&_data[offset]);
+        return reinterpret_cast<StringRef*>(&_data[offset]);
     }
 
-    const StringValue* get_string_slot(int offset) const {
+    const StringRef* get_string_slot(int offset) const {
         DCHECK(offset != -1); // -1 offset indicates non-materialized slot
-        return reinterpret_cast<const StringValue*>(&_data[offset]);
+        return reinterpret_cast<const StringRef*>(&_data[offset]);
     }
 
     CollectionValue* get_collection_slot(int offset) {
diff --git a/be/src/runtime/type_limit.h b/be/src/runtime/type_limit.h
index 923f2891e9..378c27e9e1 100644
--- a/be/src/runtime/type_limit.h
+++ b/be/src/runtime/type_limit.h
@@ -19,7 +19,7 @@
 
 #include "runtime/datetime_value.h"
 #include "runtime/decimalv2_value.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -30,9 +30,9 @@ struct type_limit {
 };
 
 template <>
-struct type_limit<StringValue> {
-    static StringValue min() { return StringValue::min_string_val(); }
-    static StringValue max() { return StringValue::max_string_val(); }
+struct type_limit<StringRef> {
+    static StringRef min() { return StringRef::min_string_val(); }
+    static StringRef max() { return StringRef::max_string_val(); }
 };
 
 template <>
diff --git a/be/src/udf/udf.h b/be/src/udf/udf.h
index 49758f40d1..734f6eab1e 100644
--- a/be/src/udf/udf.h
+++ b/be/src/udf/udf.h
@@ -32,7 +32,7 @@
 namespace doris {
 class FunctionContextImpl;
 struct ColumnPtrWrapper;
-struct StringValue;
+struct StringRef;
 class BitmapValue;
 class DecimalV2Value;
 class DateTimeValue;
@@ -303,7 +303,8 @@ private:
 // the same lifetime as results for the UDF. In other words, the UDF can return
 // memory from input arguments without making copies. For example, a function like
 // substring will not need to allocate and copy the smaller string. For cases where
-// the UDF needs a buffer, it should use the StringValue(FunctionContext, len) c'tor.
+// the UDF needs a buffer, it should use the StringRef(FunctionContext, len) c'tor.
+// TODO: things above is not right. StringRef shouldn't use here.
 //
 // The UDF can optionally specify a Prepare function. The prepare function is called
 // once before any calls to the Udf to evaluate values. This is the appropriate time for
@@ -743,6 +744,7 @@ struct DateTimeV2Val : public AnyVal {
     bool operator!=(const DateTimeV2Val& other) const { return !(*this == other); }
 };
 
+// TODO: need to set explicit align?
 // Note: there is a difference between a nullptr string (is_null == true) and an
 // empty string (len == 0).
 struct StringVal : public AnyVal {
diff --git a/be/src/util/bitmap_intersect.h b/be/src/util/bitmap_intersect.h
index 94ff8dc283..bc4a8c6923 100644
--- a/be/src/util/bitmap_intersect.h
+++ b/be/src/util/bitmap_intersect.h
@@ -17,9 +17,9 @@
 #pragma once
 #include <parallel_hashmap/phmap.h>
 
-#include "runtime/string_value.h"
 #include "udf/udf.h"
 #include "util/bitmap_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -62,9 +62,9 @@ public:
 };
 
 template <>
-inline StringValue Helper::get_val<StringVal>(const StringVal& x) {
+inline StringRef Helper::get_val<StringVal>(const StringVal& x) {
     DCHECK(!x.is_null);
-    return StringValue::from_string_val(x);
+    return StringRef(x);
 }
 
 template <>
@@ -98,11 +98,11 @@ inline char* Helper::write_to<DecimalV2Value>(const DecimalV2Value& v, char* des
 }
 
 template <>
-inline char* Helper::write_to<StringValue>(const StringValue& v, char* dest) {
-    *(int32_t*)dest = v.len;
+inline char* Helper::write_to<StringRef>(const StringRef& v, char* dest) {
+    *(int32_t*)dest = v.size;
     dest += 4;
-    memcpy(dest, v.ptr, v.len);
-    dest += v.len;
+    memcpy(dest, v.data, v.size);
+    dest += v.size;
     return dest;
 }
 
@@ -127,8 +127,8 @@ inline int32_t Helper::serialize_size<DecimalV2Value>(const DecimalV2Value& v) {
 }
 
 template <>
-inline int32_t Helper::serialize_size<StringValue>(const StringValue& v) {
-    return v.len + 4;
+inline int32_t Helper::serialize_size<StringRef>(const StringRef& v) {
+    return v.size + 4;
 }
 
 template <>
@@ -157,10 +157,10 @@ inline void Helper::read_from<DecimalV2Value>(const char** src, DecimalV2Value*
 }
 
 template <>
-inline void Helper::read_from<StringValue>(const char** src, StringValue* result) {
+inline void Helper::read_from<StringRef>(const char** src, StringRef* result) {
     int32_t length = *(int32_t*)(*src);
     *src += 4;
-    *result = StringValue((char*)*src, length);
+    *result = StringRef((char*)*src, length);
     *src += length;
 }
 
diff --git a/be/src/util/simd/vstring_function.h b/be/src/util/simd/vstring_function.h
index 3c1a4e7f32..d716e8e573 100644
--- a/be/src/util/simd/vstring_function.h
+++ b/be/src/util/simd/vstring_function.h
@@ -27,8 +27,8 @@
 #include <sse2neon.h>
 #endif
 
-#include "runtime/string_value.hpp"
 #include "util/simd/lower_upper_impl.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
diff --git a/be/src/util/static_asserts.cpp b/be/src/util/static_asserts.cpp
index b578646f9b..b740349593 100644
--- a/be/src/util/static_asserts.cpp
+++ b/be/src/util/static_asserts.cpp
@@ -19,16 +19,16 @@
 // and modified by Doris
 
 #include "runtime/datetime_value.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 #include "vec/runtime/vdatetime_value.h"
 namespace doris {
 // This class is unused.  It contains static (compile time) asserts.
 // This is useful to validate struct sizes and other similar things
-// at compile time.  If these asserts fail, the compile will fail.
+// at compile time.  If these assertions fail, the compiling will fail.
 class UnusedClass {
 private:
-    static_assert(sizeof(StringValue) == 16);
-    static_assert(offsetof(StringValue, len) == 8);
+    static_assert(sizeof(StringRef) == 16);
+    static_assert(offsetof(StringRef, size) == 8);
     // Datetime value
     static_assert(sizeof(DateTimeValue) == 16);
     static_assert(sizeof(doris::vectorized::VecDateTimeValue) == 8);
diff --git a/be/src/util/url_parser.cpp b/be/src/util/url_parser.cpp
index 06ed454251..e54281eb2c 100644
--- a/be/src/util/url_parser.cpp
+++ b/be/src/util/url_parser.cpp
@@ -19,25 +19,25 @@
 
 #include <string>
 
-#include "runtime/string_value.hpp"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
-const StringValue UrlParser::_s_url_authority(const_cast<char*>("AUTHORITY"), 9);
-const StringValue UrlParser::_s_url_file(const_cast<char*>("FILE"), 4);
-const StringValue UrlParser::_s_url_host(const_cast<char*>("HOST"), 4);
-const StringValue UrlParser::_s_url_path(const_cast<char*>("PATH"), 4);
-const StringValue UrlParser::_s_url_protocol(const_cast<char*>("PROTOCOL"), 8);
-const StringValue UrlParser::_s_url_query(const_cast<char*>("QUERY"), 5);
-const StringValue UrlParser::_s_url_ref(const_cast<char*>("REF"), 3);
-const StringValue UrlParser::_s_url_userinfo(const_cast<char*>("USERINFO"), 8);
-const StringValue UrlParser::_s_url_port(const_cast<char*>("PORT"), 4);
-const StringValue UrlParser::_s_protocol(const_cast<char*>("://"), 3);
-const StringValue UrlParser::_s_at(const_cast<char*>("@"), 1);
-const StringValue UrlParser::_s_slash(const_cast<char*>("/"), 1);
-const StringValue UrlParser::_s_colon(const_cast<char*>(":"), 1);
-const StringValue UrlParser::_s_question(const_cast<char*>("?"), 1);
-const StringValue UrlParser::_s_hash(const_cast<char*>("#"), 1);
+const StringRef UrlParser::_s_url_authority(const_cast<char*>("AUTHORITY"), 9);
+const StringRef UrlParser::_s_url_file(const_cast<char*>("FILE"), 4);
+const StringRef UrlParser::_s_url_host(const_cast<char*>("HOST"), 4);
+const StringRef UrlParser::_s_url_path(const_cast<char*>("PATH"), 4);
+const StringRef UrlParser::_s_url_protocol(const_cast<char*>("PROTOCOL"), 8);
+const StringRef UrlParser::_s_url_query(const_cast<char*>("QUERY"), 5);
+const StringRef UrlParser::_s_url_ref(const_cast<char*>("REF"), 3);
+const StringRef UrlParser::_s_url_userinfo(const_cast<char*>("USERINFO"), 8);
+const StringRef UrlParser::_s_url_port(const_cast<char*>("PORT"), 4);
+const StringRef UrlParser::_s_protocol(const_cast<char*>("://"), 3);
+const StringRef UrlParser::_s_at(const_cast<char*>("@"), 1);
+const StringRef UrlParser::_s_slash(const_cast<char*>("/"), 1);
+const StringRef UrlParser::_s_colon(const_cast<char*>(":"), 1);
+const StringRef UrlParser::_s_question(const_cast<char*>("?"), 1);
+const StringRef UrlParser::_s_hash(const_cast<char*>("#"), 1);
 const StringSearch UrlParser::_s_protocol_search(&_s_protocol);
 const StringSearch UrlParser::_s_at_search(&_s_at);
 const StringSearch UrlParser::_s_slash_search(&_s_slash);
@@ -45,11 +45,11 @@ const StringSearch UrlParser::_s_colon_search(&_s_colon);
 const StringSearch UrlParser::_s_question_search(&_s_question);
 const StringSearch UrlParser::_s_hash_search(&_s_hash);
 
-bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* result) {
-    result->ptr = nullptr;
-    result->len = 0;
+bool UrlParser::parse_url(const StringRef& url, UrlPart part, StringRef* result) {
+    result->data = nullptr;
+    result->size = 0;
     // Remove leading and trailing spaces.
-    StringValue trimmed_url = url.trim();
+    StringRef trimmed_url = url.trim();
 
     // All parts require checking for the _s_protocol.
     int32_t protocol_pos = _s_protocol_search.search(&trimmed_url);
@@ -58,7 +58,7 @@ bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* res
     }
 
     // Positioned to first char after '://'.
-    StringValue protocol_end = trimmed_url.substring(protocol_pos + _s_protocol.len);
+    StringRef protocol_end = trimmed_url.substring(protocol_pos + _s_protocol.size);
 
     switch (part) {
     case AUTHORITY: {
@@ -78,7 +78,7 @@ bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* res
             return true;
         }
 
-        StringValue path_start = protocol_end.substring(start_pos);
+        StringRef path_start = protocol_end.substring(start_pos);
         int32_t end_pos;
 
         if (part == FILE) {
@@ -107,10 +107,10 @@ bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* res
             start_pos = 0;
         } else {
             // Skip '@'.
-            start_pos += _s_at.len;
+            start_pos += _s_at.size;
         }
 
-        StringValue host_start = protocol_end.substring(start_pos);
+        StringRef host_start = protocol_end.substring(start_pos);
         // Find ':' to strip out port.
         int32_t end_pos = _s_colon_search.search(&host_start);
 
@@ -137,7 +137,7 @@ bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* res
             return false;
         }
 
-        StringValue query_start = protocol_end.substring(start_pos + _s_question.len);
+        StringRef query_start = protocol_end.substring(start_pos + _s_question.size);
         // End string _s_at next '#'.
         int32_t end_pos = _s_hash_search.search(&query_start);
         *result = query_start.substring(0, end_pos);
@@ -153,7 +153,7 @@ bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* res
             return false;
         }
 
-        *result = protocol_end.substring(start_pos + _s_hash.len);
+        *result = protocol_end.substring(start_pos + _s_hash.size);
         break;
     }
 
@@ -179,10 +179,10 @@ bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* res
             start_pos = 0;
         } else {
             // Skip '@'.
-            start_pos += _s_at.len;
+            start_pos += _s_at.size;
         }
 
-        StringValue host_start = protocol_end.substring(start_pos);
+        StringRef host_start = protocol_end.substring(start_pos);
         // Find ':' to strip out port.
         int32_t end_pos = _s_colon_search.search(&host_start);
         //no port found
@@ -190,7 +190,7 @@ bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* res
             return false;
         }
 
-        StringValue port_start_str = protocol_end.substring(end_pos + _s_colon.len);
+        StringRef port_start_str = protocol_end.substring(end_pos + _s_colon.size);
         int32_t port_end_pos = _s_slash_search.search(&port_start_str);
         //if '/' not found, try to find '?'
         if (port_end_pos < 0) {
@@ -207,20 +207,20 @@ bool UrlParser::parse_url(const StringValue& url, UrlPart part, StringValue* res
     return true;
 }
 
-bool UrlParser::parse_url_key(const StringValue& url, UrlPart part, const StringValue& key,
-                              StringValue* result) {
+bool UrlParser::parse_url_key(const StringRef& url, UrlPart part, const StringRef& key,
+                              StringRef* result) {
     // Part must be query to ask for a specific query key.
     if (part != QUERY) {
         return false;
     }
 
     // Remove leading and trailing spaces.
-    StringValue trimmed_url = url.trim();
+    StringRef trimmed_url = url.trim();
 
     // Search for the key in the url, ignoring malformed URLs for now.
     StringSearch key_search(&key);
 
-    while (trimmed_url.len > 0) {
+    while (trimmed_url.size > 0) {
         // Search for the key in the current substring.
         int32_t key_pos = key_search.search(&trimmed_url);
         bool match = true;
@@ -232,31 +232,31 @@ bool UrlParser::parse_url_key(const StringValue& url, UrlPart part, const String
         // Key pos must be != 0 because it must be preceded by a '?' or a '&'.
         // Check that the char before key_pos is either '?' or '&'.
         if (key_pos == 0 ||
-            (trimmed_url.ptr[key_pos - 1] != '?' && trimmed_url.ptr[key_pos - 1] != '&')) {
+            (trimmed_url.data[key_pos - 1] != '?' && trimmed_url.data[key_pos - 1] != '&')) {
             match = false;
         }
 
         // Advance substring beyond matching key.
-        trimmed_url = trimmed_url.substring(key_pos + key.len);
+        trimmed_url = trimmed_url.substring(key_pos + key.size);
 
         if (!match) {
             continue;
         }
 
-        if (trimmed_url.len <= 0) {
+        if (trimmed_url.size <= 0) {
             break;
         }
 
         // Next character must be '=', otherwise the match cannot be a key in the query part.
-        if (trimmed_url.ptr[0] != '=') {
+        if (trimmed_url.data[0] != '=') {
             continue;
         }
 
         int32_t pos = 1;
 
         // Find ending position of key's value by matching '#' or '&'.
-        while (pos < trimmed_url.len) {
-            switch (trimmed_url.ptr[pos]) {
+        while (pos < trimmed_url.size) {
+            switch (trimmed_url.data[pos]) {
             case '#':
             case '&':
                 *result = trimmed_url.substring(1, pos - 1);
@@ -274,13 +274,13 @@ bool UrlParser::parse_url_key(const StringValue& url, UrlPart part, const String
     return false;
 }
 
-UrlParser::UrlPart UrlParser::get_url_part(const StringValue& part) {
+UrlParser::UrlPart UrlParser::get_url_part(const StringRef& part) {
     // Quick filter on requested URL part, based on first character.
     // Hive requires the requested URL part to be all upper case.
     std::string part_str = part.to_string();
     transform(part_str.begin(), part_str.end(), part_str.begin(), ::toupper);
-    StringValue newPart = StringValue(part_str);
-    switch (newPart.ptr[0]) {
+    StringRef newPart = StringRef(part_str);
+    switch (newPart.data[0]) {
     case 'A': {
         if (!newPart.eq(_s_url_authority)) {
             return INVALID;
@@ -346,10 +346,10 @@ UrlParser::UrlPart UrlParser::get_url_part(const StringValue& part) {
     }
 }
 
-StringValue UrlParser::extract_url(StringValue url, StringValue name) {
-    StringValue result("", 0);
+StringRef UrlParser::extract_url(StringRef url, StringRef name) {
+    StringRef result("", 0);
     // Remove leading and trailing spaces.
-    StringValue trimmed_url = url.trim();
+    StringRef trimmed_url = url.trim();
     // find '?'
     int32_t question_pos = _s_question_search.search(&trimmed_url);
     if (question_pos < 0) {
@@ -360,9 +360,9 @@ StringValue UrlParser::extract_url(StringValue url, StringValue name) {
 
     // find '#'
     int32_t hash_pos = _s_hash_search.search(&trimmed_url);
-    StringValue sub_url;
+    StringRef sub_url;
     if (hash_pos < 0) {
-        sub_url = trimmed_url.substring(question_pos + 1, trimmed_url.len - question_pos - 1);
+        sub_url = trimmed_url.substring(question_pos + 1, trimmed_url.size - question_pos - 1);
     } else {
         sub_url = trimmed_url.substring(question_pos + 1, hash_pos - question_pos - 1);
     }
@@ -370,8 +370,8 @@ StringValue UrlParser::extract_url(StringValue url, StringValue name) {
     // find '&' and '=', and extract target parameter
     // Example: k1=aa&k2=bb&k3=cc&test=dd
     int64_t and_pod;
-    auto len = sub_url.len;
-    StringValue key_url;
+    auto len = sub_url.size;
+    StringRef key_url;
     while (true) {
         if (len <= 0) {
             break;
@@ -385,14 +385,14 @@ StringValue UrlParser::extract_url(StringValue url, StringValue name) {
             key_url = end_pos == -1 ? sub_url : sub_url.substring(0, end_pos);
             sub_url = result;
         }
-        len = sub_url.len;
+        len = sub_url.size;
 
         auto eq_pod = key_url.find_first_of('=');
         if (eq_pod == -1) {
             // invalid url. like: k1&k2=bb
             continue;
         }
-        int32_t key_len = key_url.len;
+        int32_t key_len = key_url.size;
         auto key = key_url.substring(0, eq_pod);
         if (name == key) {
             return key_url.substring(eq_pod + 1, key_len - eq_pod - 1);
diff --git a/be/src/util/url_parser.h b/be/src/util/url_parser.h
index e2a7ca6872..882ae251ff 100644
--- a/be/src/util/url_parser.h
+++ b/be/src/util/url_parser.h
@@ -18,7 +18,7 @@
 #pragma once
 
 #include "runtime/string_search.hpp"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -47,42 +47,42 @@ public:
     // Tries to parse the part from url. Places the result in result.
     // Returns false if the URL is malformed or if part is invalid. True otherwise.
     // If false is returned the contents of results are undefined.
-    static bool parse_url(const StringValue& url, UrlPart part, StringValue* result);
+    static bool parse_url(const StringRef& url, UrlPart part, StringRef* result);
 
     // Tries to parse key from url. Places the result in result.
     // Returns false if the URL is malformed or if part is invalid. True otherwise.
     // If false is returned the contents of results are undefined.
-    static bool parse_url_key(const StringValue& url, UrlPart part, const StringValue& key,
-                              StringValue* result);
+    static bool parse_url_key(const StringRef& url, UrlPart part, const StringRef& key,
+                              StringRef* result);
 
     // Compares part against url_authority, url_file, url_host, etc.,
     // and returns the corresponding enum.
     // If part did not match any of the url part constants, returns INVALID.
-    static UrlPart get_url_part(const StringValue& part);
+    static UrlPart get_url_part(const StringRef& part);
 
     // Extract parameter value from url
     // Example for url:
     // http://doris.apache.org?k1=aa&k2=bb&k3=cc&test=dd#999
-    static StringValue extract_url(StringValue url, StringValue name);
+    static StringRef extract_url(StringRef url, StringRef name);
 
 private:
     // Constants representing parts of a URL.
-    static const StringValue _s_url_authority;
-    static const StringValue _s_url_file;
-    static const StringValue _s_url_host;
-    static const StringValue _s_url_path;
-    static const StringValue _s_url_protocol;
-    static const StringValue _s_url_query;
-    static const StringValue _s_url_ref;
-    static const StringValue _s_url_userinfo;
-    static const StringValue _s_url_port;
+    static const StringRef _s_url_authority;
+    static const StringRef _s_url_file;
+    static const StringRef _s_url_host;
+    static const StringRef _s_url_path;
+    static const StringRef _s_url_protocol;
+    static const StringRef _s_url_query;
+    static const StringRef _s_url_ref;
+    static const StringRef _s_url_userinfo;
+    static const StringRef _s_url_port;
     // Constants used in searching for URL parts.
-    static const StringValue _s_protocol;
-    static const StringValue _s_at;
-    static const StringValue _s_slash;
-    static const StringValue _s_colon;
-    static const StringValue _s_question;
-    static const StringValue _s_hash;
+    static const StringRef _s_protocol;
+    static const StringRef _s_at;
+    static const StringRef _s_slash;
+    static const StringRef _s_colon;
+    static const StringRef _s_question;
+    static const StringRef _s_hash;
     static const StringSearch _s_protocol_search;
     static const StringSearch _s_at_search;
     static const StringSearch _s_slash_search;
diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt
index 4c42d341c9..b2c6e36c5c 100644
--- a/be/src/vec/CMakeLists.txt
+++ b/be/src/vec/CMakeLists.txt
@@ -59,6 +59,7 @@ set(VEC_FILES
   common/exception.cpp
   common/mremap.cpp
   common/pod_array.cpp
+  common/string_ref.cpp
   common/sort/heap_sorter.cpp
   common/sort/sorter.cpp
   common/sort/topn_sorter.cpp
diff --git a/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.cpp b/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.cpp
index 4194befae2..59aeec80f4 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.cpp
@@ -34,7 +34,7 @@ AggregateFunctionPtr create_aggregate_function_orthogonal(const std::string& nam
         LOG(WARNING) << "Incorrect number of arguments for aggregate function " << name;
         return nullptr;
     } else if (argument_types.size() == 1) {
-        return std::make_shared<AggFunctionOrthBitmapFunc<Impl<StringValue>>>(argument_types);
+        return std::make_shared<AggFunctionOrthBitmapFunc<Impl<StringRef>>>(argument_types);
     } else {
         const IDataType& argument_type = *argument_types[1];
         AggregateFunctionPtr res(create_with_numeric_type<AggFunctionOrthBitmapFunc, Impl>(
diff --git a/be/src/vec/columns/column_dictionary.h b/be/src/vec/columns/column_dictionary.h
index 95cd848763..354fb78e2e 100644
--- a/be/src/vec/columns/column_dictionary.h
+++ b/be/src/vec/columns/column_dictionary.h
@@ -21,10 +21,10 @@
 
 #include <algorithm>
 
-#include "runtime/string_value.h"
 #include "vec/columns/column.h"
 #include "vec/columns/column_string.h"
 #include "vec/columns/predicate_column.h"
+#include "vec/common/string_ref.h"
 #include "vec/core/types.h"
 
 namespace doris::vectorized {
@@ -57,7 +57,7 @@ public:
     using Self = ColumnDictionary;
     using value_type = T;
     using Container = PaddedPODArray<value_type>;
-    using DictContainer = PaddedPODArray<StringValue>;
+    using DictContainer = PaddedPODArray<StringRef>;
     using HashValueContainer = PaddedPODArray<uint32_t>; // used for bloom filter
 
     bool is_column_dictionary() const override { return true; }
@@ -193,9 +193,9 @@ public:
         size_t length = 0;
         for (size_t i = 0; i != sel_size; ++i) {
             auto& value = _dict.get_value(_codes[sel[i]]);
-            strings[i].data = value.ptr;
-            strings[i].size = value.len;
-            length += value.len;
+            strings[i].data = value.data;
+            strings[i].size = value.size;
+            length += value.size;
         }
         res_col->get_offsets().reserve(sel_size + res_col->get_offsets().size());
         res_col->get_chars().reserve(length + res_col->get_chars().size());
@@ -217,7 +217,7 @@ public:
     void insert_many_dict_data(const StringRef* dict_array, uint32_t dict_num) {
         _dict.reserve(_dict.size() + dict_num);
         for (uint32_t i = 0; i < dict_num; ++i) {
-            auto value = StringValue(dict_array[i].data, dict_array[i].size);
+            auto value = StringRef(dict_array[i].data, dict_array[i].size);
             _dict.insert_value(value);
         }
     }
@@ -228,7 +228,7 @@ public:
         if (_dict.empty()) {
             _dict.reserve(dict_num);
             for (uint32_t i = 0; i < dict_num; ++i) {
-                auto value = StringValue(dict_array[i].data, dict_array[i].size);
+                auto value = StringRef(dict_array[i].data, dict_array[i].size);
                 _dict.insert_value(value);
             }
         }
@@ -259,9 +259,9 @@ public:
         }
     }
 
-    int32_t find_code(const StringValue& value) const { return _dict.find_code(value); }
+    int32_t find_code(const StringRef& value) const { return _dict.find_code(value); }
 
-    int32_t find_code_by_bound(const StringValue& value, bool greater, bool eq) const {
+    int32_t find_code_by_bound(const StringRef& value, bool greater, bool eq) const {
         return _dict.find_code_by_bound(value, greater, eq);
     }
 
@@ -271,7 +271,7 @@ public:
 
     uint32_t get_hash_value(uint32_t idx) const { return _dict.get_hash_value(_codes[idx]); }
 
-    void find_codes(const phmap::flat_hash_set<StringValue>& values,
+    void find_codes(const phmap::flat_hash_set<StringRef>& values,
                     std::vector<vectorized::UInt8>& selected) const {
         return _dict.find_codes(values, selected);
     }
@@ -297,19 +297,19 @@ public:
         for (size_t i = 0; i < _codes.size(); ++i) {
             auto& code = reinterpret_cast<T&>(_codes[i]);
             auto value = _dict.get_value(code);
-            res->insert_data(value.ptr, value.len);
+            res->insert_data(value.data, value.size);
         }
         clear();
         _dict.clear();
         return res;
     }
 
-    inline const StringValue& get_value(value_type code) const { return _dict.get_value(code); }
+    inline const StringRef& get_value(value_type code) const { return _dict.get_value(code); }
 
-    inline StringValue get_shrink_value(value_type code) const {
-        StringValue result = _dict.get_value(code);
+    inline StringRef get_shrink_value(value_type code) const {
+        StringRef result = _dict.get_value(code);
         if (_type == OLAP_FIELD_TYPE_CHAR) {
-            result.len = strnlen(result.ptr, result.len);
+            result.size = strnlen(result.data, result.size);
         }
         return result;
     }
@@ -324,12 +324,12 @@ public:
 
         void reserve(size_t n) { _dict_data->reserve(n); }
 
-        void insert_value(StringValue& value) {
+        void insert_value(StringRef& value) {
             _dict_data->push_back_without_reserve(value);
-            _total_str_len += value.len;
+            _total_str_len += value.size;
         }
 
-        int32_t find_code(const StringValue& value) const {
+        int32_t find_code(const StringRef& value) const {
             for (size_t i = 0; i < _dict_data->size(); i++) {
                 if ((*_dict_data)[i] == value) {
                     return i;
@@ -340,9 +340,9 @@ public:
 
         T get_null_code() const { return -1; }
 
-        inline StringValue& get_value(T code) { return (*_dict_data)[code]; }
+        inline StringRef& get_value(T code) { return (*_dict_data)[code]; }
 
-        inline const StringValue& get_value(T code) const { return (*_dict_data)[code]; }
+        inline const StringRef& get_value(T code) const { return (*_dict_data)[code]; }
 
         // The function is only used in the runtime filter feature
         inline void generate_hash_values_for_runtime_filter(FieldType type) {
@@ -357,13 +357,13 @@ public:
                     // Remove the suffix 0
                     // When writing data, use the CharField::consume function to fill in the trailing 0.
 
-                    // For dictionary data of char type, sv.len is the schema length,
+                    // For dictionary data of char type, sv.size is the schema length,
                     // so use strnlen to remove the 0 at the end to get the actual length.
-                    int32_t len = sv.len;
+                    int32_t len = sv.size;
                     if (type == OLAP_FIELD_TYPE_CHAR) {
-                        len = strnlen(sv.ptr, sv.len);
+                        len = strnlen(sv.data, sv.size);
                     }
-                    uint32_t hash_val = HashUtil::murmur_hash3_32(sv.ptr, len, 0);
+                    uint32_t hash_val = HashUtil::murmur_hash3_32(sv.data, len, 0);
                     _hash_values[i] = hash_val;
                 }
             }
@@ -388,7 +388,7 @@ public:
         //  so upper_bound is the code 0 of b, then evaluate code < 0 and returns empty
         // If the predicate is col <= 'a' and upper_bound-1 is -1,
         //  then evaluate code <= -1 and returns empty
-        int32_t find_code_by_bound(const StringValue& value, bool greater, bool eq) const {
+        int32_t find_code_by_bound(const StringRef& value, bool greater, bool eq) const {
             auto code = find_code(value);
             if (code >= 0) {
                 return code;
@@ -398,7 +398,7 @@ public:
             return greater ? bound - greater + eq : bound - eq;
         }
 
-        void find_codes(const phmap::flat_hash_set<StringValue>& values,
+        void find_codes(const phmap::flat_hash_set<StringRef>& values,
                         std::vector<vectorized::UInt8>& selected) const {
             size_t dict_word_num = _dict_data->size();
             selected.resize(dict_word_num);
@@ -476,8 +476,8 @@ public:
         }
 
     private:
-        StringValue _null_value = StringValue();
-        StringValue::Comparator _comparator;
+        StringRef _null_value = StringRef();
+        StringRef::Comparator _comparator;
         // dict code -> dict value
         std::unique_ptr<DictContainer> _dict_data;
         std::vector<T> _code_convert_table;
diff --git a/be/src/vec/columns/predicate_column.h b/be/src/vec/columns/predicate_column.h
index f307d12473..e2a464b5ea 100644
--- a/be/src/vec/columns/predicate_column.h
+++ b/be/src/vec/columns/predicate_column.h
@@ -21,11 +21,11 @@
 #include "olap/uint24.h"
 #include "runtime/mem_pool.h"
 #include "runtime/primitive_type.h"
-#include "runtime/string_value.h"
 #include "vec/columns/column.h"
 #include "vec/columns/column_decimal.h"
 #include "vec/columns/column_string.h"
 #include "vec/columns/column_vector.h"
+#include "vec/common/string_ref.h"
 #include "vec/core/types.h"
 
 namespace doris::vectorized {
@@ -95,10 +95,10 @@ private:
         size_t length = 0;
         for (size_t i = 0; i < sel_size; i++) {
             uint16_t n = sel[i];
-            auto& sv = reinterpret_cast<StringValue&>(data[n]);
-            refs[i].data = sv.ptr;
-            refs[i].size = sv.len;
-            length += sv.len;
+            auto& sv = reinterpret_cast<StringRef&>(data[n]);
+            refs[i].data = sv.data;
+            refs[i].size = sv.size;
+            length += sv.size;
         }
         res_ptr->get_offsets().reserve(sel_size + res_ptr->get_offsets().size());
         res_ptr->get_chars().reserve(length + res_ptr->get_chars().size());
@@ -187,7 +187,7 @@ public:
     }
 
     void insert_string_value(const char* data_ptr, size_t length) {
-        StringValue sv((char*)data_ptr, length);
+        StringRef sv((char*)data_ptr, length);
         data.push_back_without_reserve(sv);
     }
 
@@ -211,7 +211,7 @@ public:
     }
 
     void insert_data(const char* data_ptr, size_t length) override {
-        if constexpr (std::is_same_v<T, StringValue>) {
+        if constexpr (std::is_same_v<T, StringRef>) {
             insert_string_value(data_ptr, length);
         } else if constexpr (std::is_same_v<T, decimal12_t>) {
             insert_decimal_value(data_ptr, length);
@@ -242,7 +242,7 @@ public:
             insert_many_in_copy_way(data_ptr, num);
         } else if constexpr (std::is_same_v<T, doris::vectorized::Int128>) {
             insert_many_in_copy_way(data_ptr, num);
-        } else if constexpr (std::is_same_v<T, StringValue>) {
+        } else if constexpr (std::is_same_v<T, StringRef>) {
             // here is unreachable, just for compilation to be able to pass
         } else if constexpr (Type == TYPE_DATE) {
             insert_many_date(data_ptr, num);
@@ -253,7 +253,7 @@ public:
 
     void insert_many_dict_data(const int32_t* data_array, size_t start_index, const StringRef* dict,
                                size_t num, uint32_t /*dict_num*/) override {
-        if constexpr (std::is_same_v<T, StringValue>) {
+        if constexpr (std::is_same_v<T, StringRef>) {
             for (size_t end_index = start_index + num; start_index < end_index; ++start_index) {
                 int32_t codeword = data_array[start_index];
                 insert_string_value(dict[codeword].data, dict[codeword].size);
@@ -266,7 +266,7 @@ public:
         if (UNLIKELY(num == 0)) {
             return;
         }
-        if constexpr (std::is_same_v<T, StringValue>) {
+        if constexpr (std::is_same_v<T, StringRef>) {
             if (_pool == nullptr) {
                 _pool.reset(new MemPool());
             }
@@ -278,10 +278,10 @@ public:
 
             auto* data_ptr = &data[org_elem_num];
             for (size_t i = 0; i != num; ++i) {
-                data_ptr[i].ptr = destination + offsets[i] - offsets[0];
-                data_ptr[i].len = offsets[i + 1] - offsets[i];
+                data_ptr[i].data = destination + offsets[i] - offsets[0];
+                data_ptr[i].size = offsets[i + 1] - offsets[i];
             }
-            DCHECK(data_ptr[num - 1].ptr + data_ptr[num - 1].len == destination + total_mem_size);
+            DCHECK(data_ptr[num - 1].data + data_ptr[num - 1].size == destination + total_mem_size);
         }
     }
 
@@ -290,7 +290,7 @@ public:
         if (num == 0) {
             return;
         }
-        if constexpr (std::is_same_v<T, StringValue>) {
+        if constexpr (std::is_same_v<T, StringRef>) {
             if (_pool == nullptr) {
                 _pool.reset(new MemPool());
             }
@@ -307,8 +307,8 @@ public:
             uint32_t fragment_start_offset = start_offset_array[0];
             size_t fragment_len = 0;
             for (size_t i = 0; i < num; i++) {
-                data[org_elem_num + i].ptr = destination + fragment_len;
-                data[org_elem_num + i].len = len_array[i];
+                data[org_elem_num + i].data = destination + fragment_len;
+                data[org_elem_num + i].size = len_array[i];
                 fragment_len += len_array[i];
                 // Compute the largest continuous memcpy block and copy them.
                 // If this is the last element in data array, then should copy the current memory block.
@@ -447,7 +447,7 @@ public:
     }
 
     Status filter_by_selector(const uint16_t* sel, size_t sel_size, IColumn* col_ptr) override {
-        if constexpr (std::is_same_v<T, StringValue>) {
+        if constexpr (std::is_same_v<T, StringRef>) {
             insert_string_to_res_column(sel, sel_size,
                                         reinterpret_cast<vectorized::ColumnString*>(col_ptr));
         } else if constexpr (std::is_same_v<T, decimal12_t>) {
diff --git a/be/src/vec/common/hash_table/hash.h b/be/src/vec/common/hash_table/hash.h
index 6d63a16735..28288a4813 100644
--- a/be/src/vec/common/hash_table/hash.h
+++ b/be/src/vec/common/hash_table/hash.h
@@ -23,6 +23,7 @@
 #include <type_traits>
 
 #include "parallel_hashmap/phmap_utils.h"
+#include "vec/common/string_ref.h"
 #include "vec/common/uint128.h"
 #include "vec/core/types.h"
 
@@ -100,7 +101,7 @@ struct DefaultHash<doris::vectorized::Int128I> {
 };
 
 template <>
-struct DefaultHash<StringRef> : public StringRefHash {};
+struct DefaultHash<doris::StringRef> : public doris::StringRefHash {};
 
 template <typename T>
 struct HashCRC32;
diff --git a/be/src/vec/common/hash_table/hash_table.h b/be/src/vec/common/hash_table/hash_table.h
index 337d3ed8f9..41e7a4b446 100644
--- a/be/src/vec/common/hash_table/hash_table.h
+++ b/be/src/vec/common/hash_table/hash_table.h
@@ -163,7 +163,7 @@ struct HashTableCell {
 
     Key key;
 
-    HashTableCell() {}
+    HashTableCell() = default;
 
     /// Create a cell with the given key / key and value.
     HashTableCell(const Key& key_, const State&) : key(key_) {}
diff --git a/be/src/vec/common/hash_table/hash_table_key_holder.h b/be/src/vec/common/hash_table/hash_table_key_holder.h
index eb507d16a5..a422ad7eef 100644
--- a/be/src/vec/common/hash_table/hash_table_key_holder.h
+++ b/be/src/vec/common/hash_table/hash_table_key_holder.h
@@ -99,7 +99,8 @@ struct ArenaKeyHolder {
 
 } // namespace doris::vectorized
 
-inline StringRef& ALWAYS_INLINE key_holder_get_key(doris::vectorized::ArenaKeyHolder& holder) {
+inline doris::StringRef& ALWAYS_INLINE
+key_holder_get_key(doris::vectorized::ArenaKeyHolder& holder) {
     return holder.key;
 }
 
@@ -125,7 +126,8 @@ struct SerializedKeyHolder {
 
 } // namespace doris::vectorized
 
-inline StringRef& ALWAYS_INLINE key_holder_get_key(doris::vectorized::SerializedKeyHolder& holder) {
+inline doris::StringRef& ALWAYS_INLINE
+key_holder_get_key(doris::vectorized::SerializedKeyHolder& holder) {
     return holder.key;
 }
 
diff --git a/be/src/vec/common/hash_table/string_hash_map.h b/be/src/vec/common/hash_table/string_hash_map.h
index 344f468dec..99624cab42 100644
--- a/be/src/vec/common/hash_table/string_hash_map.h
+++ b/be/src/vec/common/hash_table/string_hash_map.h
@@ -30,7 +30,7 @@ struct StringHashMapCell : public HashMapCell<Key, TMapped, StringHashTableHash,
     using Base::Base;
     static constexpr bool need_zero_value_storage = false;
     // external
-    const StringRef get_key() const { return to_string_ref(this->value.first); } /// NOLINT
+    const doris::StringRef get_key() const { return to_string_ref(this->value.first); } /// NOLINT
     // internal
     static const Key& get_key(const value_type& value_) { return value_.first; }
 };
@@ -50,7 +50,7 @@ struct StringHashMapCell<StringKey16, TMapped>
     void set_zero() { this->value.first.high = 0; }
 
     // external
-    const StringRef get_key() const { return to_string_ref(this->value.first); } /// NOLINT
+    const doris::StringRef get_key() const { return to_string_ref(this->value.first); } /// NOLINT
     // internal
     static const StringKey16& get_key(const value_type& value_) { return value_.first; }
 };
@@ -70,24 +70,24 @@ struct StringHashMapCell<StringKey24, TMapped>
     void set_zero() { this->value.first.c = 0; }
 
     // external
-    const StringRef get_key() const { return to_string_ref(this->value.first); } /// NOLINT
+    const doris::StringRef get_key() const { return to_string_ref(this->value.first); } /// NOLINT
     // internal
     static const StringKey24& get_key(const value_type& value_) { return value_.first; }
 };
 
 template <typename TMapped>
-struct StringHashMapCell<StringRef, TMapped>
-        : public HashMapCellWithSavedHash<StringRef, TMapped, StringHashTableHash,
+struct StringHashMapCell<doris::StringRef, TMapped>
+        : public HashMapCellWithSavedHash<doris::StringRef, TMapped, StringHashTableHash,
                                           HashTableNoState> {
-    using Base =
-            HashMapCellWithSavedHash<StringRef, TMapped, StringHashTableHash, HashTableNoState>;
+    using Base = HashMapCellWithSavedHash<doris::StringRef, TMapped, StringHashTableHash,
+                                          HashTableNoState>;
     using value_type = typename Base::value_type;
     using Base::Base;
     static constexpr bool need_zero_value_storage = false;
     // external
     using Base::get_key;
     // internal
-    static const StringRef& get_key(const value_type& value_) { return value_.first; }
+    static const doris::StringRef& get_key(const value_type& value_) { return value_.first; }
 
     template <typename Key>
     StringHashMapCell(const StringHashMapCell<Key, TMapped>& other) {
@@ -107,21 +107,21 @@ struct StringHashMapCell<StringRef, TMapped>
 
 template <typename TMapped, typename Allocator>
 struct StringHashMapSubMaps {
-    using T0 = StringHashTableEmpty<StringHashMapCell<StringRef, TMapped>>;
+    using T0 = StringHashTableEmpty<StringHashMapCell<doris::StringRef, TMapped>>;
     using T1 = HashMapTable<StringKey8, StringHashMapCell<StringKey8, TMapped>, StringHashTableHash,
                             StringHashTableGrower<>, Allocator>;
     using T2 = HashMapTable<StringKey16, StringHashMapCell<StringKey16, TMapped>,
                             StringHashTableHash, StringHashTableGrower<>, Allocator>;
     using T3 = HashMapTable<StringKey24, StringHashMapCell<StringKey24, TMapped>,
                             StringHashTableHash, StringHashTableGrower<>, Allocator>;
-    using Ts = HashMapTable<StringRef, StringHashMapCell<StringRef, TMapped>, StringHashTableHash,
-                            StringHashTableGrower<>, Allocator>;
+    using Ts = HashMapTable<doris::StringRef, StringHashMapCell<doris::StringRef, TMapped>,
+                            StringHashTableHash, StringHashTableGrower<>, Allocator>;
 };
 
 template <typename TMapped, typename Allocator = HashTableAllocator>
 class StringHashMap : public StringHashTable<StringHashMapSubMaps<TMapped, Allocator>> {
 public:
-    using Key = StringRef;
+    using Key = doris::StringRef;
     using Base = StringHashTable<StringHashMapSubMaps<TMapped, Allocator>>;
     using Self = StringHashMap;
     using LookupResult = typename Base::LookupResult;
@@ -177,7 +177,7 @@ public:
     template <typename Func>
     void ALWAYS_INLINE for_each_value(Func&& func) {
         if (this->m0.size()) {
-            func(StringRef {}, this->m0.zero_value()->get_second());
+            func(doris::StringRef {}, this->m0.zero_value()->get_second());
         }
 
         for (auto& v : this->m1) {
diff --git a/be/src/vec/common/hash_table/string_hash_table.h b/be/src/vec/common/hash_table/string_hash_table.h
index ad164b1b8f..00d76c4a24 100644
--- a/be/src/vec/common/hash_table/string_hash_table.h
+++ b/be/src/vec/common/hash_table/string_hash_table.h
@@ -35,15 +35,15 @@ struct StringKey24 {
     bool operator==(const StringKey24 rhs) const { return a == rhs.a && b == rhs.b && c == rhs.c; }
 };
 
-inline StringRef ALWAYS_INLINE to_string_ref(const StringKey8& n) {
+inline doris::StringRef ALWAYS_INLINE to_string_ref(const StringKey8& n) {
     assert(n != 0);
     return {reinterpret_cast<const char*>(&n), 8ul - (__builtin_clzll(n) >> 3)};
 }
-inline StringRef ALWAYS_INLINE to_string_ref(const StringKey16& n) {
+inline doris::StringRef ALWAYS_INLINE to_string_ref(const StringKey16& n) {
     assert(n.high != 0);
     return {reinterpret_cast<const char*>(&n), 16ul - (__builtin_clzll(n.high) >> 3)};
 }
-inline StringRef ALWAYS_INLINE to_string_ref(const StringKey24& n) {
+inline doris::StringRef ALWAYS_INLINE to_string_ref(const StringKey24& n) {
     assert(n.c != 0);
     return {reinterpret_cast<const char*>(&n), 24ul - (__builtin_clzll(n.c) >> 3)};
 }
@@ -79,7 +79,9 @@ struct StringHashTableHash {
         return util_hash::CityHash64(reinterpret_cast<const char*>(&key), 24);
     }
 #endif
-    size_t ALWAYS_INLINE operator()(StringRef key) const { return StringRefHash()(key); }
+    size_t ALWAYS_INLINE operator()(doris::StringRef key) const {
+        return doris::StringRefHash()(key);
+    }
 };
 
 template <typename Cell>
@@ -107,7 +109,9 @@ public:
 
     void clear_has_zero() {
         _has_zero = false;
-        if (!std::is_trivially_destructible_v<Cell>) zero_value()->~Cell();
+        if (!std::is_trivially_destructible_v<Cell>) {
+            zero_value()->~Cell();
+        }
     }
 
     Cell* zero_value() { return std::launder(reinterpret_cast<Cell*>(&zero_value_storage)); }
@@ -125,8 +129,9 @@ public:
             const auto& key = key_holder_get_key(key_holder);
             set_has_zero(key);
             inserted = true;
-        } else
+        } else {
             inserted = false;
+        }
         it = zero_value();
     }
 
@@ -219,7 +224,7 @@ protected:
     using T2 = typename SubMaps::T2;
     using T3 = typename SubMaps::T3;
 
-    // Long strings are stored as StringRef along with saved hash
+    // Long strings are stored as doris::StringRef along with saved hash
     using Ts = typename SubMaps::Ts;
     using Self = StringHashTable;
 
@@ -250,7 +255,7 @@ protected:
         friend class StringHashTable;
 
     public:
-        iterator_base() {}
+        iterator_base() = default;
         iterator_base(Container* container_, bool end = false) : container(container_) {
             if (end) {
                 sub_table_index = 4;
@@ -373,8 +378,6 @@ protected:
                 }
                 }
             }
-            while (need_switch_to_next)
-                ;
 
             return static_cast<Derived&>(*this);
         }
@@ -449,7 +452,7 @@ protected:
     };
 
 public:
-    using Key = StringRef;
+    using Key = doris::StringRef;
     using key_type = Key;
     using mapped_type = typename Ts::mapped_type;
     using value_type = typename Ts::value_type;
@@ -487,7 +490,7 @@ public:
     template <typename Self, typename KeyHolder, typename Func>
     static auto ALWAYS_INLINE dispatch(Self& self, KeyHolder&& key_holder, Func&& func) {
         StringHashTableHash hash;
-        const StringRef& x = key_holder_get_key(key_holder);
+        const doris::StringRef& x = key_holder_get_key(key_holder);
         const size_t sz = x.size;
         if (sz == 0) {
             key_holder_discard_key(key_holder);
diff --git a/be/src/vec/common/string_ref.cpp b/be/src/vec/common/string_ref.cpp
new file mode 100644
index 0000000000..20cfbad708
--- /dev/null
+++ b/be/src/vec/common/string_ref.cpp
@@ -0,0 +1,81 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+// This file is copied from
+// https://github.com/ClickHouse/ClickHouse/blob/master/base/base/StringRef.
+// And modified by Doris
+
+#include "string_ref.h"
+
+namespace doris {
+
+StringRef StringRef::trim() const {
+    // Remove leading and trailing spaces.
+    int32_t begin = 0;
+
+    while (begin < size && data[begin] == ' ') {
+        ++begin;
+    }
+
+    int32_t end = size - 1;
+
+    while (end > begin && data[end] == ' ') {
+        --end;
+    }
+
+    return StringRef(data + begin, end - begin + 1);
+}
+
+// TODO: rewrite in AVX2
+size_t StringRef::find_first_of(char c) const {
+    const char* p = static_cast<const char*>(memchr(data, c, size));
+    return p == nullptr ? -1 : p - data;
+}
+
+StringRef StringRef::min_string_val() {
+    return StringRef((char*)(&StringRef::MIN_CHAR), 0);
+}
+
+StringRef StringRef::max_string_val() {
+    return StringRef((char*)(&StringRef::MAX_CHAR), 1);
+}
+
+bool StringRef::start_with(const StringRef& search_string) const {
+    DCHECK(size >= search_string.size);
+    if (search_string.size == 0) {
+        return true;
+    }
+
+#if defined(__SSE2__) || defined(__aarch64__)
+    return memequalSSE2Wide(data, search_string.data, search_string.size);
+#else
+    return 0 == memcmp(data, search_string.data, search_string.size);
+#endif
+}
+bool StringRef::end_with(const StringRef& search_string) const {
+    DCHECK(size >= search_string.size);
+    if (search_string.size == 0) {
+        return true;
+    }
+
+#if defined(__SSE2__) || defined(__aarch64__)
+    return memequalSSE2Wide(data + size - search_string.size, search_string.data,
+                            search_string.size);
+#else
+    return 0 == memcmp(data + size - search_string.size, search_string.data, search_string.size);
+#endif
+}
+} // namespace doris
diff --git a/be/src/vec/common/string_ref.h b/be/src/vec/common/string_ref.h
index c146b48187..3eef585a60 100644
--- a/be/src/vec/common/string_ref.h
+++ b/be/src/vec/common/string_ref.h
@@ -29,7 +29,10 @@
 #include "gutil/hash/city.h"
 #include "gutil/hash/hash128to64.h"
 #include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
 #include "util/slice.h"
+#include "vec/common/string_ref.h"
 #include "vec/common/unaligned.h"
 #include "vec/core/types.h"
 
@@ -40,12 +43,19 @@
 #if defined(__SSE4_2__)
 #include <nmmintrin.h>
 #include <smmintrin.h>
+
+#include "util/sse_util.hpp"
 #endif
 
 #if defined(__aarch64__)
 #include <sse2neon.h>
 #endif
 
+namespace doris {
+
+/// unnamed namespace packaging simd-style equality compare functions.
+namespace {
+
 #if defined(__SSE2__) || defined(__aarch64__)
 
 /** Compare strings for equality.
@@ -112,19 +122,26 @@ inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
             p1 += 64;
             p2 += 64;
             size -= 64;
-        } else
+        } else {
             return false;
+        }
     }
 
     switch (size / 16) {
     case 3:
-        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        if (!compareSSE2(p1 + 32, p2 + 32)) {
+            return false;
+        }
         [[fallthrough]];
     case 2:
-        if (!compareSSE2(p1 + 16, p2 + 16)) return false;
+        if (!compareSSE2(p1 + 16, p2 + 16)) {
+            return false;
+        }
         [[fallthrough]];
     case 1:
-        if (!compareSSE2(p1, p2)) return false;
+        if (!compareSSE2(p1, p2)) {
+            return false;
+        }
     }
 
     return compareSSE2(p1 + size - 16, p2 + size - 16);
@@ -132,24 +149,85 @@ inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
 
 #endif
 
+// Compare two strings using sse4.2 intrinsics if they are available. This code assumes
+// that the trivial cases are already handled (i.e. one string is empty).
+// Returns:
+//   < 0 if s1 < s2
+//   0 if s1 == s2
+//   > 0 if s1 > s2
+// The SSE code path is just under 2x faster than the non-sse code path.
+//   - s1/n1: ptr/len for the first string
+//   - s2/n2: ptr/len for the second string
+//   - len: min(n1, n2) - this can be more cheaply passed in by the caller
+inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2, int64_t len) {
+    DCHECK_EQ(len, std::min(n1, n2));
+#ifdef __SSE4_2__
+    while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) {
+        __m128i xmm0 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s1));
+        __m128i xmm1 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s2));
+        int chars_match = _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1,
+                                       sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE);
+        if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) {
+            return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match];
+        }
+        len -= sse_util::CHARS_PER_128_BIT_REGISTER;
+        s1 += sse_util::CHARS_PER_128_BIT_REGISTER;
+        s2 += sse_util::CHARS_PER_128_BIT_REGISTER;
+    }
+#endif
+    unsigned char u1, u2;
+    while (len-- > 0) {
+        u1 = (unsigned char)*s1++;
+        u2 = (unsigned char)*s2++;
+        if (u1 != u2) {
+            return u1 - u2;
+        }
+        if (u1 == '\0') {
+            return n1 - n2;
+        }
+    }
+
+    return n1 - n2;
+}
+
+} // unnamed namespace
+
 /// The thing to avoid creating strings to find substrings in the hash table.
+/// User should make sure data source is const.
+/// maybe considering rewrite it with std::span / std::basic_string_view is meaningful.
 struct StringRef {
+    // TODO: opening member accessing really damages.
     const char* data = nullptr;
     size_t size = 0;
 
+    StringRef() = default;
     StringRef(const char* data_, size_t size_) : data(data_), size(size_) {}
     StringRef(const unsigned char* data_, size_t size_)
-            : data(reinterpret_cast<const char*>(data_)), size(size_) {}
+            : StringRef(reinterpret_cast<const char*>(data_), size_) {}
+
     StringRef(const std::string& s) : data(s.data()), size(s.size()) {}
-    StringRef() = default;
+    StringRef(const StringVal& src) : StringRef(src.ptr, src.len) {}
+    explicit StringRef(const char* str) : data(str), size(strlen(str)) {}
+
+    static StringRef from_string_val(const StringVal& src) { return StringRef(src); }
 
     std::string to_string() const { return std::string(data, size); }
+    std::string debug_string() const { return to_string(); }
     std::string_view to_string_view() const { return std::string_view(data, size); }
-    doris::Slice to_slice() const { return doris::Slice(data, size); }
+    Slice to_slice() const { return doris::Slice(data, size); }
 
-    // this is just for show, eg. print data to error log, to avoid print large string.
+    // this is just for show, e.g. print data to error log, to avoid print large string.
     std::string to_prefix(size_t length) const { return std::string(data, std::min(length, size)); }
 
+    // TODO: this function is dangerous!
+    StringVal to_string_val() const {
+        return StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+
+    void to_string_val(StringVal* sv) const {
+        *sv = StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+
     explicit operator std::string() const { return to_string(); }
     operator std::string_view() const { return std::string_view {data, size}; }
 
@@ -157,66 +235,94 @@ struct StringRef {
         return StringRef(data + start_pos, (new_len < 0) ? (size - start_pos) : new_len);
     }
 
-    StringVal to_string_val() {
-        return StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    StringRef substring(int start_pos) const { return substring(start_pos, size - start_pos); }
+
+    // Trims leading and trailing spaces.
+    StringRef trim() const;
+
+    // support for type_limit
+    static constexpr char MIN_CHAR = 0x00;
+    static constexpr char MAX_CHAR = 0xFF;
+    static StringRef min_string_val();
+    static StringRef max_string_val();
+
+    bool start_with(const StringRef& search_string) const;
+    bool end_with(const StringRef& search_string) const;
+
+    // Byte-by-byte comparison. Returns:
+    // this < other: -1
+    // this == other: 0
+    // this > other: 1
+    int compare(const StringRef& other) const {
+        int l = std::min(size, other.size);
+
+        if (l == 0) {
+            if (size == other.size) {
+                return 0;
+            } else if (size == 0) {
+                return -1;
+            } else {
+                DCHECK_EQ(other.size, 0);
+                return 1;
+            }
+        }
+
+        return string_compare(this->data, this->size, other.data, other.size, l);
     }
 
-    static StringRef from_string_val(StringVal sv) {
-        return StringRef(reinterpret_cast<char*>(sv.ptr), sv.len);
+    void replace(const char* ptr, int len) {
+        this->data = ptr;
+        this->size = len;
     }
 
-    bool start_with(StringRef& search_string) const {
-        DCHECK(size >= search_string.size);
-        if (search_string.size == 0) return true;
+    // Find the first position char of appear, return -1 if not found
+    size_t find_first_of(char c) const;
 
-#if defined(__SSE2__) || defined(__aarch64__)
-        return memequalSSE2Wide(data, search_string.data, search_string.size);
-#else
-        return 0 == memcmp(data, search_string.data, search_string.size);
+    // ==
+    bool eq(const StringRef& other) const {
+        if (this->size != other.size) {
+            return false;
+        }
+#if defined(__SSE2__)
+        return memequalSSE2Wide(this->data, other.data, this->size);
 #endif
+        return string_compare(this->data, this->size, other.data, other.size, this->size) == 0;
     }
-    bool end_with(StringRef& search_string) const {
-        DCHECK(size >= search_string.size);
-        if (search_string.size == 0) return true;
 
-#if defined(__SSE2__) || defined(__aarch64__)
-        return memequalSSE2Wide(data + size - search_string.size, search_string.data,
-                                search_string.size);
-#else
-        return 0 ==
-               memcmp(data + size - search_string.size, search_string.data, search_string.size);
-#endif
-    }
-};
+    bool operator==(const StringRef& other) const { return eq(other); }
+    // !=
+    bool ne(const StringRef& other) const { return !eq(other); }
+    // <=
+    bool le(const StringRef& other) const { return compare(other) <= 0; }
+    // >=
+    bool ge(const StringRef& other) const { return compare(other) >= 0; }
+    // <
+    bool lt(const StringRef& other) const { return compare(other) < 0; }
+    // >
+    bool gt(const StringRef& other) const { return compare(other) > 0; }
 
-using StringRefs = std::vector<StringRef>;
+    bool operator!=(const StringRef& other) const { return ne(other); }
 
-inline bool operator==(StringRef lhs, StringRef rhs) {
-    if (lhs.size != rhs.size) return false;
+    bool operator<=(const StringRef& other) const { return le(other); }
 
-    if (lhs.size == 0) return true;
+    bool operator>=(const StringRef& other) const { return ge(other); }
 
-#if defined(__SSE2__) || defined(__aarch64__)
-    return memequalSSE2Wide(lhs.data, rhs.data, lhs.size);
-#else
-    return 0 == memcmp(lhs.data, rhs.data, lhs.size);
-#endif
-}
+    bool operator<(const StringRef& other) const { return lt(other); }
 
-inline bool operator!=(StringRef lhs, StringRef rhs) {
-    return !(lhs == rhs);
-}
+    bool operator>(const StringRef& other) const { return gt(other); }
 
-inline bool operator<(StringRef lhs, StringRef rhs) {
-    int cmp = memcmp(lhs.data, rhs.data, std::min(lhs.size, rhs.size));
-    return cmp < 0 || (cmp == 0 && lhs.size < rhs.size);
-}
+    struct Comparator {
+        bool operator()(const StringRef& a, const StringRef& b) const { return a.compare(b) < 0; }
+    };
+}; // class StringRef
 
-inline bool operator>(StringRef lhs, StringRef rhs) {
-    int cmp = memcmp(lhs.data, rhs.data, std::min(lhs.size, rhs.size));
-    return cmp > 0 || (cmp == 0 && lhs.size > rhs.size);
+// This function must be called 'hash_value' to be picked up by boost.
+inline std::size_t hash_value(const StringRef& v) {
+    return HashUtil::hash(v.data, v.size, 0);
 }
 
+using StringRefs = std::vector<StringRef>;
+
 /** Hash functions.
   * You can use either CityHash64,
   *  or a function based on the crc32 statement,
@@ -278,11 +384,13 @@ inline size_t hash_less_than16(const char* data, size_t size) {
 }
 
 struct CRC32Hash {
-    size_t operator()(StringRef x) const {
+    size_t operator()(const StringRef& x) const {
         const char* pos = x.data;
         size_t size = x.size;
 
-        if (size == 0) return 0;
+        if (size == 0) {
+            return 0;
+        }
 
         if (size < 8) {
             return hash_less_than8(x.data, x.size);
@@ -318,22 +426,21 @@ struct CRC32Hash {
 
 struct StringRefHash : StringRefHash64 {};
 
-#endif
+#endif // end of hash functions
 
-template <>
-struct std::hash<StringRef> : public StringRefHash {};
+inline std::ostream& operator<<(std::ostream& os, const StringRef& str) {
+    return os << str.to_string();
+}
+} // namespace doris
 
 namespace ZeroTraits {
-inline bool check(const StringRef& x) {
+inline bool check(const doris::StringRef& x) {
     return 0 == x.size;
 }
-inline void set(StringRef& x) {
+inline void set(doris::StringRef& x) {
     x.size = 0;
 }
 } // namespace ZeroTraits
 
-inline std::ostream& operator<<(std::ostream& os, const StringRef& str) {
-    if (str.data) os.write(str.data, str.size);
-
-    return os;
-}
+template <>
+struct std::hash<doris::StringRef> : public doris::StringRefHash {};
diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp
index a23ecb2a75..4202f76304 100644
--- a/be/src/vec/core/block.cpp
+++ b/be/src/vec/core/block.cpp
@@ -855,7 +855,7 @@ void Block::deep_copy_slot(void* dst, MemPool* pool, const doris::TypeDescriptor
                 auto item_offset = offset + i;
                 const auto& data_ref = item_type_desc.type != TYPE_ARRAY
                                                ? item_column->get_data_at(item_offset)
-                                               : StringRef();
+                                               : StringRef {};
                 if (item_type_desc.is_date_type()) {
                     // In CollectionValue, date type data is stored as either uint24_t or uint64_t.
                     DateTimeValue datetime_value;
@@ -890,33 +890,37 @@ void Block::deep_copy_slot(void* dst, MemPool* pool, const doris::TypeDescriptor
         auto size = bitmap_value->getSizeInBytes();
 
         // serialize the content of string
-        auto string_slot = reinterpret_cast<StringValue*>(dst);
-        string_slot->ptr = reinterpret_cast<char*>(pool->allocate(size));
-        bitmap_value->write(string_slot->ptr);
-        string_slot->len = size;
+        // TODO: NEED TO REWRITE COMPLETELY. the way writing now is WRONG.
+        // StringRef shouldn't managing exclusive memory cause it will break RAII.
+        // besides, accessing object which is essentially const by non-const object
+        // is UB!
+        auto string_slot = reinterpret_cast<StringRef*>(dst);
+        string_slot->data = reinterpret_cast<char*>(pool->allocate(size));
+        bitmap_value->write(const_cast<char*>(string_slot->data)); //!
+        string_slot->size = size;
     } else if (type_desc.type == TYPE_HLL) {
         auto hll_value = (HyperLogLog*)(data_ref.data);
         auto size = hll_value->max_serialized_size();
-        auto string_slot = reinterpret_cast<StringValue*>(dst);
-        string_slot->ptr = reinterpret_cast<char*>(pool->allocate(size));
-        size_t actual_size = hll_value->serialize((uint8_t*)string_slot->ptr);
-        string_slot->len = actual_size;
+        auto string_slot = reinterpret_cast<StringRef*>(dst);
+        string_slot->data = reinterpret_cast<char*>(pool->allocate(size));
+        size_t actual_size = hll_value->serialize((uint8_t*)string_slot->data);
+        string_slot->size = actual_size;
     } else if (type_desc.is_string_type()) { // TYPE_OBJECT and TYPE_HLL must be handled before.
         memcpy(dst, (const void*)(&data_ref), sizeof(data_ref));
         // Copy the content of string
         if (padding_char && type_desc.type == TYPE_CHAR) {
             // serialize the content of string
-            auto string_slot = reinterpret_cast<StringValue*>(dst);
-            string_slot->ptr = reinterpret_cast<char*>(pool->allocate(type_desc.len));
-            string_slot->len = type_desc.len;
-            memset(string_slot->ptr, 0, type_desc.len);
-            memcpy(string_slot->ptr, data_ref.data, data_ref.size);
+            auto string_slot = reinterpret_cast<StringRef*>(dst);
+            string_slot->data = reinterpret_cast<char*>(pool->allocate(type_desc.len));
+            string_slot->size = type_desc.len;
+            memset(const_cast<char*>(string_slot->data), 0, type_desc.len);             //!
+            memcpy(const_cast<char*>(string_slot->data), data_ref.data, data_ref.size); //!
         } else {
             auto str_ptr = pool->allocate(data_ref.size);
             memcpy(str_ptr, data_ref.data, data_ref.size);
-            auto string_slot = reinterpret_cast<StringValue*>(dst);
-            string_slot->ptr = reinterpret_cast<char*>(str_ptr);
-            string_slot->len = data_ref.size;
+            auto string_slot = reinterpret_cast<StringRef*>(dst);
+            string_slot->data = reinterpret_cast<char*>(str_ptr);
+            string_slot->size = data_ref.size;
         }
     } else {
         memcpy(dst, data_ref.data, data_ref.size);
diff --git a/be/src/vec/core/types.h b/be/src/vec/core/types.h
index 332d310d57..7f0fefc0ad 100644
--- a/be/src/vec/core/types.h
+++ b/be/src/vec/core/types.h
@@ -144,8 +144,8 @@ struct TypeName<uint24_t> {
     static const char* get() { return "uint24_t"; }
 };
 template <>
-struct TypeName<StringValue> {
-    static const char* get() { return "StringValue"; }
+struct TypeName<StringRef> {
+    static const char* get() { return "StringRef"; }
 };
 
 template <>
diff --git a/be/src/vec/exec/data_gen_functions/vnumbers_tvf.cpp b/be/src/vec/exec/data_gen_functions/vnumbers_tvf.cpp
index d8107ba925..e00808fb63 100644
--- a/be/src/vec/exec/data_gen_functions/vnumbers_tvf.cpp
+++ b/be/src/vec/exec/data_gen_functions/vnumbers_tvf.cpp
@@ -22,9 +22,9 @@
 #include "exec/exec_node.h"
 #include "gen_cpp/PlanNodes_types.h"
 #include "runtime/runtime_state.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple_row.h"
 #include "util/runtime_profile.h"
+#include "vec/common/string_ref.h"
 
 namespace doris::vectorized {
 
diff --git a/be/src/vec/exec/format/orc/vorc_reader.cpp b/be/src/vec/exec/format/orc/vorc_reader.cpp
index 0c27d78add..339c150422 100644
--- a/be/src/vec/exec/format/orc/vorc_reader.cpp
+++ b/be/src/vec/exec/format/orc/vorc_reader.cpp
@@ -308,8 +308,8 @@ static std::tuple<bool, orc::Literal> convert_to_orc_literal(const orc::Type* ty
         case orc::TypeKind::BINARY:
         case orc::TypeKind::CHAR:
         case orc::TypeKind::VARCHAR: {
-            StringValue* string_value = (StringValue*)value;
-            return std::make_tuple(true, orc::Literal(string_value->ptr, string_value->len));
+            StringRef* string_value = (StringRef*)value;
+            return std::make_tuple(true, orc::Literal(string_value->data, string_value->size));
         }
         case orc::TypeKind::DECIMAL: {
             int128_t decimal_value;
diff --git a/be/src/vec/exec/format/parquet/parquet_pred_cmp.h b/be/src/vec/exec/format/parquet/parquet_pred_cmp.h
index 06db698a9c..49016b5266 100644
--- a/be/src/vec/exec/format/parquet/parquet_pred_cmp.h
+++ b/be/src/vec/exec/format/parquet/parquet_pred_cmp.h
@@ -128,7 +128,7 @@ static bool _eval_in_val(const ColumnMinMaxParams& params) {
     case TYPE_CHAR: {
         std::vector<std::string> in_values;
         for (auto val : params.in_pred_values) {
-            in_values.emplace_back(((StringValue*)val)->to_string());
+            in_values.emplace_back(((StringRef*)val)->to_string());
         }
         if (in_values.empty()) {
             return false;
@@ -216,7 +216,7 @@ static bool _eval_eq(const ColumnMinMaxParams& params) {
     case TYPE_STRING:
     case TYPE_VARCHAR:
     case TYPE_CHAR: {
-        std::string conjunct_value = ((StringValue*)params.value)->to_string();
+        std::string conjunct_value = ((StringRef*)params.value)->to_string();
         if (strcmp(conjunct_value.data(), params.min_bytes) < 0 ||
             strcmp(conjunct_value.data(), params.max_bytes) > 0) {
             return true;
@@ -292,7 +292,7 @@ static bool _eval_gt(const ColumnMinMaxParams& params, bool is_eq) {
     case TYPE_STRING:
     case TYPE_VARCHAR:
     case TYPE_CHAR: {
-        std::string conjunct_value = ((StringValue*)params.value)->to_string();
+        std::string conjunct_value = ((StringRef*)params.value)->to_string();
         if (!is_eq && strcmp(params.max_bytes, conjunct_value.data()) <= 0) {
             return true;
         } else if (strcmp(params.max_bytes, conjunct_value.data()) < 0) {
@@ -343,7 +343,7 @@ static bool _eval_lt(const ColumnMinMaxParams& params, bool is_eq) {
     case TYPE_STRING:
     case TYPE_VARCHAR:
     case TYPE_CHAR: {
-        std::string conjunct_value = ((StringValue*)params.value)->to_string();
+        std::string conjunct_value = ((StringRef*)params.value)->to_string();
         if (!is_eq && strcmp(params.min_bytes, conjunct_value.data()) >= 0) {
             return true;
         } else if (strcmp(params.min_bytes, conjunct_value.data()) > 0) {
diff --git a/be/src/vec/exec/format/table/iceberg_reader.cpp b/be/src/vec/exec/format/table/iceberg_reader.cpp
index 4848b0b1c6..5c9f1c52cb 100644
--- a/be/src/vec/exec/format/table/iceberg_reader.cpp
+++ b/be/src/vec/exec/format/table/iceberg_reader.cpp
@@ -302,7 +302,7 @@ Status IcebergTableReader::_position_delete(
         parquet_reader->set_delete_rows(&_delete_rows);
         COUNTER_UPDATE(_iceberg_profile.num_delete_rows, num_delete_rows);
     }
-    // the delete rows are copy out, we can erase them.
+    // the deleted rows are copy out, we can erase them.
     for (auto& erase_item : erase_data) {
         erase_item->erase(data_file_path);
     }
diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp
index 21a38c6595..906f7a18d7 100644
--- a/be/src/vec/exec/scan/vscan_node.cpp
+++ b/be/src/vec/exec/scan/vscan_node.cpp
@@ -768,7 +768,7 @@ Status VScanNode::_normalize_in_and_eq_predicate(VExpr* expr, VExprContext* expr
         if (value.data != nullptr) {
             if constexpr (T == TYPE_CHAR || T == TYPE_VARCHAR || T == TYPE_STRING ||
                           T == TYPE_HLL) {
-                auto val = StringValue(value.data, value.size);
+                auto val = StringRef(value.data, value.size);
                 RETURN_IF_ERROR(_change_value_range<true>(
                         temp_range, reinterpret_cast<void*>(&val),
                         ColumnValueRange<T>::add_fixed_value_range, fn_name));
@@ -845,7 +845,7 @@ Status VScanNode::_normalize_not_in_and_not_eq_predicate(VExpr* expr, VExprConte
             auto fn_name = std::string("");
             if constexpr (T == TYPE_CHAR || T == TYPE_VARCHAR || T == TYPE_STRING ||
                           T == TYPE_HLL) {
-                auto val = StringValue(value.data, value.size);
+                auto val = StringRef(value.data, value.size);
                 if (is_fixed_range) {
                     RETURN_IF_ERROR(_change_value_range<true>(
                             range, reinterpret_cast<void*>(&val),
@@ -934,7 +934,7 @@ Status VScanNode::_normalize_noneq_binary_predicate(VExpr* expr, VExprContext* e
             if (value.data != nullptr) {
                 if constexpr (T == TYPE_CHAR || T == TYPE_VARCHAR || T == TYPE_STRING ||
                               T == TYPE_HLL) {
-                    auto val = StringValue(value.data, value.size);
+                    auto val = StringRef(value.data, value.size);
                     RETURN_IF_ERROR(_change_value_range<false>(range, reinterpret_cast<void*>(&val),
                                                                ColumnValueRange<T>::add_value_range,
                                                                fn_name, slot_ref_child));
@@ -1047,7 +1047,7 @@ Status VScanNode::_normalize_binary_in_compound_predicate(vectorized::VExpr* exp
             if (value.data != nullptr) {
                 if constexpr (T == TYPE_CHAR || T == TYPE_VARCHAR || T == TYPE_STRING ||
                               T == TYPE_HLL) {
-                    auto val = StringValue(value.data, value.size);
+                    auto val = StringRef(value.data, value.size);
                     RETURN_IF_ERROR(_change_value_range<false>(
                             range, reinterpret_cast<void*>(&val),
                             ColumnValueRange<T>::add_compound_value_range, fn_name,
@@ -1104,7 +1104,7 @@ Status VScanNode::_normalize_match_predicate(VExpr* expr, VExprContext* expr_ctx
                 using CppType = typename PrimitiveTypeTraits<T>::CppType;
                 if constexpr (T == TYPE_CHAR || T == TYPE_VARCHAR || T == TYPE_STRING ||
                               T == TYPE_HLL) {
-                    auto val = StringValue(value.data, value.size);
+                    auto val = StringRef(value.data, value.size);
                     ColumnValueRange<T>::add_match_value_range(temp_range,
                                                                to_match_type(expr->op()),
                                                                reinterpret_cast<CppType*>(&val));
diff --git a/be/src/vec/exec/vbroker_scan_node.cpp b/be/src/vec/exec/vbroker_scan_node.cpp
index 1b46e36022..d0ebe5c6cb 100644
--- a/be/src/vec/exec/vbroker_scan_node.cpp
+++ b/be/src/vec/exec/vbroker_scan_node.cpp
@@ -20,12 +20,12 @@
 #include "gen_cpp/PlanNodes_types.h"
 #include "runtime/memory/mem_tracker.h"
 #include "runtime/runtime_state.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple.h"
 #include "runtime/tuple_row.h"
 #include "util/runtime_profile.h"
 #include "util/thread.h"
 #include "util/types.h"
+#include "vec/common/string_ref.h"
 #include "vec/exec/vbroker_scanner.h"
 #include "vec/exec/vjson_scanner.h"
 #include "vec/exec/vorc_scanner.h"
diff --git a/be/src/vec/exec/vbroker_scanner.cpp b/be/src/vec/exec/vbroker_scanner.cpp
index 528218f548..c39b2003fb 100644
--- a/be/src/vec/exec/vbroker_scanner.cpp
+++ b/be/src/vec/exec/vbroker_scanner.cpp
@@ -232,9 +232,9 @@ Status VBrokerScanner::_line_to_src_tuple(const Slice& line) {
         }
         _src_tuple->set_not_null(slot_desc->null_indicator_offset());
         void* slot = _src_tuple->get_slot(slot_desc->tuple_offset());
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-        str_slot->ptr = value.data;
-        str_slot->len = value.size;
+        Slice* str_slot = reinterpret_cast<Slice*>(slot);
+        str_slot->data = value.data;
+        str_slot->size = value.size;
     }
 
     const TBrokerRangeDesc& range = _ranges.at(_next_range - 1);
diff --git a/be/src/vec/exec/vdata_gen_scan_node.cpp b/be/src/vec/exec/vdata_gen_scan_node.cpp
index e91626d49c..f30a2a14f5 100644
--- a/be/src/vec/exec/vdata_gen_scan_node.cpp
+++ b/be/src/vec/exec/vdata_gen_scan_node.cpp
@@ -22,7 +22,6 @@
 #include "common/status.h"
 #include "gen_cpp/PlanNodes_types.h"
 #include "runtime/runtime_state.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple_row.h"
 #include "util/runtime_profile.h"
 #include "vec/exec/data_gen_functions/vnumbers_tvf.h"
diff --git a/be/src/vec/exec/vjson_scanner.cpp b/be/src/vec/exec/vjson_scanner.cpp
index dee107f06a..7585af4d98 100644
--- a/be/src/vec/exec/vjson_scanner.cpp
+++ b/be/src/vec/exec/vjson_scanner.cpp
@@ -1456,14 +1456,18 @@ std::string VJsonReader::_print_json_value(const rapidjson::Value& value) {
     return std::string(buffer.GetString());
 }
 
+// TODO: NEED TO REWRITE COMPLETELY. the way writing now is WRONG.
+// StringRef shouldn't managing exclusive memory cause it will break RAII.
+// besides, accessing object which is essentially const by non-const object
+// is UB!
 void VJsonReader::_fill_slot(doris::Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool,
                              const uint8_t* value, int32_t len) {
     tuple->set_not_null(slot_desc->null_indicator_offset());
     void* slot = tuple->get_slot(slot_desc->tuple_offset());
-    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
-    memcpy(str_slot->ptr, value, len);
-    str_slot->len = len;
+    StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+    str_slot->data = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(const_cast<char*>(str_slot->data), value, len);
+    str_slot->size = len;
 }
 
 Status VJsonReader::_write_data_to_tuple(rapidjson::Value::ConstValueIterator value,
diff --git a/be/src/vec/exec/vmysql_scan_node.cpp b/be/src/vec/exec/vmysql_scan_node.cpp
index dcc7593ee9..a7a77f29db 100644
--- a/be/src/vec/exec/vmysql_scan_node.cpp
+++ b/be/src/vec/exec/vmysql_scan_node.cpp
@@ -21,10 +21,10 @@
 #include "exec/text_converter.hpp"
 #include "gen_cpp/PlanNodes_types.h"
 #include "runtime/runtime_state.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple_row.h"
 #include "util/runtime_profile.h"
 #include "util/types.h"
+#include "vec/common/string_ref.h"
 namespace doris::vectorized {
 
 VMysqlScanNode::VMysqlScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
diff --git a/be/src/vec/exec/vschema_scan_node.cpp b/be/src/vec/exec/vschema_scan_node.cpp
index 65588eb11b..f99de7ff85 100644
--- a/be/src/vec/exec/vschema_scan_node.cpp
+++ b/be/src/vec/exec/vschema_scan_node.cpp
@@ -21,10 +21,10 @@
 #include "exec/text_converter.hpp"
 #include "gen_cpp/PlanNodes_types.h"
 #include "runtime/runtime_state.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple_row.h"
 #include "util/runtime_profile.h"
 #include "util/types.h"
+#include "vec/common/string_ref.h"
 #include "vec/core/types.h"
 namespace doris::vectorized {
 
@@ -348,9 +348,9 @@ Status VSchemaScanNode::write_slot_to_vectorized_column(void* slot, SlotDescript
     case TYPE_VARCHAR:
     case TYPE_CHAR:
     case TYPE_STRING: {
-        StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
-        reinterpret_cast<vectorized::ColumnString*>(col_ptr)->insert_data(str_slot->ptr,
-                                                                          str_slot->len);
+        StringRef* str_slot = reinterpret_cast<StringRef*>(slot);
+        reinterpret_cast<vectorized::ColumnString*>(col_ptr)->insert_data(str_slot->data,
+                                                                          str_slot->size);
         break;
     }
 
diff --git a/be/src/vec/exprs/table_function/vexplode_json_array.h b/be/src/vec/exprs/table_function/vexplode_json_array.h
index 00531b9a22..ab171c7cef 100644
--- a/be/src/vec/exprs/table_function/vexplode_json_array.h
+++ b/be/src/vec/exprs/table_function/vexplode_json_array.h
@@ -20,8 +20,8 @@
 #include <rapidjson/document.h>
 #include <rapidjson/stringbuffer.h>
 
-#include "runtime/string_value.h"
 #include "vec/columns/column.h"
+#include "vec/common/string_ref.h"
 #include "vec/exprs/table_function/table_function.h"
 
 namespace doris::vectorized {
@@ -35,7 +35,7 @@ struct ParsedData {
     // The number parsed from json array
     // the `_backup` saved the real number entity.
     std::vector<void*> _data;
-    std::vector<StringValue> _data_string;
+    std::vector<StringRef> _data_string;
     std::vector<int64_t> _backup_int;
     std::vector<double> _backup_double;
     std::vector<std::string> _backup_string;
diff --git a/be/src/vec/exprs/table_function/vexplode_split.h b/be/src/vec/exprs/table_function/vexplode_split.h
index 63396bff91..53935b6a0b 100644
--- a/be/src/vec/exprs/table_function/vexplode_split.h
+++ b/be/src/vec/exprs/table_function/vexplode_split.h
@@ -18,8 +18,8 @@
 #pragma once
 
 #include "gutil/strings/stringpiece.h"
-#include "runtime/string_value.h"
 #include "vec/columns/column.h"
+#include "vec/common/string_ref.h"
 #include "vec/exprs/table_function/table_function.h"
 
 namespace doris::vectorized {
diff --git a/be/src/vec/exprs/vbloom_predicate.cpp b/be/src/vec/exprs/vbloom_predicate.cpp
index 5cdaa048ed..f154ab7856 100644
--- a/be/src/vec/exprs/vbloom_predicate.cpp
+++ b/be/src/vec/exprs/vbloom_predicate.cpp
@@ -68,7 +68,7 @@ Status VBloomPredicate::execute(VExprContext* context, Block* block, int* result
     if (type.is_string_or_fixed_string()) {
         for (size_t i = 0; i < sz; i++) {
             auto ele = argument_column->get_data_at(i);
-            const StringValue v(ele.data, ele.size);
+            const StringRef v(ele.data, ele.size);
             ptr[i] = _filter->find(reinterpret_cast<const void*>(&v));
         }
     } else if (_be_exec_version > 0 && (type.is_int_or_uint() || type.is_float())) {
diff --git a/be/src/vec/exprs/vdirect_in_predicate.h b/be/src/vec/exprs/vdirect_in_predicate.h
index 458e6c0a0e..b717f00d18 100644
--- a/be/src/vec/exprs/vdirect_in_predicate.h
+++ b/be/src/vec/exprs/vdirect_in_predicate.h
@@ -48,7 +48,7 @@ public:
         if (type.is_string_or_fixed_string()) {
             for (size_t i = 0; i < sz; i++) {
                 auto ele = argument_column->get_data_at(i);
-                StringValue v(ele.data, ele.size);
+                StringRef v(ele.data, ele.size);
                 ptr[i] = _filter->find(reinterpret_cast<const void*>(&v));
             }
         } else if (type.is_int_or_uint() || type.is_float()) {
diff --git a/be/src/vec/functions/function_regexp.cpp b/be/src/vec/functions/function_regexp.cpp
index 0fc3681a1e..bd651e85c1 100644
--- a/be/src/vec/functions/function_regexp.cpp
+++ b/be/src/vec/functions/function_regexp.cpp
@@ -20,8 +20,8 @@
 #include <random>
 
 #include "exprs/string_functions.h"
-#include "runtime/string_value.h"
 #include "udf/udf.h"
+#include "vec/common/string_ref.h"
 #include "vec/data_types/data_type_number.h"
 #include "vec/data_types/data_type_string.h"
 #include "vec/functions/function_string.h"
diff --git a/be/src/vec/functions/function_string.cpp b/be/src/vec/functions/function_string.cpp
index ad8499dcc7..660dce0551 100644
--- a/be/src/vec/functions/function_string.cpp
+++ b/be/src/vec/functions/function_string.cpp
@@ -93,7 +93,7 @@ struct StringUtf8LengthImpl {
         for (int i = 0; i < size; ++i) {
             const char* raw_str = reinterpret_cast<const char*>(&data[offsets[i - 1]]);
             int str_size = offsets[i] - offsets[i - 1];
-            res[i] = get_char_len(StringValue(raw_str, str_size), str_size);
+            res[i] = get_char_len(StringRef(raw_str, str_size), str_size);
         }
         return Status::OK();
     }
@@ -186,8 +186,8 @@ struct InStrOP {
             return;
         }
 
-        StringValue str_sv(strl.data(), strl.length());
-        StringValue substr_sv(strr.data(), strr.length());
+        StringRef str_sv(strl.data(), strl.length());
+        StringRef substr_sv(strr.data(), strr.length());
         StringSearch search(&substr_sv);
         // Hive returns positions starting from 1.
         int loc = search.search(&str_sv);
diff --git a/be/src/vec/functions/function_string.h b/be/src/vec/functions/function_string.h
index 0339bbb22f..06af3c6c19 100644
--- a/be/src/vec/functions/function_string.h
+++ b/be/src/vec/functions/function_string.h
@@ -66,6 +66,7 @@
 
 namespace doris::vectorized {
 
+//TODO: these three functions could be merged.
 inline size_t get_char_len(const std::string_view& str, std::vector<size_t>* str_index) {
     size_t char_len = 0;
     for (size_t i = 0, char_size = 0; i < str.length(); i += char_size) {
@@ -86,10 +87,10 @@ inline size_t get_char_len(const StringVal& str, std::vector<size_t>* str_index)
     return char_len;
 }
 
-inline size_t get_char_len(const StringValue& str, size_t end_pos) {
+inline size_t get_char_len(const StringRef& str, size_t end_pos) {
     size_t char_len = 0;
-    for (size_t i = 0, char_size = 0; i < std::min(str.len, end_pos); i += char_size) {
-        char_size = UTF8_BYTE_LENGTH[(unsigned char)(str.ptr)[i]];
+    for (size_t i = 0, char_size = 0; i < std::min(str.size, end_pos); i += char_size) {
+        char_size = UTF8_BYTE_LENGTH[(unsigned char)(str.data)[i]];
         ++char_len;
     }
     return char_len;
@@ -1809,7 +1810,7 @@ public:
             auto param = parameter_col->get_data_at(i);
             auto res = extract_url(source, param);
 
-            col_res->insert_data(res.ptr, res.len);
+            col_res->insert_data(res.data, res.size);
         }
 
         block.replace_by_position(result, std::move(col_res));
@@ -1817,11 +1818,11 @@ public:
     }
 
 private:
-    StringValue extract_url(StringRef url, StringRef parameter) {
+    StringRef extract_url(StringRef url, StringRef parameter) {
         if (url.size == 0 || parameter.size == 0) {
-            return StringValue("", 0);
+            return StringRef("", 0);
         }
-        return UrlParser::extract_url(StringValue(url), StringValue(parameter));
+        return UrlParser::extract_url(url, parameter);
     }
 };
 
@@ -1875,18 +1876,18 @@ public:
             }
 
             auto part = part_col->get_data_at(i);
-            StringValue p(const_cast<char*>(part.data), part.size);
+            StringRef p(const_cast<char*>(part.data), part.size);
             UrlParser::UrlPart url_part = UrlParser::get_url_part(p);
-            StringValue url_key;
+            StringRef url_key;
             if (has_key) {
                 auto key = key_col->get_data_at(i);
-                url_key = StringValue(const_cast<char*>(key.data), key.size);
+                url_key = StringRef(const_cast<char*>(key.data), key.size);
             }
 
             auto source = url_col->get_data_at(i);
-            StringValue url_val(const_cast<char*>(source.data), source.size);
+            StringRef url_val(const_cast<char*>(source.data), source.size);
 
-            StringValue parse_res;
+            StringRef parse_res;
             bool success = false;
             if (has_key) {
                 success = UrlParser::parse_url_key(url_val, url_part, url_key, &parse_res);
@@ -1908,7 +1909,7 @@ public:
                 }
             }
 
-            StringOP::push_value_string(std::string_view(parse_res.ptr, parse_res.len), i,
+            StringOP::push_value_string(std::string_view(parse_res.data, parse_res.size), i,
                                         res_chars, res_offsets);
         }
         block.get_by_position(result).column =
@@ -2136,11 +2137,11 @@ private:
         if (start_pos <= 0 || start_pos > str.len || start_pos > char_len) {
             return 0;
         }
-        StringValue substr_sv = StringValue::from_string_val(substr);
+        StringRef substr_sv = StringRef(substr);
         StringSearch search(&substr_sv);
         // Input start_pos starts from 1.
-        StringValue adjusted_str(reinterpret_cast<char*>(str.ptr) + index[start_pos - 1],
-                                 str.len - index[start_pos - 1]);
+        StringRef adjusted_str(reinterpret_cast<char*>(str.ptr) + index[start_pos - 1],
+                               str.len - index[start_pos - 1]);
         int32_t match_pos = search.search(&adjusted_str);
         if (match_pos >= 0) {
             // Hive returns the position in the original string starting from 1.
diff --git a/be/src/vec/functions/like.cpp b/be/src/vec/functions/like.cpp
index 4e96389b81..e8b0ad6321 100644
--- a/be/src/vec/functions/like.cpp
+++ b/be/src/vec/functions/like.cpp
@@ -17,9 +17,8 @@
 
 #include "vec/functions/like.h"
 
-#include "runtime/string_value.h"
-#include "runtime/string_value.hpp"
 #include "vec/columns/columns_number.h"
+#include "vec/common/string_ref.h"
 #include "vec/functions/simple_function_factory.h"
 
 namespace doris::vectorized {
@@ -64,7 +63,7 @@ Status LikeSearchState::clone(LikeSearchState& cloned) {
 }
 
 Status FunctionLikeBase::constant_starts_with_fn(LikeSearchState* state, const ColumnString& val,
-                                                 const StringValue& pattern,
+                                                 const StringRef& pattern,
                                                  ColumnUInt8::Container& result) {
     auto sz = val.size();
     for (size_t i = 0; i < sz; i++) {
@@ -76,7 +75,7 @@ Status FunctionLikeBase::constant_starts_with_fn(LikeSearchState* state, const C
 }
 
 Status FunctionLikeBase::constant_ends_with_fn(LikeSearchState* state, const ColumnString& val,
-                                               const StringValue& pattern,
+                                               const StringRef& pattern,
                                                ColumnUInt8::Container& result) {
     auto sz = val.size();
     for (size_t i = 0; i < sz; i++) {
@@ -88,7 +87,7 @@ Status FunctionLikeBase::constant_ends_with_fn(LikeSearchState* state, const Col
 }
 
 Status FunctionLikeBase::constant_equals_fn(LikeSearchState* state, const ColumnString& val,
-                                            const StringValue& pattern,
+                                            const StringRef& pattern,
                                             ColumnUInt8::Container& result) {
     auto sz = val.size();
     for (size_t i = 0; i < sz; i++) {
@@ -98,7 +97,7 @@ Status FunctionLikeBase::constant_equals_fn(LikeSearchState* state, const Column
 }
 
 Status FunctionLikeBase::constant_substring_fn(LikeSearchState* state, const ColumnString& val,
-                                               const StringValue& pattern,
+                                               const StringRef& pattern,
                                                ColumnUInt8::Container& result) {
     auto sz = val.size();
     for (size_t i = 0; i < sz; i++) {
@@ -112,7 +111,7 @@ Status FunctionLikeBase::constant_substring_fn(LikeSearchState* state, const Col
 
 Status FunctionLikeBase::constant_starts_with_fn_predicate(
         LikeSearchState* state, const PredicateColumnType<TYPE_STRING>& val,
-        const StringValue& pattern, ColumnUInt8::Container& result, uint16_t* sel, size_t sz) {
+        const StringRef& pattern, ColumnUInt8::Container& result, const uint16_t* sel, size_t sz) {
     auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
     for (size_t i = 0; i < sz; i++) {
         result[i] = (data_ptr[sel[i]].size >= state->search_string_sv.size) &&
@@ -124,7 +123,7 @@ Status FunctionLikeBase::constant_starts_with_fn_predicate(
 
 Status FunctionLikeBase::constant_ends_with_fn_predicate(
         LikeSearchState* state, const PredicateColumnType<TYPE_STRING>& val,
-        const StringValue& pattern, ColumnUInt8::Container& result, uint16_t* sel, size_t sz) {
+        const StringRef& pattern, ColumnUInt8::Container& result, const uint16_t* sel, size_t sz) {
     auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
     for (size_t i = 0; i < sz; i++) {
         result[i] =
@@ -138,9 +137,9 @@ Status FunctionLikeBase::constant_ends_with_fn_predicate(
 
 Status FunctionLikeBase::constant_equals_fn_predicate(LikeSearchState* state,
                                                       const PredicateColumnType<TYPE_STRING>& val,
-                                                      const StringValue& pattern,
-                                                      ColumnUInt8::Container& result, uint16_t* sel,
-                                                      size_t sz) {
+                                                      const StringRef& pattern,
+                                                      ColumnUInt8::Container& result,
+                                                      const uint16_t* sel, size_t sz) {
     auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
     for (size_t i = 0; i < sz; i++) {
         result[i] = (data_ptr[sel[i]] == state->search_string_sv);
@@ -150,7 +149,7 @@ Status FunctionLikeBase::constant_equals_fn_predicate(LikeSearchState* state,
 
 Status FunctionLikeBase::constant_substring_fn_predicate(
         LikeSearchState* state, const PredicateColumnType<TYPE_STRING>& val,
-        const StringValue& pattern, ColumnUInt8::Container& result, uint16_t* sel, size_t sz) {
+        const StringRef& pattern, ColumnUInt8::Container& result, const uint16_t* sel, size_t sz) {
     auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
     for (size_t i = 0; i < sz; i++) {
         if (state->search_string_sv.size == 0) {
@@ -163,7 +162,7 @@ Status FunctionLikeBase::constant_substring_fn_predicate(
 
 Status FunctionLikeBase::constant_starts_with_fn_scalar(LikeSearchState* state,
                                                         const StringRef& val,
-                                                        const StringValue& pattern,
+                                                        const StringRef& pattern,
                                                         unsigned char* result) {
     *result = (val.size >= state->search_string_sv.size) &&
               (state->search_string_sv == val.substring(0, state->search_string_sv.size));
@@ -171,7 +170,7 @@ Status FunctionLikeBase::constant_starts_with_fn_scalar(LikeSearchState* state,
 }
 
 Status FunctionLikeBase::constant_ends_with_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                                      const StringValue& pattern,
+                                                      const StringRef& pattern,
                                                       unsigned char* result) {
     *result = (val.size >= state->search_string_sv.size) &&
               (state->search_string_sv == val.substring(val.size - state->search_string_sv.size,
@@ -180,14 +179,14 @@ Status FunctionLikeBase::constant_ends_with_fn_scalar(LikeSearchState* state, co
 }
 
 Status FunctionLikeBase::constant_equals_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                                   const StringValue& pattern,
+                                                   const StringRef& pattern,
                                                    unsigned char* result) {
     *result = (val == state->search_string_sv);
     return Status::OK();
 }
 
 Status FunctionLikeBase::constant_substring_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                                      const StringValue& pattern,
+                                                      const StringRef& pattern,
                                                       unsigned char* result) {
     if (state->search_string_sv.size == 0) {
         *result = true;
@@ -198,10 +197,9 @@ Status FunctionLikeBase::constant_substring_fn_scalar(LikeSearchState* state, co
 }
 
 Status FunctionLikeBase::constant_regex_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                                  const StringValue& pattern,
-                                                  unsigned char* result) {
+                                                  const StringRef& pattern, unsigned char* result) {
     auto ret = hs_scan(state->hs_database.get(), val.data, val.size, 0, state->hs_scratch.get(),
-                       state->hs_match_handler, (void*)result);
+                       doris::vectorized::LikeSearchState::hs_match_handler, (void*)result);
     if (ret != HS_SUCCESS && ret != HS_SCAN_TERMINATED) {
         return Status::RuntimeError(fmt::format("hyperscan error: {}", ret));
     }
@@ -210,15 +208,15 @@ Status FunctionLikeBase::constant_regex_fn_scalar(LikeSearchState* state, const
 }
 
 Status FunctionLikeBase::regexp_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                          const StringValue& pattern, unsigned char* result) {
-    std::string_view re_pattern(pattern.ptr, pattern.len);
+                                          const StringRef& pattern, unsigned char* result) {
+    std::string_view re_pattern(pattern.data, pattern.size);
 
     hs_database_t* database = nullptr;
     hs_scratch_t* scratch = nullptr;
     RETURN_IF_ERROR(hs_prepare(nullptr, re_pattern.data(), &database, &scratch));
 
-    auto ret = hs_scan(database, val.data, val.size, 0, scratch, state->hs_match_handler,
-                       (void*)result);
+    auto ret = hs_scan(database, val.data, val.size, 0, scratch,
+                       doris::vectorized::LikeSearchState::hs_match_handler, (void*)result);
     if (ret != HS_SUCCESS && ret != HS_SCAN_TERMINATED) {
         return Status::RuntimeError(fmt::format("hyperscan error: {}", ret));
     }
@@ -230,14 +228,14 @@ Status FunctionLikeBase::regexp_fn_scalar(LikeSearchState* state, const StringRe
 }
 
 Status FunctionLikeBase::constant_regex_fn(LikeSearchState* state, const ColumnString& val,
-                                           const StringValue& pattern,
+                                           const StringRef& pattern,
                                            ColumnUInt8::Container& result) {
     auto sz = val.size();
     for (size_t i = 0; i < sz; i++) {
         const auto& str_ref = val.get_data_at(i);
-        auto ret = hs_scan(state->hs_database.get(), str_ref.data, str_ref.size, 0,
-                           state->hs_scratch.get(), state->hs_match_handler,
-                           (void*)(result.data() + i));
+        auto ret = hs_scan(
+                state->hs_database.get(), str_ref.data, str_ref.size, 0, state->hs_scratch.get(),
+                doris::vectorized::LikeSearchState::hs_match_handler, (void*)(result.data() + i));
         if (ret != HS_SUCCESS && ret != HS_SCAN_TERMINATED) {
             return Status::RuntimeError(fmt::format("hyperscan error: {}", ret));
         }
@@ -247,8 +245,8 @@ Status FunctionLikeBase::constant_regex_fn(LikeSearchState* state, const ColumnS
 }
 
 Status FunctionLikeBase::regexp_fn(LikeSearchState* state, const ColumnString& val,
-                                   const StringValue& pattern, ColumnUInt8::Container& result) {
-    std::string_view re_pattern(pattern.ptr, pattern.len);
+                                   const StringRef& pattern, ColumnUInt8::Container& result) {
+    std::string_view re_pattern(pattern.data, pattern.size);
 
     hs_database_t* database = nullptr;
     hs_scratch_t* scratch = nullptr;
@@ -258,7 +256,8 @@ Status FunctionLikeBase::regexp_fn(LikeSearchState* state, const ColumnString& v
     for (size_t i = 0; i < sz; i++) {
         const auto& str_ref = val.get_data_at(i);
         auto ret = hs_scan(database, str_ref.data, str_ref.size, 0, scratch,
-                           state->hs_match_handler, (void*)(result.data() + i));
+                           doris::vectorized::LikeSearchState::hs_match_handler,
+                           (void*)(result.data() + i));
         if (ret != HS_SUCCESS && ret != HS_SCAN_TERMINATED) {
             return Status::RuntimeError(fmt::format("hyperscan error: {}", ret));
         }
@@ -272,13 +271,14 @@ Status FunctionLikeBase::regexp_fn(LikeSearchState* state, const ColumnString& v
 
 Status FunctionLikeBase::constant_regex_fn_predicate(LikeSearchState* state,
                                                      const PredicateColumnType<TYPE_STRING>& val,
-                                                     const StringValue& pattern,
-                                                     ColumnUInt8::Container& result, uint16_t* sel,
-                                                     size_t sz) {
+                                                     const StringRef& pattern,
+                                                     ColumnUInt8::Container& result,
+                                                     const uint16_t* sel, size_t sz) {
     auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
     for (size_t i = 0; i < sz; i++) {
         auto ret = hs_scan(state->hs_database.get(), data_ptr[sel[i]].data, data_ptr[sel[i]].size,
-                           0, state->hs_scratch.get(), state->hs_match_handler,
+                           0, state->hs_scratch.get(),
+                           doris::vectorized::LikeSearchState::hs_match_handler,
                            (void*)(result.data() + i));
         if (ret != HS_SUCCESS && ret != HS_SCAN_TERMINATED) {
             return Status::RuntimeError(fmt::format("hyperscan error: {}", ret));
@@ -290,10 +290,10 @@ Status FunctionLikeBase::constant_regex_fn_predicate(LikeSearchState* state,
 
 Status FunctionLikeBase::regexp_fn_predicate(LikeSearchState* state,
                                              const PredicateColumnType<TYPE_STRING>& val,
-                                             const StringValue& pattern,
-                                             ColumnUInt8::Container& result, uint16_t* sel,
+                                             const StringRef& pattern,
+                                             ColumnUInt8::Container& result, const uint16_t* sel,
                                              size_t sz) {
-    std::string_view re_pattern(pattern.ptr, pattern.len);
+    std::string_view re_pattern(pattern.data, pattern.size);
 
     hs_database_t* database = nullptr;
     hs_scratch_t* scratch = nullptr;
@@ -302,7 +302,8 @@ Status FunctionLikeBase::regexp_fn_predicate(LikeSearchState* state,
     auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
     for (size_t i = 0; i < sz; i++) {
         auto ret = hs_scan(database, data_ptr[sel[i]].data, data_ptr[sel[i]].size, 0, scratch,
-                           state->hs_match_handler, (void*)(result.data() + i));
+                           doris::vectorized::LikeSearchState::hs_match_handler,
+                           (void*)(result.data() + i));
         if (ret != HS_SUCCESS && ret != HS_SCAN_TERMINATED) {
             return Status::RuntimeError(fmt::format("hyperscan error: {}", ret));
         }
@@ -318,11 +319,13 @@ Status FunctionLikeBase::regexp_fn_predicate(LikeSearchState* state,
 Status FunctionLikeBase::hs_prepare(FunctionContext* context, const char* expression,
                                     hs_database_t** database, hs_scratch_t** scratch) {
     hs_compile_error_t* compile_err;
-    auto res = hs_compile(expression, HS_FLAG_DOTALL | HS_FLAG_ALLOWEMPTY, HS_MODE_BLOCK, NULL,
+    auto res = hs_compile(expression, HS_FLAG_DOTALL | HS_FLAG_ALLOWEMPTY, HS_MODE_BLOCK, nullptr,
                           database, &compile_err);
     if (res != HS_SUCCESS) {
         *database = nullptr;
-        if (context) context->set_error("hs_compile regex pattern error");
+        if (context) {
+            context->set_error("hs_compile regex pattern error");
+        }
         return Status::RuntimeError("hs_compile regex pattern error:" +
                                     std::string(compile_err->message));
         hs_free_compile_error(compile_err);
@@ -333,7 +336,9 @@ Status FunctionLikeBase::hs_prepare(FunctionContext* context, const char* expres
         hs_free_database(*database);
         *database = nullptr;
         *scratch = nullptr;
-        if (context) context->set_error("hs_alloc_scratch allocate scratch space error");
+        if (context) {
+            context->set_error("hs_alloc_scratch allocate scratch space error");
+        }
         return Status::RuntimeError("hs_alloc_scratch allocate scratch space error");
     }
 
@@ -359,9 +364,8 @@ Status FunctionLikeBase::execute_impl(FunctionContext* context, Block& block,
             context->get_function_state(FunctionContext::THREAD_LOCAL));
     // for constant_substring_fn, use long run length search for performance
     if (constant_substring_fn ==
-        *(state->function
-                  .target<doris::Status (*)(LikeSearchState * state, const ColumnString&,
-                                            const StringValue&, ColumnUInt8::Container&)>())) {
+        *(state->function.target<doris::Status (*)(LikeSearchState * state, const ColumnString&,
+                                                   const StringRef&, ColumnUInt8::Container&)>())) {
         RETURN_IF_ERROR(execute_substring(values->get_chars(), values->get_offsets(), vec_res,
                                           &state->search_state));
     } else {
@@ -412,11 +416,15 @@ Status FunctionLikeBase::execute_substring(const ColumnString::Chars& values,
     while (pos < end) {
         // search return matched substring start offset
         pos = (UInt8*)search_state->substring_pattern.search((char*)pos, end - pos);
-        if (pos >= end) break;
+        if (pos >= end) {
+            break;
+        }
 
         /// Determine which index it refers to.
         /// begin + value_offsets[i] is the start offset of string at i+1
-        while (begin + value_offsets[i] < pos) ++i;
+        while (begin + value_offsets[i] < pos) {
+            ++i;
+        }
 
         /// We check that the entry does not pass through the boundaries of strings.
         if (pos + needle_size <= begin + value_offsets[i]) {
@@ -435,36 +443,36 @@ Status FunctionLikeBase::vector_const(const ColumnString& values, const StringRe
                                       ColumnUInt8::Container& result, const LikeFn& function,
                                       LikeSearchState* search_state) {
     RETURN_IF_ERROR((function)(search_state, values,
-                               *reinterpret_cast<const StringValue*>(pattern_val), result));
+                               *reinterpret_cast<const StringRef*>(pattern_val), result));
     return Status::OK();
 }
 
 Status FunctionLike::like_fn(LikeSearchState* state, const ColumnString& val,
-                             const StringValue& pattern, ColumnUInt8::Container& result) {
+                             const StringRef& pattern, ColumnUInt8::Container& result) {
     std::string re_pattern;
-    convert_like_pattern(state, std::string(pattern.ptr, pattern.len), &re_pattern);
+    convert_like_pattern(state, std::string(pattern.data, pattern.size), &re_pattern);
 
-    return regexp_fn(state, val, {re_pattern.c_str(), (int)re_pattern.size()}, result);
+    return regexp_fn(state, val, {re_pattern.c_str(), re_pattern.size()}, result);
 }
 
 Status FunctionLike::like_fn_predicate(LikeSearchState* state,
                                        const PredicateColumnType<TYPE_STRING>& val,
-                                       const StringValue& pattern, ColumnUInt8::Container& result,
+                                       const StringRef& pattern, ColumnUInt8::Container& result,
                                        uint16_t* sel, size_t sz) {
     std::string re_pattern;
-    convert_like_pattern(state, std::string(pattern.ptr, pattern.len), &re_pattern);
+    convert_like_pattern(state, std::string(pattern.data, pattern.size), &re_pattern);
 
-    return regexp_fn_predicate(state, val, {re_pattern.c_str(), (int)re_pattern.size()}, result,
-                               sel, sz);
+    return regexp_fn_predicate(state, val, {re_pattern.c_str(), re_pattern.size()}, result, sel,
+                               sz);
 }
 
-Status FunctionLike::like_fn_scalar(LikeSearchState* state, const StringValue& val,
-                                    const StringValue& pattern, unsigned char* result) {
+Status FunctionLike::like_fn_scalar(LikeSearchState* state, const StringRef& val,
+                                    const StringRef& pattern, unsigned char* result) {
     std::string re_pattern;
-    convert_like_pattern(state, std::string(pattern.ptr, pattern.len), &re_pattern);
+    convert_like_pattern(state, std::string(pattern.data, pattern.size), &re_pattern);
 
-    return regexp_fn_scalar(state, StringRef(val.ptr, val.len),
-                            {re_pattern.c_str(), (int)re_pattern.size()}, result);
+    return regexp_fn_scalar(state, StringRef(val.data, val.size),
+                            {re_pattern.c_str(), re_pattern.size()}, result);
 }
 
 void FunctionLike::convert_like_pattern(LikeSearchState* state, const std::string& pattern,
diff --git a/be/src/vec/functions/like.h b/be/src/vec/functions/like.h
index b44831a93d..40235faafa 100644
--- a/be/src/vec/functions/like.h
+++ b/be/src/vec/functions/like.h
@@ -23,11 +23,11 @@
 #include <memory>
 
 #include "runtime/string_search.hpp"
-#include "runtime/string_value.h"
 #include "vec/columns/column_const.h"
 #include "vec/columns/column_set.h"
 #include "vec/columns/columns_number.h"
 #include "vec/columns/predicate_column.h"
+#include "vec/common/string_ref.h"
 #include "vec/data_types/data_type_nullable.h"
 #include "vec/data_types/data_type_number.h"
 #include "vec/exprs/vexpr.h"
@@ -41,7 +41,7 @@ namespace doris::vectorized {
 struct LikeSearchState {
     char escape_char;
 
-    /// Holds the string the StringValue points to and is set any time StringValue is
+    /// Holds the string the StringRef points to and is set any time StringRef is
     /// used.
     std::string search_string;
 
@@ -97,15 +97,15 @@ struct LikeSearchState {
     }
 };
 
-using LikeFn = std::function<doris::Status(LikeSearchState*, const ColumnString&,
-                                           const StringValue&, ColumnUInt8::Container&)>;
+using LikeFn = std::function<doris::Status(LikeSearchState*, const ColumnString&, const StringRef&,
+                                           ColumnUInt8::Container&)>;
 
 using LikePredicateFn = std::function<doris::Status(
-        LikeSearchState*, const PredicateColumnType<TYPE_STRING>&, const StringValue&,
+        LikeSearchState*, const PredicateColumnType<TYPE_STRING>&, const StringRef&,
         ColumnUInt8::Container&, uint16_t* sel, size_t sz)>;
 
 using ScalarLikeFn = std::function<doris::Status(LikeSearchState*, const StringRef&,
-                                                 const StringValue&, unsigned char*)>;
+                                                 const StringRef&, unsigned char*)>;
 
 struct LikeState {
     LikeSearchState search_state;
@@ -140,77 +140,76 @@ protected:
                              ColumnUInt8::Container& result, LikeSearchState* search_state);
 
     static Status constant_starts_with_fn(LikeSearchState* state, const ColumnString& val,
-                                          const StringValue& pattern,
-                                          ColumnUInt8::Container& result);
+                                          const StringRef& pattern, ColumnUInt8::Container& result);
 
     static Status constant_ends_with_fn(LikeSearchState* state, const ColumnString& val,
-                                        const StringValue& pattern, ColumnUInt8::Container& result);
+                                        const StringRef& pattern, ColumnUInt8::Container& result);
 
     static Status constant_equals_fn(LikeSearchState* state, const ColumnString& val,
-                                     const StringValue& pattern, ColumnUInt8::Container& result);
+                                     const StringRef& pattern, ColumnUInt8::Container& result);
 
     static Status constant_substring_fn(LikeSearchState* state, const ColumnString& val,
-                                        const StringValue& pattern, ColumnUInt8::Container& result);
+                                        const StringRef& pattern, ColumnUInt8::Container& result);
 
     static Status constant_regex_fn(LikeSearchState* state, const ColumnString& val,
-                                    const StringValue& pattern, ColumnUInt8::Container& result);
+                                    const StringRef& pattern, ColumnUInt8::Container& result);
 
     static Status regexp_fn(LikeSearchState* state, const ColumnString& val,
-                            const StringValue& pattern, ColumnUInt8::Container& result);
+                            const StringRef& pattern, ColumnUInt8::Container& result);
 
     // These functions below are used only for predicate.
     static Status constant_regex_fn_predicate(LikeSearchState* state,
                                               const PredicateColumnType<TYPE_STRING>& val,
-                                              const StringValue& pattern,
-                                              ColumnUInt8::Container& result, uint16_t* sel,
+                                              const StringRef& pattern,
+                                              ColumnUInt8::Container& result, const uint16_t* sel,
                                               size_t sz);
 
     static Status regexp_fn_predicate(LikeSearchState* state,
                                       const PredicateColumnType<TYPE_STRING>& val,
-                                      const StringValue& pattern, ColumnUInt8::Container& result,
-                                      uint16_t* sel, size_t sz);
+                                      const StringRef& pattern, ColumnUInt8::Container& result,
+                                      const uint16_t* sel, size_t sz);
 
     static Status constant_starts_with_fn_predicate(LikeSearchState* state,
                                                     const PredicateColumnType<TYPE_STRING>& val,
-                                                    const StringValue& pattern,
-                                                    ColumnUInt8::Container& result, uint16_t* sel,
-                                                    size_t sz);
+                                                    const StringRef& pattern,
+                                                    ColumnUInt8::Container& result,
+                                                    const uint16_t* sel, size_t sz);
 
     static Status constant_ends_with_fn_predicate(LikeSearchState* state,
                                                   const PredicateColumnType<TYPE_STRING>& val,
-                                                  const StringValue& pattern,
-                                                  ColumnUInt8::Container& result, uint16_t* sel,
-                                                  size_t sz);
+                                                  const StringRef& pattern,
+                                                  ColumnUInt8::Container& result,
+                                                  const uint16_t* sel, size_t sz);
 
     static Status constant_equals_fn_predicate(LikeSearchState* state,
                                                const PredicateColumnType<TYPE_STRING>& val,
-                                               const StringValue& pattern,
-                                               ColumnUInt8::Container& result, uint16_t* sel,
+                                               const StringRef& pattern,
+                                               ColumnUInt8::Container& result, const uint16_t* sel,
                                                size_t sz);
 
     static Status constant_substring_fn_predicate(LikeSearchState* state,
                                                   const PredicateColumnType<TYPE_STRING>& val,
-                                                  const StringValue& pattern,
-                                                  ColumnUInt8::Container& result, uint16_t* sel,
-                                                  size_t sz);
+                                                  const StringRef& pattern,
+                                                  ColumnUInt8::Container& result,
+                                                  const uint16_t* sel, size_t sz);
 
     static Status constant_starts_with_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                                 const StringValue& pattern, unsigned char* result);
+                                                 const StringRef& pattern, unsigned char* result);
 
     static Status constant_ends_with_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                               const StringValue& pattern, unsigned char* result);
+                                               const StringRef& pattern, unsigned char* result);
 
     static Status constant_equals_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                            const StringValue& pattern, unsigned char* result);
+                                            const StringRef& pattern, unsigned char* result);
 
     static Status constant_substring_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                               const StringValue& pattern, unsigned char* result);
+                                               const StringRef& pattern, unsigned char* result);
 
     static Status constant_regex_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                           const StringValue& pattern, unsigned char* result);
+                                           const StringRef& pattern, unsigned char* result);
 
     static Status regexp_fn_scalar(LikeSearchState* state, const StringRef& val,
-                                   const StringValue& pattern, unsigned char* result);
+                                   const StringRef& pattern, unsigned char* result);
 
     // hyperscan compile expression to database and allocate scratch space
     static Status hs_prepare(FunctionContext* context, const char* expression,
@@ -230,16 +229,16 @@ public:
     friend struct LikeSearchState;
 
 private:
-    static Status like_fn(LikeSearchState* state, const ColumnString& val,
-                          const StringValue& pattern, ColumnUInt8::Container& result);
+    static Status like_fn(LikeSearchState* state, const ColumnString& val, const StringRef& pattern,
+                          ColumnUInt8::Container& result);
 
     static Status like_fn_predicate(LikeSearchState* state,
                                     const PredicateColumnType<TYPE_STRING>& val,
-                                    const StringValue& pattern, ColumnUInt8::Container& result,
+                                    const StringRef& pattern, ColumnUInt8::Container& result,
                                     uint16_t* sel, size_t sz);
 
-    static Status like_fn_scalar(LikeSearchState* state, const StringValue& val,
-                                 const StringValue& pattern, unsigned char* result);
+    static Status like_fn_scalar(LikeSearchState* state, const StringRef& val,
+                                 const StringRef& pattern, unsigned char* result);
 
     static void convert_like_pattern(LikeSearchState* state, const std::string& pattern,
                                      std::string* re_pattern);
diff --git a/be/src/vec/io/io_helper.h b/be/src/vec/io/io_helper.h
index 990b67fd62..5c848188c5 100644
--- a/be/src/vec/io/io_helper.h
+++ b/be/src/vec/io/io_helper.h
@@ -37,7 +37,7 @@
 namespace doris::vectorized {
 
 // Define in the namespace and avoid defining global macros,
-// because it maybe conflict with other libs
+// because it maybe conflicts with other libs
 static constexpr size_t DEFAULT_MAX_STRING_SIZE = 1073741824; // 1GB
 static constexpr size_t DEFAULT_MAX_JSON_SIZE = 1073741824;   // 1GB
 static constexpr auto WRITE_HELPERS_MAX_INT_WIDTH = 40U;
@@ -134,7 +134,7 @@ inline void write_string_binary(const StringRef& s, BufferWritable& buf) {
 }
 
 inline void write_string_binary(const char* s, BufferWritable& buf) {
-    write_string_binary(StringRef {s}, buf);
+    write_string_binary(StringRef {std::string(s)}, buf);
 }
 
 inline void write_json_binary(JsonbField s, BufferWritable& buf) {
diff --git a/be/src/vec/olap/vertical_merge_iterator.cpp b/be/src/vec/olap/vertical_merge_iterator.cpp
index 7523dc3237..605e110e06 100644
--- a/be/src/vec/olap/vertical_merge_iterator.cpp
+++ b/be/src/vec/olap/vertical_merge_iterator.cpp
@@ -28,11 +28,11 @@ RowSource::RowSource(uint16_t source_num, bool agg_flag) {
     _data = agg_flag ? (_data | AGG_FLAG) : (_data & SOURCE_FLAG);
 }
 
-uint16_t RowSource::get_source_num() {
+uint16_t RowSource::get_source_num() const {
     return _data & SOURCE_FLAG;
 }
 
-bool RowSource::agg_flag() {
+bool RowSource::agg_flag() const {
     return (_data & AGG_FLAG) != 0;
 }
 
@@ -605,8 +605,7 @@ std::shared_ptr<RowwiseIterator> new_vertical_heap_merge_iterator(
 std::shared_ptr<RowwiseIterator> new_vertical_mask_merge_iterator(
         const std::vector<RowwiseIterator*>& inputs, size_t ori_return_cols,
         RowSourcesBuffer* row_sources) {
-    return std::make_shared<VerticalMaskMergeIterator>(std::move(inputs), ori_return_cols,
-                                                       row_sources);
+    return std::make_shared<VerticalMaskMergeIterator>(inputs, ori_return_cols, row_sources);
 }
 
 } // namespace vectorized
diff --git a/be/src/vec/olap/vertical_merge_iterator.h b/be/src/vec/olap/vertical_merge_iterator.h
index 6ad5613f36..a7712192b4 100644
--- a/be/src/vec/olap/vertical_merge_iterator.h
+++ b/be/src/vec/olap/vertical_merge_iterator.h
@@ -37,8 +37,8 @@ public:
     RowSource(uint16_t data) : _data(data) {}
     RowSource(uint16_t source_num, bool agg_flag);
 
-    uint16_t get_source_num();
-    bool agg_flag();
+    uint16_t get_source_num() const;
+    bool agg_flag() const;
     void set_agg_flag(bool agg_flag);
     uint16_t data() const;
 
diff --git a/be/src/vec/runtime/vfile_result_writer.cpp b/be/src/vec/runtime/vfile_result_writer.cpp
index c2778543ba..256bb8bf19 100644
--- a/be/src/vec/runtime/vfile_result_writer.cpp
+++ b/be/src/vec/runtime/vfile_result_writer.cpp
@@ -28,11 +28,11 @@
 #include "runtime/descriptors.h"
 #include "runtime/large_int_value.h"
 #include "runtime/runtime_state.h"
-#include "runtime/string_value.h"
 #include "service/backend_options.h"
 #include "util/file_utils.h"
 #include "util/mysql_global.h"
 #include "util/mysql_row_buffer.h"
+#include "vec/common/string_ref.h"
 #include "vec/core/block.h"
 #include "vec/exprs/vexpr.h"
 #include "vec/exprs/vexpr_context.h"
diff --git a/be/src/vec/runtime/vorc_writer.cpp b/be/src/vec/runtime/vorc_writer.cpp
index 4129a86831..5737d0bf94 100644
--- a/be/src/vec/runtime/vorc_writer.cpp
+++ b/be/src/vec/runtime/vorc_writer.cpp
@@ -158,12 +158,12 @@ void VOrcWriterWrapper::close() {
                 cur_batch->notNull[row_id] = 1;                                                    \
                 int len = binary_cast<FROM, TO>(                                                   \
                                   assert_cast<const ColumnVector<FROM>&>(*col).get_data()[row_id]) \
-                                  .to_buffer(buffer.ptr);                                          \
+                                  .to_buffer((char*)buffer.ptr);                                   \
                 while (buffer.len < offset + len) {                                                \
                     char* new_ptr = (char*)malloc(buffer.len + BUFFER_UNIT_SIZE);                  \
                     memcpy(new_ptr, buffer.ptr, buffer.len);                                       \
                     free(buffer.ptr);                                                              \
-                    buffer.ptr = new_ptr;                                                          \
+                    buffer.ptr = (uint8_t*)new_ptr;                                                \
                     buffer.len = buffer.len + BUFFER_UNIT_SIZE;                                    \
                 }                                                                                  \
                 cur_batch->length[row_id] = len;                                                   \
@@ -175,7 +175,7 @@ void VOrcWriterWrapper::close() {
             if (null_data[row_id] != 0) {                                                          \
                 cur_batch->notNull[row_id] = 0;                                                    \
             } else {                                                                               \
-                cur_batch->data[row_id] = buffer.ptr + offset;                                     \
+                cur_batch->data[row_id] = (char*)buffer.ptr + offset;                              \
                 offset += cur_batch->length[row_id];                                               \
             }                                                                                      \
         }                                                                                          \
@@ -183,12 +183,12 @@ void VOrcWriterWrapper::close() {
                        check_and_get_column<const ColumnVector<FROM>>(col)) {                      \
         for (size_t row_id = 0; row_id < sz; row_id++) {                                           \
             int len = binary_cast<FROM, TO>(not_null_column->get_data()[row_id])                   \
-                              .to_buffer(buffer.ptr);                                              \
+                              .to_buffer((char*)buffer.ptr);                                       \
             while (buffer.len < offset + len) {                                                    \
                 char* new_ptr = (char*)malloc(buffer.len + BUFFER_UNIT_SIZE);                      \
                 memcpy(new_ptr, buffer.ptr, buffer.len);                                           \
                 free(buffer.ptr);                                                                  \
-                buffer.ptr = new_ptr;                                                              \
+                buffer.ptr = (uint8_t*)new_ptr;                                                    \
                 buffer.len = buffer.len + BUFFER_UNIT_SIZE;                                        \
             }                                                                                      \
             cur_batch->length[row_id] = len;                                                       \
@@ -196,7 +196,7 @@ void VOrcWriterWrapper::close() {
         }                                                                                          \
         offset = 0;                                                                                \
         for (size_t row_id = 0; row_id < sz; row_id++) {                                           \
-            cur_batch->data[row_id] = buffer.ptr + offset;                                         \
+            cur_batch->data[row_id] = (char*)buffer.ptr + offset;                                  \
             offset += cur_batch->length[row_id];                                                   \
         }                                                                                          \
     } else {                                                                                       \
@@ -258,7 +258,7 @@ Status VOrcWriterWrapper::write(const Block& block) {
 
     // Buffer used by date type
     char* ptr = (char*)malloc(BUFFER_UNIT_SIZE);
-    StringValue buffer(ptr, BUFFER_UNIT_SIZE);
+    ::StringVal buffer(ptr, BUFFER_UNIT_SIZE);
 
     size_t sz = block.rows();
     auto row_batch = _create_row_batch(sz);
@@ -347,12 +347,12 @@ Status VOrcWriterWrapper::write(const Block& block) {
                             int len = binary_cast<UInt64, DateV2Value<DateTimeV2ValueType>>(
                                               assert_cast<const ColumnVector<UInt64>&>(*col)
                                                       .get_data()[row_id])
-                                              .to_buffer(buffer.ptr, output_scale);
+                                              .to_buffer((char*)buffer.ptr, output_scale);
                             while (buffer.len < offset + len) {
                                 char* new_ptr = (char*)malloc(buffer.len + BUFFER_UNIT_SIZE);
                                 memcpy(new_ptr, buffer.ptr, buffer.len);
                                 free(buffer.ptr);
-                                buffer.ptr = new_ptr;
+                                buffer.ptr = (uint8_t*)new_ptr;
                                 buffer.len = buffer.len + BUFFER_UNIT_SIZE;
                             }
                             cur_batch->length[row_id] = len;
@@ -364,7 +364,7 @@ Status VOrcWriterWrapper::write(const Block& block) {
                         if (null_data[row_id] != 0) {
                             cur_batch->notNull[row_id] = 0;
                         } else {
-                            cur_batch->data[row_id] = buffer.ptr + offset;
+                            cur_batch->data[row_id] = (char*)buffer.ptr + offset;
                             offset += cur_batch->length[row_id];
                         }
                     }
@@ -374,12 +374,12 @@ Status VOrcWriterWrapper::write(const Block& block) {
                         int output_scale = _output_vexpr_ctxs[i]->root()->type().scale;
                         int len = binary_cast<UInt64, DateV2Value<DateTimeV2ValueType>>(
                                           not_null_column->get_data()[row_id])
-                                          .to_buffer(buffer.ptr, output_scale);
+                                          .to_buffer((char*)buffer.ptr, output_scale);
                         while (buffer.len < offset + len) {
                             char* new_ptr = (char*)malloc(buffer.len + BUFFER_UNIT_SIZE);
                             memcpy(new_ptr, buffer.ptr, buffer.len);
                             free(buffer.ptr);
-                            buffer.ptr = new_ptr;
+                            buffer.ptr = (uint8_t*)new_ptr;
                             buffer.len = buffer.len + BUFFER_UNIT_SIZE;
                         }
                         cur_batch->length[row_id] = len;
@@ -387,7 +387,7 @@ Status VOrcWriterWrapper::write(const Block& block) {
                     }
                     offset = 0;
                     for (size_t row_id = 0; row_id < sz; row_id++) {
-                        cur_batch->data[row_id] = buffer.ptr + offset;
+                        cur_batch->data[row_id] = (char*)buffer.ptr + offset;
                         offset += cur_batch->length[row_id];
                     }
                 } else {
diff --git a/be/test/exprs/bitmap_function_test.cpp b/be/test/exprs/bitmap_function_test.cpp
index d76e30f333..aa3150995d 100644
--- a/be/test/exprs/bitmap_function_test.cpp
+++ b/be/test/exprs/bitmap_function_test.cpp
@@ -310,8 +310,7 @@ TEST_F(BitmapFunctionsTest, test_bitmap_intersect) {
     date_time_value.to_datetime_val(&datetime2);
     test_bitmap_intersect<DateTimeVal, DateTimeValue>(ctx, datetime1, datetime2);
 
-    test_bitmap_intersect<StringVal, StringValue>(ctx, StringVal("20191211"),
-                                                  StringVal("20191212"));
+    test_bitmap_intersect<StringVal, StringRef>(ctx, StringVal("20191211"), StringVal("20191212"));
 }
 
 TEST_F(BitmapFunctionsTest, bitmap_or) {
diff --git a/be/test/exprs/bloom_filter_predicate_test.cpp b/be/test/exprs/bloom_filter_predicate_test.cpp
index 3f2fec11bc..ef87562b86 100644
--- a/be/test/exprs/bloom_filter_predicate_test.cpp
+++ b/be/test/exprs/bloom_filter_predicate_test.cpp
@@ -20,7 +20,7 @@
 #include "exprs/bloomfilter_predicate.h"
 #include "exprs/create_predicate_function.h"
 #include "gtest/gtest.h"
-#include "runtime/string_value.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 class BloomFilterPredicateTest : public testing::Test {
@@ -55,10 +55,10 @@ TEST_F(BloomFilterPredicateTest, bloom_filter_func_stringval_test) {
     EXPECT_TRUE(func->init(1024, 0.05).ok());
     ObjectPool obj_pool;
     const int data_size = 1024;
-    StringValue data[data_size];
+    StringRef data[data_size];
     for (int i = 0; i < data_size; i++) {
         auto str = obj_pool.add(new std::string(std::to_string(i)));
-        data[i] = StringValue(*str);
+        data[i] = StringRef(*str);
         func->insert((const void*)&data[i]);
     }
     for (int i = 0; i < data_size; i++) {
@@ -66,7 +66,7 @@ TEST_F(BloomFilterPredicateTest, bloom_filter_func_stringval_test) {
     }
     // test not exist value
     std::string not_exist_str = "0x3355ff";
-    StringValue not_exist_val(not_exist_str);
+    StringRef not_exist_val(not_exist_str);
     EXPECT_FALSE(func->find((const void*)&not_exist_val));
 
     // test fixed char
@@ -74,24 +74,24 @@ TEST_F(BloomFilterPredicateTest, bloom_filter_func_stringval_test) {
     EXPECT_TRUE(func->init(1024, 0.05).ok());
 
     auto varchar_true_str = obj_pool.add(new std::string("true"));
-    StringValue varchar_true(*varchar_true_str);
+    StringRef varchar_true(*varchar_true_str);
     func->insert((const void*)&varchar_true);
 
     auto varchar_false_str = obj_pool.add(new std::string("false"));
-    StringValue varchar_false(*varchar_false_str);
+    StringRef varchar_false(*varchar_false_str);
     func->insert((const void*)&varchar_false);
 
-    StringValue fixed_char_true;
+    StringRef fixed_char_true;
     char true_buf[100] = "true";
     memset(true_buf + strlen(true_buf), 0, 100 - strlen(true_buf));
-    fixed_char_true.ptr = true_buf;
-    fixed_char_true.len = 10;
+    fixed_char_true.data = true_buf;
+    fixed_char_true.size = 10;
 
-    StringValue fixed_char_false;
+    StringRef fixed_char_false;
     char false_buf[100] = "false";
     memset(false_buf + strlen(false_buf), 0, 100 - strlen(false_buf));
-    fixed_char_false.ptr = false_buf;
-    fixed_char_false.len = 10;
+    fixed_char_false.data = false_buf;
+    fixed_char_false.size = 10;
 
     EXPECT_TRUE(func->find_olap_engine((const void*)&fixed_char_true));
     EXPECT_TRUE(func->find_olap_engine((const void*)&fixed_char_false));
diff --git a/be/test/exprs/hybrid_set_test.cpp b/be/test/exprs/hybrid_set_test.cpp
index 83e05c6ce3..33ea07d2f1 100644
--- a/be/test/exprs/hybrid_set_test.cpp
+++ b/be/test/exprs/hybrid_set_test.cpp
@@ -278,52 +278,52 @@ TEST_F(HybridSetTest, double) {
 }
 TEST_F(HybridSetTest, string) {
     HybridSetBase* set = create_set(TYPE_VARCHAR);
-    StringValue a;
+    StringRef a;
 
     char buf[100];
 
     snprintf(buf, 100, "abcdefghigk");
-    a.ptr = buf;
+    a.data = buf;
 
-    a.len = 0;
+    a.size = 0;
     set->insert(&a);
-    a.len = 1;
+    a.size = 1;
     set->insert(&a);
-    a.len = 2;
+    a.size = 2;
     set->insert(&a);
-    a.len = 3;
+    a.size = 3;
     set->insert(&a);
-    a.len = 4;
+    a.size = 4;
     set->insert(&a);
-    a.len = 4;
+    a.size = 4;
     set->insert(&a);
 
     EXPECT_EQ(5, set->size());
     HybridSetBase::IteratorBase* base = set->begin();
 
     while (base->has_next()) {
-        LOG(INFO) << ((StringValue*)base->get_value())->ptr;
+        LOG(INFO) << ((StringRef*)base->get_value())->data;
         base->next();
     }
 
-    StringValue b;
+    StringRef b;
 
     char buf1[100];
 
     snprintf(buf1, 100, "abcdefghigk");
-    b.ptr = buf1;
+    b.data = buf1;
 
-    b.len = 0;
+    b.size = 0;
     EXPECT_TRUE(set->find(&b));
-    b.len = 1;
+    b.size = 1;
     EXPECT_TRUE(set->find(&b));
-    b.len = 2;
+    b.size = 2;
     EXPECT_TRUE(set->find(&b));
-    b.len = 3;
+    b.size = 3;
     EXPECT_TRUE(set->find(&b));
-    b.len = 4;
+    b.size = 4;
     EXPECT_TRUE(set->find(&b));
-    b.len = 5;
+    b.size = 5;
     EXPECT_FALSE(set->find(&b));
 }
 TEST_F(HybridSetTest, timestamp) {
diff --git a/be/test/olap/block_column_predicate_test.cpp b/be/test/olap/block_column_predicate_test.cpp
index 49d7c30e5a..b1fc0985d1 100644
--- a/be/test/olap/block_column_predicate_test.cpp
+++ b/be/test/olap/block_column_predicate_test.cpp
@@ -25,8 +25,8 @@
 #include "olap/field.h"
 #include "olap/wrapper_field.h"
 #include "runtime/mem_pool.h"
-#include "runtime/string_value.hpp"
 #include "vec/columns/predicate_column.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
diff --git a/be/test/runtime/string_buffer_test.cpp b/be/test/runtime/string_buffer_test.cpp
index 2064b282c9..2e8d7a1b33 100644
--- a/be/test/runtime/string_buffer_test.cpp
+++ b/be/test/runtime/string_buffer_test.cpp
@@ -30,7 +30,7 @@ void validate_string(const std::string& std_str, const StringBuffer& str) {
     EXPECT_EQ((int)std_str.size(), str.size());
 
     if (std_str.size() > 0) {
-        EXPECT_EQ(strncmp(std_str.c_str(), str.str().ptr, std_str.size()), 0);
+        EXPECT_EQ(strncmp(std_str.c_str(), str.str().data, std_str.size()), 0);
     }
 }
 
diff --git a/be/test/runtime/string_value_test.cpp b/be/test/runtime/string_value_test.cpp
index 5e063929b6..6d41d35bff 100644
--- a/be/test/runtime/string_value_test.cpp
+++ b/be/test/runtime/string_value_test.cpp
@@ -15,25 +15,24 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "runtime/string_value.hpp"
-
 #include <gtest/gtest.h>
 
 #include <string>
 
 #include "util/cpu_info.h"
+#include "vec/common/string_ref.h"
 
 using std::string;
 
 namespace doris {
 
-StringValue FromStdString(const string& str) {
+StringRef FromStdString(const string& str) {
     char* ptr = const_cast<char*>(str.c_str());
     int len = str.size();
-    return StringValue(ptr, len);
+    return StringRef(ptr, len);
 }
 
-TEST(StringValueTest, TestCompare) {
+TEST(StringRefTest, TestCompare) {
     std::string empty_str = "";
     std::string str1_str = "abc";
     std::string str2_str = "abcdef";
@@ -42,7 +41,7 @@ TEST(StringValueTest, TestCompare) {
     const int NUM_STRINGS = 4;
 
     // Must be in lexical order
-    StringValue svs[NUM_STRINGS];
+    StringRef svs[NUM_STRINGS];
     svs[0] = FromStdString(empty_str);
     svs[1] = FromStdString(str1_str);
     svs[2] = FromStdString(str2_str);
diff --git a/be/test/util/array_parser_test.cpp b/be/test/util/array_parser_test.cpp
index c88ef91753..1b0f93823b 100644
--- a/be/test/util/array_parser_test.cpp
+++ b/be/test/util/array_parser_test.cpp
@@ -22,8 +22,8 @@
 
 #include "olap/tablet_schema.h"
 #include "olap/types.h"
-#include "runtime/string_value.h"
 #include "testutil/array_utils.h"
+#include "vec/common/string_ref.h"
 
 namespace doris {
 
@@ -81,7 +81,7 @@ TEST(ArrayParserTest, TestParseVarcharArray) {
 
     char data[] = {'a', 'b', 'c'};
     int num_items = sizeof(data) / sizeof(data[0]);
-    StringValue string_values[] = {
+    StringRef string_values[] = {
             {&data[0], 1},
             {&data[1], 1},
             {&data[2], 1},
diff --git a/be/test/vec/core/block_test.cpp b/be/test/vec/core/block_test.cpp
index a4de5e9129..991a82229e 100644
--- a/be/test/vec/core/block_test.cpp
+++ b/be/test/vec/core/block_test.cpp
@@ -26,13 +26,13 @@
 #include "agent/be_exec_version_manager.h"
 #include "exec/schema_scanner.h"
 #include "gen_cpp/data.pb.h"
-#include "runtime/string_value.h"
 #include "runtime/tuple_row.h"
 #include "vec/columns/column_array.h"
 #include "vec/columns/column_decimal.h"
 #include "vec/columns/column_nullable.h"
 #include "vec/columns/column_string.h"
 #include "vec/columns/column_vector.h"
+#include "vec/common/string_ref.h"
 #include "vec/data_types/data_type.h"
 #include "vec/data_types/data_type_array.h"
 #include "vec/data_types/data_type_bitmap.h"
diff --git a/be/test/vec/exec/parquet/parquet_thrift_test.cpp b/be/test/vec/exec/parquet/parquet_thrift_test.cpp
index f41b2da5d9..c838d11b92 100644
--- a/be/test/vec/exec/parquet/parquet_thrift_test.cpp
+++ b/be/test/vec/exec/parquet/parquet_thrift_test.cpp
@@ -26,9 +26,9 @@
 #include "io/buffered_reader.h"
 #include "io/fs/local_file_system.h"
 #include "olap/iterators.h"
-#include "runtime/string_value.h"
 #include "util/runtime_profile.h"
 #include "util/timezone_utils.h"
+#include "vec/common/string_ref.h"
 #include "vec/core/block.h"
 #include "vec/core/column_with_type_and_name.h"
 #include "vec/data_types/data_type_factory.hpp"
@@ -235,14 +235,14 @@ static void create_block(std::unique_ptr<vectorized::Block>& block) {
             {"boolean_col", TYPE_BOOLEAN, sizeof(bool), true},
             {"float_col", TYPE_FLOAT, sizeof(float_t), true},
             {"double_col", TYPE_DOUBLE, sizeof(double_t), true},
-            {"string_col", TYPE_STRING, sizeof(StringValue), true},
+            {"string_col", TYPE_STRING, sizeof(StringRef), true},
             // binary is not supported, use string instead
-            {"binary_col", TYPE_STRING, sizeof(StringValue), true},
+            {"binary_col", TYPE_STRING, sizeof(StringRef), true},
             // 64-bit-length, see doris::get_slot_size in primitive_type.cpp
             {"timestamp_col", TYPE_DATETIME, sizeof(DateTimeValue), true},
             {"decimal_col", TYPE_DECIMALV2, sizeof(DecimalV2Value), true},
-            {"char_col", TYPE_CHAR, sizeof(StringValue), true},
-            {"varchar_col", TYPE_VARCHAR, sizeof(StringValue), true},
+            {"char_col", TYPE_CHAR, sizeof(StringRef), true},
+            {"varchar_col", TYPE_VARCHAR, sizeof(StringRef), true},
             {"date_col", TYPE_DATE, sizeof(DateTimeValue), true},
             {"date_v2_col", TYPE_DATEV2, sizeof(uint32_t), true},
             {"timestamp_v2_col", TYPE_DATETIMEV2, sizeof(DateTimeValue), true, 18, 0}};
@@ -355,12 +355,12 @@ TEST_F(ParquetThriftReaderTest, group_reader) {
             {"boolean_col", TYPE_BOOLEAN, sizeof(bool), true},
             {"float_col", TYPE_FLOAT, sizeof(float_t), true},
             {"double_col", TYPE_DOUBLE, sizeof(double_t), true},
-            {"string_col", TYPE_STRING, sizeof(StringValue), true},
-            {"binary_col", TYPE_STRING, sizeof(StringValue), true},
+            {"string_col", TYPE_STRING, sizeof(StringRef), true},
+            {"binary_col", TYPE_STRING, sizeof(StringRef), true},
             {"timestamp_col", TYPE_DATETIME, sizeof(DateTimeValue), true},
             {"decimal_col", TYPE_DECIMALV2, sizeof(DecimalV2Value), true},
-            {"char_col", TYPE_CHAR, sizeof(StringValue), true},
-            {"varchar_col", TYPE_VARCHAR, sizeof(StringValue), true},
+            {"char_col", TYPE_CHAR, sizeof(StringRef), true},
+            {"varchar_col", TYPE_VARCHAR, sizeof(StringRef), true},
             {"date_col", TYPE_DATE, sizeof(DateTimeValue), true}};
     int num_cols = sizeof(column_descs) / sizeof(SchemaScanner::ColumnDesc);
     SchemaScanner schema_scanner(column_descs, num_cols);
diff --git a/gensrc/script/gen_functions.py b/gensrc/script/gen_functions.py
index 303e9fda02..bdc83e4b1d 100755
--- a/gensrc/script/gen_functions.py
+++ b/gensrc/script/gen_functions.py
@@ -585,7 +585,7 @@ cc_preamble = '\
 #include "gen_cpp/opcode/functions.h"\n\
 #include "exprs/expr.h"\n\
 #include "exprs/case_expr.h"\n\
-#include "runtime/string_value.hpp"\n\
+#include "vec/common/string_tmp.h"\n\
 #include "runtime/tuple_row.h"\n\
 #include "util/mysql_dtoa.h"\n\
 #include "util/string_parser.hpp"\n\
diff --git a/gensrc/script/gen_vector_functions.py b/gensrc/script/gen_vector_functions.py
index f77742b769..073e2f8655 100755
--- a/gensrc/script/gen_vector_functions.py
+++ b/gensrc/script/gen_vector_functions.py
@@ -393,7 +393,7 @@ cc_preamble = '\
 #include "exprs/case_expr.h"\n\
 #include "exprs/expr.h"\n\
 #include "exprs/in_predicate.h"\n\
-#include "runtime/string_value.hpp"\n\
+#include "vec/common/string_tmp.h"\n\
 #include "runtime/vectorized_row_batch.h"\n\
 #include "util/string_parser.hpp"\n\
 #include <boost/lexical_cast.hpp>\n\


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