This is an automated email from the ASF dual-hosted git repository. jianliangqi pushed a commit to branch branch-2.1 in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-2.1 by this push: new 12bb38eab37 [fix](build index)Fix build index failed on renamed column (#42882)(#43246)(#43336) (#43266) 12bb38eab37 is described below commit 12bb38eab37e092b9f523acb8db1156d0f21d02f Author: qiye <jianliang5...@gmail.com> AuthorDate: Thu Nov 7 21:32:02 2024 +0800 [fix](build index)Fix build index failed on renamed column (#42882)(#43246)(#43336) (#43266) bp #42882 #43246 #43336 We pick these PRs together because they all fix one problem and if not do so, the regression test won't be passed. --------- Co-authored-by: Sun Chenyang <csun5...@gmail.com> Co-authored-by: qidaye <l...@selectdb.com> --- be/src/olap/tablet_schema.cpp | 10 +- be/src/olap/task/index_builder.cpp | 41 +++++-- .../apache/doris/alter/SchemaChangeHandler.java | 3 + .../apache/doris/analysis/BuildIndexClause.java | 2 +- .../apache/doris/analysis/CreateIndexClause.java | 2 +- .../org/apache/doris/analysis/CreateTableStmt.java | 3 +- .../java/org/apache/doris/analysis/IndexDef.java | 5 + .../main/java/org/apache/doris/catalog/Index.java | 20 +++- .../doris/catalog/MaterializedIndexMeta.java | 7 ++ .../trees/plans/commands/info/IndexDefinition.java | 2 +- .../org/apache/doris/catalog/OlapTableTest.java | 2 +- .../doris/common/proc/IndexesProcNodeTest.java | 8 +- .../persist/TableAddOrDropColumnsInfoTest.java | 2 +- gensrc/thrift/Descriptors.thrift | 1 + .../test_index_change_on_renamed_column.out | 11 ++ ...est_match_without_index_fault_injection.groovy} | 4 +- .../test_index_change_on_renamed_column.groovy | 121 +++++++++++++++++++++ 17 files changed, 217 insertions(+), 27 deletions(-) diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp index 62d80fb28c1..655e6d2e1e4 100644 --- a/be/src/olap/tablet_schema.cpp +++ b/be/src/olap/tablet_schema.cpp @@ -739,7 +739,15 @@ void TabletIndex::init_from_thrift(const TOlapTableIndex& index, if (column_idx >= 0) { col_unique_ids[i] = tablet_schema.column(column_idx).unique_id(); } else { - col_unique_ids[i] = -1; + // if column unique id not found by column name, find by column unique id + // column unique id can not bigger than tablet schema column size, if bigger than column size means + // this column is a new column added by light schema change + if (index.__isset.column_unique_ids && + index.column_unique_ids[i] < tablet_schema.num_columns()) { + col_unique_ids[i] = index.column_unique_ids[i]; + } else { + col_unique_ids[i] = -1; + } } } _col_unique_ids = std::move(col_unique_ids); diff --git a/be/src/olap/task/index_builder.cpp b/be/src/olap/task/index_builder.cpp index 490f0654e90..a6cd46ca41f 100644 --- a/be/src/olap/task/index_builder.cpp +++ b/be/src/olap/task/index_builder.cpp @@ -83,10 +83,16 @@ Status IndexBuilder::update_inverted_index_info() { auto column_name = t_inverted_index.columns[0]; auto column_idx = output_rs_tablet_schema->field_index(column_name); if (column_idx < 0) { - LOG(WARNING) << "referenced column was missing. " - << "[column=" << column_name << " referenced_column=" << column_idx - << "]"; - continue; + if (!t_inverted_index.column_unique_ids.empty()) { + auto column_unique_id = t_inverted_index.column_unique_ids[0]; + column_idx = output_rs_tablet_schema->field_index(column_unique_id); + } + if (column_idx < 0) { + LOG(WARNING) << "referenced column was missing. " + << "[column=" << column_name + << " referenced_column=" << column_idx << "]"; + continue; + } } auto column = output_rs_tablet_schema->column(column_idx); const auto* index_meta = output_rs_tablet_schema->get_inverted_index(column); @@ -346,10 +352,16 @@ Status IndexBuilder::handle_single_rowset(RowsetMetaSharedPtr output_rowset_meta auto column_name = inverted_index.columns[0]; auto column_idx = output_rowset_schema->field_index(column_name); if (column_idx < 0) { - LOG(WARNING) << "referenced column was missing. " - << "[column=" << column_name << " referenced_column=" << column_idx - << "]"; - continue; + if (!inverted_index.column_unique_ids.empty()) { + column_idx = output_rowset_schema->field_index( + inverted_index.column_unique_ids[0]); + } + if (column_idx < 0) { + LOG(WARNING) << "referenced column was missing. " + << "[column=" << column_name + << " referenced_column=" << column_idx << "]"; + continue; + } } auto column = output_rowset_schema->column(column_idx); if (!InvertedIndexColumnWriter::check_support_inverted_index(column)) { @@ -478,9 +490,16 @@ Status IndexBuilder::_write_inverted_index_data(TabletSchemaSPtr tablet_schema, auto column_name = inverted_index.columns[0]; auto column_idx = tablet_schema->field_index(column_name); if (column_idx < 0) { - LOG(WARNING) << "referenced column was missing. " - << "[column=" << column_name << " referenced_column=" << column_idx << "]"; - continue; + if (!inverted_index.column_unique_ids.empty()) { + auto column_unique_id = inverted_index.column_unique_ids[0]; + column_idx = tablet_schema->field_index(column_unique_id); + } + if (column_idx < 0) { + LOG(WARNING) << "referenced column was missing. " + << "[column=" << column_name << " referenced_column=" << column_idx + << "]"; + continue; + } } auto column = tablet_schema->column(column_idx); auto writer_sign = std::make_pair(segment_idx, index_id); 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 73a4865c4b6..2f93cbd681a 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 @@ -2012,6 +2012,7 @@ public class SchemaChangeHandler extends AlterHandler { index.setIndexId(existedIdx.getIndexId()); index.setColumns(existedIdx.getColumns()); index.setProperties(existedIdx.getProperties()); + index.setColumnUniqueIds(existedIdx.getColumnUniqueIds()); if (indexDef.getPartitionNames().isEmpty()) { invertedIndexOnPartitions.put(index.getIndexId(), olapTable.getPartitionNames()); } else { @@ -2588,6 +2589,7 @@ public class SchemaChangeHandler extends AlterHandler { if (column != null) { indexDef.checkColumn(column, olapTable.getKeysType(), olapTable.getTableProperty().getEnableUniqueKeyMergeOnWrite()); + indexDef.getColumnUniqueIds().add(column.getUniqueId()); } else { throw new DdlException("index column does not exist in table. invalid column: " + col); } @@ -2598,6 +2600,7 @@ public class SchemaChangeHandler extends AlterHandler { // so here update column name in CreateIndexClause after checkColumn for indexDef, // there will use the column name in olapTable insead of the column name in CreateIndexClause. alterIndex.setColumns(indexDef.getColumns()); + alterIndex.setColumnUniqueIds(indexDef.getColumnUniqueIds()); newIndexes.add(alterIndex); return false; } 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 cb7ec08de78..c65766a1ae8 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 @@ -73,7 +73,7 @@ public class BuildIndexClause extends AlterTableClause { indexDef.analyze(); this.index = new Index(Env.getCurrentEnv().getNextId(), indexDef.getIndexName(), indexDef.getColumns(), indexDef.getIndexType(), - indexDef.getProperties(), indexDef.getComment()); + indexDef.getProperties(), indexDef.getComment(), indexDef.getColumnUniqueIds()); } @Override 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 b39c0df4a85..86df87453ad 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 @@ -73,7 +73,7 @@ public class CreateIndexClause extends AlterTableClause { indexDef.analyze(); this.index = new Index(Env.getCurrentEnv().getNextId(), indexDef.getIndexName(), indexDef.getColumns(), indexDef.getIndexType(), - indexDef.getProperties(), indexDef.getComment()); + indexDef.getProperties(), indexDef.getComment(), indexDef.getColumnUniqueIds()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java index 97073c088e5..fc36416b299 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java @@ -581,7 +581,8 @@ public class CreateTableStmt extends DdlStmt { } } indexes.add(new Index(Env.getCurrentEnv().getNextId(), indexDef.getIndexName(), indexDef.getColumns(), - indexDef.getIndexType(), indexDef.getProperties(), indexDef.getComment())); + indexDef.getIndexType(), indexDef.getProperties(), indexDef.getComment(), + indexDef.getColumnUniqueIds())); distinct.add(indexDef.getIndexName()); distinctCol.add(Pair.of(indexDef.getIndexType(), indexDef.getColumns().stream().map(String::toUpperCase).collect(Collectors.toList()))); 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 index 7db8c460b2f..fc122cc82a5 100644 --- 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 @@ -43,6 +43,7 @@ public class IndexDef { private Map<String, String> properties; private boolean isBuildDeferred = false; private PartitionNames partitionNames; + private List<Integer> columnUniqueIds = Lists.newArrayList(); public static final String NGRAM_SIZE_KEY = "gram_size"; public static final String NGRAM_BF_SIZE_KEY = "bf_size"; @@ -197,6 +198,10 @@ public class IndexDef { return partitionNames == null ? Lists.newArrayList() : partitionNames.getPartitionNames(); } + public List<Integer> getColumnUniqueIds() { + return columnUniqueIds; + } + public enum IndexType { BITMAP, INVERTED, 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 032f396fddd..453d1a71d78 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 @@ -47,7 +47,7 @@ import java.util.Set; /** * Internal representation of index, including index type, name, columns and comments. - * This class will used in olaptable + * This class will be used in olap table */ public class Index implements Writable { public static final int INDEX_ID_INIT_VALUE = -1; @@ -64,15 +64,19 @@ public class Index implements Writable { private Map<String, String> properties; @SerializedName(value = "comment") private String comment; + @SerializedName(value = "cui", alternate = {"columnUniqueIds"}) + private List<Integer> columnUniqueIds; public Index(long indexId, String indexName, List<String> columns, - IndexDef.IndexType indexType, Map<String, String> properties, String comment) { + IndexDef.IndexType indexType, Map<String, String> properties, String comment, + List<Integer> columnUniqueIds) { 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; + this.columnUniqueIds = columnUniqueIds == null ? Lists.newArrayList() : Lists.newArrayList(columnUniqueIds); if (indexType == IndexDef.IndexType.INVERTED) { if (this.properties != null && !this.properties.isEmpty()) { if (this.properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY)) { @@ -96,6 +100,7 @@ public class Index implements Writable { this.indexType = null; this.properties = null; this.comment = null; + this.columnUniqueIds = null; } public long getIndexId() { @@ -185,6 +190,14 @@ public class Index implements Writable { this.comment = comment; } + public List<Integer> getColumnUniqueIds() { + return columnUniqueIds; + } + + public void setColumnUniqueIds(List<Integer> columnUniqueIds) { + this.columnUniqueIds = columnUniqueIds; + } + @Override public void write(DataOutput out) throws IOException { Text.writeString(out, GsonUtils.GSON.toJson(this)); @@ -202,7 +215,7 @@ public class Index implements Writable { public Index clone() { return new Index(indexId, indexName, new ArrayList<>(columns), - indexType, new HashMap<>(properties), comment); + indexType, new HashMap<>(properties), comment, columnUniqueIds); } @Override @@ -246,6 +259,7 @@ public class Index implements Writable { if (properties != null) { tIndex.setProperties(properties); } + tIndex.setColumnUniqueIds(columnUniqueIds); return tIndex; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java index ca52afd3b6d..58fa196a079 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java @@ -387,6 +387,13 @@ public class MaterializedIndexMeta implements Writable, GsonPostProcessable { maxColUniqueId = Column.COLUMN_UNIQUE_ID_INIT_VALUE; this.schema.forEach(column -> { column.setUniqueId(incAndGetMaxColUniqueId()); + this.indexes.forEach(index -> { + index.getColumns().forEach(col -> { + if (col.equalsIgnoreCase(column.getName())) { + index.getColumnUniqueIds().add(column.getUniqueId()); + } + }); + }); if (LOG.isDebugEnabled()) { LOG.debug("indexId: {}, column:{}, uniqueId:{}", indexId, column, column.getUniqueId()); 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 bd37616ea2d..dc2968f53b8 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 @@ -211,6 +211,6 @@ public class IndexDefinition { public Index translateToCatalogStyle() { return new Index(Env.getCurrentEnv().getNextId(), name, cols, indexType, properties, - comment); + comment, null); } } 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 8526b2d2271..64c35482433 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 @@ -58,7 +58,7 @@ public class OlapTableTest { } OlapTable tbl = (OlapTable) table; tbl.setIndexes(Lists.newArrayList(new Index(0, "index", Lists.newArrayList("col"), - IndexDef.IndexType.BITMAP, null, "xxxxxx"))); + IndexDef.IndexType.BITMAP, null, "xxxxxx", Lists.newArrayList(1)))); 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 aeb5bc471fe..966f6c38b5b 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 @@ -42,18 +42,18 @@ public class IndexesProcNodeTest { public void testFetchResult() throws AnalysisException { List<Index> indexes = new ArrayList<>(); Index indexBitmap = new Index(1, "bitmap_index", Lists.newArrayList("col_1"), - IndexType.BITMAP, null, "bitmap index on col_1"); + IndexType.BITMAP, null, "bitmap index on col_1", Lists.newArrayList(1)); Map<String, String> invertedProperties = new HashMap<>(); invertedProperties.put("parser", "unicode"); Index indexInverted = new Index(2, "inverted_index", Lists.newArrayList("col_2"), - IndexType.INVERTED, invertedProperties, "inverted index on col_2"); + IndexType.INVERTED, invertedProperties, "inverted index on col_2", Lists.newArrayList(2)); Index indexBf = new Index(3, "bloomfilter_index", Lists.newArrayList("col_3"), - IndexType.BLOOMFILTER, null, "bloomfilter index on col_3"); + IndexType.BLOOMFILTER, null, "bloomfilter index on col_3", Lists.newArrayList(3)); Map<String, String> ngramProperties = new HashMap<>(); ngramProperties.put("gram_size", "3"); ngramProperties.put("bf_size", "256"); Index indexNgramBf = new Index(4, "ngram_bf_index", Lists.newArrayList("col_4"), - IndexType.NGRAM_BF, ngramProperties, "ngram_bf index on col_4"); + IndexType.NGRAM_BF, ngramProperties, "ngram_bf index on col_4", Lists.newArrayList(4)); indexes.add(indexBitmap); indexes.add(indexInverted); indexes.add(indexBf); 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 0e04bddd681..be71998eac3 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 @@ -65,7 +65,7 @@ public class TableAddOrDropColumnsInfoTest { indexSchemaMap.put(tableId, fullSchema); List<Index> indexes = Lists.newArrayList( - new Index(0, "index", Lists.newArrayList("testCol1"), IndexDef.IndexType.INVERTED, null, "xxxxxx")); + new Index(0, "index", Lists.newArrayList("testCol1"), IndexDef.IndexType.INVERTED, null, "xxxxxx", Lists.newArrayList(1))); TableAddOrDropColumnsInfo tableAddOrDropColumnsInfo1 = new TableAddOrDropColumnsInfo("", dbId, tableId, indexSchemaMap, indexes, jobId); diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index f0c7372460b..1cb6f79cf97 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -227,6 +227,7 @@ struct TOlapTableIndex { 4: optional string comment 5: optional i64 index_id 6: optional map<string, string> properties + 7: optional list<i32> column_unique_ids } struct TOlapTableIndexSchema { diff --git a/regression-test/data/inverted_index_p0/index_change/test_index_change_on_renamed_column.out b/regression-test/data/inverted_index_p0/index_change/test_index_change_on_renamed_column.out new file mode 100644 index 00000000000..4bf30965b8b --- /dev/null +++ b/regression-test/data/inverted_index_p0/index_change/test_index_change_on_renamed_column.out @@ -0,0 +1,11 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select1 -- +1 hello world +2 welcome to the world + +-- !select2 -- +1 hello world +2 welcome to the world + +-- !select3 -- +2 welcome to the world \ No newline at end of file diff --git a/regression-test/suites/inverted_index_p0/test_match_without_index.groovy b/regression-test/suites/fault_injection_p0/test_match_without_index_fault_injection.groovy similarity index 96% rename from regression-test/suites/inverted_index_p0/test_match_without_index.groovy rename to regression-test/suites/fault_injection_p0/test_match_without_index_fault_injection.groovy index a008cf703d8..db03b75bee6 100644 --- a/regression-test/suites/inverted_index_p0/test_match_without_index.groovy +++ b/regression-test/suites/fault_injection_p0/test_match_without_index_fault_injection.groovy @@ -16,9 +16,9 @@ // under the License. -suite("test_match_without_index", "p0") { +suite("test_match_without_index_fault_injection", "nonConcurrent") { - def testTable = "test_match_without_index" + def testTable = "test_match_without_index_fault_injection" sql "DROP TABLE IF EXISTS ${testTable}" sql """ CREATE TABLE ${testTable} ( diff --git a/regression-test/suites/inverted_index_p0/index_change/test_index_change_on_renamed_column.groovy b/regression-test/suites/inverted_index_p0/index_change/test_index_change_on_renamed_column.groovy new file mode 100644 index 00000000000..4cda3a66489 --- /dev/null +++ b/regression-test/suites/inverted_index_p0/index_change/test_index_change_on_renamed_column.groovy @@ -0,0 +1,121 @@ +// 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. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("test_index_change_on_renamed_column") { + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + + def timeout = 60000 + def delta_time = 1000 + def alter_res = "null" + def useTime = 0 + + def wait_for_latest_op_on_table_finish = { table_name, OpTimeout -> + for(int t = delta_time; t <= OpTimeout; t += delta_time){ + alter_res = sql """SHOW ALTER TABLE COLUMN WHERE TableName = "${table_name}" ORDER BY CreateTime DESC LIMIT 1;""" + alter_res = alter_res.toString() + if(alter_res.contains("FINISHED")) { + sleep(3000) // wait change table state to normal + logger.info(table_name + " latest alter job finished, detail: " + alter_res) + break + } + useTime = t + sleep(delta_time) + } + assertTrue(useTime <= OpTimeout, "wait_for_latest_op_on_table_finish timeout") + } + + def wait_for_build_index_on_partition_finish = { table_name, OpTimeout -> + for(int t = delta_time; t <= OpTimeout; t += delta_time){ + alter_res = sql """SHOW BUILD INDEX WHERE TableName = "${table_name}";""" + def expected_finished_num = alter_res.size(); + def finished_num = 0; + for (int i = 0; i < expected_finished_num; i++) { + logger.info(table_name + " build index job state: " + alter_res[i][7] + i) + if (alter_res[i][7] == "FINISHED") { + ++finished_num; + } + } + if (finished_num == expected_finished_num) { + logger.info(table_name + " all build index jobs finished, detail: " + alter_res) + break + } + useTime = t + sleep(delta_time) + } + assertTrue(useTime <= OpTimeout, "wait_for_latest_build_index_on_partition_finish timeout") + } + + def tableName = "test_index_change_on_renamed_column" + + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + `id` INT COMMENT "", + `s` STRING COMMENT "" + ) + DUPLICATE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1", "inverted_index_storage_format" = "V2" ); + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, 'hello world'); + """ + sql """ INSERT INTO ${tableName} VALUES + (2, 'welcome to the world'); + """ + + // create inverted + sql """ alter table ${tableName} add index idx_s(s) USING INVERTED PROPERTIES('parser' = 'english')""" + wait_for_latest_op_on_table_finish(tableName, timeout) + + qt_select1 """ SELECT * FROM ${tableName} order by id; """ + + // rename column + sql """ alter table ${tableName} rename column s s1; """ + + // build inverted index on renamed column + if (!isCloudMode()) { + sql """ build index idx_s on ${tableName} """ + wait_for_build_index_on_partition_finish(tableName, timeout) + } + + def show_result = sql "show index from ${tableName}" + logger.info("show index from " + tableName + " result: " + show_result) + assertEquals(show_result.size(), 1) + assertEquals(show_result[0][2], "idx_s") + + qt_select2 """ SELECT * FROM ${tableName} order by id; """ + qt_select3 """ SELECT /*+ SET_VAR(enable_inverted_index_query = true) */ * FROM ${tableName} where s1 match 'welcome'; """ + + def tablets = sql_return_maparray """ show tablets from ${tableName}; """ + String tablet_id = tablets[0].TabletId + String backend_id = tablets[0].BackendId + String ip = backendId_to_backendIP.get(backend_id) + String port = backendId_to_backendHttpPort.get(backend_id) + check_nested_index_file(ip, port, tablet_id, 3, 1, "V2") + + // drop inverted index on renamed column + sql """ alter table ${tableName} drop index idx_s; """ + wait_for_latest_op_on_table_finish(tableName, timeout) + show_result = sql "show index from ${tableName}" + logger.info("show index from " + tableName + " result: " + show_result) + assertEquals(show_result.size(), 0) +} --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org