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]