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

eldenmoon pushed a commit to branch cs_opt_version-3.1
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/cs_opt_version-3.1 by this 
push:
     new b6a36df7d9d support external meta for variant type (#57719)
b6a36df7d9d is described below

commit b6a36df7d9de6e87f5d6e7edfc15148d6b766b8e
Author: lihangyu <[email protected]>
AuthorDate: Wed Nov 5 14:25:03 2025 +0800

    support external meta for variant type (#57719)
---
 be/src/common/config.cpp                           |   2 +-
 be/src/common/config.h                             |   2 +
 .../olap/rowset/segment_v2/binary_prefix_page.cpp  |   3 +-
 be/src/olap/rowset/segment_v2/column_reader.cpp    |   1 +
 be/src/olap/rowset/segment_v2/column_reader.h      |   6 +
 .../olap/rowset/segment_v2/column_reader_cache.cpp |  47 ++++-
 .../olap/rowset/segment_v2/column_reader_cache.h   |   3 +
 be/src/olap/rowset/segment_v2/segment.cpp          |  44 ++--
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |  16 +-
 be/src/olap/rowset/segment_v2/segment_writer.cpp   |   9 +-
 be/src/olap/rowset/segment_v2/stream_reader.h      |   2 +-
 .../segment_v2/variant/variant_column_reader.cpp   | 197 ++++++++++++-----
 .../segment_v2/variant/variant_column_reader.h     |  39 +++-
 .../segment_v2/variant/variant_ext_meta_writer.cpp | 159 ++++++++++++++
 .../segment_v2/variant/variant_ext_meta_writer.h   |  71 +++++++
 .../variant/variant_external_meta_reader.cpp       | 232 +++++++++++++++++++++
 .../variant/variant_external_meta_reader.h         |  85 ++++++++
 .../segment_v2/variant_column_writer_impl.cpp      |   8 +-
 .../rowset/segment_v2/vertical_segment_writer.cpp  |   8 +
 be/src/service/internal_service.cpp                |   4 +
 be/src/vec/common/schema_util.cpp                  |  28 ++-
 be/src/vec/data_types/data_type_factory.cpp        |   2 +-
 22 files changed, 884 insertions(+), 84 deletions(-)

diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp
index d19f6381028..f1c2a8bb513 100644
--- a/be/src/common/config.cpp
+++ b/be/src/common/config.cpp
@@ -1085,6 +1085,7 @@ DEFINE_mBool(variant_use_cloud_schema_dict_cache, "true");
 DEFINE_mDouble(variant_ratio_of_defaults_as_sparse_column, "1");
 DEFINE_mInt64(variant_threshold_rows_to_estimate_sparse_column, "2048");
 DEFINE_mBool(variant_throw_exeception_on_invalid_json, "false");
+DEFINE_mBool(enable_variant_external_meta, "true");
 
 // block file cache
 DEFINE_Bool(enable_file_cache, "false");
@@ -1210,7 +1211,6 @@ DEFINE_mInt64(hdfs_write_batch_buffer_size_mb, "1"); // 
1MB
 DEFINE_mBool(enable_shrink_memory, "false");
 DEFINE_mInt32(schema_cache_capacity, "1024");
 DEFINE_mInt32(schema_cache_sweep_time_sec, "100");
-
 // max number of segment cache, default -1 for backward compatibility 
fd_number*2/5
 DEFINE_Int32(segment_cache_capacity, "-1");
 DEFINE_Int32(segment_cache_fd_percentage, "20");
diff --git a/be/src/common/config.h b/be/src/common/config.h
index a17768fae6e..5807605d78b 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -1313,6 +1313,8 @@ DECLARE_mBool(variant_use_cloud_schema_dict_cache);
 // Threshold to estimate a column is sparsed
 // Treat invalid json format str as string, instead of throwing exception if 
false
 DECLARE_mBool(variant_throw_exeception_on_invalid_json);
+// Enable variant external meta
+DECLARE_mBool(enable_variant_external_meta);
 
 DECLARE_mBool(enable_merge_on_write_correctness_check);
 // USED FOR DEBUGING
diff --git a/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp 
b/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp
index 34eb14951ae..0e5c1fdba45 100644
--- a/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp
+++ b/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp
@@ -210,7 +210,8 @@ Status 
BinaryPrefixPageDecoder::seek_at_or_after_value(const void* value, bool*
         _cur_pos++;
         auto st = _read_next_value();
         if (st.is<ErrorCode::END_OF_FILE>()) {
-            return Status::Error<ErrorCode::ENTRY_NOT_FOUND>("all value small 
than the value");
+            return Status::Error<ErrorCode::ENTRY_NOT_FOUND, false>(
+                    "all value small than the value");
         }
         if (!st.ok()) {
             return st;
diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp 
b/be/src/olap/rowset/segment_v2/column_reader.cpp
index 4fe1a0cae57..5074421ad30 100644
--- a/be/src/olap/rowset/segment_v2/column_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/column_reader.cpp
@@ -299,6 +299,7 @@ ColumnReader::ColumnReader(const ColumnReaderOptions& opts, 
const ColumnMetaPB&
     if (_meta_type == FieldType::OLAP_FIELD_TYPE_ARRAY) {
         _meta_children_column_type = 
(FieldType)meta.children_columns(0).type();
     }
+    _data_type = 
vectorized::DataTypeFactory::instance().create_data_type(meta);
     _meta_is_nullable = meta.is_nullable();
     _meta_dict_page = meta.dict_page();
     _meta_compression = meta.compression();
diff --git a/be/src/olap/rowset/segment_v2/column_reader.h 
b/be/src/olap/rowset/segment_v2/column_reader.h
index 76dd06ccbbc..ad423d44fae 100644
--- a/be/src/olap/rowset/segment_v2/column_reader.h
+++ b/be/src/olap/rowset/segment_v2/column_reader.h
@@ -234,6 +234,8 @@ public:
 
     void disable_index_meta_cache() { _use_index_page_cache = false; }
 
+    vectorized::DataTypePtr get_vec_data_type() { return _data_type; }
+
     virtual FieldType get_meta_type() { return _meta_type; }
 
     int64_t get_metadata_size() const override;
@@ -281,6 +283,8 @@ private:
     Status _calculate_row_ranges(const std::vector<uint32_t>& page_indexes, 
RowRanges* row_ranges,
                                  const ColumnIteratorOptions& iter_opts);
 
+    friend class VariantColumnReader;
+
 private:
     int64_t _meta_length;
     FieldType _meta_type;
@@ -299,6 +303,8 @@ private:
 
     DictEncodingType _dict_encoding_type;
 
+    vectorized::DataTypePtr _data_type;
+
     TypeInfoPtr _type_info =
             TypeInfoPtr(nullptr, nullptr); // initialized in init(), may 
changed by subclasses.
     const EncodingInfo* _encoding_info =
diff --git a/be/src/olap/rowset/segment_v2/column_reader_cache.cpp 
b/be/src/olap/rowset/segment_v2/column_reader_cache.cpp
index 30b6e9528f5..34cedc1f338 100644
--- a/be/src/olap/rowset/segment_v2/column_reader_cache.cpp
+++ b/be/src/olap/rowset/segment_v2/column_reader_cache.cpp
@@ -87,6 +87,20 @@ Status ColumnReaderCache::_insert(const 
ColumnReaderCacheKey& key, const ColumnR
     return Status::OK();
 }
 
+void ColumnReaderCache::_insert_direct(const ColumnReaderCacheKey& key,
+                                       const std::shared_ptr<ColumnReader>& 
column_reader) {
+    std::lock_guard<std::mutex> lock(_cache_mutex);
+    if (_cache_map.size() >= config::max_segment_partial_column_cache_size) {
+        g_segment_column_cache_evict_count << 1;
+        auto last_it = _lru_list.end();
+        --last_it;
+        _cache_map.erase(last_it->key);
+        _lru_list.pop_back();
+    }
+    _lru_list.push_front(CacheNode {key, column_reader, 
std::chrono::steady_clock::now()});
+    _cache_map[key] = _lru_list.begin();
+}
+
 std::map<int32_t, std::shared_ptr<ColumnReader>> 
ColumnReaderCache::get_available_readers(
         bool include_subcolumns) {
     std::lock_guard<std::mutex> lock(_cache_mutex);
@@ -153,9 +167,11 @@ Status ColumnReaderCache::get_path_column_reader(uint32_t 
col_uid,
                                  ->get_subcolumn_meta_by_path(relative_path)
                        : nullptr;
     }
-    if (node != nullptr) {
+
+    // if footer ordinal is valid, try to get the column reader from footer
+    // otherwise, try to get the column reader from external meta
+    if (node != nullptr && node->data.footer_ordinal >= 0) {
         // lazy create column reader from footer
-        DCHECK_GE(node->data.footer_ordinal, 0);
         std::shared_ptr<SegmentFooterPB> footer_pb_shared;
         {
             std::lock_guard<std::mutex> lock(_cache_mutex);
@@ -177,7 +193,32 @@ Status ColumnReaderCache::get_path_column_reader(uint32_t 
col_uid,
                                 _segment->num_rows(), column_reader));
         return Status::OK();
     }
-    // no such column in this segment, return nullptr
+
+    // Fallback: try external meta directly when node is not found but ext 
meta exists.
+    {
+        // Ensure variant root reader is available
+        if (!variant_column_reader) {
+            RETURN_IF_ERROR(get_column_reader(col_uid, &variant_column_reader, 
stats));
+        }
+        ColumnReaderOptions ropts {.kept_in_memory = 
_segment->tablet_schema()->is_in_memory(),
+                                   .be_exec_version = _be_exec_version,
+                                   .tablet_schema = _segment->tablet_schema()};
+        std::shared_ptr<ColumnReader> ext_reader;
+        Status st = 
static_cast<VariantColumnReader*>(variant_column_reader.get())
+                            
->create_reader_from_external_meta(relative_path.get_path(), ropts,
+                                                               
_segment->_file_reader,
+                                                               
_segment->num_rows(), &ext_reader);
+        if (!st.is<ErrorCode::NOT_FOUND>() && !st.ok()) {
+            return st;
+        }
+        if (st.ok()) {
+            _insert_direct({col_uid, relative_path}, ext_reader);
+            *column_reader = std::move(ext_reader);
+            return Status::OK();
+        }
+    }
+
+    // no such column
     *column_reader = nullptr;
     return Status::OK();
 }
diff --git a/be/src/olap/rowset/segment_v2/column_reader_cache.h 
b/be/src/olap/rowset/segment_v2/column_reader_cache.h
index ce4060a5f26..1825e4fca56 100644
--- a/be/src/olap/rowset/segment_v2/column_reader_cache.h
+++ b/be/src/olap/rowset/segment_v2/column_reader_cache.h
@@ -65,6 +65,9 @@ private:
                    const SegmentFooterPB& footer, int32_t column_id,
                    const io::FileReaderSPtr& file_reader, size_t num_rows,
                    std::shared_ptr<ColumnReader>* column_reader);
+    // Insert an already-created reader directly into cache
+    void _insert_direct(const ColumnReaderCacheKey& key,
+                        const std::shared_ptr<ColumnReader>& column_reader);
     // keep _lru_list and _cache_map thread safe
     std::mutex _cache_mutex;
     // Doubly-linked list to maintain LRU order
diff --git a/be/src/olap/rowset/segment_v2/segment.cpp 
b/be/src/olap/rowset/segment_v2/segment.cpp
index 4fd5f2755f9..1fbffe95f8a 100644
--- a/be/src/olap/rowset/segment_v2/segment.cpp
+++ b/be/src/olap/rowset/segment_v2/segment.cpp
@@ -580,19 +580,20 @@ vectorized::DataTypePtr Segment::get_data_type_of(const 
TabletColumn& column,
 
     // Path exists, proceed with variant logic.
     vectorized::PathInData relative_path = path->copy_pop_front();
-    int32_t unique_id = column.unique_id() > 0 ? column.unique_id() : 
column.parent_unique_id();
+    int32_t unique_id = column.unique_id() >= 0 ? column.unique_id() : 
column.parent_unique_id();
 
     // Find the reader for the base variant column.
     if (!_column_uid_to_footer_ordinal.contains(unique_id)) {
         return 
vectorized::DataTypeFactory::instance().create_data_type(column);
     }
 
-    std::shared_ptr<ColumnReader> reader;
+    std::shared_ptr<ColumnReader> v_reader;
+
     // get the parent variant column reader
     OlapReaderStatistics stats;
     // If status is not ok, it will throw exception(data corruption)
-    THROW_IF_ERROR(get_column_reader(unique_id, &reader, &stats));
-    const auto* variant_reader = static_cast<const 
VariantColumnReader*>(reader.get());
+    THROW_IF_ERROR(get_column_reader(unique_id, &v_reader, &stats));
+    const auto* variant_reader = static_cast<const 
VariantColumnReader*>(v_reader.get());
 
     // Find the specific node within the variant structure using the relative 
path.
     const auto* node = 
variant_reader->get_subcolumn_meta_by_path(relative_path);
@@ -601,11 +602,27 @@ vectorized::DataTypePtr Segment::get_data_type_of(const 
TabletColumn& column,
         return 
vectorized::DataTypeFactory::instance().create_data_type(column);
     }
 
-    // Case 1: Node not found for the given path within the variant reader.
-    // If relative_path is empty, it means the original path pointed to the 
root
-    // of the variant column itself. We should return the Variant type.
+    // Use variant type when the path is a prefix of any existing subcolumn 
path.
+    if (variant_reader->has_prefix_path(relative_path)) {
+        return 
vectorized::DataTypeFactory::instance().create_data_type(column);
+    }
+
+    // try to get the reader from cache and return it's data type
+    // usually when leaf node is in cache
+    if (_column_reader_cache->get_path_column_reader(unique_id, relative_path, 
&v_reader,
+                                                     nullptr) &&
+        v_reader != nullptr) {
+        return v_reader->get_vec_data_type();
+    }
+
+    // Node not found for the given path within the variant reader.
     // If node is nullptr, it means the path is not exist in the variant sub 
columns.
-    if (node == nullptr || relative_path.empty()) {
+    if (node == nullptr) {
+        // nested subcolumn is not exist in the sparse column
+        if (column.is_nested_subcolumn()) {
+            return 
vectorized::DataTypeFactory::instance().create_data_type(column);
+        }
+
         // when the path is in the sparse column or exceeded the limit, return 
the variant type.
         if (variant_reader->exist_in_sparse_column(relative_path) ||
             variant_reader->is_exceeded_sparse_column_limit()) {
@@ -624,19 +641,18 @@ vectorized::DataTypePtr Segment::get_data_type_of(const 
TabletColumn& column,
         }
     }
 
-    bool exist_in_sparse = 
variant_reader->exist_in_sparse_column(relative_path);
-    bool is_physical_leaf = node->children.empty();
-
-    if (is_physical_leaf && column.is_nested_subcolumn()) {
+    if (column.is_nested_subcolumn()) {
         return node->data.file_column_type;
     }
 
+    bool exist_in_sparse = 
variant_reader->exist_in_sparse_column(relative_path);
+
     // Condition to return the specific underlying type of the node:
     // 1. We are reading flat leaves (ignoring hierarchy).
     // 2. OR It's a leaf in the physical column structure AND it doesn't 
*also* exist
     //    in the sparse column (meaning it's purely a materialized leaf).
-    if (read_flat_leaves || (is_physical_leaf && !exist_in_sparse &&
-                             
!variant_reader->is_exceeded_sparse_column_limit())) {
+    if (read_flat_leaves ||
+        (!exist_in_sparse && 
!variant_reader->is_exceeded_sparse_column_limit())) {
         return node->data.file_column_type;
     }
 
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp 
b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index facfba00027..4622576fa9f 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -306,8 +306,6 @@ Status SegmentIterator::_init_impl(const 
StorageReadOptions& opts) {
         _record_rowids = true;
     }
 
-    RETURN_IF_ERROR(init_iterators());
-
     if (opts.output_columns != nullptr) {
         _output_columns = *(opts.output_columns);
     }
@@ -345,6 +343,8 @@ Status SegmentIterator::_init_impl(const 
StorageReadOptions& opts) {
         }
     }
 
+    RETURN_IF_ERROR(init_iterators());
+
     RETURN_IF_ERROR(_construct_compound_expr_context());
     _enable_common_expr_pushdown = !_common_expr_ctxs_push_down.empty();
     _initialize_predicate_results();
@@ -1089,6 +1089,8 @@ Status SegmentIterator::_init_inverted_index_iterators() {
             // We use this column to locate the metadata for the inverted 
index, which requires a unique_id and path.
             const auto& column = _opts.tablet_schema->column(cid);
             std::vector<const TabletIndex*> inverted_indexs;
+            // Keep shared_ptr alive to prevent use-after-free when accessing 
raw pointers
+            TabletIndexes inverted_indexs_holder;
             // If the column is an extracted column, we need to find the 
sub-column in the parent column reader.
             std::shared_ptr<ColumnReader> column_reader;
             if (column.is_extracted_column()) {
@@ -1097,8 +1099,14 @@ Status SegmentIterator::_init_inverted_index_iterators() 
{
                     column_reader == nullptr) {
                     continue;
                 }
-                inverted_indexs = 
assert_cast<VariantColumnReader*>(column_reader.get())
-                                          
->find_subcolumn_tablet_indexes(column.suffix_path());
+                inverted_indexs_holder =
+                        assert_cast<VariantColumnReader*>(column_reader.get())
+                                ->find_subcolumn_tablet_indexes(column,
+                                                                
_storage_name_and_type[cid].second);
+                // Extract raw pointers from shared_ptr for iteration
+                for (const auto& index_ptr : inverted_indexs_holder) {
+                    inverted_indexs.push_back(index_ptr.get());
+                }
             }
             // If the column is not an extracted column, we can directly get 
the inverted index metadata from the tablet schema.
             else {
diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp 
b/be/src/olap/rowset/segment_v2/segment_writer.cpp
index 3046871864a..52d3284d10a 100644
--- a/be/src/olap/rowset/segment_v2/segment_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp
@@ -50,6 +50,7 @@
 #include "olap/rowset/segment_v2/inverted_index_writer.h"
 #include "olap/rowset/segment_v2/page_io.h"
 #include "olap/rowset/segment_v2/page_pointer.h"
+#include "olap/rowset/segment_v2/variant/variant_ext_meta_writer.h"
 #include "olap/rowset/segment_v2/variant_stats_calculator.h"
 #include "olap/segment_loader.h"
 #include "olap/short_key_index.h"
@@ -73,7 +74,6 @@
 #include "vec/jsonb/serialize.h"
 #include "vec/olap/olap_data_convertor.h"
 #include "vec/runtime/vdatetime_value.h"
-
 namespace doris {
 namespace segment_v2 {
 
@@ -1208,6 +1208,13 @@ Status SegmentWriter::_write_primary_key_index() {
 Status SegmentWriter::_write_footer() {
     _footer.set_num_rows(_row_count);
 
+    if (config::enable_variant_external_meta) {
+        // Externalize variant subcolumns into ext meta and prune them from 
footer.columns.
+        auto variant_ext_meta_agg =
+                std::make_unique<VariantExtMetaWriter>(_file_writer, 
_opts.compression_type);
+        
RETURN_IF_ERROR(variant_ext_meta_agg->externalize_from_footer(&_footer));
+    }
+
     // Footer := SegmentFooterPB, FooterPBSize(4), FooterPBChecksum(4), 
MagicNumber(4)
     std::string footer_buf;
     VLOG_DEBUG << "footer " << _footer.DebugString();
diff --git a/be/src/olap/rowset/segment_v2/stream_reader.h 
b/be/src/olap/rowset/segment_v2/stream_reader.h
index 134792b1ffd..43f95e29306 100644
--- a/be/src/olap/rowset/segment_v2/stream_reader.h
+++ b/be/src/olap/rowset/segment_v2/stream_reader.h
@@ -19,7 +19,7 @@
 
 #include <memory>
 
-// #include "olap/rowset/segment_v2/column_reader.h"
+#include "olap/rowset/segment_v2/column_reader.h"
 #include "vec/columns/column.h"
 #include "vec/columns/subcolumn_tree.h"
 #include "vec/data_types/data_type.h"
diff --git a/be/src/olap/rowset/segment_v2/variant/variant_column_reader.cpp 
b/be/src/olap/rowset/segment_v2/variant/variant_column_reader.cpp
index f4a1431bcfe..abe8aefabea 100644
--- a/be/src/olap/rowset/segment_v2/variant/variant_column_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/variant/variant_column_reader.cpp
@@ -28,15 +28,18 @@
 #include "io/fs/file_reader.h"
 #include "olap/rowset/segment_v2/column_reader.h"
 #include "olap/rowset/segment_v2/column_reader_cache.h"
+#include "olap/rowset/segment_v2/indexed_column_reader.h"
 #include "olap/rowset/segment_v2/page_handle.h"
 #include "olap/rowset/segment_v2/segment.h"
 #include "olap/rowset/segment_v2/variant/hierarchical_data_iterator.h"
 #include "olap/rowset/segment_v2/variant/sparse_column_extract_iterator.h"
 #include "olap/rowset/segment_v2/variant/sparse_column_merge_iterator.h"
 #include "olap/tablet_schema.h"
+#include "util/slice.h"
 #include "vec/columns/column_array.h"
 #include "vec/columns/column_nullable.h"
 #include "vec/columns/column_object.h"
+#include "vec/columns/column_string.h"
 #include "vec/common/assert_cast.h"
 #include "vec/common/schema_util.h"
 #include "vec/data_types/data_type_factory.hpp"
@@ -114,6 +117,10 @@ Status 
VariantColumnReader::_create_hierarchical_reader(ColumnIteratorUPtr* read
                                                         const 
SubcolumnColumnMetaInfo::Node* root,
                                                         ColumnReaderCache* 
column_reader_cache,
                                                         OlapReaderStatistics* 
stats) {
+    // make sure external meta is loaded otherwise can't find any meta data 
for extracted columns
+    // TODO(lhy): this will load all external meta if not loaded, and memory 
will be consumed.
+    RETURN_IF_ERROR(load_external_meta_once());
+
     // Node contains column with children columns or has correspoding sparse 
columns
     // Create reader with hirachical data.
     std::unique_ptr<SubstreamIterator> sparse_iter;
@@ -124,6 +131,9 @@ Status 
VariantColumnReader::_create_hierarchical_reader(ColumnIteratorUPtr* read
         sparse_iter = std::make_unique<SubstreamIterator>(
                 vectorized::ColumnObject::create_sparse_column_fn(), 
std::move(iter), nullptr);
     }
+    if (node == nullptr) {
+        node = _subcolumns_meta_info->find_exact(path);
+    }
     // If read the full path of variant read in MERGE_ROOT, otherwise 
READ_DIRECT
     HierarchicalDataIterator::ReadType read_type =
             (path == root->path) ? 
HierarchicalDataIterator::ReadType::MERGE_ROOT
@@ -229,6 +239,9 @@ Status 
VariantColumnReader::_new_iterator_with_flat_leaves(ColumnIteratorUPtr* i
                                                            bool 
exceeded_sparse_column_limit,
                                                            bool 
existed_in_sparse_column,
                                                            ColumnReaderCache* 
column_reader_cache) {
+    // make sure external meta is loaded otherwise can't find any meta data 
for extracted columns
+    RETURN_IF_ERROR(load_external_meta_once());
+
     DCHECK(opts != nullptr);
     auto relative_path = target_col.path_info_ptr()->copy_pop_front();
     // compaction need to read flat leaves nodes data to prevent from 
amplification
@@ -289,6 +302,43 @@ Status 
VariantColumnReader::_new_iterator_with_flat_leaves(ColumnIteratorUPtr* i
     return Status::OK();
 }
 
+bool VariantColumnReader::has_prefix_path(const vectorized::PathInData& 
relative_path) const {
+    if (relative_path.empty()) {
+        return true;
+    }
+    const std::string path = relative_path.get_path();
+    const std::string dot_prefix = relative_path.get_path() + ".";
+
+    // 1) exact node exists and has children.
+    if (const auto* node = _subcolumns_meta_info->find_exact(relative_path)) {
+        if (!node->children.empty()) {
+            return true;
+        }
+    }
+
+    // 2) Check sparse column stats: use lower_bound to test the `p.` prefix 
range
+    // example sparse columns path: a.b.c, a.b.e, access prefix: a.b.
+    // then we must read the sparse columns
+    if (_statistics && !_statistics->sparse_column_non_null_size.empty()) {
+        auto it = 
_statistics->sparse_column_non_null_size.lower_bound(dot_prefix);
+        if (it != _statistics->sparse_column_non_null_size.end() &&
+            it->first.starts_with(dot_prefix)) {
+            return true;
+        }
+    }
+
+    // 3) Check external meta store (if available).
+    if (_ext_meta_reader && _ext_meta_reader->available()) {
+        bool has = false;
+        // Pass strict prefix `p.` to avoid false positives like `a.b` 
matching `a.bc`.
+        if (_ext_meta_reader->has_prefix(dot_prefix, &has).ok() && has) {
+            return true;
+        }
+    }
+
+    return false;
+}
+
 Status VariantColumnReader::new_iterator(ColumnIteratorUPtr* iterator,
                                          const TabletColumn* target_col,
                                          const StorageReadOptions* opt) {
@@ -319,8 +369,7 @@ Status 
VariantColumnReader::new_iterator(ColumnIteratorUPtr* iterator,
     // Check if path exist in sparse column
     bool existed_in_sparse_column =
             !_statistics->sparse_column_non_null_size.empty() &&
-            
_statistics->sparse_column_non_null_size.find(relative_path.get_path()) !=
-                    _statistics->sparse_column_non_null_size.end();
+            
_statistics->sparse_column_non_null_size.contains(relative_path.get_path());
 
     // Otherwise the prefix is not exist and the sparse column size is reached 
limit
     // which means the path maybe exist in sparse_column
@@ -346,15 +395,8 @@ Status 
VariantColumnReader::new_iterator(ColumnIteratorUPtr* iterator,
     }
 
     // Check if path is prefix, example sparse columns path: a.b.c, a.b.e, 
access prefix: a.b.
-    // then we must read the sparse columns
-    const std::string& prefix = relative_path.get_path() + ".";
-    bool prefix_existed_in_sparse_column =
-            !_statistics->sparse_column_non_null_size.empty() &&
-            (_statistics->sparse_column_non_null_size.lower_bound(prefix) !=
-             _statistics->sparse_column_non_null_size.end()) &&
-            
_statistics->sparse_column_non_null_size.lower_bound(prefix)->first.starts_with(prefix);
-    // if prefix exists in sparse column, read sparse column with hierarchical 
reader
-    if (prefix_existed_in_sparse_column || exceeded_sparse_column_limit) {
+    // Or access root path
+    if (has_prefix_path(relative_path)) {
         // Example {"b" : {"c":456,"e":7.111}}
         // b.c is sparse column, b.e is subcolumn, so b is both the prefix of 
sparse column and subcolumn
         return _create_hierarchical_reader(iterator, col_uid, relative_path, 
node, root,
@@ -376,21 +418,39 @@ Status 
VariantColumnReader::new_iterator(ColumnIteratorUPtr* iterator,
         return Status::OK();
     }
 
+    // read leaf node
     if (node != nullptr) {
-        // relative_path means the root node, should always use 
HierarchicalDataIterator
-        if (node->is_leaf_node() && !relative_path.empty()) {
-            // Node contains column without any child sub columns and no 
corresponding sparse columns
-            // Direct read extracted columns
-            const auto* node = _subcolumns_meta_info->find_leaf(relative_path);
-            std::shared_ptr<ColumnReader> column_reader;
-            RETURN_IF_ERROR(column_reader_cache->get_path_column_reader(
-                    col_uid, node->path, &column_reader, opt->stats, node));
-            RETURN_IF_ERROR(column_reader->new_iterator(iterator, nullptr));
-        } else {
-            RETURN_IF_ERROR(_create_hierarchical_reader(iterator, col_uid, 
relative_path, node,
-                                                        root, 
column_reader_cache, opt->stats));
-        }
+        DCHECK(node->is_leaf_node());
+        // Node contains column without any child sub columns and no 
corresponding sparse columns
+        // Direct read extracted columns
+        const auto* leaf_node = 
_subcolumns_meta_info->find_leaf(relative_path);
+        std::shared_ptr<ColumnReader> leaf_column_reader;
+        RETURN_IF_ERROR(column_reader_cache->get_path_column_reader(
+                col_uid, leaf_node->path, &leaf_column_reader, opt->stats, 
leaf_node));
+        RETURN_IF_ERROR(leaf_column_reader->new_iterator(iterator, nullptr));
     } else {
+        if (_ext_meta_reader && _ext_meta_reader->available()) {
+            // Get path reader from external meta
+            std::shared_ptr<ColumnReader> leaf_column_reader;
+            Status st = column_reader_cache->get_path_column_reader(
+                    col_uid, relative_path, &leaf_column_reader, opt->stats, 
nullptr);
+            DCHECK(!has_prefix_path(relative_path));
+            RETURN_IF_ERROR(st);
+            if (st.ok() && leaf_column_reader != nullptr) {
+                // Try external meta fallback: build a leaf reader on demand 
from externalized meta
+                RETURN_IF_ERROR(leaf_column_reader->new_iterator(iterator, 
nullptr));
+                return Status::OK();
+            }
+            // if (!st.is<ErrorCode::NOT_FOUND>()) {
+            //     return st;
+            // }
+            // not found, need continue
+        }
+        if (exceeded_sparse_column_limit) {
+            // maybe exist prefix path in sparse column
+            return _create_hierarchical_reader(iterator, col_uid, 
relative_path, node, root,
+                                               column_reader_cache, 
opt->stats);
+        }
         // Sparse column not exists and not reached stats limit, then the 
target path is not exist, get a default iterator
         RETURN_IF_ERROR(Segment::new_default_iterator(*target_col, iterator));
     }
@@ -404,12 +464,14 @@ Status VariantColumnReader::init(const 
ColumnReaderOptions& opts, const SegmentF
     _subcolumns_meta_info = std::make_unique<SubcolumnColumnMetaInfo>();
     _statistics = std::make_unique<VariantStatistics>();
     const ColumnMetaPB& self_column_pb = footer.columns(column_id);
+    _data_type = 
vectorized::DataTypeFactory::instance().create_data_type(self_column_pb);
+    _root_unique_id = self_column_pb.unique_id();
     const auto& parent_index = 
opts.tablet_schema->inverted_indexs(self_column_pb.unique_id());
     // record variant_sparse_column_statistics_size from parent column
     _variant_sparse_column_statistics_size =
             opts.tablet_schema->column_by_uid(self_column_pb.unique_id())
                     .variant_max_sparse_column_statistics_size();
-
+    _tablet_schema = opts.tablet_schema;
     for (int32_t ordinal = 0; ordinal < footer.columns_size(); ++ordinal) {
         const ColumnMetaPB& column_pb = footer.columns(ordinal);
         // Find all columns belonging to the current variant column
@@ -474,19 +536,19 @@ Status VariantColumnReader::init(const 
ColumnReaderOptions& opts, const SegmentF
                                                               
column_pb.none_null_size());
             }
             _subcolumns_meta_info->add(relative_path, SubcolumnMeta 
{get_data_type_fn(), ordinal});
-            TabletSchema::SubColumnInfo sub_column_info;
-            // if subcolumn has index, add index to _variant_subcolumns_indexes
-            if (vectorized::schema_util::generate_sub_column_info(
-                        *opts.tablet_schema, self_column_pb.unique_id(), 
relative_path.get_path(),
-                        &sub_column_info) &&
-                !sub_column_info.indexes.empty()) {
-                _variant_subcolumns_indexes[path.get_path()] = 
std::move(sub_column_info.indexes);
-            }
-            // if parent column has index, add index to 
_variant_subcolumns_indexes
-            else if (!parent_index.empty()) {
-                vectorized::schema_util::inherit_index(
-                        parent_index, 
_variant_subcolumns_indexes[path.get_path()], column_pb);
-            }
+            // TabletSchema::SubColumnInfo sub_column_info;
+            // // if subcolumn has index, add index to 
_variant_subcolumns_indexes
+            // if (vectorized::schema_util::generate_sub_column_info(
+            //             *opts.tablet_schema, self_column_pb.unique_id(), 
relative_path.get_path(),
+            //             &sub_column_info) &&
+            //     !sub_column_info.indexes.empty()) {
+            //     _variant_subcolumns_indexes[path.get_path()] = 
std::move(sub_column_info.indexes);
+            // }
+            // // if parent column has index, add index to 
_variant_subcolumns_indexes
+            // else if (!parent_index.empty()) {
+            //     vectorized::schema_util::inherit_index(
+            //             parent_index, 
_variant_subcolumns_indexes[path.get_path()], column_pb);
+            // }
         }
     }
 
@@ -498,19 +560,60 @@ Status VariantColumnReader::init(const 
ColumnReaderOptions& opts, const SegmentF
             _statistics->sparse_column_non_null_size.emplace(path, size);
         }
     }
+    _segment_file_reader = file_reader;
+    _num_rows = num_rows;
+    // try build external meta readers (optional)
+    _ext_meta_reader = std::make_unique<VariantExternalMetaReader>();
+    RETURN_IF_ERROR(_ext_meta_reader->init_from_footer(footer, file_reader, 
_root_unique_id));
     return Status::OK();
 }
+Status VariantColumnReader::create_reader_from_external_meta(const 
std::string& path,
+                                                             const 
ColumnReaderOptions& opts,
+                                                             const 
io::FileReaderSPtr& file_reader,
+                                                             uint64_t num_rows,
+                                                             
std::shared_ptr<ColumnReader>* out) {
+    if (!_ext_meta_reader || !_ext_meta_reader->available()) {
+        return Status::Error<ErrorCode::NOT_FOUND, false>("no external variant 
meta");
+    }
+    ColumnMetaPB meta;
+    RETURN_IF_ERROR(_ext_meta_reader->lookup_meta_by_path(path, &meta));
+    return ColumnReader::create(opts, meta, num_rows, file_reader, out);
+}
 
-std::vector<const TabletIndex*> 
VariantColumnReader::find_subcolumn_tablet_indexes(
-        const std::string& path) {
-    auto it = _variant_subcolumns_indexes.find(path);
-    std::vector<const TabletIndex*> indexes;
-    if (it != _variant_subcolumns_indexes.end()) {
-        for (const auto& index : it->second) {
-            indexes.push_back(index.get());
-        }
+Status VariantColumnReader::load_external_meta_once() {
+    if (!_ext_meta_reader || !_ext_meta_reader->available()) {
+        return Status::OK();
+    }
+    return _ext_meta_reader->load_all_once(_subcolumns_meta_info.get(), 
_statistics.get());
+}
+
+TabletIndexes VariantColumnReader::find_subcolumn_tablet_indexes(
+        const TabletColumn& column, const vectorized::DataTypePtr& data_type) {
+    TabletSchema::SubColumnInfo sub_column_info;
+    const auto& parent_index = 
_tablet_schema->inverted_indexs(column.parent_unique_id());
+    auto relative_path = column.path_info_ptr()->copy_pop_front();
+    // if subcolumn has index, add index to _variant_subcolumns_indexes
+    if (vectorized::schema_util::generate_sub_column_info(
+                *_tablet_schema, column.parent_unique_id(), 
relative_path.get_path(),
+                &sub_column_info) &&
+        !sub_column_info.indexes.empty()) {
+    }
+    // if parent column has index, add index to _variant_subcolumns_indexes
+    else if (!parent_index.empty() &&
+             data_type->get_storage_field_type() != 
doris::FieldType::OLAP_FIELD_TYPE_VARIANT &&
+             data_type->get_storage_field_type() !=
+                     doris::FieldType::OLAP_FIELD_TYPE_MAP /*SPARSE COLUMN*/) {
+        // type in column maynot be real type, so use data_type to get the 
real type
+        TabletColumn target_column = 
vectorized::schema_util::get_column_by_type(
+                data_type, column.name(),
+                {.unique_id = -1,
+                 .parent_unique_id = column.parent_unique_id(),
+                 .path_info = *column.path_info_ptr()});
+        vectorized::schema_util::inherit_index(parent_index, 
sub_column_info.indexes,
+                                               target_column);
     }
-    return indexes;
+    // Return shared_ptr directly to maintain object lifetime
+    return sub_column_info.indexes;
 }
 
 void VariantColumnReader::get_subcolumns_types(
diff --git a/be/src/olap/rowset/segment_v2/variant/variant_column_reader.h 
b/be/src/olap/rowset/segment_v2/variant/variant_column_reader.h
index f22809eed52..039d3e1de35 100644
--- a/be/src/olap/rowset/segment_v2/variant/variant_column_reader.h
+++ b/be/src/olap/rowset/segment_v2/variant/variant_column_reader.h
@@ -27,9 +27,12 @@
 #include <vector>
 
 #include "olap/rowset/segment_v2/column_reader.h"
+#include "olap/rowset/segment_v2/indexed_column_reader.h"
 #include "olap/rowset/segment_v2/page_handle.h"
+#include "olap/rowset/segment_v2/variant/variant_external_meta_reader.h"
 #include "olap/rowset/segment_v2/variant_statistics.h"
 #include "olap/tablet_schema.h"
+#include "util/once.h"
 #include "vec/columns/column_object.h"
 #include "vec/columns/subcolumn_tree.h"
 #include "vec/json/path_in_data.h"
@@ -38,6 +41,7 @@ namespace doris {
 
 class TabletIndex;
 class StorageReadOptions;
+class TabletSchema;
 
 namespace segment_v2 {
 
@@ -70,7 +74,9 @@ public:
 
     int64_t get_metadata_size() const override;
 
-    std::vector<const TabletIndex*> find_subcolumn_tablet_indexes(const 
std::string&);
+    // Return shared_ptr to ensure the lifetime of TabletIndex objects
+    TabletIndexes find_subcolumn_tablet_indexes(const TabletColumn& 
target_column,
+                                                const vectorized::DataTypePtr& 
data_type);
 
     bool exist_in_sparse_column(const vectorized::PathInData& path) const;
 
@@ -80,15 +86,34 @@ public:
         return _subcolumns_meta_info.get();
     }
 
+    // Get the types of all subcolumns in the variant column.
     void get_subcolumns_types(
             std::unordered_map<vectorized::PathInData, vectorized::DataTypes,
                                vectorized::PathInData::Hash>* 
subcolumns_types) const;
 
+    // Get the typed paths in the variant column.
     void get_typed_paths(std::unordered_set<std::string>* typed_paths) const;
 
+    // Get the nested paths in the variant column.
     void get_nested_paths(std::unordered_set<vectorized::PathInData, 
vectorized::PathInData::Hash>*
                                   nested_paths) const;
 
+    // Try create a ColumnReader from externalized meta (path -> ColumnMetaPB 
bytes) if present.
+    Status create_reader_from_external_meta(const std::string& path,
+                                            const ColumnReaderOptions& opts,
+                                            const io::FileReaderSPtr& 
file_reader,
+                                            uint64_t num_rows, 
std::shared_ptr<ColumnReader>* out);
+
+    // Ensure external meta is loaded only once across concurrent callers.
+    Status load_external_meta_once();
+
+    // Determine whether `path` is a strict prefix of any existing subcolumn 
path.
+    // Consider three sources:
+    // 1) Extracted subcolumns in `_subcolumns_meta_info`
+    // 2) Sparse column statistics in 
`_statistics->sparse_column_non_null_size`
+    // 3) Externalized metas via `_ext_meta_reader`
+    bool has_prefix_path(const vectorized::PathInData& relative_path) const;
+
 private:
     // init for compaction read
     Status _new_default_iter_with_same_nested(ColumnIteratorUPtr* iterator, 
const TabletColumn& col,
@@ -114,11 +139,19 @@ private:
     std::shared_ptr<ColumnReader> _sparse_column_reader;
     std::shared_ptr<ColumnReader> _root_column_reader;
     std::unique_ptr<VariantStatistics> _statistics;
-    // key: subcolumn path, value: subcolumn indexes
-    std::unordered_map<std::string, TabletIndexes> _variant_subcolumns_indexes;
+    std::shared_ptr<TabletSchema> _tablet_schema;
     // variant_sparse_column_statistics_size
     size_t _variant_sparse_column_statistics_size =
             BeConsts::DEFAULT_VARIANT_MAX_SPARSE_COLUMN_STATS_SIZE;
+
+    // Externalized meta reader (optional)
+    std::unique_ptr<VariantExternalMetaReader> _ext_meta_reader;
+
+    io::FileReaderSPtr _segment_file_reader;
+    uint64_t _num_rows {0};
+    uint32_t _root_unique_id {0};
+
+    // call-once guard moved into VariantExternalMetaReader
 };
 
 class VariantRootColumnIterator : public ColumnIterator {
diff --git a/be/src/olap/rowset/segment_v2/variant/variant_ext_meta_writer.cpp 
b/be/src/olap/rowset/segment_v2/variant/variant_ext_meta_writer.cpp
new file mode 100644
index 00000000000..db5acac9114
--- /dev/null
+++ b/be/src/olap/rowset/segment_v2/variant/variant_ext_meta_writer.cpp
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "olap/rowset/segment_v2/variant/variant_ext_meta_writer.h"
+
+#include <utility>
+
+#include "olap/rowset/segment_v2/indexed_column_writer.h"
+#include "olap/types.h"
+#include "vec/common/schema_util.h"
+
+namespace doris::segment_v2 {
+
+Status VariantExtMetaWriter::_ensure_inited(Writers* w) {
+    if (w->inited) {
+        return Status::OK();
+    }
+
+    // key writer: VARCHAR, value index ON, ordinal index OFF
+    IndexedColumnWriterOptions dict_opts;
+    dict_opts.write_value_index = true;
+    dict_opts.write_ordinal_index = false;
+    dict_opts.encoding = PREFIX_ENCODING;
+    dict_opts.compression = _comp;
+    const TypeInfo* dict_type = 
get_scalar_type_info<FieldType::OLAP_FIELD_TYPE_VARCHAR>();
+    w->key_writer = std::make_unique<IndexedColumnWriter>(dict_opts, 
dict_type, _fw);
+    RETURN_IF_ERROR(w->key_writer->init());
+
+    // value writer: VARCHAR, value index OFF, ordinal index ON
+    IndexedColumnWriterOptions vals_opts;
+    vals_opts.write_value_index = false;
+    vals_opts.write_ordinal_index = true;
+    vals_opts.encoding = PLAIN_ENCODING;
+    vals_opts.compression = _comp;
+    const TypeInfo* vals_type = 
get_scalar_type_info<FieldType::OLAP_FIELD_TYPE_VARCHAR>();
+    w->val_writer = std::make_unique<IndexedColumnWriter>(vals_opts, 
vals_type, _fw);
+    RETURN_IF_ERROR(w->val_writer->init());
+
+    w->inited = true;
+    return Status::OK();
+}
+
+Status VariantExtMetaWriter::add(int32_t root_uid, const Slice& key, const 
Slice& val) {
+    auto& w = _writers_by_uid[root_uid];
+    RETURN_IF_ERROR(_ensure_inited(&w));
+    RETURN_IF_ERROR(w.key_writer->add(&key));
+    RETURN_IF_ERROR(w.val_writer->add(&val));
+    ++w.count;
+    return Status::OK();
+}
+
+Status VariantExtMetaWriter::flush_to_footer(SegmentFooterPB* footer) {
+    for (auto& [uid, w] : _writers_by_uid) {
+        if (!w.inited || w.count == 0) {
+            continue;
+        }
+        doris::segment_v2::IndexedColumnMetaPB key_meta;
+        doris::segment_v2::IndexedColumnMetaPB val_meta;
+        RETURN_IF_ERROR(w.key_writer->finish(&key_meta));
+        RETURN_IF_ERROR(w.val_writer->finish(&val_meta));
+
+        // keys
+        std::string k = std::string("variant_meta_keys.") + 
std::to_string(uid);
+        std::string v;
+        key_meta.AppendToString(&v);
+        auto* p1 = footer->add_file_meta_datas();
+        p1->set_key(k);
+        p1->set_value(v);
+
+        // values
+        std::string k2 = std::string("variant_meta_values.") + 
std::to_string(uid);
+        std::string v2;
+        val_meta.AppendToString(&v2);
+        auto* p2 = footer->add_file_meta_datas();
+        p2->set_key(k2);
+        p2->set_value(v2);
+    }
+    _writers_by_uid.clear();
+    return Status::OK();
+}
+
+Status VariantExtMetaWriter::externalize_from_footer(SegmentFooterPB* footer) {
+    // Collect variant subcolumns first, then write in sorted order to keep 
stability.
+    std::vector<ColumnMetaPB> kept;
+    kept.reserve(footer->columns_size());
+    std::unordered_map<int32_t, std::vector<std::pair<std::string, 
std::string>>>
+            pending; // uid -> [(path, meta_bytes)]
+    pending.reserve(8);
+    size_t kept_count = 0;
+    size_t externalized_count = 0;
+
+    std::string meta_bytes;
+    for (int i = 0; i < footer->columns_size(); ++i) {
+        const ColumnMetaPB& col = footer->columns(i);
+        if (!col.has_column_path_info()) {
+            kept.emplace_back(col);
+            kept_count++;
+            continue;
+        }
+        vectorized::PathInData full_path;
+        full_path.from_protobuf(col.column_path_info());
+        vectorized::PathInData rel = full_path.copy_pop_front();
+        if (rel.empty()) {
+            kept.emplace_back(col);
+            kept_count++;
+            continue; // root variant column
+        }
+        std::string rel_path = rel.get_path();
+        // Check if this is a sparse column or sub column
+        // Treat both single sparse column and bucketized sparse columns 
(.b{i}) as sparse
+        if (rel_path.find("__DORIS_VARIANT_SPARSE__") != std::string::npos) {
+            kept.emplace_back(col);
+            kept_count++;
+            continue;
+        }
+        int32_t root_uid = col.column_path_info().parrent_column_unique_id();
+        meta_bytes.clear();
+        col.AppendToString(&meta_bytes);
+        pending[root_uid].emplace_back(std::move(rel_path), meta_bytes);
+        externalized_count++;
+    }
+
+    // Write keys/values per uid in sorted path order
+    for (auto& [uid, vec] : pending) {
+        std::sort(vec.begin(), vec.end(),
+                  [](const auto& a, const auto& b) { return a.first < b.first; 
});
+        for (auto& kv : vec) {
+            RETURN_IF_ERROR(add(uid, Slice(kv.first), Slice(kv.second)));
+        }
+    }
+    RETURN_IF_ERROR(flush_to_footer(footer));
+
+    // Replace columns with kept ones (prune externalized subcolumns)
+    footer->clear_columns();
+    for (const auto& c : kept) {
+        auto* dst = footer->add_columns();
+        dst->CopyFrom(c);
+    }
+    VLOG_DEBUG << "VariantExtMetaWriter::externalize_from_footer, externalized 
subcolumns: "
+               << externalized_count << ", kept columns: " << kept_count
+               << ", total columns: " << footer->columns_size();
+    return Status::OK();
+}
+
+} // namespace doris::segment_v2
diff --git a/be/src/olap/rowset/segment_v2/variant/variant_ext_meta_writer.h 
b/be/src/olap/rowset/segment_v2/variant/variant_ext_meta_writer.h
new file mode 100644
index 00000000000..5dc81761fd4
--- /dev/null
+++ b/be/src/olap/rowset/segment_v2/variant/variant_ext_meta_writer.h
@@ -0,0 +1,71 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include "gen_cpp/segment_v2.pb.h"
+#include "io/fs/file_reader_writer_fwd.h"
+#include "olap/rowset/segment_v2/indexed_column_writer.h"
+#include "olap/types.h"
+#include "util/slice.h"
+#include "vec/json/path_in_data.h"
+
+namespace doris::segment_v2 {
+
+// Aggregates externalized meta for Variant subcolumns and flushes
+// them as two IndexedColumns into footer.file_meta_datas:
+// - variant_meta_keys.<root_uid>
+// - variant_meta_values.<root_uid>
+class VariantExtMetaWriter {
+public:
+    VariantExtMetaWriter(io::FileWriter* fw, CompressionTypePB comp) : 
_fw(fw), _comp(comp) {}
+
+    // Add one path->meta mapping for the specified variant root uid.
+    // key: subcolumn path (VARCHAR)
+    // val: serialized ColumnMetaPB (VARCHAR)
+    Status add(int32_t root_uid, const Slice& key, const Slice& val);
+
+    // Finish writers and append their metas into footer.file_meta_datas.
+    Status flush_to_footer(SegmentFooterPB* footer);
+
+    // Scan footer.columns, find variant extracted subcolumns, externalize 
them into
+    // ext meta index and remove them from footer.columns. This method both 
writes
+    // keys/values metas to footer.file_meta_datas and prunes subcolumns from 
footer.
+    Status externalize_from_footer(SegmentFooterPB* footer);
+
+    bool empty() const { return _writers_by_uid.empty(); }
+
+private:
+    struct Writers {
+        std::unique_ptr<IndexedColumnWriter> key_writer; // value index
+        std::unique_ptr<IndexedColumnWriter> val_writer; // ordinal index
+        size_t count = 0;
+        bool inited = false;
+    };
+
+    Status _ensure_inited(Writers* w);
+
+    io::FileWriter* _fw;
+    CompressionTypePB _comp;
+    std::unordered_map<int32_t, Writers> _writers_by_uid;
+};
+
+} // namespace doris::segment_v2
diff --git 
a/be/src/olap/rowset/segment_v2/variant/variant_external_meta_reader.cpp 
b/be/src/olap/rowset/segment_v2/variant/variant_external_meta_reader.cpp
new file mode 100644
index 00000000000..8faf12bdea9
--- /dev/null
+++ b/be/src/olap/rowset/segment_v2/variant/variant_external_meta_reader.cpp
@@ -0,0 +1,232 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "olap/rowset/segment_v2/variant/variant_external_meta_reader.h"
+
+#include <gen_cpp/segment_v2.pb.h>
+
+#include <memory>
+#include <string>
+#include <string_view>
+
+#include "common/status.h"
+#include "olap/rowset/segment_v2/indexed_column_reader.h"
+#include "vec/columns/column_string.h"
+#include "vec/data_types/data_type_factory.hpp"
+
+namespace doris::segment_v2 {
+
+#include "common/compile_check_begin.h"
+
+Status VariantExternalMetaReader::_find_pairs(const SegmentFooterPB& footer, 
int32_t root_uid,
+                                              const MetadataPairPB** 
keys_meta_pair,
+                                              const MetadataPairPB** 
vals_meta_pair) const {
+    *keys_meta_pair = nullptr;
+    *vals_meta_pair = nullptr;
+    // prefer suffixed pairs
+    std::string suffix = "." + std::to_string(root_uid);
+    for (const auto& m : footer.file_meta_datas()) {
+        if (m.key() == std::string("variant_meta_keys") + suffix) {
+            *keys_meta_pair = &m;
+        }
+        if (m.key() == std::string("variant_meta_values") + suffix) {
+            *vals_meta_pair = &m;
+        }
+    }
+    // fallback: legacy single-variant footer
+    if (!*keys_meta_pair || !*vals_meta_pair) {
+        for (const auto& m : footer.file_meta_datas()) {
+            if (!*keys_meta_pair && m.key() == "variant_meta_keys") {
+                *keys_meta_pair = &m;
+            }
+            if (!*vals_meta_pair && m.key() == "variant_meta_values") {
+                *vals_meta_pair = &m;
+            }
+        }
+    }
+    return Status::OK();
+}
+
+Status VariantExternalMetaReader::init_from_footer(const SegmentFooterPB& 
footer,
+                                                   const io::FileReaderSPtr& 
file_reader,
+                                                   int32_t root_uid) {
+    const MetadataPairPB* keys_meta_pair = nullptr;
+    const MetadataPairPB* vals_meta_pair = nullptr;
+    RETURN_IF_ERROR(_find_pairs(footer, root_uid, &keys_meta_pair, 
&vals_meta_pair));
+    if (!keys_meta_pair || !vals_meta_pair) {
+        // External meta not present, keep unavailable state.
+        return Status::OK();
+    }
+
+    doris::segment_v2::IndexedColumnMetaPB key_meta;
+    doris::segment_v2::IndexedColumnMetaPB val_meta;
+    if (!key_meta.ParseFromArray(keys_meta_pair->value().data(),
+                                 
static_cast<int>(keys_meta_pair->value().size()))) {
+        return Status::Corruption("bad variant_meta_keys meta");
+    }
+    if (!val_meta.ParseFromArray(vals_meta_pair->value().data(),
+                                 
static_cast<int>(vals_meta_pair->value().size()))) {
+        return Status::Corruption("bad variant_meta_values meta");
+    }
+
+    _key_reader = 
std::make_unique<segment_v2::IndexedColumnReader>(file_reader, key_meta);
+    _val_reader = 
std::make_unique<segment_v2::IndexedColumnReader>(file_reader, val_meta);
+    RETURN_IF_ERROR(_key_reader->load(true, false));
+    RETURN_IF_ERROR(_val_reader->load(true, false));
+    return Status::OK();
+}
+
+Status VariantExternalMetaReader::lookup_meta_by_path(const std::string& 
rel_path,
+                                                      ColumnMetaPB* out_meta) 
const {
+    if (!available()) {
+        return Status::Error<ErrorCode::NOT_FOUND, false>("no external variant 
meta");
+    }
+    segment_v2::IndexedColumnIterator key_it(_key_reader.get());
+    bool exact = false;
+    Status st = key_it.seek_at_or_after(&rel_path, &exact);
+    if (st.is<ErrorCode::ENTRY_NOT_FOUND>()) {
+        return Status::Error<ErrorCode::NOT_FOUND, false>("variant meta key 
not found");
+    }
+    if (!exact) {
+        return Status::Error<ErrorCode::NOT_FOUND, false>("variant meta key 
not found");
+    }
+    if (!st.ok()) {
+        return st;
+    }
+    auto ord = key_it.get_current_ordinal();
+    segment_v2::IndexedColumnIterator val_it(_val_reader.get());
+    RETURN_IF_ERROR(val_it.seek_to_ordinal(ord));
+    size_t n = 1;
+    auto col = vectorized::ColumnString::create();
+    vectorized::MutableColumnPtr dst = std::move(col);
+    RETURN_IF_ERROR(val_it.next_batch(&n, dst));
+    if (n != 1) {
+        return Status::Corruption("variant meta value read failed");
+    }
+    auto* s = assert_cast<vectorized::ColumnString*>(dst.get());
+    auto ref = s->get_data_at(0);
+    if (!out_meta->ParseFromArray(ref.data, static_cast<int>(ref.size))) {
+        return Status::Corruption("bad ColumnMetaPB in variant external meta");
+    }
+    return Status::OK();
+}
+
+Status VariantExternalMetaReader::load_all(SubcolumnColumnMetaInfo* 
out_meta_tree,
+                                           VariantStatistics* out_stats) {
+    segment_v2::IndexedColumnIterator val_it(_val_reader.get());
+    RETURN_IF_ERROR(val_it.seek_to_ordinal(0));
+    auto total = static_cast<size_t>(_val_reader->num_values());
+    size_t built = 0;
+    while (built < total) {
+        size_t n = total - built;
+        auto col = vectorized::ColumnString::create();
+        vectorized::MutableColumnPtr dst = std::move(col);
+        RETURN_IF_ERROR(val_it.next_batch(&n, dst));
+        if (n == 0) {
+            break;
+        }
+        auto* s = assert_cast<vectorized::ColumnString*>(dst.get());
+        for (size_t i = 0; i < n; ++i) {
+            auto ref = s->get_data_at(i);
+            ColumnMetaPB meta;
+            if (!meta.ParseFromArray(ref.data, static_cast<int>(ref.size))) {
+                return Status::Corruption("bad ColumnMetaPB in variant 
external meta");
+            }
+            if (!meta.has_column_path_info()) {
+                continue;
+            }
+            vectorized::PathInData full_path;
+            full_path.from_protobuf(meta.column_path_info());
+            auto relative_path = full_path.copy_pop_front();
+            if (relative_path.empty()) {
+                continue; // skip root
+            }
+            if (out_meta_tree->find_leaf(relative_path)) {
+                continue; // already exists
+            }
+            if (meta.has_none_null_size() && out_stats != nullptr) {
+                
out_stats->subcolumns_non_null_size.emplace(relative_path.get_path(),
+                                                            
meta.none_null_size());
+            }
+            auto file_type = 
vectorized::DataTypeFactory::instance().create_data_type(meta);
+            out_meta_tree->add(relative_path,
+                               SubcolumnMeta {.file_column_type = file_type, 
.footer_ordinal = -1});
+        }
+        built += n;
+        if (built < total) {
+            
RETURN_IF_ERROR(val_it.seek_to_ordinal(static_cast<ordinal_t>(built)));
+        }
+    }
+    return Status::OK();
+}
+
+Status VariantExternalMetaReader::has_prefix(const std::string& prefix, bool* 
out) const {
+    // english only in comments
+    DCHECK(out != nullptr);
+    DCHECK(available());
+    *out = false;
+    // Empty prefix means everything matches; guard for safety
+    if (prefix.empty()) {
+        *out = true;
+        return Status::OK();
+    }
+
+    segment_v2::IndexedColumnIterator key_it(_key_reader.get());
+    bool exact = false;
+    Status st = key_it.seek_at_or_after(&prefix, &exact);
+    if (st.is<ErrorCode::ENTRY_NOT_FOUND>()) {
+        *out = false;
+        return Status::OK();
+    }
+    if (!st.ok()) {
+        return st;
+    }
+
+    size_t n = 1;
+    auto col = vectorized::ColumnString::create();
+    vectorized::MutableColumnPtr dst = std::move(col);
+    RETURN_IF_ERROR(key_it.next_batch(&n, dst));
+    if (n == 0) {
+        *out = false;
+        return Status::OK();
+    }
+    auto* s = assert_cast<vectorized::ColumnString*>(dst.get());
+    auto ref = s->get_data_at(0);
+    std::string_view key_sv(ref.data, ref.size);
+    // starts_with check
+    *out = key_sv.size() >= prefix.size() && key_sv.starts_with(prefix);
+    return Status::OK();
+}
+
+Status VariantExternalMetaReader::load_all_once(SubcolumnColumnMetaInfo* 
out_meta_tree,
+                                                VariantStatistics* out_stats) {
+    if (!available()) {
+        return Status::OK();
+    }
+    return _load_once_call.call([&]() -> Status {
+        if (_loaded) {
+            return Status::OK();
+        }
+        RETURN_IF_ERROR(load_all(out_meta_tree, out_stats));
+        _loaded = true;
+        return Status::OK();
+    });
+}
+
+#include "common/compile_check_end.h"
+
+} // namespace doris::segment_v2
diff --git 
a/be/src/olap/rowset/segment_v2/variant/variant_external_meta_reader.h 
b/be/src/olap/rowset/segment_v2/variant/variant_external_meta_reader.h
new file mode 100644
index 00000000000..93a43debff2
--- /dev/null
+++ b/be/src/olap/rowset/segment_v2/variant/variant_external_meta_reader.h
@@ -0,0 +1,85 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <gen_cpp/segment_v2.pb.h>
+
+#include <memory>
+#include <string>
+
+#include "common/status.h"
+#include "io/fs/file_reader.h"
+#include "olap/rowset/segment_v2/indexed_column_reader.h"
+#include "olap/rowset/segment_v2/stream_reader.h"
+#include "olap/rowset/segment_v2/variant_statistics.h"
+#include "util/once.h"
+#include "vec/columns/subcolumn_tree.h"
+#include "vec/common/schema_util.h"
+#include "vec/json/path_in_data.h"
+
+namespace doris::segment_v2 {
+
+#include "common/compile_check_begin.h"
+
+// Encapsulates reading of externalized Variant subcolumn metas.
+// It discovers key/value indexed-columns from SegmentFooterPB, supports:
+// - availability check
+// - lookup ColumnMetaPB by relative path
+// - one-time bulk load of all metas into SubcolumnColumnMetaInfo and 
VariantStatistics
+class VariantExternalMetaReader {
+public:
+    VariantExternalMetaReader() = default;
+
+    // Initialize by locating and opening ext meta key/value indexed columns.
+    // root_uid is used to find suffixed keys like: 
variant_meta_keys.<root_uid>
+    Status init_from_footer(const SegmentFooterPB& footer, const 
io::FileReaderSPtr& file_reader,
+                            int32_t root_uid);
+
+    bool available() const { return _key_reader != nullptr && _val_reader != 
nullptr; }
+
+    // Lookup a single ColumnMetaPB by relative path. Returns NOT_FOUND if 
missing/unavailable.
+    Status lookup_meta_by_path(const std::string& rel_path, ColumnMetaPB* 
out_meta) const;
+
+    // Check whether there exists any key in external meta that starts with 
`prefix`.
+    // This performs a lower_bound (seek_at_or_after) on the sorted key column
+    // and verifies the first key is prefixed by `prefix`.
+    Status has_prefix(const std::string& prefix, bool* out) const;
+
+    // Ensure external metas are loaded exactly once and merged into provided 
structures.
+    Status load_all_once(SubcolumnColumnMetaInfo* out_meta_tree, 
VariantStatistics* out_stats);
+
+    // Load and merge all external metas without call-once guard.
+    Status load_all(SubcolumnColumnMetaInfo* out_meta_tree, VariantStatistics* 
out_stats);
+
+private:
+    // helpers
+    Status _find_pairs(const SegmentFooterPB& footer, int32_t root_uid,
+                       const MetadataPairPB** keys_meta_pair,
+                       const MetadataPairPB** vals_meta_pair) const;
+
+    std::unique_ptr<IndexedColumnReader> _key_reader;
+    std::unique_ptr<IndexedColumnReader> _val_reader;
+
+    // call-once guard for bulk loading
+    DorisCallOnce<Status> _load_once_call;
+    bool _loaded = false;
+};
+
+#include "common/compile_check_end.h"
+
+} // namespace doris::segment_v2
diff --git a/be/src/olap/rowset/segment_v2/variant_column_writer_impl.cpp 
b/be/src/olap/rowset/segment_v2/variant_column_writer_impl.cpp
index 6cb59d186da..c5fcefa0f5b 100644
--- a/be/src/olap/rowset/segment_v2/variant_column_writer_impl.cpp
+++ b/be/src/olap/rowset/segment_v2/variant_column_writer_impl.cpp
@@ -31,9 +31,12 @@
 #include "olap/rowset/rowset_fwd.h"
 #include "olap/rowset/rowset_writer_context.h"
 #include "olap/rowset/segment_v2/column_writer.h"
+#include "olap/rowset/segment_v2/indexed_column_writer.h"
 #include "olap/segment_loader.h"
 #include "olap/tablet_schema.h"
+#include "olap/types.h"
 #include "util/simd/bits.h"
+#include "util/slice.h"
 #include "vec/columns/column.h"
 #include "vec/columns/column_nullable.h"
 #include "vec/columns/column_object.h"
@@ -250,6 +253,9 @@ Status 
VariantColumnWriterImpl::_process_subcolumns(vectorized::ColumnObject* pt
     };
     _subcolumns_indexes.resize(ptr->get_subcolumns().size());
     // convert sub column data from engine format to storage layer format
+    // NOTE: We only keep up to variant_max_subcolumns_count as extracted 
columns; others are externalized.
+    // uint32_t extracted = 0;
+    // uint32_t extract_limit = _tablet_column->variant_max_subcolumns_count();
     for (const auto& entry :
          
vectorized::schema_util::get_sorted_subcolumns(ptr->get_subcolumns())) {
         const auto& least_common_type = entry->data.get_least_common_type();
@@ -263,7 +269,7 @@ Status 
VariantColumnWriterImpl::_process_subcolumns(vectorized::ColumnObject* pt
         }
         CHECK(entry->data.is_finalized());
 
-        // create subcolumn writer
+        // create subcolumn writer if under limit; otherwise externalize 
ColumnMetaPB via IndexedColumn
         int current_column_id = column_id++;
         TabletColumn tablet_column;
         int64_t none_null_value_size = entry->data.get_non_null_value_size();
diff --git a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp 
b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp
index adb4986c2be..94e08f60590 100644
--- a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp
@@ -53,6 +53,7 @@
 #include "olap/rowset/segment_v2/inverted_index_file_writer.h"
 #include "olap/rowset/segment_v2/page_io.h"
 #include "olap/rowset/segment_v2/page_pointer.h"
+#include "olap/rowset/segment_v2/variant/variant_ext_meta_writer.h"
 #include "olap/segment_loader.h"
 #include "olap/short_key_index.h"
 #include "olap/tablet_schema.h"
@@ -1447,6 +1448,13 @@ Status VerticalSegmentWriter::_write_primary_key_index() 
{
 Status VerticalSegmentWriter::_write_footer() {
     _footer.set_num_rows(_row_count);
 
+    if (config::enable_variant_external_meta) {
+        // Externalize variant subcolumns into ext meta and prune them from 
footer.columns.
+        auto variant_ext_meta_agg =
+                std::make_unique<VariantExtMetaWriter>(_file_writer, 
_opts.compression_type);
+        
RETURN_IF_ERROR(variant_ext_meta_agg->externalize_from_footer(&_footer));
+    }
+
     // Footer := SegmentFooterPB, FooterPBSize(4), FooterPBChecksum(4), 
MagicNumber(4)
     VLOG_DEBUG << "footer " << _footer.DebugString();
     std::string footer_buf;
diff --git a/be/src/service/internal_service.cpp 
b/be/src/service/internal_service.cpp
index a1c5c98ce45..6396fe9bcc4 100644
--- a/be/src/service/internal_service.cpp
+++ b/be/src/service/internal_service.cpp
@@ -1132,6 +1132,10 @@ void 
PInternalService::fetch_remote_tablet_schema(google::protobuf::RpcControlle
     bool ret = _heavy_work_pool.try_offer([request, response, done]() {
         brpc::ClosureGuard closure_guard(done);
         Status st = Status::OK();
+        // Bind thread MemTracker to avoid Doris allocator crash on Orphan 
tracker
+        std::shared_ptr<MemTrackerLimiter> mem_tracker = 
MemTrackerLimiter::create_shared(
+                MemTrackerLimiter::Type::OTHER, 
"InternalService::fetch_remote_tablet_schema");
+        SCOPED_ATTACH_TASK(mem_tracker);
         if (request->is_coordinator()) {
             // Spawn rpc request to none coordinator nodes, and finally merge 
them all
             PFetchRemoteSchemaRequest remote_request(*request);
diff --git a/be/src/vec/common/schema_util.cpp 
b/be/src/vec/common/schema_util.cpp
index aa21db80d29..49a94630a37 100644
--- a/be/src/vec/common/schema_util.cpp
+++ b/be/src/vec/common/schema_util.cpp
@@ -759,8 +759,10 @@ Status aggregate_path_to_stats(
             }
 
             CHECK(column_reader->get_meta_type() == 
FieldType::OLAP_FIELD_TYPE_VARIANT);
-            const auto* variant_column_reader =
-                    assert_cast<const 
segment_v2::VariantColumnReader*>(column_reader.get());
+            auto* variant_column_reader =
+                    
assert_cast<segment_v2::VariantColumnReader*>(column_reader.get());
+            // load external meta before getting stats
+            RETURN_IF_ERROR(variant_column_reader->load_external_meta_once());
             const auto* source_stats = variant_column_reader->get_stats();
             CHECK(source_stats);
 
@@ -798,8 +800,10 @@ Status aggregate_variant_extended_info(
             }
 
             CHECK(column_reader->get_meta_type() == 
FieldType::OLAP_FIELD_TYPE_VARIANT);
-            const auto* variant_column_reader =
-                    assert_cast<const 
segment_v2::VariantColumnReader*>(column_reader.get());
+            auto* variant_column_reader =
+                    
assert_cast<segment_v2::VariantColumnReader*>(column_reader.get());
+            // load external meta before getting stats
+            RETURN_IF_ERROR(variant_column_reader->load_external_meta_once());
             const auto* source_stats = variant_column_reader->get_stats();
             CHECK(source_stats);
 
@@ -819,14 +823,17 @@ Status aggregate_variant_extended_info(
             auto& paths_types =
                     
(*uid_to_variant_extended_info)[column->unique_id()].path_to_data_types;
             variant_column_reader->get_subcolumns_types(&paths_types);
+            VLOG_DEBUG << "path_to_data_types size: " << paths_types.size();
 
             // 3. extract typed paths
             auto& typed_paths = 
(*uid_to_variant_extended_info)[column->unique_id()].typed_paths;
             variant_column_reader->get_typed_paths(&typed_paths);
+            VLOG_DEBUG << "typed_paths size: " << typed_paths.size();
 
             // 4. extract nested paths
             auto& nested_paths = 
(*uid_to_variant_extended_info)[column->unique_id()].nested_paths;
             variant_column_reader->get_nested_paths(&nested_paths);
+            VLOG_DEBUG << "nested_paths size: " << nested_paths.size();
         }
     }
     return Status::OK();
@@ -1490,9 +1497,16 @@ TabletSchemaSPtr calculate_variant_extended_schema(const 
std::vector<RowsetShare
                 }
 
                 CHECK(column_reader->get_meta_type() == 
FieldType::OLAP_FIELD_TYPE_VARIANT);
-                const auto* subcolumn_meta_info =
-                        assert_cast<VariantColumnReader*>(column_reader.get())
-                                ->get_subcolumns_meta_info();
+                auto* variant_column_reader =
+                        
assert_cast<segment_v2::VariantColumnReader*>(column_reader.get());
+                // load external meta before getting subcolumn meta info
+                st = variant_column_reader->load_external_meta_once();
+                if (!st.ok()) {
+                    LOG(WARNING) << "Failed to load external meta for column: 
" << column->name()
+                                 << " error: " << st.to_string();
+                    continue;
+                }
+                const auto* subcolumn_meta_info = 
variant_column_reader->get_subcolumns_meta_info();
                 for (const auto& entry : *subcolumn_meta_info) {
                     if (entry->path.empty()) {
                         continue;
diff --git a/be/src/vec/data_types/data_type_factory.cpp 
b/be/src/vec/data_types/data_type_factory.cpp
index bfcda534ecc..61ee589729e 100644
--- a/be/src/vec/data_types/data_type_factory.cpp
+++ b/be/src/vec/data_types/data_type_factory.cpp
@@ -445,7 +445,7 @@ DataTypePtr 
DataTypeFactory::_create_primitive_data_type(const FieldType& type,
         result = vectorized::create_decimal(precision, scale, false);
         break;
     default:
-        DCHECK(false) << "Invalid FieldType:" << (int)type;
+        // DCHECK(false) << "Invalid FieldType:" << (int)type;
         result = nullptr;
         break;
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to