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

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


The following commit(s) were added to refs/heads/master by this push:
     new bcaaf60fa30 [Enhancement](nereids) Replace IndexDef with 
IndexDefinition (#57886)
bcaaf60fa30 is described below

commit bcaaf60fa305a2bdb99f0fe5104752e2b4a02136
Author: yaoxiao <[email protected]>
AuthorDate: Mon Nov 17 11:09:23 2025 +0800

    [Enhancement](nereids) Replace IndexDef with IndexDefinition (#57886)
---
 .../apache/doris/alter/SchemaChangeHandler.java    |  32 +-
 .../apache/doris/analysis/BuildIndexClause.java    |  27 +-
 .../apache/doris/analysis/CreateIndexClause.java   |  13 +-
 .../java/org/apache/doris/analysis/IndexDef.java   | 356 ---------------------
 .../main/java/org/apache/doris/catalog/Column.java |   6 +-
 .../main/java/org/apache/doris/catalog/Index.java  |  34 +-
 .../java/org/apache/doris/catalog/OlapTable.java   |  12 +-
 .../rules/implementation/AggregateStrategies.java  |   2 +-
 .../rewrite/PushDownVectorTopNIntoOlapScan.java    |   2 +-
 .../trees/plans/commands/info/BuildIndexOp.java    |  21 +-
 .../trees/plans/commands/info/CreateIndexOp.java   |  11 +-
 .../trees/plans/commands/info/CreateTableInfo.java |   2 +-
 .../trees/plans/commands/info/IndexDefinition.java | 180 +++++++++--
 .../org/apache/doris/alter/CloudIndexTest.java     |  31 +-
 .../org/apache/doris/alter/IndexChangeJobTest.java |  80 ++---
 .../doris/alter/SchemaChangeHandlerTest.java       |   6 +-
 .../doris/catalog/IndexPropertiesOrderTest.java    |  10 +-
 .../java/org/apache/doris/catalog/IndexTest.java   |  20 +-
 .../org/apache/doris/catalog/OlapTableTest.java    |   4 +-
 .../doris/common/proc/IndexesProcNodeTest.java     |   2 +-
 .../persist/TableAddOrDropColumnsInfoTest.java     |   4 +-
 .../index_p0/test_ngram_bloomfilter_index.groovy   |   2 +-
 .../inverted_index_p0/test_array_index2.groovy     |  11 +-
 23 files changed, 321 insertions(+), 547 deletions(-)

diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java 
b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
index b8af31e5049..9fd16645a41 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
@@ -27,8 +27,6 @@ import org.apache.doris.analysis.CreateIndexClause;
 import org.apache.doris.analysis.DropColumnClause;
 import org.apache.doris.analysis.DropIndexClause;
 import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.IndexDef;
-import org.apache.doris.analysis.IndexDef.IndexType;
 import org.apache.doris.analysis.InvertedIndexUtil;
 import org.apache.doris.analysis.ModifyColumnClause;
 import org.apache.doris.analysis.ModifyTablePropertiesClause;
@@ -90,6 +88,8 @@ import 
org.apache.doris.nereids.trees.plans.commands.AlterCommand;
 import org.apache.doris.nereids.trees.plans.commands.CancelAlterTableCommand;
 import org.apache.doris.nereids.trees.plans.commands.CancelBuildIndexCommand;
 import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
+import 
org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition.IndexType;
 import org.apache.doris.persist.AlterLightSchemaChangeInfo;
 import org.apache.doris.persist.RemoveAlterJobV2OperationLog;
 import org.apache.doris.persist.TableAddOrDropColumnsInfo;
@@ -2124,7 +2124,7 @@ public class SchemaChangeHandler extends AlterHandler {
                     }
                 } else if (alterClause instanceof BuildIndexClause) {
                     BuildIndexClause buildIndexClause = (BuildIndexClause) 
alterClause;
-                    IndexDef indexDef = buildIndexClause.getIndexDef();
+                    IndexDefinition indexDef = buildIndexClause.getIndexDef();
                     Index index = buildIndexClause.getIndex();
 
                     for (Column column : olapTable.getBaseSchema()) {
@@ -2141,11 +2141,11 @@ public class SchemaChangeHandler extends AlterHandler {
                         }
                     }
 
-                    if (indexDef.getPartitionNamesInfo().isEmpty()) {
+                    if (indexDef.getPartitionNames().isEmpty()) {
                         indexOnPartitions.put(index.getIndexId(), 
olapTable.getPartitionNames());
                     } else {
                         indexOnPartitions.put(
-                                index.getIndexId(), new 
HashSet<>(indexDef.getPartitionNamesInfo()));
+                                index.getIndexId(), new 
HashSet<>(indexDef.getPartitionNames()));
                     }
 
                     alterIndexes.add(index);
@@ -2173,11 +2173,11 @@ public class SchemaChangeHandler extends AlterHandler {
                         if (enableAddIndexForNewData) {
                             supportLightIndexChange = (
                                     found.getIndexType() == IndexType.NGRAM_BF
-                                            || found.getIndexType() == 
IndexDef.IndexType.INVERTED);
+                                            || found.getIndexType() == 
IndexType.INVERTED);
                         }
                     } else {
-                        supportLightIndexChange = found.getIndexType() == 
IndexDef.IndexType.INVERTED
-                                || found.getIndexType() == 
IndexDef.IndexType.ANN;
+                        supportLightIndexChange = found.getIndexType() == 
IndexType.INVERTED
+                                || found.getIndexType() == IndexType.ANN;
                     }
                     if (found != null && supportLightIndexChange) {
                         alterIndexes.add(found);
@@ -2662,7 +2662,7 @@ public class SchemaChangeHandler extends AlterHandler {
             List<AlterJobV2> schemaChangeJobV2List = 
getUnfinishedAlterJobV2ByTableId(olapTable.getId());
             // current schemaChangeJob job doesn't support batch operation,so 
just need to get one job
             schemaChangeJobV2 = schemaChangeJobV2List.size() == 0 ? null
-                : Iterables.getOnlyElement(schemaChangeJobV2List);
+                    : Iterables.getOnlyElement(schemaChangeJobV2List);
             if (schemaChangeJobV2 == null) {
                 throw new DdlException(
                     "Table[" + tableName + "] is under schema change state" + 
" but could not find related job");
@@ -2750,9 +2750,9 @@ public class SchemaChangeHandler extends AlterHandler {
         }
 
         List<Index> existedIndexes = olapTable.getIndexes();
-        IndexDef indexDef = alterClause.getIndexDef();
+        IndexDefinition indexDef = alterClause.getIndexDef();
         Set<String> newColset = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
-        newColset.addAll(indexDef.getColumns());
+        newColset.addAll(indexDef.getColumnNames());
         Set<Long> existedIndexIdSet = Sets.newHashSet();
         if (checkDuplicateIndexes(existedIndexes, indexDef, newColset, 
existedIndexIdSet, olapTable)) {
             return true;
@@ -2775,7 +2775,7 @@ public class SchemaChangeHandler extends AlterHandler {
             
AnnIndexPropertiesChecker.checkProperties(indexDef.getProperties());
         }
 
-        for (String col : indexDef.getColumns()) {
+        for (String col : indexDef.getColumnNames()) {
             Column column = olapTable.getColumn(col);
             if (column != null) {
                 indexDef.checkColumn(column, olapTable.getKeysType(),
@@ -2793,12 +2793,12 @@ public class SchemaChangeHandler extends AlterHandler {
         // when send index description to BE, there maybe cannot find column 
by name,
         // so here update column name in CreateIndexClause after checkColumn 
for indexDef,
         // there will use the column name in olapTable instead of the column 
name in CreateIndexClause.
-        alterIndex.setColumns(indexDef.getColumns());
+        alterIndex.setColumns(indexDef.getColumnNames());
         newIndexes.add(alterIndex);
         return false;
     }
 
-    private boolean checkDuplicateIndexes(List<Index> indexes, IndexDef 
indexDef, Set<String> newColset,
+    private boolean checkDuplicateIndexes(List<Index> indexes, IndexDefinition 
indexDef, Set<String> newColset,
                                           Set<Long> existedIndexIdSet, 
OlapTable olapTable) throws DdlException {
         for (Index index : indexes) {
             if 
(index.getIndexName().equalsIgnoreCase(indexDef.getIndexName())) {
@@ -2815,7 +2815,7 @@ public class SchemaChangeHandler extends AlterHandler {
                 if (newColset.size() == 1
                         && olapTable.getInvertedIndexFileStorageFormat()
                             .compareTo(TInvertedIndexFileStorageFormat.V2) >= 
0) {
-                    String columnName = indexDef.getColumns().get(0);
+                    String columnName = indexDef.getColumnNames().get(0);
                     Column column = olapTable.getColumn(columnName);
                     if (column != null && (column.getType().isStringType() || 
column.getType().isVariantType())) {
                         boolean isExistingIndexAnalyzer = 
index.isAnalyzedInvertedIndex();
@@ -2831,7 +2831,7 @@ public class SchemaChangeHandler extends AlterHandler {
                     }
                 } else {
                     throw new DdlException(
-                        indexDef.getIndexType() + " index for columns (" + 
String.join(",", indexDef.getColumns())
+                        indexDef.getIndexType() + " index for columns (" + 
String.join(",", indexDef.getColumnNames())
                             + ") already exist.");
                 }
             }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/analysis/BuildIndexClause.java 
b/fe/fe-core/src/main/java/org/apache/doris/analysis/BuildIndexClause.java
index a52b2e9494a..6753af4c043 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BuildIndexClause.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BuildIndexClause.java
@@ -18,7 +18,6 @@
 package org.apache.doris.analysis;
 
 import org.apache.doris.alter.AlterOpType;
-import org.apache.doris.analysis.IndexDef.IndexType;
 import org.apache.doris.catalog.DatabaseIf;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.Index;
@@ -29,6 +28,8 @@ import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.info.PartitionNamesInfo;
 import org.apache.doris.info.TableNameInfo;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
+import 
org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition.IndexType;
 
 import com.google.common.collect.Maps;
 
@@ -39,7 +40,7 @@ public class BuildIndexClause extends AlterTableClause {
     // in which table the index on, only used when alter = false
     private TableNameInfo tableName;
     // index definition class
-    private IndexDef indexDef;
+    private IndexDefinition indexDef;
     // when alter = true, clause like: alter table add index xxxx
     // when alter = false, clause like: create index xx on table xxxx
     private boolean alter;
@@ -58,7 +59,7 @@ public class BuildIndexClause extends AlterTableClause {
     }
 
     // for nereids
-    public BuildIndexClause(TableNameInfo tableName, IndexDef indexDef, Index 
index, boolean alter) {
+    public BuildIndexClause(TableNameInfo tableName, IndexDefinition indexDef, 
Index index, boolean alter) {
         super(AlterOpType.SCHEMA_CHANGE);
         this.tableName = tableName;
         this.indexDef = indexDef;
@@ -75,7 +76,7 @@ public class BuildIndexClause extends AlterTableClause {
         return index;
     }
 
-    public IndexDef getIndexDef() {
+    public IndexDefinition getIndexDef() {
         return indexDef;
     }
 
@@ -110,8 +111,8 @@ public class BuildIndexClause extends AlterTableClause {
                 existedIdx = index;
                 if (!existedIdx.isLightIndexChangeSupported()) {
                     throw new AnalysisException("BUILD INDEX operation failed: 
The index "
-                        + existedIdx.getIndexName() + " of type " + 
existedIdx.getIndexType()
-                        + " does not support lightweight index changes.");
+                            + existedIdx.getIndexName() + " of type " + 
existedIdx.getIndexType()
+                            + " does not support lightweight index changes.");
                 }
                 break;
             }
@@ -120,21 +121,21 @@ public class BuildIndexClause extends AlterTableClause {
             throw new AnalysisException("Index[" + indexName + "] is not exist 
in table[" + tableName.getTbl() + "]");
         }
 
-        IndexDef.IndexType indexType = existedIdx.getIndexType();
-        if ((Config.isNotCloudMode() && indexType == 
IndexDef.IndexType.NGRAM_BF)
+        IndexDefinition.IndexType indexType = existedIdx.getIndexType();
+        if ((Config.isNotCloudMode() && indexType == 
IndexDefinition.IndexType.NGRAM_BF)
                 || (Config.isCloudMode() && indexType == IndexType.INVERTED & 
!existedIdx.isInvertedIndexParserNone())
-                || indexType == IndexDef.IndexType.BLOOMFILTER) {
+                || indexType == IndexType.BLOOMFILTER) {
             throw new AnalysisException("bloomfilter index is not needed to 
build.");
         }
-        indexDef = new IndexDef(indexName, partitionNames, indexType, true);
+        indexDef = new IndexDefinition(indexName, partitionNames, indexType);
         if (!table.isPartitionedTable()) {
-            List<String> specifiedPartitions = 
indexDef.getPartitionNamesInfo();
+            List<String> specifiedPartitions = indexDef.getPartitionNames();
             if (!specifiedPartitions.isEmpty()) {
                 throw new AnalysisException("table " + table.getName()
-                    + " is not partitioned, cannot build index with 
partitions.");
+                        + " is not partitioned, cannot build index with 
partitions.");
             }
         }
-        indexDef.analyze();
+        indexDef.validate();
         this.index = existedIdx.clone();
     }
 
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java 
b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java
index 0c931a65fd1..b143ba4f829 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java
@@ -22,6 +22,7 @@ import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.Index;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.info.TableNameInfo;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
 
 import com.google.common.collect.Maps;
 
@@ -31,14 +32,14 @@ public class CreateIndexClause extends AlterTableClause {
     // in which table the index on, only used when alter = false
     private TableNameInfo tableNameInfo;
     // index definition class
-    private IndexDef indexDef;
+    private IndexDefinition indexDef;
     // when alter = true, clause like: alter table add index xxxx
     // when alter = false, clause like: create index xx on table xxxx
     private boolean alter;
     // index internal class
     private Index index;
 
-    public CreateIndexClause(TableNameInfo tableNameInfo, IndexDef indexDef, 
boolean alter) {
+    public CreateIndexClause(TableNameInfo tableNameInfo, IndexDefinition 
indexDef, boolean alter) {
         super(AlterOpType.SCHEMA_CHANGE);
         this.tableNameInfo = tableNameInfo;
         this.indexDef = indexDef;
@@ -46,7 +47,7 @@ public class CreateIndexClause extends AlterTableClause {
     }
 
     // for nereids
-    public CreateIndexClause(TableNameInfo tableNameInfo, IndexDef indexDef, 
Index index, boolean alter) {
+    public CreateIndexClause(TableNameInfo tableNameInfo, IndexDefinition 
indexDef, Index index, boolean alter) {
         super(AlterOpType.SCHEMA_CHANGE);
         this.tableNameInfo = tableNameInfo;
         this.indexDef = indexDef;
@@ -63,7 +64,7 @@ public class CreateIndexClause extends AlterTableClause {
         return index;
     }
 
-    public IndexDef getIndexDef() {
+    public IndexDefinition getIndexDef() {
         return indexDef;
     }
 
@@ -80,9 +81,9 @@ public class CreateIndexClause extends AlterTableClause {
         if (indexDef == null) {
             throw new AnalysisException("index definition expected.");
         }
-        indexDef.analyze();
+        indexDef.validate();
         this.index = new Index(Env.getCurrentEnv().getNextId(), 
indexDef.getIndexName(),
-                indexDef.getColumns(), indexDef.getIndexType(),
+                indexDef.getColumnNames(), indexDef.getIndexType(),
                 indexDef.getProperties(), indexDef.getComment());
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java 
b/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java
deleted file mode 100644
index 393cd5f1820..00000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java
+++ /dev/null
@@ -1,356 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-package org.apache.doris.analysis;
-
-import org.apache.doris.catalog.ArrayType;
-import org.apache.doris.catalog.Column;
-import org.apache.doris.catalog.KeysType;
-import org.apache.doris.catalog.PrimitiveType;
-import org.apache.doris.catalog.Type;
-import org.apache.doris.common.AnalysisException;
-import org.apache.doris.common.Config;
-import org.apache.doris.common.util.SqlUtils;
-import org.apache.doris.info.PartitionNamesInfo;
-import org.apache.doris.thrift.TInvertedIndexFileStorageFormat;
-
-import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeSet;
-
-public class IndexDef {
-    private String indexName;
-    private boolean ifNotExists;
-    private List<String> columns;
-    // add the column name of olapTable column into caseSensitivityColumns
-    // instead of the column name which from sql_parser analyze
-    private List<String> caseSensitivityColumns = Lists.newArrayList();
-    private IndexType indexType;
-    private String comment;
-    private Map<String, String> properties;
-    private boolean isBuildDeferred = false;
-    private PartitionNamesInfo partitionNamesInfo;
-    public static final int MIN_NGRAM_SIZE = 1;
-    public static final int MAX_NGRAM_SIZE = 255;
-    public static final int MIN_BF_SIZE = 64;
-    public static final int MAX_BF_SIZE = 65535;
-
-    public static final String NGRAM_SIZE_KEY = "gram_size";
-    public static final String NGRAM_BF_SIZE_KEY = "bf_size";
-    public static final String DEFAULT_NGRAM_SIZE = "2";
-    public static final String DEFAULT_NGRAM_BF_SIZE = "256";
-
-
-    public IndexDef(String indexName, boolean ifNotExists, List<String> 
columns, IndexType indexType,
-                    Map<String, String> properties, String comment) {
-        this.indexName = indexName;
-        this.ifNotExists = ifNotExists;
-        this.columns = columns;
-        if (indexType == null) {
-            this.indexType = IndexType.INVERTED;
-        } else {
-            this.indexType = indexType;
-        }
-        if (columns == null) {
-            this.comment = "";
-        } else {
-            this.comment = comment;
-        }
-        if (properties == null) {
-            this.properties = new HashMap<>();
-        } else {
-            this.properties = properties;
-        }
-        if (indexType == IndexType.NGRAM_BF) {
-            this.properties.putIfAbsent(NGRAM_SIZE_KEY, DEFAULT_NGRAM_SIZE);
-            this.properties.putIfAbsent(NGRAM_BF_SIZE_KEY, 
DEFAULT_NGRAM_BF_SIZE);
-        }
-    }
-
-    public IndexDef(String indexName, PartitionNamesInfo partitionNamesInfo,
-                        IndexType indexType, boolean isBuildDeferred) {
-        this.indexName = indexName;
-        this.indexType = indexType;
-        this.partitionNamesInfo = partitionNamesInfo;
-        this.isBuildDeferred = isBuildDeferred;
-    }
-
-    public void analyze() throws AnalysisException {
-        if (isBuildDeferred && (indexType == IndexDef.IndexType.INVERTED || 
indexType == IndexDef.IndexType.ANN)) {
-            if (Strings.isNullOrEmpty(indexName)) {
-                throw new AnalysisException("index name cannot be blank.");
-            }
-            if (indexName.length() > 128) {
-                throw new AnalysisException("index name too long, the index 
name length at most is 128.");
-            }
-            return;
-        }
-
-        if (indexType == IndexDef.IndexType.BITMAP
-                || indexType == IndexDef.IndexType.INVERTED
-                || indexType == IndexDef.IndexType.ANN) {
-            if (columns == null || columns.size() != 1) {
-                throw new AnalysisException(indexType.toString() + " index can 
only apply to a single column.");
-            }
-            if (Strings.isNullOrEmpty(indexName)) {
-                throw new AnalysisException("index name cannot be blank.");
-            }
-            if (indexName.length() > 64) {
-                throw new AnalysisException("index name too long, the index 
name length at most is 64.");
-            }
-            TreeSet<String> distinct = new 
TreeSet<>(String.CASE_INSENSITIVE_ORDER);
-            distinct.addAll(columns);
-            if (columns.size() != distinct.size()) {
-                throw new AnalysisException("columns of index has 
duplicated.");
-            }
-        }
-    }
-
-    public String toSql() {
-        return toSql(null);
-    }
-
-    public String toSql(String tableName) {
-        StringBuilder sb = new StringBuilder("INDEX ");
-        sb.append("`" + indexName + "`");
-        if (tableName != null && !tableName.isEmpty()) {
-            sb.append(" ON ").append(tableName);
-        }
-        if (columns != null && columns.size() > 0) {
-            sb.append(" (");
-            boolean first = true;
-            for (String col : columns) {
-                if (first) {
-                    first = false;
-                } else {
-                    sb.append(",");
-                }
-                sb.append("`" + col + "`");
-            }
-            sb.append(")");
-        }
-        if (indexType != null) {
-            sb.append(" USING ").append(indexType.toString());
-        }
-        if (properties != null && properties.size() > 0) {
-            sb.append(" PROPERTIES(");
-            boolean first = true;
-            for (Map.Entry<String, String> e : properties.entrySet()) {
-                if (first) {
-                    first = false;
-                } else {
-                    sb.append(", ");
-                }
-                
sb.append("\"").append(e.getKey()).append("\"=").append("\"").append(e.getValue()).append("\"");
-            }
-            sb.append(")");
-        }
-        if (comment != null) {
-            sb.append(" COMMENT 
\"").append(SqlUtils.escapeQuota(comment)).append("\"");
-        }
-        return sb.toString();
-    }
-
-    @Override
-    public String toString() {
-        return toSql();
-    }
-
-    public String getIndexName() {
-        return indexName;
-    }
-
-    public List<String> getColumns() {
-        if (caseSensitivityColumns.size() > 0) {
-            return caseSensitivityColumns;
-        }
-        return columns;
-    }
-
-    public IndexType getIndexType() {
-        return indexType;
-    }
-
-    public Map<String, String> getProperties() {
-        return properties;
-    }
-
-    public String getComment() {
-        return comment;
-    }
-
-    public boolean isSetIfNotExists() {
-        return ifNotExists;
-    }
-
-    public List<String> getPartitionNamesInfo() {
-        return partitionNamesInfo == null ? Lists.newArrayList() : 
partitionNamesInfo.getPartitionNames();
-    }
-
-    public enum IndexType {
-        BITMAP,
-        INVERTED,
-        BLOOMFILTER,
-        NGRAM_BF,
-        ANN
-    }
-
-    public boolean isInvertedIndex() {
-        return (this.indexType == IndexType.INVERTED);
-    }
-
-    public boolean isAnnIndex() {
-        return (this.indexType == IndexType.ANN);
-    }
-
-    // Check if the column type is supported for inverted index
-    public boolean isSupportIdxType(Type colType) {
-        if (colType.isArrayType()) {
-            Type itemType = ((ArrayType) colType).getItemType();
-            if (itemType.isArrayType()) {
-                return false;
-            }
-            return isSupportIdxType(itemType);
-        }
-        PrimitiveType primitiveType = colType.getPrimitiveType();
-        return primitiveType.isDateType() || primitiveType.isDecimalV2Type() 
|| primitiveType.isDecimalV3Type()
-                || primitiveType.isFixedPointType() || 
primitiveType.isStringType()
-                || primitiveType == PrimitiveType.BOOLEAN
-                || primitiveType.isVariantType() || primitiveType.isIPType();
-    }
-
-    public void checkColumn(Column column, KeysType keysType, boolean 
enableUniqueKeyMergeOnWrite,
-            TInvertedIndexFileStorageFormat invertedIndexFileStorageFormat) 
throws AnalysisException {
-        if (indexType == IndexType.ANN) {
-            if (column.isAllowNull()) {
-                throw new AnalysisException("ANN index must be built on a 
column that is not nullable");
-            }
-
-            String indexColName = column.getName();
-            caseSensitivityColumns.add(indexColName);
-            PrimitiveType primitiveType = column.getDataType();
-            if (!primitiveType.isArrayType()) {
-                throw new AnalysisException("ANN index column must be array 
type");
-            }
-            Type columnType = column.getType();
-            Type itemType = ((ArrayType) columnType).getItemType();
-            if (!itemType.isFloatingPointType()) {
-                throw new AnalysisException("ANN index column item type must 
be float type");
-            }
-            if (keysType != KeysType.DUP_KEYS) {
-                throw new AnalysisException("ANN index can only be used in 
DUP_KEYS table");
-            }
-            if (invertedIndexFileStorageFormat == 
TInvertedIndexFileStorageFormat.V1) {
-                throw new AnalysisException("ANN index is not supported in 
index format V1");
-            }
-            return;
-        }
-
-        if (indexType == IndexType.BITMAP || indexType == IndexType.INVERTED 
|| indexType == IndexType.BLOOMFILTER
-                || indexType == IndexType.NGRAM_BF) {
-            String indexColName = column.getName();
-            caseSensitivityColumns.add(indexColName);
-            PrimitiveType colType = column.getDataType();
-            Type columnType = column.getType();
-            if (!isSupportIdxType(columnType)) {
-                throw new AnalysisException(colType + " is not supported in " 
+ indexType.toString() + " index. "
-                        + "invalid index: " + indexName);
-            }
-
-            if (indexType == IndexType.ANN && !colType.isArrayType()) {
-                throw new AnalysisException("ANN index column must be array 
type");
-            }
-
-            // In inverted index format v1, each subcolumn of a variant has 
its own index file, leading to high IOPS.
-            // when the subcolumn type changes, it may result in missing 
files, causing link file failure.
-            // There are two cases in which the inverted index format v1 is 
not supported:
-            // 1. in cloud mode
-            // 2. enable_inverted_index_v1_for_variant = false
-            boolean notSupportInvertedIndexForVariant =
-                    (invertedIndexFileStorageFormat == 
TInvertedIndexFileStorageFormat.V1
-                        || invertedIndexFileStorageFormat == 
TInvertedIndexFileStorageFormat.DEFAULT)
-                            && (Config.isCloudMode() || 
!Config.enable_inverted_index_v1_for_variant);
-
-            if (colType.isVariantType() && notSupportInvertedIndexForVariant) {
-                throw new AnalysisException(colType + " is not supported in 
inverted index format V1,"
-                        + "Please set 
properties(\"inverted_index_storage_format\"= \"v2\"),"
-                        + "or upgrade to a newer version");
-            }
-            if (!column.isKey()) {
-                if (keysType == KeysType.AGG_KEYS) {
-                    throw new AnalysisException("index should only be used in 
columns of DUP_KEYS/UNIQUE_KEYS table"
-                        + " or key columns of AGG_KEYS table. invalid index: " 
+ indexName);
-                } else if (keysType == KeysType.UNIQUE_KEYS && 
!enableUniqueKeyMergeOnWrite
-                        && indexType == IndexType.INVERTED && properties != 
null
-                        && 
(properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY)
-                            || 
properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY_ALIAS)
-                            || 
properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_CUSTOM_ANALYZER_KEY))) {
-                    throw new AnalysisException("INVERTED index with parser 
can NOT be used in value columns of"
-                        + " UNIQUE_KEYS table with merge_on_write disable. 
invalid index: " + indexName);
-                }
-            }
-
-            if (indexType == IndexType.INVERTED) {
-                InvertedIndexUtil.checkInvertedIndexParser(indexColName, 
colType, properties,
-                        invertedIndexFileStorageFormat);
-            } else if (indexType == IndexType.NGRAM_BF) {
-                if (colType != PrimitiveType.CHAR && colType != 
PrimitiveType.VARCHAR
-                        && colType != PrimitiveType.STRING) {
-                    throw new AnalysisException(colType + " is not supported 
in ngram_bf index. "
-                                                    + "invalid column: " + 
indexColName);
-                }
-                if (properties.size() != 2) {
-                    throw new AnalysisException("ngram_bf index should have 
gram_size and bf_size properties");
-                }
-
-                parseAndValidateProperty(properties, NGRAM_SIZE_KEY, 
MIN_NGRAM_SIZE, MAX_NGRAM_SIZE);
-                parseAndValidateProperty(properties, NGRAM_BF_SIZE_KEY, 
MIN_BF_SIZE, MAX_BF_SIZE);
-            }
-        } else {
-            throw new AnalysisException("Unsupported index type: " + 
indexType);
-        }
-    }
-
-    public static void parseAndValidateProperty(Map<String, String> 
properties, String key, int minValue, int maxValue)
-            throws AnalysisException {
-        String valueStr = properties.get(key);
-        if (valueStr == null) {
-            throw new AnalysisException("Property '" + key + "' is missing.");
-        }
-        try {
-            int value = Integer.parseInt(valueStr);
-            if (value < minValue || value > maxValue) {
-                throw new AnalysisException("'" + key + "' should be an 
integer between "
-                                                + minValue + " and " + 
maxValue + ".");
-            }
-        } catch (NumberFormatException e) {
-            throw new AnalysisException("Invalid value for '" + key + "': " + 
valueStr, e);
-        }
-    }
-
-    public boolean isAnalyzedInvertedIndex() {
-        return indexType == IndexDef.IndexType.INVERTED
-            && properties != null
-            && 
(properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY)
-                || 
properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY_ALIAS)
-                || 
properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_CUSTOM_ANALYZER_KEY));
-    }
-}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
index 6bd014aec59..052537ea655 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
@@ -20,12 +20,12 @@ package org.apache.doris.catalog;
 import org.apache.doris.alter.SchemaChangeHandler;
 import org.apache.doris.analysis.DefaultValueExprDef;
 import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.IndexDef;
 import org.apache.doris.analysis.SlotRef;
 import org.apache.doris.common.CaseSensibility;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.util.SqlUtils;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
 import org.apache.doris.persist.gson.GsonPostProcessable;
 import org.apache.doris.proto.OlapFile;
 import org.apache.doris.proto.OlapFile.PatternTypePB;
@@ -856,7 +856,7 @@ public class Column implements GsonPostProcessable {
 
         if (indexes != null) {
             for (Index index : indexes) {
-                if (index.getIndexType() == IndexDef.IndexType.BITMAP) {
+                if (index.getIndexType() == IndexDefinition.IndexType.BITMAP) {
                     List<String> columns = index.getColumns();
                     if (this.name.equalsIgnoreCase(columns.get(0))) {
                         builder.setHasBitmapIndex(true);
@@ -1224,7 +1224,7 @@ public class Column implements GsonPostProcessable {
     public void setIndexFlag(TColumn tColumn, OlapTable olapTable) {
         List<Index> indexes = olapTable.getIndexes();
         for (Index index : indexes) {
-            if (index.getIndexType() == IndexDef.IndexType.BITMAP) {
+            if (index.getIndexType() == IndexDefinition.IndexType.BITMAP) {
                 List<String> columns = index.getColumns();
                 if (tColumn.getColumnName().equals(columns.get(0))) {
                     tColumn.setHasBitmapIndex(true);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java
index 2bec0471602..9e8f873bf42 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java
@@ -17,8 +17,6 @@
 
 package org.apache.doris.catalog;
 
-import org.apache.doris.analysis.IndexDef;
-import org.apache.doris.analysis.IndexDef.IndexType;
 import org.apache.doris.analysis.InvertedIndexUtil;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
@@ -26,6 +24,8 @@ import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.common.util.PrintableMap;
 import org.apache.doris.common.util.SqlUtils;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
+import 
org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition.IndexType;
 import org.apache.doris.persist.gson.GsonUtils;
 import org.apache.doris.proto.OlapFile;
 import org.apache.doris.thrift.TIndexType;
@@ -62,21 +62,21 @@ public class Index implements Writable {
     @SerializedName(value = "c", alternate = {"columns"})
     private List<String> columns;
     @SerializedName(value = "it", alternate = {"indexType"})
-    private IndexDef.IndexType indexType;
+    private IndexDefinition.IndexType indexType;
     @SerializedName(value = "pt", alternate = {"properties"})
     private Map<String, String> properties;
     @SerializedName(value = "ct", alternate = {"comment"})
     private String comment;
 
     public Index(long indexId, String indexName, List<String> columns,
-            IndexDef.IndexType indexType, Map<String, String> properties, 
String comment) {
+            IndexDefinition.IndexType indexType, Map<String, String> 
properties, String comment) {
         this.indexId = indexId;
         this.indexName = indexName;
         this.columns = columns == null ? Lists.newArrayList() : 
Lists.newArrayList(columns);
         this.indexType = indexType;
         this.properties = properties == null ? Maps.newHashMap() : 
Maps.newHashMap(properties);
         this.comment = comment;
-        if (indexType == IndexDef.IndexType.INVERTED) {
+        if (indexType == IndexDefinition.IndexType.INVERTED) {
             if (this.properties != null && !this.properties.isEmpty()) {
                 if 
(this.properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY)
                         || 
this.properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY_ALIAS)
@@ -130,11 +130,11 @@ public class Index implements Writable {
         this.columns = columns;
     }
 
-    public IndexDef.IndexType getIndexType() {
+    public IndexDefinition.IndexType getIndexType() {
         return indexType;
     }
 
-    public void setIndexType(IndexDef.IndexType indexType) {
+    public void setIndexType(IndexDefinition.IndexType indexType) {
         this.indexType = indexType;
     }
 
@@ -189,7 +189,7 @@ public class Index implements Writable {
 
     // Whether the index can be changed in light mode
     public boolean isLightIndexChangeSupported() {
-        return indexType == IndexDef.IndexType.INVERTED || indexType == 
IndexType.NGRAM_BF;
+        return indexType == IndexDefinition.IndexType.INVERTED || indexType == 
IndexType.NGRAM_BF;
     }
 
     // Whether the index can be added in light mode
@@ -198,15 +198,15 @@ public class Index implements Writable {
     // the rest of the index types do not support light add
     public boolean isLightAddIndexSupported(boolean enableAddIndexForNewData) {
         if (Config.isCloudMode()) {
-            if (indexType == IndexDef.IndexType.INVERTED) {
+            if (indexType == IndexDefinition.IndexType.INVERTED) {
                 return isInvertedIndexParserNone() && enableAddIndexForNewData;
-            } else if (indexType == IndexDef.IndexType.NGRAM_BF) {
+            } else if (indexType == IndexDefinition.IndexType.NGRAM_BF) {
                 return enableAddIndexForNewData;
             }
             return false;
         }
-        return (indexType == IndexDef.IndexType.NGRAM_BF && 
enableAddIndexForNewData)
-                || (indexType == IndexDef.IndexType.INVERTED);
+        return (indexType == IndexDefinition.IndexType.NGRAM_BF && 
enableAddIndexForNewData)
+                || (indexType == IndexDefinition.IndexType.INVERTED);
     }
 
     public String getInvertedIndexCustomAnalyzer() {
@@ -359,8 +359,8 @@ public class Index implements Writable {
         bloomFilters = bloomFilters == null ? Collections.emptySet() : 
bloomFilters;
         Set<String> bfColumns = new HashSet<>();
         for (Index index : indices) {
-            if (IndexDef.IndexType.NGRAM_BF == index.getIndexType()
-                    || IndexDef.IndexType.BLOOMFILTER == index.getIndexType()) 
{
+            if (IndexDefinition.IndexType.NGRAM_BF == index.getIndexType()
+                    || IndexDefinition.IndexType.BLOOMFILTER == 
index.getIndexType()) {
                 for (String column : index.getColumns()) {
                     column = column.toLowerCase();
                     if (bfColumns.contains(column)) {
@@ -382,9 +382,9 @@ public class Index implements Writable {
     }
 
     public boolean isAnalyzedInvertedIndex() {
-        return indexType == IndexDef.IndexType.INVERTED
-            && properties != null
-            && 
(properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY)
+        return indexType == IndexDefinition.IndexType.INVERTED
+                && properties != null
+                && 
(properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY)
                 || 
properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY_ALIAS)
                 || 
properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_CUSTOM_ANALYZER_KEY));
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
index f3fb4914ff5..1f0d402d11f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
@@ -20,7 +20,6 @@ package org.apache.doris.catalog;
 import org.apache.doris.alter.MaterializedViewHandler;
 import org.apache.doris.analysis.ColumnDef;
 import org.apache.doris.analysis.DataSortInfo;
-import org.apache.doris.analysis.IndexDef;
 import org.apache.doris.backup.Status;
 import org.apache.doris.backup.Status.ErrCode;
 import org.apache.doris.catalog.DistributionInfo.DistributionInfoType;
@@ -56,6 +55,7 @@ import org.apache.doris.mtmv.MTMVVersionSnapshot;
 import org.apache.doris.nereids.hint.Hint;
 import org.apache.doris.nereids.hint.UseMvHint;
 import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
 import org.apache.doris.persist.ColocatePersistInfo;
 import org.apache.doris.persist.gson.GsonPostProcessable;
 import org.apache.doris.persist.gson.GsonUtils;
@@ -407,7 +407,7 @@ public class OlapTable extends Table implements 
MTMVRelatedTableIf, GsonPostProc
      * @param indexType The index type to check for
      * @return true if the table has at least one index of the specified type, 
false otherwise
      */
-    public boolean hasIndexOfType(IndexDef.IndexType indexType) {
+    public boolean hasIndexOfType(IndexDefinition.IndexType indexType) {
         if (indexes == null) {
             return false;
         }
@@ -3642,7 +3642,7 @@ public class OlapTable extends Table implements 
MTMVRelatedTableIf, GsonPostProc
     public Index getInvertedIndex(Column column, List<String> subPath) {
         List<Index> invertedIndexes = new ArrayList<>();
         for (Index index : indexes.getIndexes()) {
-            if (index.getIndexType() == IndexDef.IndexType.INVERTED) {
+            if (index.getIndexType() == IndexDefinition.IndexType.INVERTED) {
                 List<String> columns = index.getColumns();
                 if (columns != null && !columns.isEmpty() && 
column.getName().equals(columns.get(0))) {
                     invertedIndexes.add(index);
@@ -3652,7 +3652,7 @@ public class OlapTable extends Table implements 
MTMVRelatedTableIf, GsonPostProc
 
         if (subPath == null || subPath.isEmpty()) {
             return invertedIndexes.size() == 1 ? invertedIndexes.get(0)
-                : 
invertedIndexes.stream().filter(Index::isAnalyzedInvertedIndex).findFirst().orElse(null);
+                    : 
invertedIndexes.stream().filter(Index::isAnalyzedInvertedIndex).findFirst().orElse(null);
         }
 
         // subPath is not empty, means it is a variant column, find the field 
pattern from children
@@ -3681,7 +3681,7 @@ public class OlapTable extends Table implements 
MTMVRelatedTableIf, GsonPostProc
 
         List<Index> invertedIndexesWithFieldPattern = new ArrayList<>();
         for (Index index : indexes.getIndexes()) {
-            if (index.getIndexType() == IndexDef.IndexType.INVERTED) {
+            if (index.getIndexType() == IndexDefinition.IndexType.INVERTED) {
                 List<String> columns = index.getColumns();
                 if (columns != null && !columns.isEmpty() && 
column.getName().equals(columns.get(0))
                                         && 
fieldPattern.equals(index.getInvertedIndexFieldPattern())) {
@@ -3691,7 +3691,7 @@ public class OlapTable extends Table implements 
MTMVRelatedTableIf, GsonPostProc
         }
         if (invertedIndexesWithFieldPattern.isEmpty()) {
             return invertedIndexes.size() == 1 ? invertedIndexes.get(0)
-                : 
invertedIndexes.stream().filter(Index::isAnalyzedInvertedIndex).findFirst().orElse(null);
+                    : 
invertedIndexes.stream().filter(Index::isAnalyzedInvertedIndex).findFirst().orElse(null);
         } else {
             return invertedIndexesWithFieldPattern.size() == 1 ? 
invertedIndexesWithFieldPattern.get(0)
                                         : 
invertedIndexesWithFieldPattern.stream()
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java
index 8e52c8aa28c..8cf96ef3072 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java
@@ -17,7 +17,6 @@
 
 package org.apache.doris.nereids.rules.implementation;
 
-import org.apache.doris.analysis.IndexDef.IndexType;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Index;
 import org.apache.doris.catalog.KeysType;
@@ -45,6 +44,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.agg.Min;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.algebra.Project;
+import 
org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition.IndexType;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
 import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownVectorTopNIntoOlapScan.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownVectorTopNIntoOlapScan.java
index 26750184a70..46ef4d0af2b 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownVectorTopNIntoOlapScan.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownVectorTopNIntoOlapScan.java
@@ -17,7 +17,6 @@
 
 package org.apache.doris.nereids.rules.rewrite;
 
-import org.apache.doris.analysis.IndexDef.IndexType;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Index;
 import org.apache.doris.catalog.TableIf;
@@ -31,6 +30,7 @@ import 
org.apache.doris.nereids.trees.expressions.SlotReference;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProductApproximate;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.L2DistanceApproximate;
 import org.apache.doris.nereids.trees.plans.Plan;
+import 
org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition.IndexType;
 import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
 import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BuildIndexOp.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BuildIndexOp.java
index cf8330e7c6d..0706e92c311 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BuildIndexOp.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BuildIndexOp.java
@@ -20,8 +20,6 @@ package org.apache.doris.nereids.trees.plans.commands.info;
 import org.apache.doris.alter.AlterOpType;
 import org.apache.doris.analysis.AlterTableClause;
 import org.apache.doris.analysis.BuildIndexClause;
-import org.apache.doris.analysis.IndexDef;
-import org.apache.doris.analysis.IndexDef.IndexType;
 import org.apache.doris.catalog.DatabaseIf;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.Index;
@@ -33,6 +31,7 @@ import org.apache.doris.common.Config;
 import org.apache.doris.common.UserException;
 import org.apache.doris.info.PartitionNamesInfo;
 import org.apache.doris.info.TableNameInfo;
+import 
org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition.IndexType;
 import org.apache.doris.qe.ConnectContext;
 
 import com.google.common.collect.Maps;
@@ -126,8 +125,8 @@ public class BuildIndexOp extends AlterTableOp {
                 existedIdx = index;
                 if (!existedIdx.isLightIndexChangeSupported()) {
                     throw new AnalysisException("BUILD INDEX operation failed: 
The index "
-                        + existedIdx.getIndexName() + " of type " + 
existedIdx.getIndexType()
-                        + " does not support lightweight index changes.");
+                            + existedIdx.getIndexName() + " of type " + 
existedIdx.getIndexType()
+                            + " does not support lightweight index changes.");
                 }
                 break;
             }
@@ -136,9 +135,9 @@ public class BuildIndexOp extends AlterTableOp {
             throw new AnalysisException("Index[" + indexName + "] is not exist 
in table[" + tableName.getTbl() + "]");
         }
 
-        IndexDef.IndexType indexType = existedIdx.getIndexType();
-        if ((Config.isNotCloudMode() && indexType == 
IndexDef.IndexType.NGRAM_BF)
-                || indexType == IndexDef.IndexType.BLOOMFILTER
+        IndexType indexType = existedIdx.getIndexType();
+        if ((Config.isNotCloudMode() && indexType == IndexType.NGRAM_BF)
+                || indexType == IndexType.BLOOMFILTER
                 || (Config.isCloudMode()
                 && indexType == IndexType.INVERTED & 
!existedIdx.isInvertedIndexParserNone())) {
             throw new AnalysisException(indexType + " index is not needed to 
build.");
@@ -149,12 +148,12 @@ public class BuildIndexOp extends AlterTableOp {
             List<String> specifiedPartitions = indexDef.getPartitionNames();
             if (!specifiedPartitions.isEmpty()) {
                 throw new AnalysisException("table " + table.getName()
-                    + " is not partitioned, cannot build index with 
partitions.");
+                        + " is not partitioned, cannot build index with 
partitions.");
             }
         }
-        if (indexDef.getIndexType() == IndexDef.IndexType.ANN) {
+        if (indexDef.getIndexType() == IndexType.ANN) {
             throw new AnalysisException(
-                "ANN index can only be created during table creation, not 
through BUILD INDEX.");
+                    "ANN index can only be created during table creation, not 
through BUILD INDEX.");
         }
         indexDef.validate();
         this.index = existedIdx.clone();
@@ -163,7 +162,7 @@ public class BuildIndexOp extends AlterTableOp {
     @Override
     public AlterTableClause translateToLegacyAlterClause() {
         indexDef.getIndexType();
-        return new BuildIndexClause(tableName, 
indexDef.translateToLegacyIndexDef(), index, alter);
+        return new BuildIndexClause(tableName, indexDef, index, alter);
     }
 
     @Override
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateIndexOp.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateIndexOp.java
index 9ecd3946678..c7efb3b9fe3 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateIndexOp.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateIndexOp.java
@@ -20,7 +20,6 @@ package org.apache.doris.nereids.trees.plans.commands.info;
 import org.apache.doris.alter.AlterOpType;
 import org.apache.doris.analysis.AlterTableClause;
 import org.apache.doris.analysis.CreateIndexClause;
-import org.apache.doris.analysis.IndexDef;
 import org.apache.doris.catalog.Index;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.UserException;
@@ -43,10 +42,10 @@ public class CreateIndexOp extends AlterTableOp {
     // index internal class
     private Index index;
 
-    public CreateIndexOp(TableNameInfo tableName, IndexDefinition indexDef, 
boolean alter) {
+    public CreateIndexOp(TableNameInfo tableName, IndexDefinition 
indexDefinition, boolean alter) {
         super(AlterOpType.SCHEMA_CHANGE);
         this.tableName = tableName;
-        this.indexDef = indexDef;
+        this.indexDef = indexDefinition;
         this.alter = alter;
     }
 
@@ -80,9 +79,9 @@ public class CreateIndexOp extends AlterTableOp {
             tableName.analyze(ctx);
         }
 
-        if (indexDef.getIndexType() == IndexDef.IndexType.ANN) {
+        if (indexDef.getIndexType() == IndexDefinition.IndexType.ANN) {
             throw new AnalysisException(
-                "ANN index can only be created during table creation, not 
through CREATE INDEX.");
+                    "ANN index can only be created during table creation, not 
through CREATE INDEX.");
         }
 
         indexDef.validate();
@@ -91,7 +90,7 @@ public class CreateIndexOp extends AlterTableOp {
 
     @Override
     public AlterTableClause translateToLegacyAlterClause() {
-        return new CreateIndexClause(tableName, 
indexDef.translateToLegacyIndexDef(), index, alter);
+        return new CreateIndexClause(tableName, indexDef, index, alter);
     }
 
     @Override
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
index fec0161ed8a..ad50ed3ec3d 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
@@ -20,7 +20,6 @@ package org.apache.doris.nereids.trees.plans.commands.info;
 import org.apache.doris.analysis.AlterClause;
 import org.apache.doris.analysis.DistributionDesc;
 import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.IndexDef.IndexType;
 import org.apache.doris.analysis.InvertedIndexUtil;
 import org.apache.doris.analysis.KeysDesc;
 import org.apache.doris.analysis.PartitionDesc;
@@ -71,6 +70,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.GroupingScala
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Lambda;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.ScalarFunction;
 import 
org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter;
+import 
org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition.IndexType;
 import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation;
 import org.apache.doris.nereids.types.DataType;
 import org.apache.doris.nereids.types.VariantField;
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java
index 347e56aec47..828b7c4e582 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java
@@ -18,12 +18,13 @@
 package org.apache.doris.nereids.trees.plans.commands.info;
 
 import org.apache.doris.analysis.AnnIndexPropertiesChecker;
-import org.apache.doris.analysis.IndexDef;
-import org.apache.doris.analysis.IndexDef.IndexType;
 import org.apache.doris.analysis.InvertedIndexUtil;
+import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.Index;
 import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.catalog.Type;
 import org.apache.doris.common.Config;
 import org.apache.doris.info.PartitionNamesInfo;
 import org.apache.doris.nereids.exceptions.AnalysisException;
@@ -33,7 +34,6 @@ import org.apache.doris.nereids.util.Utils;
 import org.apache.doris.thrift.TInvertedIndexFileStorageFormat;
 
 import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.util.HashMap;
@@ -45,6 +45,15 @@ import java.util.TreeSet;
  * index definition
  */
 public class IndexDefinition {
+    public static final int MIN_NGRAM_SIZE = 1;
+    public static final int MAX_NGRAM_SIZE = 255;
+    public static final int MIN_BF_SIZE = 64;
+    public static final int MAX_BF_SIZE = 65535;
+
+    public static final String NGRAM_SIZE_KEY = "gram_size";
+    public static final String NGRAM_BF_SIZE_KEY = "bf_size";
+    public static final String DEFAULT_NGRAM_SIZE = "2";
+    public static final String DEFAULT_NGRAM_BF_SIZE = "256";
     private final String name;
     private final List<String> cols;
     private final String comment;
@@ -59,6 +68,17 @@ public class IndexDefinition {
 
     private PartitionNamesInfo partitionNames;
 
+    /**
+     * IndexType
+     */
+    public enum IndexType {
+        BITMAP,
+        INVERTED,
+        BLOOMFILTER,
+        NGRAM_BF,
+        ANN
+    }
+
     /**
      * constructor for IndexDefinition
      */
@@ -96,8 +116,8 @@ public class IndexDefinition {
         }
 
         if (indexType == IndexType.NGRAM_BF) {
-            this.properties.putIfAbsent(IndexDef.NGRAM_SIZE_KEY, 
IndexDef.DEFAULT_NGRAM_SIZE);
-            this.properties.putIfAbsent(IndexDef.NGRAM_BF_SIZE_KEY, 
IndexDef.DEFAULT_NGRAM_BF_SIZE);
+            this.properties.putIfAbsent(NGRAM_SIZE_KEY, DEFAULT_NGRAM_SIZE);
+            this.properties.putIfAbsent(NGRAM_BF_SIZE_KEY, 
DEFAULT_NGRAM_BF_SIZE);
         }
 
         this.comment = comment;
@@ -216,10 +236,8 @@ public class IndexDefinition {
                             "ngram_bf index should have gram_size and bf_size 
properties");
                 }
                 try {
-                    IndexDef.parseAndValidateProperty(properties, 
IndexDef.NGRAM_SIZE_KEY, IndexDef.MIN_NGRAM_SIZE,
-                                                      IndexDef.MAX_NGRAM_SIZE);
-                    IndexDef.parseAndValidateProperty(properties, 
IndexDef.NGRAM_BF_SIZE_KEY, IndexDef.MIN_BF_SIZE,
-                                                      IndexDef.MAX_BF_SIZE);
+                    parseAndValidateProperty(properties, NGRAM_SIZE_KEY, 
MIN_NGRAM_SIZE, MAX_NGRAM_SIZE);
+                    parseAndValidateProperty(properties, NGRAM_BF_SIZE_KEY, 
MIN_BF_SIZE, MAX_BF_SIZE);
                 } catch (Exception ex) {
                     throw new AnalysisException("invalid ngram bf index 
params:" + ex.getMessage(), ex);
                 }
@@ -229,6 +247,103 @@ public class IndexDefinition {
         }
     }
 
+    /**
+     * checkColumn
+     */
+    public void checkColumn(Column column, KeysType keysType, boolean 
enableUniqueKeyMergeOnWrite,
+                            TInvertedIndexFileStorageFormat 
invertedIndexFileStorageFormat) throws AnalysisException {
+        if (indexType == IndexType.ANN) {
+            if (column.isAllowNull()) {
+                throw new AnalysisException("ANN index must be built on a 
column that is not nullable");
+            }
+
+            String indexColName = column.getName();
+            caseSensitivityCols.add(indexColName);
+            PrimitiveType primitiveType = column.getDataType();
+            if (!primitiveType.isArrayType()) {
+                throw new AnalysisException("ANN index column must be array 
type");
+            }
+            Type columnType = column.getType();
+            Type itemType = ((org.apache.doris.catalog.ArrayType) 
columnType).getItemType();
+            if (!itemType.isFloatingPointType()) {
+                throw new AnalysisException("ANN index column item type must 
be float type");
+            }
+            if (keysType != KeysType.DUP_KEYS) {
+                throw new AnalysisException("ANN index can only be used in 
DUP_KEYS table");
+            }
+            if (invertedIndexFileStorageFormat == 
TInvertedIndexFileStorageFormat.V1) {
+                throw new AnalysisException("ANN index is not supported in 
index format V1");
+            }
+            return;
+        }
+
+        if (indexType == IndexType.BITMAP || indexType == IndexType.INVERTED 
|| indexType == IndexType.BLOOMFILTER
+                || indexType == IndexType.NGRAM_BF) {
+            String indexColName = column.getName();
+            caseSensitivityCols.add(indexColName);
+            PrimitiveType colType = column.getDataType();
+            Type columnType = column.getType();
+            if (!isSupportIdxType(DataType.fromCatalogType(columnType))) {
+                throw new AnalysisException(colType + " is not supported in " 
+ indexType.toString() + " index. "
+                    + "invalid index: " + name);
+            }
+
+            if (indexType == IndexType.ANN && !colType.isArrayType()) {
+                throw new AnalysisException("ANN index column must be array 
type");
+            }
+
+            // In inverted index format v1, each subcolumn of a variant has 
its own index file, leading to high IOPS.
+            // when the subcolumn type changes, it may result in missing 
files, causing link file failure.
+            // There are two cases in which the inverted index format v1 is 
not supported:
+            // 1. in cloud mode
+            // 2. enable_inverted_index_v1_for_variant = false
+            boolean notSupportInvertedIndexForVariant =
+                    (invertedIndexFileStorageFormat == 
TInvertedIndexFileStorageFormat.V1
+                    || invertedIndexFileStorageFormat == 
TInvertedIndexFileStorageFormat.DEFAULT)
+                    && (Config.isCloudMode() || 
!Config.enable_inverted_index_v1_for_variant);
+
+            if (colType.isVariantType() && notSupportInvertedIndexForVariant) {
+                throw new AnalysisException(colType + " is not supported in 
inverted index format V1,"
+                        + "Please set 
properties(\"inverted_index_storage_format\"= \"v2\"),"
+                        + "or upgrade to a newer version");
+            }
+            if (!column.isKey()) {
+                if (keysType == KeysType.AGG_KEYS) {
+                    throw new AnalysisException("index should only be used in 
columns of DUP_KEYS/UNIQUE_KEYS table"
+                        + " or key columns of AGG_KEYS table. invalid index: " 
+ name);
+                } else if (keysType == KeysType.UNIQUE_KEYS && 
!enableUniqueKeyMergeOnWrite
+                        && indexType == IndexType.INVERTED && properties != 
null
+                        && 
properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY)) {
+                    throw new AnalysisException("INVERTED index with parser 
can NOT be used in value columns of"
+                        + " UNIQUE_KEYS table with merge_on_write disable. 
invalid index: " + name);
+                }
+            }
+
+            if (indexType == IndexType.INVERTED) {
+                try {
+                    InvertedIndexUtil.checkInvertedIndexParser(indexColName, 
colType, properties,
+                            invertedIndexFileStorageFormat);
+                } catch (Exception e) {
+                    throw new AnalysisException("invalid INVERTED index:" + 
e.getMessage(), e);
+                }
+            } else if (indexType == IndexType.NGRAM_BF) {
+                if (colType != PrimitiveType.CHAR && colType != 
PrimitiveType.VARCHAR
+                        && colType != PrimitiveType.STRING) {
+                    throw new AnalysisException(colType + " is not supported 
in ngram_bf index. "
+                        + "invalid column: " + indexColName);
+                }
+                if (properties.size() != 2) {
+                    throw new AnalysisException("ngram_bf index should have 
gram_size and bf_size properties");
+                }
+
+                parseAndValidateProperty(properties, NGRAM_SIZE_KEY, 
MIN_NGRAM_SIZE, MAX_NGRAM_SIZE);
+                parseAndValidateProperty(properties, NGRAM_BF_SIZE_KEY, 
MIN_BF_SIZE, MAX_BF_SIZE);
+            }
+        } else {
+            throw new AnalysisException("Unsupported index type: " + 
indexType);
+        }
+    }
+
     /**
      * validate
      */
@@ -236,7 +351,7 @@ public class IndexDefinition {
         if (partitionNames != null) {
             partitionNames.validate();
         }
-        if (isBuildDeferred && indexType == IndexDef.IndexType.INVERTED) {
+        if (isBuildDeferred && indexType == IndexType.INVERTED) {
             if (Strings.isNullOrEmpty(name)) {
                 throw new AnalysisException("index name cannot be blank.");
             }
@@ -247,11 +362,11 @@ public class IndexDefinition {
             return;
         }
 
-        if (indexType == IndexDef.IndexType.ANN) {
+        if (indexType == IndexType.ANN) {
             AnnIndexPropertiesChecker.checkProperties(this.properties);
         }
 
-        if (indexType == IndexDef.IndexType.BITMAP || indexType == 
IndexDef.IndexType.INVERTED) {
+        if (indexType == IndexType.BITMAP || indexType == IndexType.INVERTED) {
             if (cols == null || cols.size() != 1) {
                 throw new AnalysisException(
                         indexType.toString() + " index can only apply to a 
single column.");
@@ -273,10 +388,9 @@ public class IndexDefinition {
 
     public List<String> getColumnNames() {
         if (!caseSensitivityCols.isEmpty()) {
-            return ImmutableList.copyOf(caseSensitivityCols);
-        } else {
-            return ImmutableList.copyOf(cols);
+            return caseSensitivityCols;
         }
+        return cols;
     }
 
     public String getIndexName() {
@@ -296,15 +410,8 @@ public class IndexDefinition {
         return partitionNames == null ? Lists.newArrayList() : 
partitionNames.getPartitionNames();
     }
 
-    /**
-     * translateToLegacyIndexDef
-     */
-    public IndexDef translateToLegacyIndexDef() {
-        if (isBuildDeferred) {
-            return new IndexDef(name, partitionNames, indexType, true);
-        } else {
-            return new IndexDef(name, ifNotExists, cols, indexType, 
properties, comment);
-        }
+    public boolean isSetIfNotExists() {
+        return ifNotExists;
     }
 
     public String toSql() {
@@ -359,6 +466,31 @@ public class IndexDefinition {
         return properties;
     }
 
+    public boolean isAnnIndex() {
+        return (this.indexType == IndexType.ANN);
+    }
+
+    public String getComment() {
+        return comment;
+    }
+
+    private void parseAndValidateProperty(Map<String, String> properties, 
String key, int minValue, int maxValue)
+            throws AnalysisException {
+        String valueStr = properties.get(key);
+        if (valueStr == null) {
+            throw new AnalysisException("Property '" + key + "' is missing.");
+        }
+        try {
+            int value = Integer.parseInt(valueStr);
+            if (value < minValue || value > maxValue) {
+                throw new AnalysisException("'" + key + "' should be an 
integer between "
+                    + minValue + " and " + maxValue + ".");
+            }
+        } catch (NumberFormatException e) {
+            throw new AnalysisException("Invalid value for '" + key + "': " + 
valueStr, e);
+        }
+    }
+
     public boolean isAnalyzedInvertedIndex() {
         return indexType == IndexType.INVERTED
                 && properties != null
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/alter/CloudIndexTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/alter/CloudIndexTest.java
index 6421d41f6db..4ca115a3ca9 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/CloudIndexTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/CloudIndexTest.java
@@ -22,8 +22,6 @@ import org.apache.doris.analysis.BuildIndexClause;
 import org.apache.doris.analysis.CreateIndexClause;
 import org.apache.doris.analysis.DataSortInfo;
 import org.apache.doris.analysis.DropIndexClause;
-import org.apache.doris.analysis.IndexDef;
-import org.apache.doris.analysis.IndexDef.IndexType;
 import org.apache.doris.analysis.ResourceTypeEnum;
 import org.apache.doris.analysis.UserIdentity;
 import org.apache.doris.catalog.CatalogTestUtil;
@@ -49,6 +47,7 @@ import 
org.apache.doris.mysql.privilege.AccessControllerManager;
 import org.apache.doris.mysql.privilege.Auth;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.nereids.trees.plans.commands.CancelBuildIndexCommand;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
 import org.apache.doris.persist.EditLog;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.resource.computegroup.ComputeGroup;
@@ -383,9 +382,9 @@ public class CloudIndexTest {
         properties.put("gram_size", "2");
         properties.put("bf_size", "256");
 
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDef = new IndexDefinition(indexName, false,
                 Lists.newArrayList(table.getBaseSchema().get(3).getName()),
-                IndexType.NGRAM_BF,
+                "NGRAM_BF",
                 properties, "ngram bf index");
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 table.getName());
@@ -433,13 +432,13 @@ public class CloudIndexTest {
         properties.put("gram_size", "2");
         properties.put("bf_size", "256");
 
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(table.getBaseSchema().get(3).getName()),
-                IndexType.NGRAM_BF,
+                "NGRAM_BF",
                 properties, "ngram bf index");
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 table.getName());
-        createIndexClause = new CreateIndexClause(tableName, indexDef, false);
+        createIndexClause = new CreateIndexClause(tableName, indexDefinition, 
false);
         createIndexClause.analyze();
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(createIndexClause);
@@ -496,13 +495,13 @@ public class CloudIndexTest {
         Map<String, String> properties = Maps.newHashMap();
         properties.put("parser", "none");
 
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(table.getBaseSchema().get(3).getName()),
-                IndexType.INVERTED,
+                "INVERTED",
                 properties, "raw inverted index");
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 table.getName());
-        createIndexClause = new CreateIndexClause(tableName, indexDef, false);
+        createIndexClause = new CreateIndexClause(tableName, indexDefinition, 
false);
         createIndexClause.analyze();
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(createIndexClause);
@@ -558,13 +557,13 @@ public class CloudIndexTest {
         // Explicitly set parser="none" for raw inverted index
         Map<String, String> properties = Maps.newHashMap();
         properties.put("parser", "none");
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(table.getBaseSchema().get(3).getName()),
-                IndexType.INVERTED,
+                "INVERTED",
                 properties, "lightweight raw inverted index");
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 table.getName());
-        createIndexClause = new CreateIndexClause(tableName, indexDef, false);
+        createIndexClause = new CreateIndexClause(tableName, indexDefinition, 
false);
         createIndexClause.analyze();
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(createIndexClause);
@@ -610,13 +609,13 @@ public class CloudIndexTest {
         properties.put("lower_case", "true");
 
         // Use VARCHAR column v1 (index 2) for string type support
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(table.getBaseSchema().get(2).getName()),
-                IndexType.INVERTED,
+                "INVERTED",
                 properties, "tokenized inverted index with english parser");
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 table.getName());
-        createIndexClause = new CreateIndexClause(tableName, indexDef, false);
+        createIndexClause = new CreateIndexClause(tableName, indexDefinition, 
false);
         createIndexClause.analyze();
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(createIndexClause);
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/alter/IndexChangeJobTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/alter/IndexChangeJobTest.java
index 2dd9b1fcd76..460e4d7869b 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/IndexChangeJobTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/IndexChangeJobTest.java
@@ -21,7 +21,6 @@ import org.apache.doris.analysis.AlterClause;
 import org.apache.doris.analysis.BuildIndexClause;
 import org.apache.doris.analysis.CreateIndexClause;
 import org.apache.doris.analysis.DropIndexClause;
-import org.apache.doris.analysis.IndexDef;
 import org.apache.doris.catalog.CatalogTestUtil;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Env;
@@ -41,6 +40,7 @@ import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.UserException;
 import org.apache.doris.info.TableNameInfo;
 import org.apache.doris.nereids.trees.plans.commands.CancelAlterTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.task.AgentTask;
 import org.apache.doris.task.AgentTaskQueue;
@@ -119,11 +119,11 @@ public class IndexChangeJobTest {
         String indexName = "index1";
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 olapTable.getName());
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(olapTable.getBaseSchema().get(1).getName()),
-                IndexDef.IndexType.INVERTED,
+                "INVERTED",
                 Maps.newHashMap(), "balabala");
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         alterClauses.add(createIndexClause);
         schemaChangeHandler.process(alterClauses, db, olapTable);
@@ -146,11 +146,11 @@ public class IndexChangeJobTest {
         String indexName = "index1";
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 olapTable.getName());
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(olapTable.getBaseSchema().get(1).getName()),
-                IndexDef.IndexType.INVERTED,
+                "INVERTED",
                 Maps.newHashMap(), "balabala");
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         alterClauses.add(createIndexClause);
         schemaChangeHandler.process(alterClauses, db, olapTable);
@@ -178,11 +178,11 @@ public class IndexChangeJobTest {
         String indexName = "index1";
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 olapTable.getName());
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(olapTable.getBaseSchema().get(1).getName()),
-                IndexDef.IndexType.INVERTED,
+                "INVERTED",
                 Maps.newHashMap(), "balabala");
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         alterClauses.add(createIndexClause);
         schemaChangeHandler.process(alterClauses, db, olapTable);
@@ -212,11 +212,11 @@ public class IndexChangeJobTest {
         String indexName = "index1";
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 olapTable.getName());
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(olapTable.getBaseSchema().get(1).getName()),
-                IndexDef.IndexType.INVERTED,
+                "INVERTED",
                 Maps.newHashMap(), "balabala");
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         alterClauses.add(createIndexClause);
         schemaChangeHandler.process(alterClauses, db, olapTable);
@@ -270,11 +270,11 @@ public class IndexChangeJobTest {
         String indexName = "index1";
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 olapTable.getName());
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(olapTable.getBaseSchema().get(1).getName()),
-                IndexDef.IndexType.INVERTED,
+                "INVERTED",
                 Maps.newHashMap(), "balabala");
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         alterClauses.add(createIndexClause);
         schemaChangeHandler.process(alterClauses, db, olapTable);
@@ -327,11 +327,11 @@ public class IndexChangeJobTest {
         String indexName = "index1";
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 olapTable.getName());
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(olapTable.getBaseSchema().get(1).getName()),
-                IndexDef.IndexType.INVERTED,
+                "INVERTED",
                 Maps.newHashMap(), "balabala");
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         alterClauses.add(createIndexClause);
         schemaChangeHandler.process(alterClauses, db, olapTable);
@@ -374,11 +374,11 @@ public class IndexChangeJobTest {
         String indexName = "index1";
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 olapTable.getName());
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(olapTable.getBaseSchema().get(1).getName()),
-                IndexDef.IndexType.INVERTED,
+                "INVERTED",
                 Maps.newHashMap(), "balabala");
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         alterClauses.add(createIndexClause);
         olapTable.setState(OlapTableState.SCHEMA_CHANGE);
@@ -454,11 +454,11 @@ public class IndexChangeJobTest {
         String indexName = "index1";
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 olapTable.getName());
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(olapTable.getBaseSchema().get(1).getName()),
-                IndexDef.IndexType.INVERTED,
+                "INVERTED",
                 Maps.newHashMap(), "balabala");
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         alterClauses.add(createIndexClause);
         olapTable.setState(OlapTableState.SCHEMA_CHANGE);
@@ -534,11 +534,11 @@ public class IndexChangeJobTest {
         String indexName = "index1";
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 olapTable.getName());
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(olapTable.getBaseSchema().get(1).getName()),
-                IndexDef.IndexType.INVERTED,
+                "INVERTED",
                 Maps.newHashMap(), "balabala");
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         alterClauses.add(createIndexClause);
         schemaChangeHandler.process(alterClauses, db, olapTable);
@@ -599,11 +599,11 @@ public class IndexChangeJobTest {
         String indexName = "index1";
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 olapTable.getName());
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(olapTable.getBaseSchema().get(1).getName()),
-                IndexDef.IndexType.INVERTED,
+                "INVERTED",
                 Maps.newHashMap(), "balabala");
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         alterClauses.add(createIndexClause);
         schemaChangeHandler.process(alterClauses, db, olapTable);
@@ -662,13 +662,13 @@ public class IndexChangeJobTest {
         CatalogTestUtil.createDupTable(db);
         OlapTable table = (OlapTable) 
db.getTableOrDdlException(CatalogTestUtil.testTableId2);
         String indexName = "ngram_bf_index";
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(table.getBaseSchema().get(3).getName()),
-                IndexDef.IndexType.NGRAM_BF,
+                "NGRAM_BF",
                 Maps.newHashMap(), "ngram bf index");
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 table.getName());
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         SchemaChangeHandler schemaChangeHandler = 
Env.getCurrentEnv().getSchemaChangeHandler();
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
@@ -697,12 +697,12 @@ public class IndexChangeJobTest {
         // Test with enable_add_index_for_new_data = false
         context.getSessionVariable().setEnableAddIndexForNewData(false);
         String indexName2 = "ngram_bf_index2";
-        IndexDef indexDef2 = new IndexDef(indexName2, false,
+        IndexDefinition indexDefinition2 = new IndexDefinition(indexName2, 
false,
                 Lists.newArrayList(table.getBaseSchema().get(3).getName()),
-                IndexDef.IndexType.NGRAM_BF,
+                "NGRAM_BF",
                 Maps.newHashMap(), "ngram bf index2");
 
-        createIndexClause = new CreateIndexClause(tableName, indexDef2, false);
+        createIndexClause = new CreateIndexClause(tableName, indexDefinition2, 
false);
         createIndexClause.analyze();
         ArrayList<AlterClause> alterClauses2 = new ArrayList<>();
         alterClauses2.add(createIndexClause);
@@ -750,13 +750,13 @@ public class IndexChangeJobTest {
         CatalogTestUtil.createDupTable(db);
         OlapTable table = (OlapTable) 
db.getTableOrDdlException(CatalogTestUtil.testTableId2);
         String indexName = "ngram_bf_index";
-        IndexDef indexDef = new IndexDef(indexName, false,
+        IndexDefinition indexDefinition = new IndexDefinition(indexName, false,
                 Lists.newArrayList(table.getBaseSchema().get(3).getName()),
-                IndexDef.IndexType.NGRAM_BF,
+                "NGRAM_BF",
                 Maps.newHashMap(), "ngram bf index");
         TableNameInfo tableName = new 
TableNameInfo(masterEnv.getInternalCatalog().getName(), db.getName(),
                 table.getName());
-        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDef, false);
+        CreateIndexClause createIndexClause = new CreateIndexClause(tableName, 
indexDefinition, false);
         createIndexClause.analyze();
         SchemaChangeHandler schemaChangeHandler = 
Env.getCurrentEnv().getSchemaChangeHandler();
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java
index b97dbb50c76..4c74b5b7abd 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java
@@ -18,7 +18,6 @@
 package org.apache.doris.alter;
 
 import org.apache.doris.analysis.ColumnPosition;
-import org.apache.doris.analysis.IndexDef;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Env;
@@ -32,6 +31,7 @@ import org.apache.doris.common.jmockit.Deencapsulation;
 import org.apache.doris.nereids.StatementContext;
 import org.apache.doris.nereids.parser.NereidsParser;
 import org.apache.doris.nereids.trees.plans.commands.AlterTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.qe.StmtExecutor;
@@ -849,13 +849,13 @@ public class SchemaChangeHandlerTest extends 
TestWithFeService {
             Assertions.assertNotNull(indexMeta);
 
             Assertions.assertEquals("idx_error_msg", 
tbl.getIndexes().get(0).getIndexName());
-            Assertions.assertEquals(IndexDef.IndexType.NGRAM_BF, 
tbl.getIndexes().get(0).getIndexType());
+            Assertions.assertEquals(IndexDefinition.IndexType.NGRAM_BF, 
tbl.getIndexes().get(0).getIndexType());
             Map<String, String> props = 
tbl.getIndexes().get(0).getProperties();
             Assertions.assertEquals("2", props.get("gram_size"));
             Assertions.assertEquals("256", props.get("bf_size"));
             Index index = tbl.getIndexes().get(0);
             LOG.warn("index:{}", index.toString());
-            Assertions.assertEquals(IndexDef.IndexType.NGRAM_BF, 
index.getIndexType());
+            Assertions.assertEquals(IndexDefinition.IndexType.NGRAM_BF, 
index.getIndexType());
             Assertions.assertTrue(index.toString().contains("USING NGRAM_BF"));
         } finally {
             tbl.readUnlock();
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/catalog/IndexPropertiesOrderTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/catalog/IndexPropertiesOrderTest.java
index 19ba0486e26..480cfa02e62 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/catalog/IndexPropertiesOrderTest.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/catalog/IndexPropertiesOrderTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.doris.catalog;
 
-import org.apache.doris.analysis.IndexDef;
+import 
org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition.IndexType;
 
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
@@ -41,11 +41,11 @@ public class IndexPropertiesOrderTest {
         List<String> columns = Arrays.asList("description");
 
         // Create multiple Index objects with the same properties
-        Index index1 = new Index(1L, "test_idx", columns, 
IndexDef.IndexType.INVERTED,
+        Index index1 = new Index(1L, "test_idx", columns, IndexType.INVERTED,
                                  new HashMap<>(properties), "test comment");
-        Index index2 = new Index(2L, "test_idx", columns, 
IndexDef.IndexType.INVERTED,
+        Index index2 = new Index(2L, "test_idx", columns, IndexType.INVERTED,
                                  new HashMap<>(properties), "test comment");
-        Index index3 = new Index(3L, "test_idx", columns, 
IndexDef.IndexType.INVERTED,
+        Index index3 = new Index(3L, "test_idx", columns, IndexType.INVERTED,
                                  new HashMap<>(properties), "test comment");
 
         // The properties part should be consistent across all instances
@@ -68,7 +68,7 @@ public class IndexPropertiesOrderTest {
         properties2.put("support_phrase", "true");
         properties2.put("parser", "english");
         properties2.put("lower_case", "true");
-        Index index4 = new Index(4L, "test_idx", columns, 
IndexDef.IndexType.INVERTED,
+        Index index4 = new Index(4L, "test_idx", columns, IndexType.INVERTED,
                                  properties2, "test comment");
         String props4 = index4.getPropertiesString();
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/IndexTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/catalog/IndexTest.java
index 776c97ca76b..715a210db67 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/IndexTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/IndexTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.doris.catalog;
 
-import org.apache.doris.analysis.IndexDef;
+import 
org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition.IndexType;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -52,7 +52,7 @@ public class IndexTest {
         List<String> indexColumns1 = new ArrayList<>();
         indexColumns1.add("col1");
         indexColumns1.add("col3");
-        Index index1 = new Index(1, "test_index1", indexColumns1, 
IndexDef.IndexType.BITMAP, null, null);
+        Index index1 = new Index(1, "test_index1", indexColumns1, 
IndexType.BITMAP, null, null);
 
         List<Integer> uniqueIds1 = index1.getColumnUniqueIds(schema);
         Assert.assertEquals(2, uniqueIds1.size());
@@ -63,7 +63,7 @@ public class IndexTest {
         List<String> indexColumns2 = new ArrayList<>();
         indexColumns2.add("CoL1");
         indexColumns2.add("COL3");
-        Index index2 = new Index(2, "test_index2", indexColumns2, 
IndexDef.IndexType.BITMAP, null, null);
+        Index index2 = new Index(2, "test_index2", indexColumns2, 
IndexType.BITMAP, null, null);
 
         List<Integer> uniqueIds2 = index2.getColumnUniqueIds(schema);
         Assert.assertEquals(2, uniqueIds2.size());
@@ -74,7 +74,7 @@ public class IndexTest {
         List<String> indexColumns3 = new ArrayList<>();
         indexColumns3.add("col1");
         indexColumns3.add("non_existent_column");
-        Index index3 = new Index(3, "test_index3", indexColumns3, 
IndexDef.IndexType.BITMAP, null, null);
+        Index index3 = new Index(3, "test_index3", indexColumns3, 
IndexType.BITMAP, null, null);
 
         List<Integer> uniqueIds3 = index3.getColumnUniqueIds(schema);
         Assert.assertEquals(1, uniqueIds3.size());
@@ -86,7 +86,7 @@ public class IndexTest {
 
         // Test case 5: Empty column list
         Index emptyColIndex = new Index(5, "empty_col_index", new 
ArrayList<>(),
-                IndexDef.IndexType.BITMAP, null, null);
+                IndexType.BITMAP, null, null);
         List<Integer> emptyColUniqueIds = 
emptyColIndex.getColumnUniqueIds(schema);
         Assert.assertEquals(0, emptyColUniqueIds.size());
 
@@ -99,7 +99,7 @@ public class IndexTest {
         dupColumns.add("col1");
         dupColumns.add("col1");  // Duplicated
         dupColumns.add("col2");
-        Index dupIndex = new Index(7, "dup_index", dupColumns, 
IndexDef.IndexType.BITMAP, null, null);
+        Index dupIndex = new Index(7, "dup_index", dupColumns, 
IndexType.BITMAP, null, null);
 
         List<Integer> dupUniqueIds = dupIndex.getColumnUniqueIds(schema);
         Assert.assertEquals(3, dupUniqueIds.size());
@@ -110,7 +110,7 @@ public class IndexTest {
         // Test case 8: Special characters in column names
         List<String> specialColList = new ArrayList<>();
         specialColList.add("special-name!@#");
-        Index specialIndex = new Index(8, "special_index", specialColList, 
IndexDef.IndexType.BITMAP, null, null);
+        Index specialIndex = new Index(8, "special_index", specialColList, 
IndexType.BITMAP, null, null);
 
         List<Integer> specialUniqueIds = 
specialIndex.getColumnUniqueIds(schema);
         Assert.assertEquals(1, specialUniqueIds.size());
@@ -119,7 +119,7 @@ public class IndexTest {
         // Test case 9: Mixed case column name
         List<String> mixedCaseList = new ArrayList<>();
         mixedCaseList.add("mixed_case");  // Testing case insensitivity with 
underscores
-        Index mixedCaseIndex = new Index(9, "mixed_case_index", mixedCaseList, 
IndexDef.IndexType.BITMAP, null, null);
+        Index mixedCaseIndex = new Index(9, "mixed_case_index", mixedCaseList, 
IndexType.BITMAP, null, null);
 
         List<Integer> mixedCaseUniqueIds = 
mixedCaseIndex.getColumnUniqueIds(schema);
         Assert.assertEquals(1, mixedCaseUniqueIds.size());
@@ -139,7 +139,7 @@ public class IndexTest {
             }
         }
 
-        Index largeIndex = new Index(10, "large_index", largeColumnList, 
IndexDef.IndexType.BITMAP, null, null);
+        Index largeIndex = new Index(10, "large_index", largeColumnList, 
IndexType.BITMAP, null, null);
         List<Integer> largeUniqueIds = 
largeIndex.getColumnUniqueIds(largeSchema);
 
         Assert.assertEquals(500, largeUniqueIds.size());
@@ -153,7 +153,7 @@ public class IndexTest {
         reverseOrderColumns.add("col2");
         reverseOrderColumns.add("col1");
 
-        Index reverseIndex = new Index(11, "reverse_index", 
reverseOrderColumns, IndexDef.IndexType.BITMAP, null, null);
+        Index reverseIndex = new Index(11, "reverse_index", 
reverseOrderColumns, IndexType.BITMAP, null, null);
         List<Integer> reverseUniqueIds = 
reverseIndex.getColumnUniqueIds(schema);
 
         Assert.assertEquals(3, reverseUniqueIds.size());
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java
index 0c983332ee5..807903cd15c 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java
@@ -17,12 +17,12 @@
 
 package org.apache.doris.catalog;
 
-import org.apache.doris.analysis.IndexDef;
 import org.apache.doris.analysis.UserIdentity;
 import org.apache.doris.catalog.TableIf.TableType;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.io.FastByteArrayOutputStream;
 import org.apache.doris.common.util.UnitTestUtil;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.resource.Tag;
 import org.apache.doris.resource.computegroup.ComputeGroup;
@@ -66,7 +66,7 @@ public class OlapTableTest {
             }
             OlapTable tbl = (OlapTable) table;
             tbl.setIndexes(Lists.newArrayList(new Index(0, "index", 
Lists.newArrayList("col"),
-                    IndexDef.IndexType.BITMAP, null, "xxxxxx")));
+                    IndexDefinition.IndexType.BITMAP, null, "xxxxxx")));
             System.out.println("orig table id: " + tbl.getId());
 
             FastByteArrayOutputStream byteArrayOutputStream = new 
FastByteArrayOutputStream();
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/common/proc/IndexesProcNodeTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/common/proc/IndexesProcNodeTest.java
index 48ff245af5c..d0172ebf719 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/common/proc/IndexesProcNodeTest.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/common/proc/IndexesProcNodeTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.doris.common.proc;
 
-import org.apache.doris.analysis.IndexDef.IndexType;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.HashDistributionInfo;
 import org.apache.doris.catalog.Index;
@@ -26,6 +25,7 @@ import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.PartitionInfo;
 import org.apache.doris.catalog.TableIndexes;
 import org.apache.doris.common.AnalysisException;
+import 
org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition.IndexType;
 
 import com.google.common.collect.Lists;
 import org.junit.Assert;
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java
index e3c55f70c07..f7b2d96f670 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java
@@ -17,12 +17,12 @@
 
 package org.apache.doris.persist;
 
-import org.apache.doris.analysis.IndexDef;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Index;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.common.io.Text;
+import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition;
 import org.apache.doris.persist.gson.GsonUtils;
 
 import com.google.common.collect.Lists;
@@ -71,7 +71,7 @@ public class TableAddOrDropColumnsInfoTest {
         indexNameToId.put("index", 1L);
 
         List<Index> indexes = Lists.newArrayList(
-                new Index(0, "index", Lists.newArrayList("testCol1"), 
IndexDef.IndexType.INVERTED, null, "xxxxxx"));
+                new Index(0, "index", Lists.newArrayList("testCol1"), 
IndexDefinition.IndexType.INVERTED, null, "xxxxxx"));
 
         TableAddOrDropColumnsInfo tableAddOrDropColumnsInfo1 = new 
TableAddOrDropColumnsInfo(
                 "", dbId, tableId, tableId,
diff --git 
a/regression-test/suites/index_p0/test_ngram_bloomfilter_index.groovy 
b/regression-test/suites/index_p0/test_ngram_bloomfilter_index.groovy
index 04148e984cd..5bcd55e96f4 100644
--- a/regression-test/suites/index_p0/test_ngram_bloomfilter_index.groovy
+++ b/regression-test/suites/index_p0/test_ngram_bloomfilter_index.groovy
@@ -81,7 +81,7 @@ suite("test_ngram_bloomfilter_index") {
         DISTRIBUTED BY HASH(`key_id`) BUCKETS 3
         PROPERTIES("replication_num" = "1");
         """
-        exception "java.sql.SQLException: errCode = 2, detailMessage = invalid 
ngram bf index params:errCode = 2, detailMessage = 'bf_size' should be an 
integer between 64 and 65535."
+        exception "java.sql.SQLException: errCode = 2, detailMessage = invalid 
ngram bf index params:'bf_size' should be an integer between 64 and 65535."
     }
 
     def tableName3 = 'test_ngram_bloomfilter_index3'
diff --git a/regression-test/suites/inverted_index_p0/test_array_index2.groovy 
b/regression-test/suites/inverted_index_p0/test_array_index2.groovy
index 9fb6747ca83..f1de5022b98 100644
--- a/regression-test/suites/inverted_index_p0/test_array_index2.groovy
+++ b/regression-test/suites/inverted_index_p0/test_array_index2.groovy
@@ -114,16 +114,15 @@ suite("test_array_index2") {
         exception "is not supported in"
     }
 
-    // Test creating index on array of float - should fail
+    // Test creating index on array of struct - should fail
     test {
-        sql """ ALTER TABLE ${tableName2} ADD INDEX idx_float_arr (float_arr) 
USING INVERTED; """
+        sql """ ALTER TABLE ${tableName2} ADD INDEX idx_struct_arr 
(struct_arr) USING INVERTED; """
         exception "is not supported in"
     }
 
-    // Test creating index on array of struct - should fail
+    // Test creating index on array of float - should success
     test {
-        sql """ ALTER TABLE ${tableName2} ADD INDEX idx_struct_arr 
(struct_arr) USING INVERTED; """
-        exception "is not supported in"
+        sql """ ALTER TABLE ${tableName2} ADD INDEX idx_float_arr (float_arr) 
USING INVERTED; """
     }
 
     // Test array_contains function
@@ -156,5 +155,5 @@ suite("test_array_index2") {
     """
 
     sql "DROP TABLE IF EXISTS ${tableName1}"
-    sql "DROP TABLE IF EXISTS ${tableName2}"
+    sql "DROP TABLE IF EXISTS ${tableName2} FORCE"
 } 


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

Reply via email to