You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/04/06 02:44:49 UTC

[incubator-doris] 03/19: [refactor][improvement](type_info) use template and single instance to refactor get type info logic (#8680)

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

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

commit 15a5479f277918099b3d7b25e95910a6ba732f13
Author: caiconghui <55...@users.noreply.github.com>
AuthorDate: Sun Apr 3 10:10:36 2022 +0800

    [refactor][improvement](type_info) use template and single instance to refactor get type info logic (#8680)
    
    1. use const pointer instead of shared_ptr
    2. Restrict array types to support only primitive types and nest up to 9 levels.
---
 be/src/olap/aggregate_func.h                       |  30 ++-
 be/src/olap/column_block.h                         |   2 +-
 be/src/olap/column_vector.cpp                      |  16 +-
 be/src/olap/column_vector.h                        |  14 +-
 be/src/olap/field.h                                |   4 +-
 be/src/olap/olap_common.h                          |   2 +
 be/src/olap/push_handler.cpp                       |   2 +-
 be/src/olap/rowset/segment_v2/binary_dict_page.cpp |   3 +-
 .../olap/rowset/segment_v2/bitmap_index_reader.h   |   6 +-
 .../olap/rowset/segment_v2/bitmap_index_writer.cpp |  12 +-
 .../olap/rowset/segment_v2/bitmap_index_writer.h   |   2 +-
 .../rowset/segment_v2/bloom_filter_index_reader.h  |   6 +-
 .../segment_v2/bloom_filter_index_writer.cpp       |   8 +-
 .../rowset/segment_v2/bloom_filter_index_writer.h  |   2 +-
 be/src/olap/rowset/segment_v2/column_reader.cpp    |   4 +-
 be/src/olap/rowset/segment_v2/column_reader.h      |   6 +-
 be/src/olap/rowset/segment_v2/column_writer.cpp    |   6 +-
 .../rowset/segment_v2/indexed_column_reader.cpp    |   4 +-
 .../olap/rowset/segment_v2/indexed_column_reader.h |   4 +-
 .../rowset/segment_v2/indexed_column_writer.cpp    |   4 +-
 .../olap/rowset/segment_v2/indexed_column_writer.h |   4 +-
 be/src/olap/rowset/segment_v2/zone_map_index.cpp   |   4 +-
 be/src/olap/schema_change.cpp                      |   2 +-
 be/src/olap/tablet_schema.cpp                      |   4 +-
 be/src/olap/types.cpp                              | 269 ++++++++++-----------
 be/src/olap/types.h                                |  34 ++-
 be/test/olap/column_vector_test.cpp                |  12 +-
 .../olap/rowset/segment_v2/bitmap_index_test.cpp   |   2 +-
 .../bloom_filter_index_reader_writer_test.cpp      |   2 +-
 .../segment_v2/column_reader_writer_test.cpp       |   4 +-
 .../olap/rowset/segment_v2/encoding_info_test.cpp  |   8 +-
 be/test/olap/schema_change_test.cpp                |  42 ++--
 be/test/olap/storage_types_test.cpp                |   6 +-
 be/test/tools/benchmark_tool.cpp                   |   2 +-
 34 files changed, 277 insertions(+), 255 deletions(-)

diff --git a/be/src/olap/aggregate_func.h b/be/src/olap/aggregate_func.h
index 9b2ba5254d..02de11b866 100644
--- a/be/src/olap/aggregate_func.h
+++ b/be/src/olap/aggregate_func.h
@@ -98,13 +98,29 @@ struct BaseAggregateFuncs {
         if (src_null) {
             return;
         }
-        if constexpr (field_type == OLAP_FIELD_TYPE_ARRAY) {
-            auto _type_info = get_collection_type_info(sub_type);
-            _type_info->deep_copy(dst->mutable_cell_ptr(), src, mem_pool);
-        } else {
-            // get type at compile time for performance
-            auto _type_info = get_scalar_type_info<field_type>();
-            _type_info->deep_copy(dst->mutable_cell_ptr(), src, mem_pool);
+        const auto* type_info = get_scalar_type_info<field_type>();
+        type_info->deep_copy(dst->mutable_cell_ptr(), src, mem_pool);
+    }
+
+    // Default update do nothing.
+    static void update(RowCursorCell* dst, const RowCursorCell& src, MemPool* mem_pool) {}
+
+    // Default finalize do nothing.
+    static void finalize(RowCursorCell* src, MemPool* mem_pool) {}
+};
+
+template <FieldType sub_type>
+struct BaseAggregateFuncs<OLAP_FIELD_TYPE_ARRAY, sub_type> {
+    static void init(RowCursorCell* dst, const char* src, bool src_null, MemPool* mem_pool,
+                     ObjectPool* agg_pool) {
+        dst->set_is_null(src_null);
+        if (src_null) {
+            return;
+        }
+        // nested array type is unsupported for base aggregate function now
+        if (sub_type != OLAP_FIELD_TYPE_ARRAY) {
+            const auto* type_info = get_collection_type_info<sub_type>();
+            type_info->deep_copy(dst->mutable_cell_ptr(), src, mem_pool);
         }
     }
 
diff --git a/be/src/olap/column_block.h b/be/src/olap/column_block.h
index 54dea1cf54..ca170174d8 100644
--- a/be/src/olap/column_block.h
+++ b/be/src/olap/column_block.h
@@ -36,7 +36,7 @@ class ColumnBlock {
 public:
     ColumnBlock(ColumnVectorBatch* batch, MemPool* pool) : _batch(batch), _pool(pool) {}
 
-    const TypeInfo* type_info() const { return _batch->type_info().get(); }
+    const TypeInfo* type_info() const { return _batch->type_info(); }
     uint8_t* data() const { return _batch->data(); }
     bool is_nullable() const { return _batch->is_nullable(); }
     MemPool* pool() const { return _pool; }
diff --git a/be/src/olap/column_vector.cpp b/be/src/olap/column_vector.cpp
index 33406282fb..ad73b8e067 100644
--- a/be/src/olap/column_vector.cpp
+++ b/be/src/olap/column_vector.cpp
@@ -31,8 +31,8 @@ Status ColumnVectorBatch::resize(size_t new_cap) {
     return Status::OK();
 }
 
-Status ColumnVectorBatch::create(size_t init_capacity, bool is_nullable, std::shared_ptr<const TypeInfo> type_info,
-                                 Field* field,
+Status ColumnVectorBatch::create(size_t init_capacity, bool is_nullable,
+                                 const TypeInfo* type_info, Field* field,
                                  std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
     if (is_scalar_type(type_info->type())) {
         std::unique_ptr<ColumnVectorBatch> local;
@@ -133,14 +133,13 @@ Status ColumnVectorBatch::create(size_t init_capacity, bool is_nullable, std::sh
             }
 
             std::unique_ptr<ColumnVectorBatch> elements;
-            auto array_type_info = dynamic_cast<const ArrayTypeInfo*>(type_info.get());
+            const auto* array_type_info = dynamic_cast<const ArrayTypeInfo*>(type_info);
             RETURN_IF_ERROR(ColumnVectorBatch::create(
                     init_capacity * 2, field->get_sub_field(0)->is_nullable(),
                     array_type_info->item_type_info(), field->get_sub_field(0), &elements));
 
             std::unique_ptr<ColumnVectorBatch> offsets;
-            auto offsets_type_info =
-                    get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_UNSIGNED_INT);
+            const auto* offsets_type_info = get_scalar_type_info<OLAP_FIELD_TYPE_UNSIGNED_INT>();
             RETURN_IF_ERROR(ColumnVectorBatch::create(init_capacity + 1, false, offsets_type_info,
                                                       nullptr, &offsets));
 
@@ -160,8 +159,8 @@ Status ColumnVectorBatch::create(size_t init_capacity, bool is_nullable, std::sh
 }
 
 template <class ScalarType>
-ScalarColumnVectorBatch<ScalarType>::ScalarColumnVectorBatch(std::shared_ptr<const TypeInfo> type_info,
-                                                             bool is_nullable)
+ScalarColumnVectorBatch<ScalarType>::ScalarColumnVectorBatch(
+        const TypeInfo* type_info, bool is_nullable)
         : ColumnVectorBatch(type_info, is_nullable), _data(0) {}
 
 template <class ScalarType>
@@ -176,7 +175,8 @@ Status ScalarColumnVectorBatch<ScalarType>::resize(size_t new_cap) {
     return Status::OK();
 }
 
-ArrayColumnVectorBatch::ArrayColumnVectorBatch(std::shared_ptr<const TypeInfo> type_info, bool is_nullable,
+ArrayColumnVectorBatch::ArrayColumnVectorBatch(const TypeInfo* type_info,
+                                               bool is_nullable,
                                                ScalarColumnVectorBatch<uint32_t>* offsets,
                                                ColumnVectorBatch* elements)
         : ColumnVectorBatch(type_info, is_nullable), _data(0) {
diff --git a/be/src/olap/column_vector.h b/be/src/olap/column_vector.h
index eeeecebced..52f6ed6fbc 100644
--- a/be/src/olap/column_vector.h
+++ b/be/src/olap/column_vector.h
@@ -54,7 +54,7 @@ public:
 // struct that contains column data(null bitmap), data array in sub class.
 class ColumnVectorBatch {
 public:
-    explicit ColumnVectorBatch(std::shared_ptr<const TypeInfo> type_info, bool is_nullable)
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
             : _type_info(type_info),
               _capacity(0),
               _delete_state(DEL_NOT_SATISFIED),
@@ -63,7 +63,7 @@ public:
 
     virtual ~ColumnVectorBatch();
 
-    std::shared_ptr<const TypeInfo> type_info() const { return _type_info; }
+    const TypeInfo* type_info() const { return _type_info; }
 
     size_t capacity() const { return _capacity; }
 
@@ -105,11 +105,11 @@ public:
     // Get thr idx's cell_ptr for write
     virtual uint8_t* mutable_cell_ptr(size_t idx) = 0;
 
-    static Status create(size_t init_capacity, bool is_nullable, std::shared_ptr<const TypeInfo> type_info,
+    static Status create(size_t init_capacity, bool is_nullable, const TypeInfo* type_info,
                          Field* field, std::unique_ptr<ColumnVectorBatch>* column_vector_batch);
 
 private:
-    std::shared_ptr<const TypeInfo> _type_info;
+    const TypeInfo* _type_info;
     size_t _capacity;
     DelCondSatisfied _delete_state;
     const bool _nullable;
@@ -119,7 +119,7 @@ private:
 template <class ScalarCppType>
 class ScalarColumnVectorBatch : public ColumnVectorBatch {
 public:
-    explicit ScalarColumnVectorBatch(std::shared_ptr<const TypeInfo> type_info, bool is_nullable);
+    explicit ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable);
 
     ~ScalarColumnVectorBatch() override;
 
@@ -150,7 +150,7 @@ private:
 class ArrayNullColumnVectorBatch : public ColumnVectorBatch {
 public:
     explicit ArrayNullColumnVectorBatch(ColumnVectorBatch* array)
-            : ColumnVectorBatch(get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_TINYINT), false),
+            : ColumnVectorBatch(get_scalar_type_info<OLAP_FIELD_TYPE_TINYINT>(), false),
               _array(array) {}
 
     ~ArrayNullColumnVectorBatch() override = default;
@@ -177,7 +177,7 @@ private:
 
 class ArrayColumnVectorBatch : public ColumnVectorBatch {
 public:
-    explicit ArrayColumnVectorBatch(std::shared_ptr<const TypeInfo> type_info, bool is_nullable,
+    explicit ArrayColumnVectorBatch(const TypeInfo* type_info, bool is_nullable,
                                     ScalarColumnVectorBatch<uint32_t>* offsets,
                                     ColumnVectorBatch* elements);
     ~ArrayColumnVectorBatch() override;
diff --git a/be/src/olap/field.h b/be/src/olap/field.h
index 770ef82af4..f306e2d427 100644
--- a/be/src/olap/field.h
+++ b/be/src/olap/field.h
@@ -278,7 +278,7 @@ public:
 
     FieldType type() const { return _type_info->type(); }
     FieldAggregationMethod aggregation() const { return _agg_info->agg_method(); }
-    std::shared_ptr<const TypeInfo> type_info() const { return _type_info; }
+    const TypeInfo* type_info() const { return _type_info; }
     bool is_nullable() const { return _is_nullable; }
 
     // similar to `full_encode_ascending`, but only encode part (the first `index_size` bytes) of the value.
@@ -301,7 +301,7 @@ public:
     Field* get_sub_field(int i) { return _sub_fields[i].get(); }
 
 protected:
-    std::shared_ptr<const TypeInfo> _type_info;
+    const TypeInfo* _type_info;
     const AggregateInfo* _agg_info;
     // unit : byte
     // except for strings, other types have fixed lengths
diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h
index 6fa64e68cf..506308139d 100644
--- a/be/src/olap/olap_common.h
+++ b/be/src/olap/olap_common.h
@@ -115,6 +115,8 @@ enum DelCondSatisfied {
     DEL_PARTIAL_SATISFIED = 2, //partially satisfy delete condition
 };
 // Define all data types supported by Field.
+// If new filed_type is defined, not only new TypeInfo may need be defined,
+// but also some functions like get_type_info in types.cpp need to be changed.
 enum FieldType {
     OLAP_FIELD_TYPE_TINYINT = 1, // MYSQL_TYPE_TINY
     OLAP_FIELD_TYPE_UNSIGNED_TINYINT = 2,
diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp
index 6ddaa37de0..dd57cc2dc4 100644
--- a/be/src/olap/push_handler.cpp
+++ b/be/src/olap/push_handler.cpp
@@ -1004,7 +1004,7 @@ OLAPStatus PushBrokerReader::fill_field_row(RowCursorCell* dst, const char* src,
         if (src_null) {
             break;
         }
-        auto type_info = get_type_info(type);
+        const auto* type_info = get_scalar_type_info(type);
         type_info->deep_copy(dst->mutable_cell_ptr(), src, mem_pool);
         break;
     }
diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
index 3c6e623475..28a6e62b36 100644
--- a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
+++ b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
@@ -207,8 +207,7 @@ Status BinaryDictPageDecoder::init() {
     if (_encoding_type == DICT_ENCODING) {
         // copy the codewords into a temporary buffer first
         // And then copy the strings corresponding to the codewords to the destination buffer
-        auto type_info = get_scalar_type_info(OLAP_FIELD_TYPE_INT);
-
+        const auto* type_info = get_scalar_type_info<OLAP_FIELD_TYPE_INT>();
         RETURN_IF_ERROR(ColumnVectorBatch::create(0, false, type_info, nullptr, &_batch));
         _data_page_decoder.reset(_bit_shuffle_ptr = new BitShufflePageDecoder<OLAP_FIELD_TYPE_INT>(_data, _options));
     } else if (_encoding_type == PLAIN_ENCODING) {
diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_reader.h b/be/src/olap/rowset/segment_v2/bitmap_index_reader.h
index 91f7bdd379..29dd26467f 100644
--- a/be/src/olap/rowset/segment_v2/bitmap_index_reader.h
+++ b/be/src/olap/rowset/segment_v2/bitmap_index_reader.h
@@ -41,7 +41,7 @@ class BitmapIndexReader {
 public:
     explicit BitmapIndexReader(const FilePathDesc& path_desc, const BitmapIndexPB* bitmap_index_meta)
             : _path_desc(path_desc), _bitmap_index_meta(bitmap_index_meta) {
-        _typeinfo = get_type_info(OLAP_FIELD_TYPE_VARCHAR);
+        _typeinfo = get_scalar_type_info<OLAP_FIELD_TYPE_VARCHAR>();
     }
 
     Status load(bool use_page_cache, bool kept_in_memory);
@@ -51,13 +51,13 @@ public:
 
     int64_t bitmap_nums() { return _bitmap_column_reader->num_values(); }
 
-    std::shared_ptr<const TypeInfo> type_info() { return _typeinfo; }
+    const TypeInfo* type_info() { return _typeinfo; }
 
 private:
     friend class BitmapIndexIterator;
 
     FilePathDesc _path_desc;
-    std::shared_ptr<const TypeInfo> _typeinfo;
+    const TypeInfo* _typeinfo;
     const BitmapIndexPB* _bitmap_index_meta;
     bool _has_null = false;
     std::unique_ptr<IndexedColumnReader> _dict_column_reader;
diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp b/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp
index 6e6d44b075..c944eea661 100644
--- a/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp
@@ -64,7 +64,7 @@ public:
     using CppType = typename CppTypeTraits<field_type>::CppType;
     using MemoryIndexType = typename BitmapIndexTraits<CppType>::MemoryIndexType;
 
-    explicit BitmapIndexWriterImpl(std::shared_ptr<const TypeInfo> typeinfo)
+    explicit BitmapIndexWriterImpl(const TypeInfo* typeinfo)
             : _typeinfo(typeinfo),
               _reverted_index_size(0),
               _tracker(new MemTracker()),
@@ -114,7 +114,7 @@ public:
             IndexedColumnWriterOptions options;
             options.write_ordinal_index = false;
             options.write_value_index = true;
-            options.encoding = EncodingInfo::get_default_encoding(_typeinfo.get(), true);
+            options.encoding = EncodingInfo::get_default_encoding(_typeinfo, true);
             options.compression = LZ4F;
 
             IndexedColumnWriter dict_column_writer(options, _typeinfo, wblock);
@@ -144,12 +144,12 @@ public:
                 bitmap_sizes.push_back(bitmap_size);
             }
 
-            auto bitmap_typeinfo = get_type_info(OLAP_FIELD_TYPE_OBJECT);
+            const auto* bitmap_typeinfo = get_scalar_type_info<OLAP_FIELD_TYPE_OBJECT>();
 
             IndexedColumnWriterOptions options;
             options.write_ordinal_index = true;
             options.write_value_index = false;
-            options.encoding = EncodingInfo::get_default_encoding(bitmap_typeinfo.get(), false);
+            options.encoding = EncodingInfo::get_default_encoding(bitmap_typeinfo, false);
             // we already store compressed bitmap, use NO_COMPRESSION to save some cpu
             options.compression = NO_COMPRESSION;
 
@@ -179,7 +179,7 @@ public:
     }
 
 private:
-    std::shared_ptr<const TypeInfo> _typeinfo;
+    const TypeInfo* _typeinfo;
     uint64_t _reverted_index_size;
     rowid_t _rid = 0;
     // row id list for null value
@@ -192,7 +192,7 @@ private:
 
 } // namespace
 
-Status BitmapIndexWriter::create(std::shared_ptr<const TypeInfo> typeinfo,
+Status BitmapIndexWriter::create(const TypeInfo* typeinfo,
                                  std::unique_ptr<BitmapIndexWriter>* res) {
     FieldType type = typeinfo->type();
     switch (type) {
diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_writer.h b/be/src/olap/rowset/segment_v2/bitmap_index_writer.h
index 15839de38b..e8868634b5 100644
--- a/be/src/olap/rowset/segment_v2/bitmap_index_writer.h
+++ b/be/src/olap/rowset/segment_v2/bitmap_index_writer.h
@@ -36,7 +36,7 @@ namespace segment_v2 {
 
 class BitmapIndexWriter {
 public:
-    static Status create(std::shared_ptr<const TypeInfo> typeinfo, std::unique_ptr<BitmapIndexWriter>* res);
+    static Status create(const TypeInfo* typeinfo, std::unique_ptr<BitmapIndexWriter>* res);
 
     BitmapIndexWriter() = default;
     virtual ~BitmapIndexWriter() = default;
diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h
index cdf6b9a2a3..460bea6bde 100644
--- a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h
+++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h
@@ -45,7 +45,7 @@ public:
     explicit BloomFilterIndexReader(const FilePathDesc& path_desc,
                                     const BloomFilterIndexPB* bloom_filter_index_meta)
             : _path_desc(path_desc), _bloom_filter_index_meta(bloom_filter_index_meta) {
-        _typeinfo = get_type_info(OLAP_FIELD_TYPE_VARCHAR);
+        _typeinfo = get_scalar_type_info<OLAP_FIELD_TYPE_VARCHAR>();
     }
 
     Status load(bool use_page_cache, bool kept_in_memory);
@@ -53,13 +53,13 @@ public:
     // create a new column iterator.
     Status new_iterator(std::unique_ptr<BloomFilterIndexIterator>* iterator);
 
-    std::shared_ptr<const TypeInfo> type_info() const { return _typeinfo; }
+    const TypeInfo* type_info() const { return _typeinfo; }
 
 private:
     friend class BloomFilterIndexIterator;
 
     FilePathDesc _path_desc;
-    std::shared_ptr<const TypeInfo> _typeinfo;
+    const TypeInfo* _typeinfo;
     const BloomFilterIndexPB* _bloom_filter_index_meta;
     std::unique_ptr<IndexedColumnReader> _bloom_filter_reader;
 };
diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp
index 72485ece21..20d68bf85a 100644
--- a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp
@@ -69,7 +69,7 @@ public:
     using ValueDict = typename BloomFilterTraits<CppType>::ValueDict;
 
     explicit BloomFilterIndexWriterImpl(const BloomFilterOptions& bf_options,
-                                        std::shared_ptr<const TypeInfo> typeinfo)
+                                        const TypeInfo* typeinfo)
             : _bf_options(bf_options),
               _typeinfo(typeinfo),
               _tracker(new MemTracker(-1, "BloomFilterIndexWriterImpl")),
@@ -131,7 +131,7 @@ public:
         meta->set_algorithm(BLOCK_BLOOM_FILTER);
 
         // write bloom filters
-        auto bf_typeinfo = get_scalar_type_info(OLAP_FIELD_TYPE_VARCHAR);
+        const auto* bf_typeinfo = get_scalar_type_info<OLAP_FIELD_TYPE_VARCHAR>();
         IndexedColumnWriterOptions options;
         options.write_ordinal_index = true;
         options.write_value_index = false;
@@ -162,7 +162,7 @@ private:
 
 private:
     BloomFilterOptions _bf_options;
-    std::shared_ptr<const TypeInfo> _typeinfo;
+    const TypeInfo* _typeinfo;
     std::shared_ptr<MemTracker> _tracker;
     MemPool _pool;
     bool _has_null;
@@ -176,7 +176,7 @@ private:
 
 // TODO currently we don't support bloom filter index for tinyint/hll/float/double
 Status BloomFilterIndexWriter::create(const BloomFilterOptions& bf_options,
-                                      std::shared_ptr<const TypeInfo> typeinfo,
+                                      const TypeInfo* typeinfo,
                                       std::unique_ptr<BloomFilterIndexWriter>* res) {
     FieldType type = typeinfo->type();
     switch (type) {
diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h
index e59d8eac97..14e2a126d1 100644
--- a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h
+++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h
@@ -38,7 +38,7 @@ struct BloomFilterOptions;
 
 class BloomFilterIndexWriter {
 public:
-    static Status create(const BloomFilterOptions& bf_options, std::shared_ptr<const TypeInfo> typeinfo,
+    static Status create(const BloomFilterOptions& bf_options, const TypeInfo* typeinfo,
                          std::unique_ptr<BloomFilterIndexWriter>* res);
 
     BloomFilterIndexWriter() = default;
diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp
index 74ee07ec0b..69fc02d8c6 100644
--- a/be/src/olap/rowset/segment_v2/column_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/column_reader.cpp
@@ -104,7 +104,7 @@ Status ColumnReader::init() {
         return Status::NotSupported(
                 strings::Substitute("unsupported typeinfo, type=$0", _meta.type()));
     }
-    RETURN_IF_ERROR(EncodingInfo::get(_type_info.get(), _meta.encoding(), &_encoding_info));
+    RETURN_IF_ERROR(EncodingInfo::get(_type_info, _meta.encoding(), &_encoding_info));
     RETURN_IF_ERROR(get_block_compression_codec(_meta.compression(), &_compress_codec));
 
     for (int i = 0; i < _meta.indexes_size(); i++) {
@@ -388,7 +388,7 @@ Status ArrayFileColumnIterator::init(const ColumnIteratorOptions& opts) {
     if (_array_reader->is_nullable()) {
         RETURN_IF_ERROR(_null_iterator->init(opts));
     }
-    auto offset_type_info = get_scalar_type_info(OLAP_FIELD_TYPE_UNSIGNED_INT);
+    const auto* offset_type_info = get_scalar_type_info<OLAP_FIELD_TYPE_UNSIGNED_INT>();
     RETURN_IF_ERROR(
             ColumnVectorBatch::create(1024, false, offset_type_info, nullptr, &_length_batch));
     return Status::OK();
diff --git a/be/src/olap/rowset/segment_v2/column_reader.h b/be/src/olap/rowset/segment_v2/column_reader.h
index 0c20b75fe9..a8b0263aac 100644
--- a/be/src/olap/rowset/segment_v2/column_reader.h
+++ b/be/src/olap/rowset/segment_v2/column_reader.h
@@ -171,7 +171,7 @@ private:
     uint64_t _num_rows;
     FilePathDesc _path_desc;
 
-    std::shared_ptr<const TypeInfo> _type_info = nullptr; // initialized in init(), may changed by subclasses.
+    const TypeInfo* _type_info = nullptr; // initialized in init(), may changed by subclasses.
     const EncodingInfo* _encoding_info =
             nullptr; // initialized in init(), used for create PageDecoder
     const BlockCompressionCodec* _compress_codec = nullptr; // initialized in init()
@@ -376,7 +376,7 @@ private:
 class DefaultValueColumnIterator : public ColumnIterator {
 public:
     DefaultValueColumnIterator(bool has_default_value, const std::string& default_value,
-                               bool is_nullable, std::shared_ptr<const TypeInfo> type_info,
+                               bool is_nullable, const TypeInfo* type_info,
                                size_t schema_length)
             : _has_default_value(has_default_value),
               _default_value(default_value),
@@ -417,7 +417,7 @@ private:
     bool _has_default_value;
     std::string _default_value;
     bool _is_nullable;
-    std::shared_ptr<const TypeInfo> _type_info;
+    const TypeInfo* _type_info;
     size_t _schema_length;
     bool _is_default_value_null;
     size_t _type_size;
diff --git a/be/src/olap/rowset/segment_v2/column_writer.cpp b/be/src/olap/rowset/segment_v2/column_writer.cpp
index 31837d0686..4c5623250f 100644
--- a/be/src/olap/rowset/segment_v2/column_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/column_writer.cpp
@@ -118,7 +118,7 @@ Status ColumnWriter::create(const ColumnWriterOptions& opts, const TabletColumn*
             length_options.meta->set_unique_id(2);
             length_options.meta->set_type(length_type);
             length_options.meta->set_is_nullable(false);
-            length_options.meta->set_length(get_scalar_type_info(length_type)->size());
+            length_options.meta->set_length(get_scalar_type_info<OLAP_FIELD_TYPE_UNSIGNED_INT>()->size());
             length_options.meta->set_encoding(DEFAULT_ENCODING);
             length_options.meta->set_compression(LZ4F);
 
@@ -145,7 +145,7 @@ Status ColumnWriter::create(const ColumnWriterOptions& opts, const TabletColumn*
                 null_options.meta->set_unique_id(3);
                 null_options.meta->set_type(null_type);
                 null_options.meta->set_is_nullable(false);
-                null_options.meta->set_length(get_scalar_type_info(null_type)->size());
+                null_options.meta->set_length(get_scalar_type_info<OLAP_FIELD_TYPE_TINYINT>()->size());
                 null_options.meta->set_encoding(DEFAULT_ENCODING);
                 null_options.meta->set_compression(LZ4F);
 
@@ -226,7 +226,7 @@ Status ScalarColumnWriter::init() {
     PageBuilder* page_builder = nullptr;
 
     RETURN_IF_ERROR(
-            EncodingInfo::get(get_field()->type_info().get(), _opts.meta->encoding(), &_encoding_info));
+            EncodingInfo::get(get_field()->type_info(), _opts.meta->encoding(), &_encoding_info));
     _opts.meta->set_encoding(_encoding_info->encoding());
     // create page builder
     PageBuilderOptions opts;
diff --git a/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp b/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp
index d370af6166..67a873fae1 100644
--- a/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp
@@ -31,12 +31,12 @@ Status IndexedColumnReader::load(bool use_page_cache, bool kept_in_memory) {
     _use_page_cache = use_page_cache;
     _kept_in_memory = kept_in_memory;
 
-    _type_info = get_type_info((FieldType)_meta.data_type());
+    _type_info = get_scalar_type_info((FieldType)_meta.data_type());
     if (_type_info == nullptr) {
         return Status::NotSupported(
                 strings::Substitute("unsupported typeinfo, type=$0", _meta.data_type()));
     }
-    RETURN_IF_ERROR(EncodingInfo::get(_type_info.get(), _meta.encoding(), &_encoding_info));
+    RETURN_IF_ERROR(EncodingInfo::get(_type_info, _meta.encoding(), &_encoding_info));
     RETURN_IF_ERROR(get_block_compression_codec(_meta.compression(), &_compress_codec));
     _value_key_coder = get_key_coder(_type_info->type());
 
diff --git a/be/src/olap/rowset/segment_v2/indexed_column_reader.h b/be/src/olap/rowset/segment_v2/indexed_column_reader.h
index e2b54c4482..a1030586ab 100644
--- a/be/src/olap/rowset/segment_v2/indexed_column_reader.h
+++ b/be/src/olap/rowset/segment_v2/indexed_column_reader.h
@@ -56,7 +56,7 @@ public:
 
     int64_t num_values() const { return _num_values; }
     const EncodingInfo* encoding_info() const { return _encoding_info; }
-    std::shared_ptr<const TypeInfo> type_info() const { return _type_info; }
+    const TypeInfo* type_info() const { return _type_info; }
     bool support_ordinal_seek() const { return _meta.has_ordinal_index_meta(); }
     bool support_value_seek() const { return _meta.has_value_index_meta(); }
 
@@ -82,7 +82,7 @@ private:
     PageHandle _ordinal_index_page_handle;
     PageHandle _value_index_page_handle;
 
-    std::shared_ptr<const TypeInfo> _type_info = nullptr;
+    const TypeInfo* _type_info = nullptr;
     const EncodingInfo* _encoding_info = nullptr;
     const BlockCompressionCodec* _compress_codec = nullptr;
     const KeyCoder* _value_key_coder = nullptr;
diff --git a/be/src/olap/rowset/segment_v2/indexed_column_writer.cpp b/be/src/olap/rowset/segment_v2/indexed_column_writer.cpp
index 450d12ac6e..088de6940e 100644
--- a/be/src/olap/rowset/segment_v2/indexed_column_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/indexed_column_writer.cpp
@@ -37,7 +37,7 @@ namespace doris {
 namespace segment_v2 {
 
 IndexedColumnWriter::IndexedColumnWriter(const IndexedColumnWriterOptions& options,
-                                         std::shared_ptr<const TypeInfo> typeinfo, fs::WritableBlock* wblock)
+                                         const TypeInfo* typeinfo, fs::WritableBlock* wblock)
         : _options(options),
           _typeinfo(typeinfo),
           _wblock(wblock),
@@ -54,7 +54,7 @@ IndexedColumnWriter::~IndexedColumnWriter() = default;
 
 Status IndexedColumnWriter::init() {
     const EncodingInfo* encoding_info;
-    RETURN_IF_ERROR(EncodingInfo::get(_typeinfo.get(), _options.encoding, &encoding_info));
+    RETURN_IF_ERROR(EncodingInfo::get(_typeinfo, _options.encoding, &encoding_info));
     _options.encoding = encoding_info->encoding();
     // should store more concrete encoding type instead of DEFAULT_ENCODING
     // because the default encoding of a data type can be changed in the future
diff --git a/be/src/olap/rowset/segment_v2/indexed_column_writer.h b/be/src/olap/rowset/segment_v2/indexed_column_writer.h
index cb219a78e9..bcb27f4343 100644
--- a/be/src/olap/rowset/segment_v2/indexed_column_writer.h
+++ b/be/src/olap/rowset/segment_v2/indexed_column_writer.h
@@ -71,7 +71,7 @@ struct IndexedColumnWriterOptions {
 class IndexedColumnWriter {
 public:
     explicit IndexedColumnWriter(const IndexedColumnWriterOptions& options,
-                                 std::shared_ptr<const TypeInfo> typeinfo, fs::WritableBlock* wblock);
+                                 const TypeInfo* typeinfo, fs::WritableBlock* wblock);
 
     ~IndexedColumnWriter();
 
@@ -88,7 +88,7 @@ private:
     Status _flush_index(IndexPageBuilder* index_builder, BTreeMetaPB* meta);
 
     IndexedColumnWriterOptions _options;
-    std::shared_ptr<const TypeInfo> _typeinfo;
+    const TypeInfo* _typeinfo;
     fs::WritableBlock* _wblock;
     // only used for `_first_value`
     std::shared_ptr<MemTracker> _mem_tracker;
diff --git a/be/src/olap/rowset/segment_v2/zone_map_index.cpp b/be/src/olap/rowset/segment_v2/zone_map_index.cpp
index b237887e5a..5137c8fcbb 100644
--- a/be/src/olap/rowset/segment_v2/zone_map_index.cpp
+++ b/be/src/olap/rowset/segment_v2/zone_map_index.cpp
@@ -107,11 +107,11 @@ Status ZoneMapIndexWriter::finish(fs::WritableBlock* wblock, ColumnIndexMetaPB*
     _segment_zone_map.to_proto(meta->mutable_segment_zone_map(), _field);
 
     // write out zone map for each data pages
-    auto typeinfo = get_type_info(OLAP_FIELD_TYPE_OBJECT);
+    const auto* typeinfo = get_scalar_type_info<OLAP_FIELD_TYPE_OBJECT>();
     IndexedColumnWriterOptions options;
     options.write_ordinal_index = true;
     options.write_value_index = false;
-    options.encoding = EncodingInfo::get_default_encoding(typeinfo.get(), false);
+    options.encoding = EncodingInfo::get_default_encoding(typeinfo, false);
     options.compression = NO_COMPRESSION; // currently not compressed
 
     IndexedColumnWriter writer(options, typeinfo, wblock);
diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp
index a0b168d572..eb500a9bc8 100644
--- a/be/src/olap/schema_change.cpp
+++ b/be/src/olap/schema_change.cpp
@@ -590,7 +590,7 @@ OLAPStatus RowBlockChanger::change_row_block(const RowBlock* ref_block, int32_t
                         const Field* ref_field = read_helper.column_schema(ref_column);
                         char* ref_value = read_helper.cell_ptr(ref_column);
                         OLAPStatus st = write_helper.convert_from(i, ref_value,
-                                                                  ref_field->type_info().get(), mem_pool);
+                                                                  ref_field->type_info(), mem_pool);
                         if (st != OLAPStatus::OLAP_SUCCESS) {
                             LOG(WARNING)
                                     << "the column type which was altered from was unsupported."
diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp
index afafd25f0f..782ae1a7ac 100644
--- a/be/src/olap/tablet_schema.cpp
+++ b/be/src/olap/tablet_schema.cpp
@@ -275,7 +275,7 @@ TabletColumn::TabletColumn(FieldAggregationMethod agg, FieldType type) {
 TabletColumn::TabletColumn(FieldAggregationMethod agg, FieldType filed_type, bool is_nullable) {
     _aggregation = agg;
     _type = filed_type;
-    _length = get_type_info(filed_type)->size();
+    _length = get_scalar_type_info(filed_type)->size();
     _is_nullable = is_nullable;
 }
 
@@ -366,7 +366,7 @@ void TabletColumn::to_schema_pb(ColumnPB* column) {
     }
     column->set_visible(_visible);
 
-    if (_type == FieldType::OLAP_FIELD_TYPE_ARRAY) {
+    if (_type == OLAP_FIELD_TYPE_ARRAY) {
         DCHECK(_sub_columns.size() == 1) << "ARRAY type has more than 1 children types.";
         ColumnPB* child = column->add_children_columns();
         _sub_columns[0].to_schema_pb(child);
diff --git a/be/src/olap/types.cpp b/be/src/olap/types.cpp
index 62ebc08f72..b0cbdfdf0b 100644
--- a/be/src/olap/types.cpp
+++ b/be/src/olap/types.cpp
@@ -22,53 +22,6 @@ namespace doris {
 
 void (*FieldTypeTraits<OLAP_FIELD_TYPE_CHAR>::set_to_max)(void*) = nullptr;
 
-class ScalarTypeInfoResolver {
-    DECLARE_SINGLETON(ScalarTypeInfoResolver);
-
-public:
-    std::shared_ptr<const TypeInfo> get_type_info(const FieldType t) {
-        auto pair = _scalar_type_mapping.find(t);
-        DCHECK(pair != _scalar_type_mapping.end()) << "Bad field type: " << t;
-        return pair->second;
-    }
-
-private:
-    template <FieldType field_type>
-    void add_mapping() {
-        TypeTraits<field_type> traits;
-        _scalar_type_mapping.emplace(field_type,
-                                     std::shared_ptr<const TypeInfo>(new ScalarTypeInfo(traits)));
-    }
-
-    std::unordered_map<FieldType, std::shared_ptr<const TypeInfo>, std::hash<size_t>>
-            _scalar_type_mapping;
-
-    DISALLOW_COPY_AND_ASSIGN(ScalarTypeInfoResolver);
-};
-
-ScalarTypeInfoResolver::ScalarTypeInfoResolver() {
-    add_mapping<OLAP_FIELD_TYPE_TINYINT>();
-    add_mapping<OLAP_FIELD_TYPE_SMALLINT>();
-    add_mapping<OLAP_FIELD_TYPE_INT>();
-    add_mapping<OLAP_FIELD_TYPE_UNSIGNED_INT>();
-    add_mapping<OLAP_FIELD_TYPE_BOOL>();
-    add_mapping<OLAP_FIELD_TYPE_BIGINT>();
-    add_mapping<OLAP_FIELD_TYPE_UNSIGNED_BIGINT>();
-    add_mapping<OLAP_FIELD_TYPE_LARGEINT>();
-    add_mapping<OLAP_FIELD_TYPE_FLOAT>();
-    add_mapping<OLAP_FIELD_TYPE_DOUBLE>();
-    add_mapping<OLAP_FIELD_TYPE_DECIMAL>();
-    add_mapping<OLAP_FIELD_TYPE_DATE>();
-    add_mapping<OLAP_FIELD_TYPE_DATETIME>();
-    add_mapping<OLAP_FIELD_TYPE_CHAR>();
-    add_mapping<OLAP_FIELD_TYPE_VARCHAR>();
-    add_mapping<OLAP_FIELD_TYPE_STRING>();
-    add_mapping<OLAP_FIELD_TYPE_HLL>();
-    add_mapping<OLAP_FIELD_TYPE_OBJECT>();
-}
-
-ScalarTypeInfoResolver::~ScalarTypeInfoResolver() {}
-
 bool is_scalar_type(FieldType field_type) {
     switch (field_type) {
     case OLAP_FIELD_TYPE_STRUCT:
@@ -80,109 +33,153 @@ bool is_scalar_type(FieldType field_type) {
     }
 }
 
-std::shared_ptr<const TypeInfo> get_scalar_type_info(FieldType field_type) {
-    return ScalarTypeInfoResolver::instance()->get_type_info(field_type);
-}
-
-class ArrayTypeInfoResolver {
-    DECLARE_SINGLETON(ArrayTypeInfoResolver);
-
-public:
-    std::shared_ptr<const TypeInfo> get_type_info(const FieldType t) {
-        auto pair = _type_mapping.find(t);
-        DCHECK(pair != _type_mapping.end()) << "Bad field type: list<" << t << ">";
-        return pair->second;
-    }
-
-    std::shared_ptr<const TypeInfo> get_type_info(const TabletColumn& column) {
-        DCHECK(column.get_subtype_count() == 1) << "more than 1 child type.";
-        const auto &sub_column = column.get_sub_column(0);
-        if (is_scalar_type(sub_column.type())) {
-            return get_type_info(sub_column.type());
-        } else {
-            return std::make_shared<const ArrayTypeInfo>(get_type_info(sub_column));
-        }
+bool is_olap_string_type(FieldType field_type) {
+    switch (field_type) {
+    case OLAP_FIELD_TYPE_CHAR:
+    case OLAP_FIELD_TYPE_VARCHAR:
+    case OLAP_FIELD_TYPE_HLL:
+    case OLAP_FIELD_TYPE_OBJECT:
+    case OLAP_FIELD_TYPE_STRING:
+        return true;
+    default:
+        return false;
     }
+}
 
-    std::shared_ptr<const TypeInfo> get_type_info(const segment_v2::ColumnMetaPB& column_meta_pb) {
-        DCHECK(column_meta_pb.children_columns_size() >= 1 && column_meta_pb.children_columns_size() <= 3)
-                << "more than 3 children or no children.";
-        const auto& child_type = column_meta_pb.children_columns(0);
-        if (is_scalar_type((FieldType)child_type.type())) {
-            return get_type_info((FieldType)child_type.type());
-        } else {
-            return std::make_shared<const ArrayTypeInfo>(get_type_info(child_type));
-        }
-    }
+const TypeInfo* get_scalar_type_info(FieldType field_type) {
+    // nullptr means that there is no TypeInfo implementation for the corresponding field_type
+    static const TypeInfo* field_type_array[] = {
+        nullptr,
+        get_scalar_type_info<OLAP_FIELD_TYPE_TINYINT>(),
+        nullptr,
+        get_scalar_type_info<OLAP_FIELD_TYPE_SMALLINT>(),
+        nullptr,
+        get_scalar_type_info<OLAP_FIELD_TYPE_INT>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_UNSIGNED_INT>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_BIGINT>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_UNSIGNED_BIGINT>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_LARGEINT>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_FLOAT>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_DOUBLE>(),
+        nullptr,
+        get_scalar_type_info<OLAP_FIELD_TYPE_CHAR>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_DATE>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_DATETIME>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_DECIMAL>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_VARCHAR>(),
+        nullptr,
+        nullptr,
+        nullptr,
+        nullptr,
+        nullptr,
+        get_scalar_type_info<OLAP_FIELD_TYPE_HLL>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_BOOL>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_OBJECT>(),
+        get_scalar_type_info<OLAP_FIELD_TYPE_STRING>(),
+    };
+    return field_type_array[field_type];
+}
 
-private:
-    template <FieldType field_type>
-    void add_mapping() {
-        _type_mapping.emplace(field_type, std::shared_ptr<const TypeInfo>(new ArrayTypeInfo(
-                                                  get_scalar_type_info(field_type))));
+#define INIT_ARRAY_TYPE_INFO_LIST(type)                       \
+    {                                                         \
+        get_init_array_type_info<type>(0),                    \
+        get_init_array_type_info<type>(1),                    \
+        get_init_array_type_info<type>(2),                    \
+        get_init_array_type_info<type>(3),                    \
+        get_init_array_type_info<type>(4),                    \
+        get_init_array_type_info<type>(5),                    \
+        get_init_array_type_info<type>(6),                    \
+        get_init_array_type_info<type>(7),                    \
+        get_init_array_type_info<type>(8)                     \
     }
 
-    // item_type_info -> list_type_info
-    std::unordered_map<FieldType, std::shared_ptr<const TypeInfo>, std::hash<size_t>> _type_mapping;
-};
-
-ArrayTypeInfoResolver::~ArrayTypeInfoResolver() = default;
-
-ArrayTypeInfoResolver::ArrayTypeInfoResolver() {
-    add_mapping<OLAP_FIELD_TYPE_TINYINT>();
-    add_mapping<OLAP_FIELD_TYPE_SMALLINT>();
-    add_mapping<OLAP_FIELD_TYPE_INT>();
-    add_mapping<OLAP_FIELD_TYPE_UNSIGNED_INT>();
-    add_mapping<OLAP_FIELD_TYPE_BOOL>();
-    add_mapping<OLAP_FIELD_TYPE_BIGINT>();
-    add_mapping<OLAP_FIELD_TYPE_LARGEINT>();
-    add_mapping<OLAP_FIELD_TYPE_FLOAT>();
-    add_mapping<OLAP_FIELD_TYPE_DOUBLE>();
-    add_mapping<OLAP_FIELD_TYPE_DECIMAL>();
-    add_mapping<OLAP_FIELD_TYPE_DATE>();
-    add_mapping<OLAP_FIELD_TYPE_DATETIME>();
-    add_mapping<OLAP_FIELD_TYPE_CHAR>();
-    add_mapping<OLAP_FIELD_TYPE_VARCHAR>();
-    add_mapping<OLAP_FIELD_TYPE_STRING>();
-}
-
-// equal to get_scalar_type_info
-std::shared_ptr<const TypeInfo> get_type_info(FieldType field_type) {
-    return get_scalar_type_info(field_type);
+template <FieldType field_type>
+inline const ArrayTypeInfo* get_init_array_type_info(int32_t iterations) {
+    static ArrayTypeInfo nested_type_info_0(get_scalar_type_info<field_type>());
+    static ArrayTypeInfo nested_type_info_1(&nested_type_info_0);
+    static ArrayTypeInfo nested_type_info_2(&nested_type_info_1);
+    static ArrayTypeInfo nested_type_info_3(&nested_type_info_2);
+    static ArrayTypeInfo nested_type_info_4(&nested_type_info_3);
+    static ArrayTypeInfo nested_type_info_5(&nested_type_info_4);
+    static ArrayTypeInfo nested_type_info_6(&nested_type_info_5);
+    static ArrayTypeInfo nested_type_info_7(&nested_type_info_6);
+    static ArrayTypeInfo nested_type_info_8(&nested_type_info_7);
+    static ArrayTypeInfo* nested_type_info_array[] = {
+        &nested_type_info_0,
+        &nested_type_info_1,
+        &nested_type_info_2,
+        &nested_type_info_3,
+        &nested_type_info_4,
+        &nested_type_info_5,
+        &nested_type_info_6,
+        &nested_type_info_7,
+        &nested_type_info_8
+    };
+    return nested_type_info_array[iterations];
 }
 
-// get array array type info
-std::shared_ptr<const TypeInfo> get_collection_type_info(FieldType sub_type) {
-    return ArrayTypeInfoResolver::instance()->get_type_info(sub_type);
+const TypeInfo* get_array_type_info(FieldType leaf_type, int32_t iterations) {
+    DCHECK(iterations <= 8) << "the depth of nested array type should not be larger than 8";
+    static constexpr int32_t depth = 9;
+    static const ArrayTypeInfo* array_type_Info_arr[][depth] = {
+        { nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr },
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_TINYINT),
+        { nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr },
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_SMALLINT),
+        { nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr },
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_INT),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_UNSIGNED_INT),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_BIGINT),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_UNSIGNED_BIGINT),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_LARGEINT),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_FLOAT),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_DOUBLE),
+        { nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr },
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_CHAR),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_DATE),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_DATETIME),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_DECIMAL),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_VARCHAR),
+        { nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr },
+        { nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr },
+        { nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr },
+        { nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr },
+        { nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, nullptr },
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_HLL),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_BOOL),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_OBJECT),
+        INIT_ARRAY_TYPE_INFO_LIST(OLAP_FIELD_TYPE_STRING),
+    };
+    return array_type_Info_arr[leaf_type][iterations];
 }
 
-std::shared_ptr<const TypeInfo> get_type_info(segment_v2::ColumnMetaPB* column_meta_pb) {
-    FieldType type = (FieldType)column_meta_pb->type();
-    if (is_scalar_type(type)) {
-        return get_scalar_type_info(type);
-    } else {
-        switch (type) {
-        case OLAP_FIELD_TYPE_ARRAY: {
-            return ArrayTypeInfoResolver::instance()->get_type_info(*column_meta_pb);
-        }
-        default:
-            DCHECK(false) << "Bad field type: " << type;
-            return nullptr;
+const TypeInfo* get_type_info(segment_v2::ColumnMetaPB* column_meta_pb) {
+    FieldType type = (FieldType) column_meta_pb->type();
+    if (UNLIKELY(type == OLAP_FIELD_TYPE_ARRAY)) {
+        int32_t iterations = 0;
+        const auto* child_column = &column_meta_pb->children_columns(0);
+        while (child_column->type() == OLAP_FIELD_TYPE_ARRAY) {
+            iterations++;
+            child_column = &child_column->children_columns(0);
         }
+        return get_array_type_info((FieldType) child_column->type(), iterations);
+    } else {
+        return get_scalar_type_info(type);
     }
 }
 
-std::shared_ptr<const TypeInfo> get_type_info(const TabletColumn* col) {
-    if (is_scalar_type(col->type())) {
-        return get_scalar_type_info(col->type());
-    } else {
-        switch (col->type()) {
-        case OLAP_FIELD_TYPE_ARRAY:
-            return ArrayTypeInfoResolver::instance()->get_type_info(*col);
-        default:
-            DCHECK(false) << "Bad field type: " << col->type();
-            return nullptr;
+const TypeInfo* get_type_info(const TabletColumn* col) {
+    auto type = col->type();
+    if (UNLIKELY(type == OLAP_FIELD_TYPE_ARRAY)) {
+        int32_t iterations = 0;
+        const auto* child_column = &col->get_sub_column(0);
+        while (child_column->type() == OLAP_FIELD_TYPE_ARRAY) {
+            iterations++;
+            child_column = &child_column->get_sub_column(0);
         }
+        return get_array_type_info(child_column->type(), iterations);
+    } else {
+        return get_scalar_type_info(type);
     }
 }
 
diff --git a/be/src/olap/types.h b/be/src/olap/types.h
index 00c45d782f..88ad282e2f 100644
--- a/be/src/olap/types.h
+++ b/be/src/olap/types.h
@@ -176,7 +176,7 @@ private:
 
 class ArrayTypeInfo : public TypeInfo {
 public:
-    explicit ArrayTypeInfo(std::shared_ptr<const TypeInfo> item_type_info)
+    explicit ArrayTypeInfo(const TypeInfo* item_type_info)
             : _item_type_info(item_type_info), _item_size(item_type_info->size()) {}
     ~ArrayTypeInfo() = default;
     inline bool equal(const void* left, const void* right) const override {
@@ -372,24 +372,20 @@ public:
 
     inline FieldType type() const override { return OLAP_FIELD_TYPE_ARRAY; }
 
-    inline std::shared_ptr<const TypeInfo> item_type_info() const { return _item_type_info; }
+    inline const TypeInfo* item_type_info() const { return _item_type_info; }
 
 private:
-    std::shared_ptr<const TypeInfo> _item_type_info;
+    const TypeInfo* _item_type_info;
     const size_t _item_size;
 };
 
 extern bool is_scalar_type(FieldType field_type);
 
-extern std::shared_ptr<const TypeInfo> get_scalar_type_info(FieldType field_type);
+extern const TypeInfo* get_scalar_type_info(FieldType field_type);
 
-extern std::shared_ptr<const TypeInfo> get_collection_type_info(FieldType sub_type);
+extern const TypeInfo* get_type_info(segment_v2::ColumnMetaPB* column_meta_pb);
 
-extern std::shared_ptr<const TypeInfo> get_type_info(FieldType field_type);
-
-extern std::shared_ptr<const TypeInfo> get_type_info(segment_v2::ColumnMetaPB* column_meta_pb);
-
-extern std::shared_ptr<const TypeInfo> get_type_info(const TabletColumn* col);
+extern const TypeInfo* get_type_info(const TabletColumn* col);
 
 // support following formats when convert varchar to date
 static const std::vector<std::string> DATE_FORMATS {
@@ -1210,10 +1206,22 @@ struct TypeTraits : public FieldTypeTraits<field_type> {
 
 // get ScalarTypeInfo at compile time for performance
 template <FieldType field_type>
-inline TypeInfo* get_scalar_type_info() {
+inline const TypeInfo* get_scalar_type_info() {
     static constexpr TypeTraits<field_type> traits;
-    static auto _scala_type_info = ScalarTypeInfo(traits);
-    return dynamic_cast<TypeInfo*>(&_scala_type_info);
+    static ScalarTypeInfo scalar_type_info(traits);
+    return &scalar_type_info;
+}
+
+template <FieldType field_type>
+inline const TypeInfo* get_collection_type_info() {
+    static ArrayTypeInfo collection_type_info(get_scalar_type_info<field_type>());
+    return &collection_type_info;
+}
+
+// nested array type is unsupported for sub_type of collection
+template <>
+inline const TypeInfo* get_collection_type_info<OLAP_FIELD_TYPE_ARRAY>() {
+    return nullptr;
 }
 
 } // namespace doris
diff --git a/be/test/olap/column_vector_test.cpp b/be/test/olap/column_vector_test.cpp
index cb297cc18a..5e65365f4c 100644
--- a/be/test/olap/column_vector_test.cpp
+++ b/be/test/olap/column_vector_test.cpp
@@ -42,7 +42,7 @@ private:
 };
 
 template <FieldType type>
-void test_read_write_scalar_column_vector(std::shared_ptr<const TypeInfo> type_info, const uint8_t* src_data,
+void test_read_write_scalar_column_vector(const TypeInfo* type_info, const uint8_t* src_data,
                                           size_t data_size) {
     using Type = typename TypeTraits<type>::CppType;
     Type* src = (Type*)src_data;
@@ -72,7 +72,7 @@ void test_read_write_scalar_column_vector(std::shared_ptr<const TypeInfo> type_i
 }
 
 template <FieldType item_type>
-void test_read_write_array_column_vector(std::shared_ptr<const TypeInfo> array_type_info, size_t array_size,
+void test_read_write_array_column_vector(const TypeInfo* array_type_info, size_t array_size,
                                          CollectionValue* result) {
     DCHECK(array_size > 1);
 
@@ -145,8 +145,8 @@ TEST_F(ColumnVectorTest, scalar_column_vector_test) {
         for (int i = 0; i < size; ++i) {
             val[i] = i;
         }
-        auto ti = get_scalar_type_info(OLAP_FIELD_TYPE_TINYINT);
-        test_read_write_scalar_column_vector<OLAP_FIELD_TYPE_TINYINT>(ti, val, size);
+        const auto* type_info = get_scalar_type_info<OLAP_FIELD_TYPE_TINYINT>();
+        test_read_write_scalar_column_vector<OLAP_FIELD_TYPE_TINYINT>(type_info, val, size);
         delete[] val;
     }
     {
@@ -155,7 +155,7 @@ TEST_F(ColumnVectorTest, scalar_column_vector_test) {
         for (int i = 0; i < size; ++i) {
             set_column_value_by_type(OLAP_FIELD_TYPE_CHAR, i, (char*)&char_vals[i], &_pool, 8);
         }
-        auto ti = get_scalar_type_info(OLAP_FIELD_TYPE_CHAR);
+        const auto* ti = get_scalar_type_info<OLAP_FIELD_TYPE_CHAR>();
         test_read_write_scalar_column_vector<OLAP_FIELD_TYPE_CHAR>(ti, (uint8_t*)char_vals, size);
         delete[] char_vals;
     }
@@ -179,7 +179,7 @@ TEST_F(ColumnVectorTest, array_column_vector_test) {
                 array_val[array_index].set_length(3);
             }
         }
-        auto type_info = ArrayTypeInfoResolver::instance()->get_type_info(OLAP_FIELD_TYPE_TINYINT);
+        const auto* type_info = get_collection_type_info<OLAP_FIELD_TYPE_TINYINT>();
         test_read_write_array_column_vector<OLAP_FIELD_TYPE_TINYINT>(type_info, num_array,
                                                                      array_val);
 
diff --git a/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp b/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp
index 62ec3ea393..4a53facdf1 100644
--- a/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp
+++ b/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp
@@ -62,7 +62,7 @@ private:
 template <FieldType type>
 void write_index_file(std::string& filename, const void* values, size_t value_count,
                       size_t null_count, ColumnIndexMetaPB* meta) {
-    auto type_info = get_type_info(type);
+    const auto* type_info = get_scalar_type_info(type);
     {
         std::unique_ptr<fs::WritableBlock> wblock;
         fs::CreateBlockOptions opts(filename);
diff --git a/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp b/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp
index ffcf919c0d..705ce99d1c 100644
--- a/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp
+++ b/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp
@@ -53,7 +53,7 @@ template <FieldType type>
 void write_bloom_filter_index_file(const std::string& file_name, const void* values,
                                    size_t value_count, size_t null_count,
                                    ColumnIndexMetaPB* index_meta) {
-    auto type_info = get_type_info(type);
+    const auto* type_info = get_scalar_type_info<type>();
     using CppType = typename CppTypeTraits<type>::CppType;
     FileUtils::create_dir(dname);
     std::string fname = dname + "/" + file_name;
diff --git a/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp b/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
index 0808b65ff0..90e2aff480 100644
--- a/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
+++ b/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
@@ -462,7 +462,7 @@ TEST_F(ColumnReaderWriterTest, test_array_type) {
 template <FieldType type>
 void test_read_default_value(string value, void* result) {
     using Type = typename TypeTraits<type>::CppType;
-    auto type_info = get_type_info(type);
+    const auto* type_info = get_scalar_type_info(type);
     // read and check
     {
         TabletColumn tablet_column = create_with_default_value<type>(value);
@@ -574,7 +574,7 @@ static vectorized::MutableColumnPtr create_vectorized_column_ptr(FieldType type)
 template <FieldType type>
 void test_v_read_default_value(string value, void* result) {
     using Type = typename TypeTraits<type>::CppType;
-    auto type_info = get_type_info(type);
+    const auto* type_info = get_scalar_type_info(type);
     // read and check
     {
         TabletColumn tablet_column = create_with_default_value<type>(value);
diff --git a/be/test/olap/rowset/segment_v2/encoding_info_test.cpp b/be/test/olap/rowset/segment_v2/encoding_info_test.cpp
index a048db0676..0a6ee9dcf8 100644
--- a/be/test/olap/rowset/segment_v2/encoding_info_test.cpp
+++ b/be/test/olap/rowset/segment_v2/encoding_info_test.cpp
@@ -35,17 +35,17 @@ public:
 };
 
 TEST_F(EncodingInfoTest, normal) {
-    auto type_info = get_scalar_type_info(OLAP_FIELD_TYPE_BIGINT);
+    const auto* type_info = get_scalar_type_info<OLAP_FIELD_TYPE_BIGINT>();
     const EncodingInfo* encoding_info = nullptr;
-    auto status = EncodingInfo::get(type_info.get(), PLAIN_ENCODING, &encoding_info);
+    auto status = EncodingInfo::get(type_info, PLAIN_ENCODING, &encoding_info);
     ASSERT_TRUE(status.ok());
     ASSERT_NE(nullptr, encoding_info);
 }
 
 TEST_F(EncodingInfoTest, no_encoding) {
-    auto type_info = get_scalar_type_info(OLAP_FIELD_TYPE_BIGINT);
+    const auto* type_info = get_scalar_type_info<OLAP_FIELD_TYPE_BIGINT>();
     const EncodingInfo* encoding_info = nullptr;
-    auto status = EncodingInfo::get(type_info.get(), DICT_ENCODING, &encoding_info);
+    auto status = EncodingInfo::get(type_info, DICT_ENCODING, &encoding_info);
     ASSERT_FALSE(status.ok());
 }
 
diff --git a/be/test/olap/schema_change_test.cpp b/be/test/olap/schema_change_test.cpp
index 7acc6ed2c4..cf4c625edd 100644
--- a/be/test/olap/schema_change_test.cpp
+++ b/be/test/olap/schema_change_test.cpp
@@ -231,7 +231,7 @@ public:
         _col_vector.reset(new ColumnVector());
         ASSERT_EQ(_column_reader->next_vector(_col_vector.get(), 1, _mem_pool.get()), OLAP_SUCCESS);
         char* data = reinterpret_cast<char*>(_col_vector->col_data());
-        auto st = read_row.convert_from(0, data, write_row.column_schema(0)->type_info().get(),
+        auto st = read_row.convert_from(0, data, write_row.column_schema(0)->type_info(),
                                         _mem_pool.get());
         ASSERT_EQ(st, expected_st);
         if (st == OLAP_SUCCESS) {
@@ -239,8 +239,8 @@ public:
             ASSERT_TRUE(dst_str.compare(0, expected_val.size(), expected_val) == 0);
         }
 
-        auto tp = get_type_info(OLAP_FIELD_TYPE_HLL);
-        st = read_row.convert_from(0, read_row.cell_ptr(0), tp.get(), _mem_pool.get());
+        const auto* tp = get_scalar_type_info<OLAP_FIELD_TYPE_HLL>();
+        st = read_row.convert_from(0, read_row.cell_ptr(0), tp, _mem_pool.get());
         ASSERT_EQ(st, OLAP_ERR_INVALID_SCHEMA);
     }
 
@@ -275,7 +275,7 @@ public:
         _col_vector.reset(new ColumnVector());
         ASSERT_EQ(_column_reader->next_vector(_col_vector.get(), 1, _mem_pool.get()), OLAP_SUCCESS);
         char* data = reinterpret_cast<char*>(_col_vector->col_data());
-        auto st = read_row.convert_from(0, data, write_row.column_schema(0)->type_info().get(),
+        auto st = read_row.convert_from(0, data, write_row.column_schema(0)->type_info(),
                                         _mem_pool.get());
         ASSERT_EQ(st, expected_st);
         if (st == OLAP_SUCCESS) {
@@ -283,8 +283,8 @@ public:
             ASSERT_TRUE(dst_str.compare(0, value.size(), value) == 0);
         }
 
-        auto tp = get_scalar_type_info(OLAP_FIELD_TYPE_HLL);
-        st = read_row.convert_from(0, read_row.cell_ptr(0), tp.get(), _mem_pool.get());
+        const auto* tp = get_scalar_type_info<OLAP_FIELD_TYPE_HLL>();
+        st = read_row.convert_from(0, read_row.cell_ptr(0), tp, _mem_pool.get());
         ASSERT_EQ(st, OLAP_ERR_INVALID_SCHEMA);
     }
 
@@ -346,7 +346,7 @@ TEST_F(TestColumn, ConvertFloatToDouble) {
     _col_vector.reset(new ColumnVector());
     ASSERT_EQ(_column_reader->next_vector(_col_vector.get(), 2, _mem_pool.get()), OLAP_SUCCESS);
     char* data = reinterpret_cast<char*>(_col_vector->col_data());
-    read_row.convert_from(0, data, write_row.column_schema(0)->type_info().get(), _mem_pool.get());
+    read_row.convert_from(0, data, write_row.column_schema(0)->type_info(), _mem_pool.get());
     //float val1 = *reinterpret_cast<float*>(read_row.cell_ptr(0));
     double val2 = *reinterpret_cast<double*>(read_row.cell_ptr(0));
 
@@ -358,8 +358,8 @@ TEST_F(TestColumn, ConvertFloatToDouble) {
     ASSERT_EQ(v2, 1.234);
 
     //test not support type
-    auto tp = get_scalar_type_info(OLAP_FIELD_TYPE_HLL);
-    OLAPStatus st = read_row.convert_from(0, data, tp.get(), _mem_pool.get());
+    const auto* tp = get_scalar_type_info<OLAP_FIELD_TYPE_HLL>();
+    OLAPStatus st = read_row.convert_from(0, data, tp, _mem_pool.get());
     ASSERT_TRUE(st == OLAP_ERR_INVALID_SCHEMA);
 }
 
@@ -397,13 +397,13 @@ TEST_F(TestColumn, ConvertDatetimeToDate) {
     _col_vector.reset(new ColumnVector());
     ASSERT_EQ(_column_reader->next_vector(_col_vector.get(), 1, _mem_pool.get()), OLAP_SUCCESS);
     char* data = reinterpret_cast<char*>(_col_vector->col_data());
-    read_row.convert_from(0, data, write_row.column_schema(0)->type_info().get(), _mem_pool.get());
+    read_row.convert_from(0, data, write_row.column_schema(0)->type_info(), _mem_pool.get());
     std::string dest_string = read_row.column_schema(0)->to_string(read_row.cell_ptr(0));
     ASSERT_TRUE(strncmp(dest_string.c_str(), "2019-11-25", strlen("2019-11-25")) == 0);
 
     //test not support type
-    auto tp = get_type_info(OLAP_FIELD_TYPE_HLL);
-    OLAPStatus st = read_row.convert_from(0, data, tp.get(), _mem_pool.get());
+    const auto* tp = get_scalar_type_info<OLAP_FIELD_TYPE_HLL>();
+    OLAPStatus st = read_row.convert_from(0, data, tp, _mem_pool.get());
     ASSERT_TRUE(st == OLAP_ERR_INVALID_SCHEMA);
 }
 
@@ -442,13 +442,13 @@ TEST_F(TestColumn, ConvertDateToDatetime) {
     ASSERT_EQ(_column_reader->next_vector(_col_vector.get(), 1, _mem_pool.get()), OLAP_SUCCESS);
     char* data = reinterpret_cast<char*>(_col_vector->col_data());
     read_row.set_field_content(0, data, _mem_pool.get());
-    read_row.convert_from(0, data, write_row.column_schema(0)->type_info().get(), _mem_pool.get());
+    read_row.convert_from(0, data, write_row.column_schema(0)->type_info(), _mem_pool.get());
     std::string dest_string = read_row.column_schema(0)->to_string(read_row.cell_ptr(0));
     ASSERT_TRUE(dest_string.compare("2019-12-04 00:00:00") == 0);
 
     //test not support type
-    auto tp = get_type_info(OLAP_FIELD_TYPE_HLL);
-    OLAPStatus st = read_row.convert_from(0, data, tp.get(), _mem_pool.get());
+    const auto* tp = get_scalar_type_info<OLAP_FIELD_TYPE_HLL>();
+    OLAPStatus st = read_row.convert_from(0, data, tp, _mem_pool.get());
     ASSERT_TRUE(st == OLAP_ERR_INVALID_SCHEMA);
 }
 
@@ -484,13 +484,13 @@ TEST_F(TestColumn, ConvertIntToDate) {
     _col_vector.reset(new ColumnVector());
     ASSERT_EQ(_column_reader->next_vector(_col_vector.get(), 1, _mem_pool.get()), OLAP_SUCCESS);
     char* data = reinterpret_cast<char*>(_col_vector->col_data());
-    read_row.convert_from(0, data, write_row.column_schema(0)->type_info().get(), _mem_pool.get());
+    read_row.convert_from(0, data, write_row.column_schema(0)->type_info(), _mem_pool.get());
     std::string dest_string = read_row.column_schema(0)->to_string(read_row.cell_ptr(0));
     ASSERT_TRUE(strncmp(dest_string.c_str(), "2019-12-05", strlen("2019-12-05")) == 0);
 
     //test not support type
-    auto tp = get_type_info(OLAP_FIELD_TYPE_HLL);
-    OLAPStatus st = read_row.convert_from(0, read_row.cell_ptr(0), tp.get(), _mem_pool.get());
+    const auto* tp = get_scalar_type_info<OLAP_FIELD_TYPE_HLL>();
+    OLAPStatus st = read_row.convert_from(0, read_row.cell_ptr(0), tp, _mem_pool.get());
     ASSERT_TRUE(st == OLAP_ERR_INVALID_SCHEMA);
 }
 
@@ -532,7 +532,7 @@ TEST_F(TestColumn, ConvertVarcharToDate) {
         _col_vector.reset(new ColumnVector());
         ASSERT_EQ(_column_reader->next_vector(_col_vector.get(), 1, _mem_pool.get()), OLAP_SUCCESS);
         char* data = reinterpret_cast<char*>(_col_vector->col_data());
-        read_row.convert_from(0, data, write_row.column_schema(0)->type_info().get(),
+        read_row.convert_from(0, data, write_row.column_schema(0)->type_info(),
                               _mem_pool.get());
         std::string dst_str = read_row.column_schema(0)->to_string(read_row.cell_ptr(0));
         ASSERT_EQ(expected_val, dst_str);
@@ -545,8 +545,8 @@ TEST_F(TestColumn, ConvertVarcharToDate) {
     read_row.init(convert_tablet_schema);
 
     //test not support type
-    auto tp = get_type_info(OLAP_FIELD_TYPE_HLL);
-    OLAPStatus st = read_row.convert_from(0, read_row.cell_ptr(0), tp.get(), _mem_pool.get());
+    const auto* tp = get_scalar_type_info<OLAP_FIELD_TYPE_HLL>();
+    OLAPStatus st = read_row.convert_from(0, read_row.cell_ptr(0), tp, _mem_pool.get());
     ASSERT_EQ(st, OLAP_ERR_INVALID_SCHEMA);
 }
 
diff --git a/be/test/olap/storage_types_test.cpp b/be/test/olap/storage_types_test.cpp
index 2427be1818..764d9b0b0a 100644
--- a/be/test/olap/storage_types_test.cpp
+++ b/be/test/olap/storage_types_test.cpp
@@ -33,7 +33,7 @@ public:
 
 template <FieldType field_type>
 void common_test(typename TypeTraits<field_type>::CppType src_val) {
-    auto type = get_scalar_type_info(field_type);
+    const auto* type = get_scalar_type_info<field_type>();
 
     ASSERT_EQ(field_type, type->type());
     ASSERT_EQ(sizeof(src_val), type->size());
@@ -72,7 +72,7 @@ template <FieldType fieldType>
 void test_char(Slice src_val) {
     Field* field = FieldFactory::create_by_type(fieldType);
     field->_length = src_val.size;
-    auto type = field->type_info();
+    const auto* type = field->type_info();
 
     ASSERT_EQ(field->type(), fieldType);
     ASSERT_EQ(sizeof(src_val), type->size());
@@ -155,7 +155,7 @@ void common_test_array(CollectionValue src_val) {
     TabletColumn item_column(OLAP_FIELD_AGGREGATION_NONE, item_type, true, 0, item_length);
     list_column.add_sub_column(item_column);
 
-    auto array_type = dynamic_cast<const ArrayTypeInfo*>(get_type_info(&list_column).get());
+    const auto* array_type = dynamic_cast<const ArrayTypeInfo*>(get_type_info(&list_column));
     ASSERT_EQ(item_type, array_type->item_type_info()->type());
 
     { // test deep copy
diff --git a/be/test/tools/benchmark_tool.cpp b/be/test/tools/benchmark_tool.cpp
index 3a1dd00275..acd5d86d80 100644
--- a/be/test/tools/benchmark_tool.cpp
+++ b/be/test/tools/benchmark_tool.cpp
@@ -190,7 +190,7 @@ public:
 
             auto tracker = std::make_shared<MemTracker>();
             MemPool pool(tracker.get());
-            auto type_info = get_scalar_type_info(OLAP_FIELD_TYPE_VARCHAR);
+            const auto* type_info = get_scalar_type_info<OLAP_FIELD_TYPE_VARCHAR>();
             std::unique_ptr<ColumnVectorBatch> cvb;
             ColumnVectorBatch::create(num, false, type_info, nullptr, &cvb);
             ColumnBlock column_block(cvb.get(), &pool);


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