diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 0b851440bfe98bb..7c2e1927e30c3cc 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -185,9 +185,18 @@ supportedCreateStatement supportedAlterStatement : ALTER VIEW name=multipartIdentifier (LEFT_PAREN cols=simpleColumnDefs RIGHT_PAREN)? - AS query #alterView - | ALTER STORAGE VAULT name=multipartIdentifier properties=propertyClause #alterStorageVault - | ALTER ROLE role=identifier commentSpec #alterRole + AS query #alterView + | ALTER STORAGE VAULT name=multipartIdentifier properties=propertyClause #alterStorageVault + | ALTER ROLE role=identifier commentSpec #alterRole + | ALTER STORAGE VAULT name=multipartIdentifier properties=propertyClause #alterStorageVault + | ALTER TABLE tableName=multipartIdentifier + alterTableClause (COMMA alterTableClause)* #alterTable + | ALTER TABLE tableName=multipartIdentifier ADD ROLLUP + addRollupClause (COMMA addRollupClause)* #alterTableAddRollup + | ALTER TABLE tableName=multipartIdentifier DROP ROLLUP + dropRollupClause (COMMA dropRollupClause)* #alterTableDropRollup + | ALTER TABLE name=multipartIdentifier + SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterTableProperties ; supportedDropStatement @@ -534,13 +543,7 @@ privilegeList ; unsupportedAlterStatement - : ALTER TABLE tableName=multipartIdentifier - alterTableClause (COMMA alterTableClause)* #alterTable - | ALTER TABLE tableName=multipartIdentifier ADD ROLLUP - addRollupClause (COMMA addRollupClause)* #alterTableAddRollup - | ALTER TABLE tableName=multipartIdentifier DROP ROLLUP - dropRollupClause (COMMA dropRollupClause)* #alterTableDropRollup - | ALTER SYSTEM alterSystemClause #alterSystem + : ALTER SYSTEM alterSystemClause #alterSystem | ALTER DATABASE name=identifier SET (DATA |REPLICA | TRANSACTION) QUOTA INTEGER_VALUE identifier? #alterDatabaseSetQuota | ALTER DATABASE name=identifier RENAME newName=identifier #alterDatabaseRename @@ -560,8 +563,6 @@ unsupportedAlterStatement | ALTER ROUTINE LOAD FOR name=multipartIdentifier properties=propertyClause? (FROM type=identifier LEFT_PAREN propertyItemList RIGHT_PAREN)? #alterRoutineLoad | ALTER SQL_BLOCK_RULE name=identifier properties=propertyClause? #alterSqlBlockRule - | ALTER TABLE name=multipartIdentifier - SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterTableProperties | ALTER STORAGE POLICY name=identifierOrText properties=propertyClause #alterStoragePlicy | ALTER USER (IF EXISTS)? grantUserIdentify @@ -605,20 +606,19 @@ addRollupClause alterTableClause : ADD COLUMN columnDef columnPosition? toRollup? properties=propertyClause? #addColumnClause - | ADD COLUMN LEFT_PAREN columnDef (COMMA columnDef)* RIGHT_PAREN + | ADD COLUMN LEFT_PAREN columnDefs RIGHT_PAREN toRollup? properties=propertyClause? #addColumnsClause | DROP COLUMN name=identifier fromRollup? properties=propertyClause? #dropColumnClause | MODIFY COLUMN columnDef columnPosition? fromRollup? properties=propertyClause? #modifyColumnClause | ORDER BY identifierList fromRollup? properties=propertyClause? #reorderColumnsClause - | ADD TEMPORARY? (lessThanPartitionDef | fixedPartitionDef | inPartitionDef) - (LEFT_PAREN partitionProperties=propertyItemList RIGHT_PAREN)? + | ADD TEMPORARY? partitionDef (DISTRIBUTED BY (HASH hashKeys=identifierList | RANDOM) (BUCKETS (INTEGER_VALUE | autoBucket=AUTO))?)? properties=propertyClause? #addPartitionClause | DROP TEMPORARY? PARTITION (IF EXISTS)? partitionName=identifier FORCE? (FROM INDEX indexName=identifier)? #dropPartitionClause - | MODIFY TEMPORARY? PARTITION (IF EXISTS)? + | MODIFY TEMPORARY? PARTITION (partitionName=identifier | partitionNames=identifierList | LEFT_PAREN ASTERISK RIGHT_PAREN) SET LEFT_PAREN partitionProperties=propertyItemList RIGHT_PAREN #modifyPartitionClause @@ -1363,7 +1363,7 @@ indexDefs ; indexDef - : INDEX (IF NOT EXISTS)? indexName=identifier cols=identifierList (USING indexType=(BITMAP | INVERTED | NGRAM_BF))? (PROPERTIES LEFT_PAREN properties=propertyItemList RIGHT_PAREN)? (COMMENT comment=STRING_LITERAL)? + : INDEX (ifNotExists=IF NOT EXISTS)? indexName=identifier cols=identifierList (USING indexType=(BITMAP | INVERTED | NGRAM_BF))? (PROPERTIES LEFT_PAREN properties=propertyItemList RIGHT_PAREN)? (COMMENT comment=STRING_LITERAL)? ; partitionsDef diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 2b213d0558385a8..6d18548ba62034a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -68,6 +68,7 @@ import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.PropertyAnalyzer.RewriteProperty; import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.nereids.trees.plans.commands.AlterTableCommand; import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; import org.apache.doris.persist.AlterMTMV; import org.apache.doris.persist.AlterViewInfo; @@ -91,6 +92,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -147,8 +149,31 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, } stmt.rewriteAlterClause(olapTable); - // check conflict alter ops first alterClauses.addAll(stmt.getOps()); + return processAlterOlapTableInternal(alterClauses, olapTable, db, stmt.toSql()); + } + + private boolean processAlterOlapTable(AlterTableCommand command, OlapTable olapTable, + List alterClauses, + Database db) throws UserException { + if (olapTable.getDataSortInfo() != null + && olapTable.getDataSortInfo().getSortType() == TSortType.ZORDER) { + throw new UserException("z-order table can not support schema change!"); + } + + // check conflict alter ops first + alterClauses.addAll(command.getOps()); + return processAlterOlapTableInternal(alterClauses, olapTable, db, command.toSql()); + } + + private boolean processAlterOlapTableInternal(List alterClauses, OlapTable olapTable, + Database db, String sql) throws UserException { + if (olapTable.getDataSortInfo() != null + && olapTable.getDataSortInfo().getSortType() == TSortType.ZORDER) { + throw new UserException("z-order table can not support schema change!"); + } + + // check conflict alter ops first AlterOperations currentAlterOps = new AlterOperations(); currentAlterOps.checkConflict(alterClauses); @@ -226,7 +251,7 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, ((SchemaChangeHandler) schemaChangeHandler).updateBinlogConfig(db, olapTable, alterClauses); } else if (currentAlterOps.hasSchemaChangeOp()) { // if modify storage type to v2, do schema change to convert all related tablets to segment v2 format - schemaChangeHandler.process(stmt.toSql(), alterClauses, db, olapTable); + schemaChangeHandler.process(sql, alterClauses, db, olapTable); } else if (currentAlterOps.hasRollupOp()) { materializedViewHandler.process(alterClauses, db, olapTable); } else if (currentAlterOps.hasPartitionOp()) { @@ -407,10 +432,22 @@ public void replayModifyComment(ModifyCommentOperationLog operation) throws Meta } } - private void processAlterExternalTable(AlterTableStmt stmt, Table externalTable, Database db) throws UserException { + private void processAlterExternalTable(AlterTableStmt stmt, Table externalTable, Database db) + throws UserException { stmt.checkExternalTableOperationAllow(externalTable); + processAlterExternalTableInternal(stmt.getOps(), externalTable, db); + } + + private void processAlterExternalTable(AlterTableCommand command, Table externalTable, Database db) + throws UserException { + List alterClauses = new ArrayList<>(); + alterClauses.addAll(command.getOps()); + processAlterExternalTableInternal(alterClauses, externalTable, db); + } + + private void processAlterExternalTableInternal(List alterClauses, Table externalTable, Database db) + throws UserException { // check conflict alter ops first - List alterClauses = stmt.getOps(); AlterOperations currentAlterOps = new AlterOperations(); currentAlterOps.checkConflict(alterClauses); if (currentAlterOps.hasRenameOp()) { @@ -572,6 +609,91 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { } } + public void processAlterTable(AlterTableCommand command) throws UserException { + TableNameInfo dbTableName = command.getTbl(); + String ctlName = dbTableName.getCtl(); + String dbName = dbTableName.getDb(); + String tableName = dbTableName.getTbl(); + DatabaseIf dbIf = Env.getCurrentEnv().getCatalogMgr() + .getCatalogOrException(ctlName, catalog -> new DdlException("Unknown catalog " + catalog)) + .getDbOrDdlException(dbName); + TableIf tableIf = dbIf.getTableOrDdlException(tableName); + List alterClauses = Lists.newArrayList(); + // some operations will take long time to process, need to be done outside the table lock + boolean needProcessOutsideTableLock = false; + switch (tableIf.getType()) { + case MATERIALIZED_VIEW: + case OLAP: + OlapTable olapTable = (OlapTable) tableIf; + needProcessOutsideTableLock = processAlterOlapTable(command, olapTable, alterClauses, (Database) dbIf); + break; + case ODBC: + case JDBC: + case HIVE: + case MYSQL: + case ELASTICSEARCH: + processAlterExternalTable(command, (Table) tableIf, (Database) dbIf); + return; + case HMS_EXTERNAL_TABLE: + case JDBC_EXTERNAL_TABLE: + case ICEBERG_EXTERNAL_TABLE: + case PAIMON_EXTERNAL_TABLE: + case MAX_COMPUTE_EXTERNAL_TABLE: + case HUDI_EXTERNAL_TABLE: + case TRINO_CONNECTOR_EXTERNAL_TABLE: + alterClauses.addAll(command.getOps()); + setExternalTableAutoAnalyzePolicy((ExternalTable) tableIf, alterClauses); + return; + default: + throw new DdlException("Do not support alter " + + tableIf.getType().toString() + " table[" + tableName + "]"); + } + + Database db = (Database) dbIf; + // the following ops should done outside table lock. because it contain synchronized create operation + if (needProcessOutsideTableLock) { + Preconditions.checkState(alterClauses.size() == 1); + AlterClause alterClause = alterClauses.get(0); + if (alterClause instanceof AddPartitionClause) { + if (!((AddPartitionClause) alterClause).isTempPartition()) { + DynamicPartitionUtil.checkAlterAllowed( + (OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP)); + } + Env.getCurrentEnv().addPartition(db, tableName, (AddPartitionClause) alterClause, false, 0, true); + } else if (alterClause instanceof AddPartitionLikeClause) { + if (!((AddPartitionLikeClause) alterClause).getIsTempPartition()) { + DynamicPartitionUtil.checkAlterAllowed( + (OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP)); + } + Env.getCurrentEnv().addPartitionLike(db, tableName, (AddPartitionLikeClause) alterClause); + } else if (alterClause instanceof ModifyPartitionClause) { + ModifyPartitionClause clause = ((ModifyPartitionClause) alterClause); + Map properties = clause.getProperties(); + List partitionNames = clause.getPartitionNames(); + ((SchemaChangeHandler) schemaChangeHandler).updatePartitionsProperties( + db, tableName, partitionNames, properties); + OlapTable olapTable = (OlapTable) tableIf; + olapTable.writeLockOrDdlException(); + try { + modifyPartitionsProperty(db, olapTable, partitionNames, properties, clause.isTempPartition()); + } finally { + olapTable.writeUnlock(); + } + } else if (alterClause instanceof ModifyTablePropertiesClause) { + Map properties = alterClause.getProperties(); + ((SchemaChangeHandler) schemaChangeHandler).updateTableProperties(db, tableName, properties); + } else if (alterClause instanceof AlterMultiPartitionClause) { + if (!((AlterMultiPartitionClause) alterClause).isTempPartition()) { + DynamicPartitionUtil.checkAlterAllowed( + (OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP)); + } + Env.getCurrentEnv().addMultiPartitions(db, tableName, (AlterMultiPartitionClause) alterClause); + } else { + throw new DdlException("Invalid alter operation: " + alterClause.getOpType()); + } + } + } + // entry of processing replace table private void processReplaceTable(Database db, OlapTable origTable, List alterClauses) throws UserException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AddColumnClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AddColumnClause.java index f27f320481329c4..837acb5b51ee2cb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AddColumnClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AddColumnClause.java @@ -38,6 +38,7 @@ public class AddColumnClause extends AlterTableClause { private static final Logger LOG = LogManager.getLogger(AddColumnClause.class); private ColumnDef columnDef; + private String sql; // Column position private ColumnPosition colPos; // if rollupName is null, add to column to base index. @@ -68,6 +69,17 @@ public AddColumnClause(ColumnDef columnDef, ColumnPosition colPos, String rollup this.properties = properties; } + // for nereids + public AddColumnClause(String sql, Column column, ColumnPosition colPos, String rollupName, + Map properties) { + super(AlterOpType.SCHEMA_CHANGE); + this.sql = sql; + this.column = column; + this.colPos = colPos; + this.rollupName = rollupName; + this.properties = properties; + } + @Override public void analyze(Analyzer analyzer) throws AnalysisException, DdlException { if (columnDef == null) { @@ -122,15 +134,19 @@ public boolean needChangeMTMVState() { @Override public String toSql() { - StringBuilder sb = new StringBuilder(); - sb.append("ADD COLUMN ").append(columnDef.toSql()); - if (colPos != null) { - sb.append(" ").append(colPos.toSql()); - } - if (rollupName != null) { - sb.append(" IN `").append(rollupName).append("`"); + if (sql != null) { + return sql; + } else { + StringBuilder sb = new StringBuilder(); + sb.append("ADD COLUMN ").append(columnDef.toSql()); + if (colPos != null) { + sb.append(" ").append(colPos.toSql()); + } + if (rollupName != null) { + sb.append(" IN `").append(rollupName).append("`"); + } + return sb.toString(); } - return sb.toString(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AddColumnsClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AddColumnsClause.java index 04ee893bd7cee31..e58ccf2179e2cd3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AddColumnsClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AddColumnsClause.java @@ -34,6 +34,7 @@ */ public class AddColumnsClause extends AlterTableClause { private List columnDefs; + private String sql; private String rollupName; private Map properties; @@ -47,6 +48,15 @@ public AddColumnsClause(List columnDefs, String rollupName, Map columns, String rollupName, Map properties) { + super(AlterOpType.SCHEMA_CHANGE); + this.sql = sql; + this.columns = columns; + this.rollupName = rollupName; + this.properties = properties; + } + public List getColumns() { return columns; } @@ -90,21 +100,25 @@ public boolean needChangeMTMVState() { @Override public String toSql() { - StringBuilder sb = new StringBuilder(); - sb.append("ADD COLUMN ("); - int idx = 0; - for (ColumnDef columnDef : columnDefs) { - if (idx != 0) { - sb.append(", "); + if (sql != null) { + return sql; + } else { + StringBuilder sb = new StringBuilder(); + sb.append("ADD COLUMN ("); + int idx = 0; + for (ColumnDef columnDef : columnDefs) { + if (idx != 0) { + sb.append(", "); + } + sb.append(columnDef.toSql()); + idx++; } - sb.append(columnDef.toSql()); - idx++; - } - sb.append(")"); - if (rollupName != null) { - sb.append(" IN `").append(rollupName).append("`"); + sb.append(")"); + if (rollupName != null) { + sb.append(" IN `").append(rollupName).append("`"); + } + return sb.toString(); } - return sb.toString(); } @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 86df87453ad5751..09db2d5b8e2127c 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 @@ -44,6 +44,15 @@ public CreateIndexClause(TableName tableName, IndexDef indexDef, boolean alter) this.alter = alter; } + // for nereids + public CreateIndexClause(TableName tableName, IndexDef indexDef, Index index, boolean alter) { + super(AlterOpType.SCHEMA_CHANGE); + this.tableName = tableName; + this.indexDef = indexDef; + this.index = index; + this.alter = alter; + } + @Override public Map getProperties() { return Maps.newHashMap(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyColumnClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyColumnClause.java index 05a6b23d0d51b58..170c81796b84017 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyColumnClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyColumnClause.java @@ -33,6 +33,7 @@ // modify one column public class ModifyColumnClause extends AlterTableClause { private ColumnDef columnDef; + private String sql; private ColumnPosition colPos; // which rollup is to be modify, if rollup is null, modify base table. private String rollupName; @@ -67,6 +68,16 @@ public ModifyColumnClause(ColumnDef columnDef, ColumnPosition colPos, String rol this.properties = properties; } + public ModifyColumnClause(String sql, Column column, ColumnPosition colPos, String rollup, + Map properties) { + super(AlterOpType.SCHEMA_CHANGE); + this.sql = sql; + this.column = column; + this.colPos = colPos; + this.rollupName = rollup; + this.properties = properties; + } + @Override public void analyze(Analyzer analyzer) throws AnalysisException, DdlException { if (columnDef == null) { @@ -112,15 +123,19 @@ public boolean needChangeMTMVState() { @Override public String toSql() { - StringBuilder sb = new StringBuilder(); - sb.append("MODIFY COLUMN ").append(columnDef.toSql()); - if (colPos != null) { - sb.append(" ").append(colPos); - } - if (rollupName != null) { - sb.append(" IN `").append(rollupName).append("`"); + if (sql != null) { + return sql; + } else { + StringBuilder sb = new StringBuilder(); + sb.append("MODIFY COLUMN ").append(columnDef.toSql()); + if (colPos != null) { + sb.append(" ").append(colPos); + } + if (rollupName != null) { + sb.append(" IN `").append(rollupName).append("`"); + } + return sb.toString(); } - return sb.toString(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyPartitionClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyPartitionClause.java index d8cfa500ec5fc3b..f409a8875e44a4d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyPartitionClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyPartitionClause.java @@ -72,6 +72,23 @@ private ModifyPartitionClause(Map properties, boolean isTempPart this.isTempPartition = isTempPartition; } + // for nereids + public ModifyPartitionClause(List partitionNames, Map properties, + boolean isTempPartition, boolean needExpand) { + super(AlterOpType.MODIFY_PARTITION); + this.partitionNames = partitionNames; + this.properties = properties; + this.needExpand = needExpand; + // ATTN: currently, modify partition only allow 3 kinds of operations: + // 1. modify replication num + // 2. modify data property + // 3. modify in memory + // And these 3 operations does not require table to be stable. + // If other kinds of operations be added later, "needTableStable" may be changed. + this.needTableStable = false; + this.isTempPartition = isTempPartition; + } + public static ModifyPartitionClause createStarClause(Map properties, boolean isTempPartition) { return new ModifyPartitionClause(properties, isTempPartition); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java index 1fd7480c06c0e72..e8d2baa5437c3ec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java @@ -67,6 +67,16 @@ public ModifyTablePropertiesClause(Map properties) { this.properties = properties; } + // for nereids + public ModifyTablePropertiesClause(Map properties, String storagePolicy, boolean isBeingSynced, + boolean needTableStable, AlterOpType opType) { + super(opType); + this.properties = properties; + this.storagePolicy = storagePolicy; + this.isBeingSynced = isBeingSynced; + this.needTableStable = needTableStable; + } + @Override public void analyze(Analyzer analyzer) throws AnalysisException { if (properties == null || properties.isEmpty()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java index f82f497176bec21..842a63c7229fd6d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java @@ -85,6 +85,14 @@ public List getPartitionNames() { return partitionNames; } + // for nereids + public PartitionNames(boolean isTemp, List partitionNames, boolean isStar, long partitionCount) { + this.partitionNames = partitionNames; + this.isTemp = isTemp; + this.isStar = isStar; + this.count = partitionCount; + } + public boolean isTemp() { return isTemp; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ReplacePartitionClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ReplacePartitionClause.java index 4991ba6575abd75..c2092a17b155f75 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ReplacePartitionClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ReplacePartitionClause.java @@ -77,6 +77,20 @@ public ReplacePartitionClause(PartitionNames partitionNames, PartitionNames temp properties, PropertyAnalyzer.PROPERTIES_USE_TEMP_PARTITION_NAME, false); } + // for nereids + public ReplacePartitionClause(PartitionNames partitionNames, PartitionNames tempPartitionNames, + boolean isForce, Map properties, + boolean isStrictRange, boolean useTempPartitionName) { + super(AlterOpType.REPLACE_PARTITION); + this.partitionNames = partitionNames; + this.tempPartitionNames = tempPartitionNames; + this.needTableStable = false; + this.forceDropOldPartition = isForce; + this.properties = properties; + this.isStrictRange = isStrictRange; + this.useTempPartitionName = useTempPartitionName; + } + public List getPartitionNames() { return partitionNames.getPartitionNames(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ReplaceTableClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ReplaceTableClause.java index 49e6a03adc7f9de..22965d4310eb6c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ReplaceTableClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ReplaceTableClause.java @@ -47,6 +47,15 @@ public ReplaceTableClause(String tblName, Map properties, boolea this.isForce = isForce; } + // for nereids + public ReplaceTableClause(String tblName, Map properties, boolean isForce, boolean swapTable) { + super(AlterOpType.REPLACE_TABLE); + this.tblName = tblName; + this.properties = properties; + this.isForce = isForce; + this.swapTable = swapTable; + } + public String getTblName() { return tblName; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 8543e18fd5b8435..3e73597365628bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -199,6 +199,7 @@ import org.apache.doris.mysql.privilege.Auth; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.jobs.load.LabelProcessor; +import org.apache.doris.nereids.trees.plans.commands.AlterTableCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand.IdType; import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVPropertyInfo; import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVRefreshInfo; @@ -4712,6 +4713,10 @@ public void alterTable(AlterTableStmt stmt) throws UserException { this.alter.processAlterTable(stmt); } + public void alterTable(AlterTableCommand command) throws UserException { + this.alter.processAlterTable(command); + } + /** * used for handling AlterViewStmt (the ALTER VIEW command). */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index a7a0c970f0b6485..5467158afec9076 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -20,6 +20,7 @@ import org.apache.doris.analysis.ArithmeticExpr.Operator; import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.analysis.ColumnNullableType; +import org.apache.doris.analysis.ColumnPosition; import org.apache.doris.analysis.PassVar; import org.apache.doris.analysis.SetType; import org.apache.doris.analysis.StorageBackend; @@ -47,14 +48,24 @@ import org.apache.doris.mtmv.MTMVRefreshSchedule; import org.apache.doris.mtmv.MTMVRefreshTriggerInfo; import org.apache.doris.nereids.DorisParser; +import org.apache.doris.nereids.DorisParser.AddColumnClauseContext; +import org.apache.doris.nereids.DorisParser.AddColumnsClauseContext; import org.apache.doris.nereids.DorisParser.AddConstraintContext; +import org.apache.doris.nereids.DorisParser.AddIndexClauseContext; +import org.apache.doris.nereids.DorisParser.AddPartitionClauseContext; +import org.apache.doris.nereids.DorisParser.AddRollupClauseContext; import org.apache.doris.nereids.DorisParser.AggClauseContext; import org.apache.doris.nereids.DorisParser.AggStateDataTypeContext; import org.apache.doris.nereids.DorisParser.AliasQueryContext; import org.apache.doris.nereids.DorisParser.AliasedQueryContext; import org.apache.doris.nereids.DorisParser.AlterMTMVContext; +import org.apache.doris.nereids.DorisParser.AlterMultiPartitionClauseContext; import org.apache.doris.nereids.DorisParser.AlterRoleContext; import org.apache.doris.nereids.DorisParser.AlterStorageVaultContext; +import org.apache.doris.nereids.DorisParser.AlterTableAddRollupContext; +import org.apache.doris.nereids.DorisParser.AlterTableClauseContext; +import org.apache.doris.nereids.DorisParser.AlterTableContext; +import org.apache.doris.nereids.DorisParser.AlterTableDropRollupContext; import org.apache.doris.nereids.DorisParser.AlterViewContext; import org.apache.doris.nereids.DorisParser.ArithmeticBinaryContext; import org.apache.doris.nereids.DorisParser.ArithmeticUnaryContext; @@ -97,11 +108,16 @@ import org.apache.doris.nereids.DorisParser.DeleteContext; import org.apache.doris.nereids.DorisParser.DereferenceContext; import org.apache.doris.nereids.DorisParser.DropCatalogRecycleBinContext; +import org.apache.doris.nereids.DorisParser.DropColumnClauseContext; import org.apache.doris.nereids.DorisParser.DropConstraintContext; +import org.apache.doris.nereids.DorisParser.DropIndexClauseContext; import org.apache.doris.nereids.DorisParser.DropMTMVContext; +import org.apache.doris.nereids.DorisParser.DropPartitionClauseContext; import org.apache.doris.nereids.DorisParser.DropProcedureContext; import org.apache.doris.nereids.DorisParser.DropRoleContext; +import org.apache.doris.nereids.DorisParser.DropRollupClauseContext; import org.apache.doris.nereids.DorisParser.ElementAtContext; +import org.apache.doris.nereids.DorisParser.EnableFeatureClauseContext; import org.apache.doris.nereids.DorisParser.ExceptContext; import org.apache.doris.nereids.DorisParser.ExceptOrReplaceContext; import org.apache.doris.nereids.DorisParser.ExistContext; @@ -135,6 +151,12 @@ import org.apache.doris.nereids.DorisParser.LogicalBinaryContext; import org.apache.doris.nereids.DorisParser.LogicalNotContext; import org.apache.doris.nereids.DorisParser.MapLiteralContext; +import org.apache.doris.nereids.DorisParser.ModifyColumnClauseContext; +import org.apache.doris.nereids.DorisParser.ModifyColumnCommentClauseContext; +import org.apache.doris.nereids.DorisParser.ModifyDistributionClauseContext; +import org.apache.doris.nereids.DorisParser.ModifyEngineClauseContext; +import org.apache.doris.nereids.DorisParser.ModifyPartitionClauseContext; +import org.apache.doris.nereids.DorisParser.ModifyTableCommentClauseContext; import org.apache.doris.nereids.DorisParser.MultiStatementsContext; import org.apache.doris.nereids.DorisParser.MultipartIdentifierContext; import org.apache.doris.nereids.DorisParser.MvPartitionContext; @@ -169,7 +191,14 @@ import org.apache.doris.nereids.DorisParser.RefreshTriggerContext; import org.apache.doris.nereids.DorisParser.RegularQuerySpecificationContext; import org.apache.doris.nereids.DorisParser.RelationContext; +import org.apache.doris.nereids.DorisParser.RenameClauseContext; +import org.apache.doris.nereids.DorisParser.RenameColumnClauseContext; +import org.apache.doris.nereids.DorisParser.RenamePartitionClauseContext; +import org.apache.doris.nereids.DorisParser.RenameRollupClauseContext; +import org.apache.doris.nereids.DorisParser.ReorderColumnsClauseContext; import org.apache.doris.nereids.DorisParser.ReplaceContext; +import org.apache.doris.nereids.DorisParser.ReplacePartitionClauseContext; +import org.apache.doris.nereids.DorisParser.ReplaceTableClauseContext; import org.apache.doris.nereids.DorisParser.ResumeMTMVContext; import org.apache.doris.nereids.DorisParser.RollupDefContext; import org.apache.doris.nereids.DorisParser.RollupDefsContext; @@ -401,6 +430,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterStorageVaultCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterTableCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; @@ -451,16 +481,23 @@ import org.apache.doris.nereids.trees.plans.commands.UnsetVariableCommand; import org.apache.doris.nereids.trees.plans.commands.UnsupportedCommand; import org.apache.doris.nereids.trees.plans.commands.UpdateCommand; +import org.apache.doris.nereids.trees.plans.commands.info.AddColumnOp; +import org.apache.doris.nereids.trees.plans.commands.info.AddColumnsOp; +import org.apache.doris.nereids.trees.plans.commands.info.AddPartitionOp; +import org.apache.doris.nereids.trees.plans.commands.info.AddRollupOp; import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVInfo; import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVPropertyInfo; import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVRefreshInfo; import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVRenameInfo; import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVReplaceInfo; +import org.apache.doris.nereids.trees.plans.commands.info.AlterMultiPartitionOp; +import org.apache.doris.nereids.trees.plans.commands.info.AlterTableOp; import org.apache.doris.nereids.trees.plans.commands.info.AlterViewInfo; import org.apache.doris.nereids.trees.plans.commands.info.BulkLoadDataDesc; import org.apache.doris.nereids.trees.plans.commands.info.BulkStorageDesc; import org.apache.doris.nereids.trees.plans.commands.info.CancelMTMVTaskInfo; import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; +import org.apache.doris.nereids.trees.plans.commands.info.CreateIndexOp; import org.apache.doris.nereids.trees.plans.commands.info.CreateJobInfo; import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; @@ -469,7 +506,13 @@ import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; import org.apache.doris.nereids.trees.plans.commands.info.DefaultValue; import org.apache.doris.nereids.trees.plans.commands.info.DistributionDescriptor; +import org.apache.doris.nereids.trees.plans.commands.info.DropColumnOp; +import org.apache.doris.nereids.trees.plans.commands.info.DropIndexOp; import org.apache.doris.nereids.trees.plans.commands.info.DropMTMVInfo; +import org.apache.doris.nereids.trees.plans.commands.info.DropPartitionFromIndexOp; +import org.apache.doris.nereids.trees.plans.commands.info.DropPartitionOp; +import org.apache.doris.nereids.trees.plans.commands.info.DropRollupOp; +import org.apache.doris.nereids.trees.plans.commands.info.EnableFeatureOp; import org.apache.doris.nereids.trees.plans.commands.info.FixedRangePartition; import org.apache.doris.nereids.trees.plans.commands.info.FuncNameInfo; import org.apache.doris.nereids.trees.plans.commands.info.GeneratedColumnDesc; @@ -477,11 +520,26 @@ import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition; import org.apache.doris.nereids.trees.plans.commands.info.LessThanPartition; import org.apache.doris.nereids.trees.plans.commands.info.MTMVPartitionDefinition; +import org.apache.doris.nereids.trees.plans.commands.info.ModifyColumnCommentOp; +import org.apache.doris.nereids.trees.plans.commands.info.ModifyColumnOp; +import org.apache.doris.nereids.trees.plans.commands.info.ModifyDistributionOp; +import org.apache.doris.nereids.trees.plans.commands.info.ModifyEngineOp; +import org.apache.doris.nereids.trees.plans.commands.info.ModifyPartitionOp; +import org.apache.doris.nereids.trees.plans.commands.info.ModifyTableCommentOp; +import org.apache.doris.nereids.trees.plans.commands.info.ModifyTablePropertiesOp; import org.apache.doris.nereids.trees.plans.commands.info.PartitionDefinition; import org.apache.doris.nereids.trees.plans.commands.info.PartitionDefinition.MaxValue; +import org.apache.doris.nereids.trees.plans.commands.info.PartitionNamesInfo; import org.apache.doris.nereids.trees.plans.commands.info.PartitionTableInfo; import org.apache.doris.nereids.trees.plans.commands.info.PauseMTMVInfo; import org.apache.doris.nereids.trees.plans.commands.info.RefreshMTMVInfo; +import org.apache.doris.nereids.trees.plans.commands.info.RenameColumnOp; +import org.apache.doris.nereids.trees.plans.commands.info.RenamePartitionOp; +import org.apache.doris.nereids.trees.plans.commands.info.RenameRollupOp; +import org.apache.doris.nereids.trees.plans.commands.info.RenameTableOp; +import org.apache.doris.nereids.trees.plans.commands.info.ReorderColumnsOp; +import org.apache.doris.nereids.trees.plans.commands.info.ReplacePartitionOp; +import org.apache.doris.nereids.trees.plans.commands.info.ReplaceTableOp; import org.apache.doris.nereids.trees.plans.commands.info.ResumeMTMVInfo; import org.apache.doris.nereids.trees.plans.commands.info.RollupDefinition; import org.apache.doris.nereids.trees.plans.commands.info.SetCharsetAndCollateVarOp; @@ -2971,6 +3029,7 @@ public List visitIndexDefs(IndexDefsContext ctx) { @Override public IndexDefinition visitIndexDef(IndexDefContext ctx) { String indexName = ctx.indexName.getText(); + boolean ifNotExists = ctx.ifNotExists != null; List indexCols = visitIdentifierList(ctx.cols); Map properties = visitPropertyItemList(ctx.properties); String indexType = ctx.indexType != null ? ctx.indexType.getText().toUpperCase() : null; @@ -2982,7 +3041,7 @@ public IndexDefinition visitIndexDef(IndexDefContext ctx) { && "BITMAP".equalsIgnoreCase(indexType)) { indexType = "INVERTED"; } - return new IndexDefinition(indexName, indexCols, indexType, properties, comment); + return new IndexDefinition(indexName, ifNotExists, indexCols, indexType, properties, comment); } @Override @@ -4040,6 +4099,311 @@ public LogicalPlan visitShowLastInsert(ShowLastInsertContext ctx) { return new ShowLastInsertCommand(); } + @Override + public AlterTableCommand visitAlterTable(AlterTableContext ctx) { + TableNameInfo tableNameInfo = new TableNameInfo(visitMultipartIdentifier(ctx.tableName)); + List alterTableOps = new ArrayList<>(); + for (Object child : ctx.children) { + if (child instanceof AlterTableClauseContext) { + alterTableOps.add(typedVisit((AlterTableClauseContext) child)); + } + } + return new AlterTableCommand(tableNameInfo, alterTableOps); + } + + @Override + public AlterTableCommand visitAlterTableAddRollup(AlterTableAddRollupContext ctx) { + TableNameInfo tableNameInfo = new TableNameInfo(visitMultipartIdentifier(ctx.tableName)); + List alterTableOps = new ArrayList<>(); + for (Object child : ctx.children) { + if (child instanceof AddRollupClauseContext) { + alterTableOps.add(typedVisit((AddRollupClauseContext) child)); + } + } + return new AlterTableCommand(tableNameInfo, alterTableOps); + } + + @Override + public AlterTableCommand visitAlterTableDropRollup(AlterTableDropRollupContext ctx) { + TableNameInfo tableNameInfo = new TableNameInfo(visitMultipartIdentifier(ctx.tableName)); + List alterTableOps = new ArrayList<>(); + for (Object child : ctx.children) { + if (child instanceof DropRollupClauseContext) { + alterTableOps.add(typedVisit((DropRollupClauseContext) child)); + } + } + return new AlterTableCommand(tableNameInfo, alterTableOps); + } + + @Override + public AlterTableCommand visitAlterTableProperties(DorisParser.AlterTablePropertiesContext ctx) { + TableNameInfo tableNameInfo = new TableNameInfo(visitMultipartIdentifier(ctx.name)); + List alterTableOps = new ArrayList<>(); + Map properties = ctx.propertyItemList() != null + ? Maps.newHashMap(visitPropertyItemList(ctx.propertyItemList())) + : Maps.newHashMap(); + alterTableOps.add(new ModifyTablePropertiesOp(properties)); + return new AlterTableCommand(tableNameInfo, alterTableOps); + } + + @Override + public AlterTableOp visitAddColumnClause(AddColumnClauseContext ctx) { + ColumnDefinition columnDefinition = visitColumnDef(ctx.columnDef()); + ColumnPosition columnPosition = null; + if (ctx.columnPosition() != null) { + if (ctx.columnPosition().FIRST() != null) { + columnPosition = ColumnPosition.FIRST; + } else { + columnPosition = new ColumnPosition(ctx.columnPosition().position.getText()); + } + } + String rollupName = ctx.toRollup() != null ? ctx.toRollup().rollup.getText() : null; + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new AddColumnOp(columnDefinition, columnPosition, rollupName, properties); + } + + @Override + public AlterTableOp visitAddColumnsClause(AddColumnsClauseContext ctx) { + List columnDefinitions = visitColumnDefs(ctx.columnDefs()); + String rollupName = ctx.toRollup() != null ? ctx.toRollup().rollup.getText() : null; + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new AddColumnsOp(columnDefinitions, rollupName, properties); + } + + @Override + public AlterTableOp visitDropColumnClause(DropColumnClauseContext ctx) { + String columnName = ctx.name.getText(); + String rollupName = ctx.fromRollup() != null ? ctx.fromRollup().rollup.getText() : null; + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new DropColumnOp(columnName, rollupName, properties); + } + + @Override + public AlterTableOp visitModifyColumnClause(ModifyColumnClauseContext ctx) { + ColumnDefinition columnDefinition = visitColumnDef(ctx.columnDef()); + ColumnPosition columnPosition = null; + if (ctx.columnPosition() != null) { + if (ctx.columnPosition().FIRST() != null) { + columnPosition = ColumnPosition.FIRST; + } else { + columnPosition = new ColumnPosition(ctx.columnPosition().position.getText()); + } + } + String rollupName = ctx.fromRollup() != null ? ctx.fromRollup().rollup.getText() : null; + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new ModifyColumnOp(columnDefinition, columnPosition, rollupName, properties); + } + + @Override + public AlterTableOp visitReorderColumnsClause(ReorderColumnsClauseContext ctx) { + List columnsByPos = visitIdentifierList(ctx.identifierList()); + String rollupName = ctx.fromRollup() != null ? ctx.fromRollup().rollup.getText() : null; + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new ReorderColumnsOp(columnsByPos, rollupName, properties); + } + + @Override + public AlterTableOp visitAddPartitionClause(AddPartitionClauseContext ctx) { + boolean isTempPartition = ctx.TEMPORARY() != null; + PartitionDefinition partitionDefinition = visitPartitionDef(ctx.partitionDef()); + DistributionDescriptor desc = null; + int bucketNum = FeConstants.default_bucket_num; + if (ctx.INTEGER_VALUE() != null) { + bucketNum = Integer.parseInt(ctx.INTEGER_VALUE().getText()); + } + if (ctx.HASH() != null) { + desc = new DistributionDescriptor(true, ctx.autoBucket != null, bucketNum, + visitIdentifierList(ctx.hashKeys)); + } else if (ctx.RANDOM() != null) { + desc = new DistributionDescriptor(false, ctx.autoBucket != null, bucketNum, null); + } + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new AddPartitionOp(partitionDefinition, desc, properties, isTempPartition); + } + + @Override + public AlterTableOp visitDropPartitionClause(DropPartitionClauseContext ctx) { + boolean isTempPartition = ctx.TEMPORARY() != null; + boolean ifExists = ctx.IF() != null; + boolean forceDrop = ctx.FORCE() != null; + String partitionName = ctx.partitionName.getText(); + return ctx.indexName != null + ? new DropPartitionFromIndexOp(ifExists, partitionName, isTempPartition, forceDrop, + ctx.indexName.getText()) + : new DropPartitionOp(ifExists, partitionName, isTempPartition, forceDrop); + } + + @Override + public AlterTableOp visitModifyPartitionClause(ModifyPartitionClauseContext ctx) { + boolean isTempPartition = ctx.TEMPORARY() != null; + Map properties = visitPropertyItemList(ctx.partitionProperties); + if (ctx.ASTERISK() != null) { + return ModifyPartitionOp.createStarClause(properties, isTempPartition); + } else { + List partitions; + if (ctx.partitionNames != null) { + partitions = visitIdentifierList(ctx.partitionNames); + } else { + partitions = new ArrayList<>(); + partitions.add(ctx.partitionName.getText()); + } + return new ModifyPartitionOp(partitions, properties, isTempPartition); + } + } + + @Override + public AlterTableOp visitReplacePartitionClause(ReplacePartitionClauseContext ctx) { + boolean forceReplace = ctx.FORCE() != null; + PartitionNamesInfo partitionNames = null; + PartitionNamesInfo tempPartitionNames = null; + if (ctx.partitions != null) { + Pair> partitionSpec = visitPartitionSpec(ctx.partitions); + partitionNames = new PartitionNamesInfo(partitionSpec.first, partitionSpec.second); + } + if (ctx.tempPartitions != null) { + Pair> partitionSpec = visitPartitionSpec(ctx.tempPartitions); + tempPartitionNames = new PartitionNamesInfo(partitionSpec.first, partitionSpec.second); + } + + Map properties = ctx.properties != null ? new HashMap<>(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new ReplacePartitionOp(partitionNames, tempPartitionNames, forceReplace, properties); + } + + @Override + public AlterTableOp visitReplaceTableClause(ReplaceTableClauseContext ctx) { + String tableName = ctx.name.getText(); + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new ReplaceTableOp(tableName, properties, ctx.FORCE() != null); + } + + @Override + public AlterTableOp visitRenameClause(RenameClauseContext ctx) { + return new RenameTableOp(ctx.newName.getText()); + } + + @Override + public AlterTableOp visitRenameRollupClause(RenameRollupClauseContext ctx) { + return new RenameRollupOp(ctx.name.getText(), ctx.newName.getText()); + } + + @Override + public AlterTableOp visitRenamePartitionClause(RenamePartitionClauseContext ctx) { + return new RenamePartitionOp(ctx.name.getText(), ctx.newName.getText()); + } + + @Override + public AlterTableOp visitRenameColumnClause(RenameColumnClauseContext ctx) { + return new RenameColumnOp(ctx.name.getText(), ctx.newName.getText()); + } + + @Override + public AlterTableOp visitAddIndexClause(AddIndexClauseContext ctx) { + IndexDefinition indexDefinition = visitIndexDef(ctx.indexDef()); + return new CreateIndexOp(null, indexDefinition, true); + } + + @Override + public AlterTableOp visitDropIndexClause(DropIndexClauseContext ctx) { + return new DropIndexOp(ctx.name.getText(), ctx.EXISTS() != null, null, true); + } + + @Override + public AlterTableOp visitEnableFeatureClause(EnableFeatureClauseContext ctx) { + String featureName = stripQuotes(ctx.STRING_LITERAL().getText()); + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new EnableFeatureOp(featureName, properties); + } + + @Override + public AlterTableOp visitModifyDistributionClause(ModifyDistributionClauseContext ctx) { + int bucketNum = FeConstants.default_bucket_num; + if (ctx.INTEGER_VALUE() != null) { + bucketNum = Integer.parseInt(ctx.INTEGER_VALUE().getText()); + } + DistributionDescriptor desc; + if (ctx.HASH() != null) { + desc = new DistributionDescriptor(true, ctx.AUTO() != null, bucketNum, + visitIdentifierList(ctx.hashKeys)); + } else if (ctx.RANDOM() != null) { + desc = new DistributionDescriptor(false, ctx.AUTO() != null, bucketNum, null); + } else { + throw new ParseException("distribution can't be empty", ctx); + } + return new ModifyDistributionOp(desc); + } + + @Override + public AlterTableOp visitModifyTableCommentClause(ModifyTableCommentClauseContext ctx) { + return new ModifyTableCommentOp(stripQuotes(ctx.STRING_LITERAL().getText())); + } + + @Override + public AlterTableOp visitModifyColumnCommentClause(ModifyColumnCommentClauseContext ctx) { + String columnName = ctx.name.getText(); + String comment = stripQuotes(ctx.STRING_LITERAL().getText()); + return new ModifyColumnCommentOp(columnName, comment); + } + + @Override + public AlterTableOp visitModifyEngineClause(ModifyEngineClauseContext ctx) { + String engineName = ctx.name.getText(); + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new ModifyEngineOp(engineName, properties); + } + + @Override + public AlterTableOp visitAlterMultiPartitionClause(AlterMultiPartitionClauseContext ctx) { + boolean isTempPartition = ctx.TEMPORARY() != null; + List from = visitPartitionValueList(ctx.from); + List to = visitPartitionValueList(ctx.to); + int num = Integer.parseInt(ctx.INTEGER_VALUE().getText()); + String unitString = ctx.unit != null ? ctx.unit.getText() : null; + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new AlterMultiPartitionOp(from, to, num, unitString, properties, isTempPartition); + } + + @Override + public AlterTableOp visitAddRollupClause(DorisParser.AddRollupClauseContext ctx) { + String rollupName = ctx.rollupName.getText(); + List columnNames = visitIdentifierList(ctx.columns); + List dupKeys = ctx.dupKeys != null ? visitIdentifierList(ctx.dupKeys) : null; + String baseRollupName = ctx.fromRollup() != null ? ctx.fromRollup().rollup.getText() : null; + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new AddRollupOp(rollupName, columnNames, dupKeys, baseRollupName, properties); + } + + @Override + public AlterTableOp visitDropRollupClause(DorisParser.DropRollupClauseContext ctx) { + String rollupName = ctx.rollupName.getText(); + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new DropRollupOp(rollupName, properties); + } + @Override public LogicalPlan visitShowVariables(ShowVariablesContext ctx) { SetType type = SetType.DEFAULT; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 99ba411f1e8f02c..86c665ac3f9efa1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -174,6 +174,7 @@ public enum PlanType { SET_TRANSACTION_COMMAND, SET_USER_PROPERTIES_COMMAND, SET_DEFAULT_STORAGE_VAULT_COMMAND, + ALTER_TABLE_COMMAND, PREPARED_COMMAND, EXECUTE_COMMAND, SHOW_CONFIG_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterTableCommand.java new file mode 100644 index 000000000000000..4db298f33f5f9ff --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterTableCommand.java @@ -0,0 +1,294 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ColumnPosition; +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.AggregateType; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.Type; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.InternalDatabaseUtil; +import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.commands.info.AddColumnOp; +import org.apache.doris.nereids.trees.plans.commands.info.AddColumnsOp; +import org.apache.doris.nereids.trees.plans.commands.info.AddRollupOp; +import org.apache.doris.nereids.trees.plans.commands.info.AlterTableOp; +import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; +import org.apache.doris.nereids.trees.plans.commands.info.DropColumnOp; +import org.apache.doris.nereids.trees.plans.commands.info.DropRollupOp; +import org.apache.doris.nereids.trees.plans.commands.info.EnableFeatureOp; +import org.apache.doris.nereids.trees.plans.commands.info.ModifyColumnOp; +import org.apache.doris.nereids.trees.plans.commands.info.ModifyEngineOp; +import org.apache.doris.nereids.trees.plans.commands.info.RenameTableOp; +import org.apache.doris.nereids.trees.plans.commands.info.ReorderColumnsOp; +import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * AlterTableCommand + */ +public class AlterTableCommand extends Command implements ForwardWithSync { + private TableNameInfo tbl; + private List ops; + + public AlterTableCommand(TableNameInfo tbl, List ops) { + super(PlanType.ALTER_TABLE_COMMAND); + this.tbl = tbl; + this.ops = ops; + } + + public TableNameInfo getTbl() { + return tbl; + } + + /** + * getOps + */ + public List getOps() { + List alterTableClauses = new ArrayList<>(ops.size()); + for (AlterTableOp op : ops) { + AlterTableClause alter = op.translateToLegacyAlterClause(); + alter.setTableName(tbl.transferToTableName()); + alterTableClauses.add(alter); + } + return alterTableClauses; + } + + /** + * validate + */ + private void validate(ConnectContext ctx) throws UserException { + if (tbl == null) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_TABLES_USED); + } + tbl.analyze(ctx); + InternalDatabaseUtil.checkDatabase(tbl.getDb(), ConnectContext.get()); + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), tbl.getCtl(), tbl.getDb(), tbl.getTbl(), + PrivPredicate.ALTER)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "ALTER TABLE", + ConnectContext.get().getQualifiedUser(), + ConnectContext.get().getRemoteIP(), + tbl.getDb() + ": " + tbl.getTbl()); + } + if (ops == null || ops.isEmpty()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_ALTER_OPERATION); + } + for (AlterTableOp op : ops) { + op.setTableName(tbl); + op.validate(ctx); + } + String ctlName = tbl.getCtl(); + String dbName = tbl.getDb(); + String tableName = tbl.getTbl(); + DatabaseIf dbIf = Env.getCurrentEnv().getCatalogMgr() + .getCatalogOrException(ctlName, catalog -> new DdlException("Unknown catalog " + catalog)) + .getDbOrDdlException(dbName); + TableIf tableIf = dbIf.getTableOrDdlException(tableName); + if (tableIf instanceof OlapTable) { + rewriteAlterOpForOlapTable(ctx, (OlapTable) tableIf); + } else { + checkExternalTableOperationAllow(tableIf); + } + } + + private void rewriteAlterOpForOlapTable(ConnectContext ctx, OlapTable table) throws UserException { + List alterTableOps = new ArrayList<>(); + for (AlterTableOp alterClause : ops) { + if (alterClause instanceof EnableFeatureOp) { + EnableFeatureOp.Features alterFeature = ((EnableFeatureOp) alterClause).getFeature(); + if (alterFeature == null || alterFeature == EnableFeatureOp.Features.UNKNOWN) { + throw new AnalysisException("unknown feature for alter clause"); + } + if (table.getKeysType() != KeysType.UNIQUE_KEYS + && alterFeature == EnableFeatureOp.Features.BATCH_DELETE) { + throw new AnalysisException("Batch delete only supported in unique tables."); + } + if (table.getKeysType() != KeysType.UNIQUE_KEYS + && alterFeature == EnableFeatureOp.Features.SEQUENCE_LOAD) { + throw new AnalysisException("Sequence load only supported in unique tables."); + } + if (alterFeature == EnableFeatureOp.Features.UPDATE_FLEXIBLE_COLUMNS) { + if (!(table.getKeysType() == KeysType.UNIQUE_KEYS && table.getEnableUniqueKeyMergeOnWrite())) { + throw new AnalysisException("Update flexible columns feature is only supported" + + " on merge-on-write unique tables."); + } + if (table.hasSkipBitmapColumn()) { + throw new AnalysisException("table " + table.getName() + + " has enabled update flexible columns feature already."); + } + } + // analyse sequence column + Type sequenceColType = null; + if (alterFeature == EnableFeatureOp.Features.SEQUENCE_LOAD) { + Map propertyMap = alterClause.getProperties(); + try { + sequenceColType = PropertyAnalyzer.analyzeSequenceType(propertyMap, table.getKeysType()); + if (sequenceColType == null) { + throw new AnalysisException("unknown sequence column type"); + } + } catch (Exception e) { + throw new AnalysisException(e.getMessage()); + } + } + + // has rollup table + if (table.getVisibleIndex().size() > 1) { + for (MaterializedIndex idx : table.getVisibleIndex()) { + // add a column to rollup index it will add to base table automatically, + // if add a column here it will duplicated + if (idx.getId() == table.getBaseIndexId()) { + continue; + } + AddColumnOp addColumnOp = null; + if (alterFeature == EnableFeatureOp.Features.BATCH_DELETE) { + addColumnOp = new AddColumnOp(ColumnDefinition.newDeleteSignColumnDefinition(), null, + table.getIndexNameById(idx.getId()), null); + } else if (alterFeature == EnableFeatureOp.Features.SEQUENCE_LOAD) { + addColumnOp = new AddColumnOp( + ColumnDefinition.newSequenceColumnDefinition( + DataType.fromCatalogType(sequenceColType)), + null, + table.getIndexNameById(idx.getId()), null); + } else { + throw new AnalysisException("unknown feature : " + alterFeature); + } + addColumnOp.validate(ctx); + alterTableOps.add(addColumnOp); + } + } else { + // no rollup tables + AddColumnOp addColumnOp = null; + if (alterFeature == EnableFeatureOp.Features.BATCH_DELETE) { + addColumnOp = new AddColumnOp(ColumnDefinition.newDeleteSignColumnDefinition(), null, + null, null); + } else if (alterFeature == EnableFeatureOp.Features.SEQUENCE_LOAD) { + addColumnOp = new AddColumnOp( + ColumnDefinition.newSequenceColumnDefinition(DataType.fromCatalogType(sequenceColType)), + null, + null, null); + } else if (alterFeature == EnableFeatureOp.Features.UPDATE_FLEXIBLE_COLUMNS) { + ColumnDefinition skipBItmapCol = ColumnDefinition.newSkipBitmapColumnDef(AggregateType.NONE); + List fullSchema = table.getBaseSchema(true); + String lastCol = fullSchema.get(fullSchema.size() - 1).getName(); + addColumnOp = new AddColumnOp(skipBItmapCol, new ColumnPosition(lastCol), null, null); + } + addColumnOp.validate(ctx); + alterTableOps.add(addColumnOp); + } + // add hidden column to rollup table + } else { + alterTableOps.add(alterClause); + } + } + ops = alterTableOps; + } + + /** + * checkExternalTableOperationAllow + */ + private void checkExternalTableOperationAllow(TableIf table) throws UserException { + List alterTableOps = new ArrayList<>(); + for (AlterTableOp alterClause : ops) { + if (alterClause instanceof RenameTableOp + || alterClause instanceof AddColumnOp + || alterClause instanceof AddColumnsOp + || alterClause instanceof DropColumnOp + || alterClause instanceof ModifyColumnOp + || alterClause instanceof ReorderColumnsOp + || alterClause instanceof ModifyEngineOp) { + alterTableOps.add(alterClause); + } else { + throw new AnalysisException(table.getType().toString() + " [" + table.getName() + "] " + + "do not support " + alterClause.getOpType().toString() + " clause now"); + } + } + ops = alterTableOps; + } + + /** + * toSql + */ + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("ALTER TABLE ").append(tbl.toSql()).append(" "); + int idx = 0; + for (AlterTableOp op : ops) { + if (idx != 0) { + sb.append(", \n"); + } + if (op instanceof AddRollupOp) { + if (idx == 0) { + sb.append("ADD ROLLUP"); + } + sb.append(op.toSql().replace("ADD ROLLUP", "")); + } else if (op instanceof DropRollupOp) { + if (idx == 0) { + sb.append("DROP ROLLUP "); + } + sb.append(((DropRollupOp) op).getRollupName()); + } else { + sb.append(op.toSql()); + } + idx++; + } + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } + + @Override + public StmtType stmtType() { + return StmtType.ALTER; + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitAlterTableCommand(this, context); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(ctx); + ctx.getEnv().alterTable(this); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnOp.java new file mode 100644 index 000000000000000..b69a8a7a865ae8c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnOp.java @@ -0,0 +1,173 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AddColumnClause; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ColumnPosition; +import org.apache.doris.catalog.AggregateType; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; +import com.google.common.collect.Sets; + +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * AddColumnOp + */ +public class AddColumnOp extends AlterTableOp { + private ColumnDefinition columnDef; + // Column position + private ColumnPosition colPos; + // if rollupName is null, add to column to base index. + private String rollupName; + + private Map properties; + // set in analyze + private Column column; + + public AddColumnOp(ColumnDefinition columnDef, ColumnPosition colPos, String rollupName, + Map properties) { + super(AlterOpType.SCHEMA_CHANGE); + this.columnDef = columnDef; + this.colPos = colPos; + this.rollupName = rollupName; + this.properties = properties; + } + + public Column getColumn() { + return column; + } + + public ColumnPosition getColPos() { + return colPos; + } + + public String getRollupName() { + return rollupName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + validateColumnDef(tableName, columnDef); + if (colPos != null) { + colPos.analyze(); + } + if (columnDef.getAggType() != null && colPos != null && colPos.isFirst()) { + throw new AnalysisException("Cannot add value column[" + columnDef.getName() + "] at first"); + } + if (Strings.isNullOrEmpty(rollupName)) { + rollupName = null; + } + + column = columnDef.translateToCatalogStyle(); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new AddColumnClause(toSql(), column, colPos, rollupName, properties); + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("ADD COLUMN ").append(columnDef.toSql()); + if (colPos != null) { + sb.append(" ").append(colPos.toSql()); + } + if (rollupName != null) { + sb.append(" IN `").append(rollupName).append("`"); + } + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } + + /** + * validateColumnDef + */ + public static void validateColumnDef(TableNameInfo tableName, ColumnDefinition columnDef) throws UserException { + if (columnDef == null) { + throw new AnalysisException("No column definition in add column clause."); + } + boolean isOlap = false; + Table table = null; + Set keysSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + if (tableName != null) { + table = Env.getCurrentInternalCatalog().getDbOrDdlException(tableName.getDb()) + .getTableOrDdlException(tableName.getTbl()); + if (table instanceof OlapTable && ((OlapTable) table).getKeysType() == KeysType.AGG_KEYS + && (columnDef.getAggType() == null || columnDef.getAggType() == AggregateType.NONE)) { + columnDef.setIsKey(true); + keysSet.add(columnDef.getName()); + } + if (table instanceof OlapTable) { + columnDef.setKeysType(((OlapTable) table).getKeysType()); + isOlap = true; + } + } + + boolean isEnableMergeOnWrite = false; + KeysType keysType = KeysType.DUP_KEYS; + Set clusterKeySet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + if (isOlap) { + OlapTable olapTable = (OlapTable) table; + isEnableMergeOnWrite = olapTable.getEnableUniqueKeyMergeOnWrite(); + keysType = olapTable.getKeysType(); + keysSet.addAll( + olapTable.getBaseSchemaKeyColumns().stream().map(Column::getName).collect(Collectors.toList())); + clusterKeySet.addAll(olapTable.getBaseSchema().stream().filter(Column::isClusterKey).map(Column::getName) + .collect(Collectors.toList())); + } + columnDef.validate(isOlap, keysSet, clusterKeySet, isEnableMergeOnWrite, keysType); + if (!columnDef.isNullable() && !columnDef.hasDefaultValue()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DEFAULT_FOR_FIELD, columnDef.getName()); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnsOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnsOp.java new file mode 100644 index 000000000000000..4d53bec1a22943a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnsOp.java @@ -0,0 +1,122 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AddColumnsClause; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.catalog.Column; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.Map; + +/** + * AddColumnsOp + */ +public class AddColumnsOp extends AlterTableOp { + private List columnDefs; + private String rollupName; + + private Map properties; + // set in analyze + private List columns; + + public AddColumnsOp(List columnDefs, String rollupName, Map properties) { + super(AlterOpType.SCHEMA_CHANGE); + this.columnDefs = columnDefs; + this.rollupName = rollupName; + this.properties = properties; + } + + public List getColumns() { + return columns; + } + + public String getRollupName() { + return rollupName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (columnDefs == null || columnDefs.isEmpty()) { + throw new AnalysisException("Columns is empty in add columns clause."); + } + for (ColumnDefinition colDef : columnDefs) { + AddColumnOp.validateColumnDef(tableName, colDef); + } + + // Make sure return null if rollup name is empty. + rollupName = Strings.emptyToNull(rollupName); + + columns = Lists.newArrayList(); + for (ColumnDefinition columnDef : columnDefs) { + Column col = columnDef.translateToCatalogStyle(); + columns.add(col); + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new AddColumnsClause(toSql(), columns, rollupName, properties); + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("ADD COLUMN ("); + int idx = 0; + for (ColumnDefinition columnDef : columnDefs) { + if (idx != 0) { + sb.append(", "); + } + sb.append(columnDef.toSql()); + idx++; + } + sb.append(")"); + if (rollupName != null) { + sb.append(" IN `").append(rollupName).append("`"); + } + return sb.toString(); + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionOp.java new file mode 100644 index 000000000000000..bfb1b5cbca36619 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionOp.java @@ -0,0 +1,140 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AddPartitionClause; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.DistributionDesc; +import org.apache.doris.analysis.SinglePartitionDesc; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.UserException; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.qe.ConnectContext; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * AddPartitionOp + */ +public class AddPartitionOp extends AlterTableOp { + private PartitionDefinition partitionDesc; + private DistributionDescriptor distributionDesc; + private Map properties; + // true if this is to add a temporary partition + private boolean isTempPartition; + + /** + * AddPartitionOp + */ + public AddPartitionOp(PartitionDefinition partitionDesc, + DistributionDescriptor distributionDesc, + Map properties, + boolean isTempPartition) { + super(AlterOpType.ADD_PARTITION); + this.partitionDesc = partitionDesc; + this.distributionDesc = distributionDesc; + this.properties = properties; + this.isTempPartition = isTempPartition; + + this.needTableStable = false; + } + + private SinglePartitionDesc getSingeRangePartitionDesc() { + SinglePartitionDesc singlePartitionDesc = (SinglePartitionDesc) partitionDesc.translateToCatalogStyle(); + // TODO fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java#addPartition + // will call singlePartitionDesc.analyze method, so have to set analyzed to false to let it work + singlePartitionDesc.setAnalyzed(false); + return singlePartitionDesc; + } + + private DistributionDesc getDistributionDesc() { + return distributionDesc != null ? distributionDesc.translateToCatalogStyle() : null; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (partitionDesc instanceof StepPartition) { + throw new AnalysisException("StepPartition is not supported"); + } + String ctlName = tableName.getCtl(); + String dbName = tableName.getDb(); + String tbName = tableName.getTbl(); + DatabaseIf dbIf = Env.getCurrentEnv().getCatalogMgr() + .getCatalogOrException(ctlName, catalog -> new DdlException("Unknown catalog " + catalog)) + .getDbOrDdlException(dbName); + TableIf tableIf = dbIf.getTableOrDdlException(tbName); + if (tableIf instanceof OlapTable) { + OlapTable olapTable = (OlapTable) tableIf; + List partitionTypes = new ArrayList<>(olapTable.getPartitionColumns().size()); + for (Column col : olapTable.getPartitionColumns()) { + partitionTypes.add(DataType.fromCatalogType(col.getType())); + } + partitionDesc.setPartitionTypes(partitionTypes); + } + partitionDesc.validate(properties); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new AddPartitionClause(getSingeRangePartitionDesc(), getDistributionDesc(), properties, isTempPartition); + } + + public boolean isTempPartition() { + return isTempPartition; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("ADD "); + sb.append(getSingeRangePartitionDesc().toSql() + "\n"); + if (distributionDesc != null) { + sb.append(getDistributionDesc().toSql()); + } + return sb.toString(); + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddRollupOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddRollupOp.java new file mode 100644 index 000000000000000..eb59237e7394d11 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddRollupOp.java @@ -0,0 +1,140 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AddRollupClause; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; +import com.google.common.collect.Sets; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * AddRollupOp + */ +public class AddRollupOp extends AlterTableOp { + private String rollupName; + private List columnNames; + private String baseRollupName; + private List dupKeys; + + private Map properties; + + /** + * AddRollupOp constructor + */ + public AddRollupOp(String rollupName, List columnNames, + List dupKeys, String baseRollupName, + Map properties) { + super(AlterOpType.ADD_ROLLUP); + this.rollupName = rollupName; + this.columnNames = columnNames; + this.dupKeys = dupKeys; + this.baseRollupName = baseRollupName; + this.properties = properties; + } + + public String getRollupName() { + return rollupName; + } + + public List getColumnNames() { + return columnNames; + } + + public List getDupKeys() { + return dupKeys; + } + + public String getBaseRollupName() { + return baseRollupName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + FeNameFormat.checkTableName(rollupName); + + if (columnNames == null || columnNames.isEmpty()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLE_MUST_HAVE_COLUMNS); + } + Set colSet = Sets.newHashSet(); + for (String col : columnNames) { + if (Strings.isNullOrEmpty(col)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_COLUMN_NAME, + col, FeNameFormat.getColumnNameRegex()); + } + if (!colSet.add(col)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DUP_FIELDNAME, col); + } + } + baseRollupName = Strings.emptyToNull(baseRollupName); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new AddRollupClause(rollupName, columnNames, dupKeys, baseRollupName, properties); + } + + @Override + public boolean allowOpMTMV() { + return true; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append("ADD ROLLUP `").append(rollupName).append("` ("); + int idx = 0; + for (String column : columnNames) { + if (idx != 0) { + stringBuilder.append(", "); + } + stringBuilder.append("`").append(column).append("`"); + idx++; + } + stringBuilder.append(")"); + if (baseRollupName != null) { + stringBuilder.append(" FROM `").append(baseRollupName).append("`"); + } + return stringBuilder.toString(); + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMultiPartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMultiPartitionOp.java new file mode 100644 index 000000000000000..fcb930a8f94461e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMultiPartitionOp.java @@ -0,0 +1,128 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterMultiPartitionClause; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.PartitionKeyDesc; +import org.apache.doris.analysis.PartitionValue; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.literal.MaxLiteral; + +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * AlterMultiPartitionOp + */ +public class AlterMultiPartitionOp extends AlterTableOp { + private final List fromExpression; + private final List toExpression; + private final long unit; + private final String unitString; + private Map properties; + private boolean isTempPartition; + + /** + * AlterMultiPartitionOp + */ + public AlterMultiPartitionOp(List fromExpression, + List toExpression, long unit, String unitString, Map properties, + boolean isTempPartition) { + super(AlterOpType.ADD_PARTITION); + this.fromExpression = fromExpression; + this.toExpression = toExpression; + this.unit = unit; + this.unitString = unitString; + this.properties = properties; + this.isTempPartition = isTempPartition; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder("["); + sb.append(getPartitionValuesStr(fromExpression)).append(", ").append(getPartitionValuesStr(toExpression)); + sb.append(")"); + return String.format("ADD PARTITIONS %s", sb); + } + + @Override + public String toString() { + return toSql(); + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + public Map getProperties() { + return properties; + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + List fromValues = fromExpression.stream() + .map(this::toLegacyPartitionValue) + .collect(Collectors.toList()); + List toValues = toExpression.stream() + .map(this::toLegacyPartitionValue) + .collect(Collectors.toList()); + PartitionKeyDesc partitionKeyDesc = (unitString == null + ? PartitionKeyDesc.createMultiFixed(fromValues, toValues, unit) + : PartitionKeyDesc.createMultiFixed(fromValues, toValues, unit, unitString)); + return new AlterMultiPartitionClause(partitionKeyDesc, properties, isTempPartition); + } + + public boolean isTempPartition() { + return isTempPartition; + } + + private String getPartitionValuesStr(List values) { + StringBuilder sb = new StringBuilder("("); + Joiner.on(", ").appendTo(sb, Lists.transform(values, v -> { + if (v instanceof MaxLiteral) { + return v.toSql(); + } else { + return "'" + v.toSql() + "'"; + } + })).append(")"); + return sb.toString(); + } + + private PartitionValue toLegacyPartitionValue(Expression e) { + if (e.isLiteral()) { + return new PartitionValue(((Literal) e).getStringValue(), e.isNullLiteral()); + } else if (e instanceof PartitionDefinition.MaxValue) { + return PartitionValue.MAX_VALUE; + } + throw new AnalysisException("Unsupported partition value"); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterTableOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterTableOp.java new file mode 100644 index 000000000000000..4537d0d45f3f9b7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterTableOp.java @@ -0,0 +1,62 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import java.util.Map; + +/** + * AlterTableOp + */ +public abstract class AlterTableOp { + // if set to true, the corresponding table should be stable before processing this operation on it. + protected boolean needTableStable = true; + + protected AlterOpType opType; + + protected TableNameInfo tableName; + + public AlterTableOp(AlterOpType opType) { + this.opType = opType; + } + + public AlterOpType getOpType() { + return opType; + } + + public void setTableName(TableNameInfo tableName) { + this.tableName = tableName; + } + + public abstract boolean allowOpMTMV(); + + public abstract boolean needChangeMTMVState(); + + public abstract String toSql(); + + public abstract Map getProperties(); + + public void validate(ConnectContext ctx) throws UserException { + } + + public abstract AlterTableClause translateToLegacyAlterClause(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java index 047ef2c1b51f5b2..82345e1cdaeea7e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java @@ -41,7 +41,6 @@ import org.apache.doris.nereids.types.coercion.CharacterType; import org.apache.doris.qe.SessionVariable; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import java.util.ArrayList; @@ -69,6 +68,7 @@ public class ColumnDefinition { private int clusterKeyId = -1; private Optional generatedColumnDesc = Optional.empty(); private Set generatedColumnsThatReferToThis = new HashSet<>(); + private KeysType keysType; public ColumnDefinition(String name, DataType type, boolean isKey, AggregateType aggType, boolean isNullable, Optional defaultValue, String comment) { @@ -176,6 +176,48 @@ public void setClusterKeyId(int clusterKeyId) { this.clusterKeyId = clusterKeyId; } + public void setKeysType(KeysType keysType) { + this.keysType = keysType; + } + + public boolean hasDefaultValue() { + return defaultValue.isPresent(); + } + + /** + * toSql + */ + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("`").append(name).append("` "); + sb.append(type.toSql()).append(" "); + + if (aggType != null) { + sb.append(aggType.name()).append(" "); + } + + if (!isNullable) { + sb.append("NOT NULL "); + } else { + // should append NULL to make result can be executed right. + sb.append("NULL "); + } + + if (autoIncInitValue != -1) { + sb.append("AUTO_INCREMENT "); + sb.append("("); + sb.append(autoIncInitValue); + sb.append(")"); + } + + if (defaultValue.isPresent()) { + sb.append(defaultValue.get()).append(" "); + } + sb.append("COMMENT \"").append(comment).append("\""); + + return sb.toString(); + } + private DataType updateCharacterTypeLength(DataType dataType) { if (dataType instanceof ArrayType) { return ArrayType.of(updateCharacterTypeLength(((ArrayType) dataType).getItemType())); @@ -250,37 +292,27 @@ public void validate(boolean isOlap, Set keysSet, Set clusterKey } } if (type.isHllType() || type.isQuantileStateType() || type.isBitmapType()) { - if (isNullable) { - throw new AnalysisException("complex type column must be not nullable, column:" + name); - } - } - - // check keys type - if (keysSet.contains(name)) { - isKey = true; - if (aggType != null) { - throw new AnalysisException( - String.format("Key column %s can not set aggregation type", name)); + if (isKey) { + throw new AnalysisException("Key column can not set complex type:" + name); } - checkKeyColumnType(isOlap); - } else if (aggType == null && isOlap) { - Preconditions.checkState(keysType != null, "keysType is null"); - if (keysType.equals(KeysType.DUP_KEYS)) { - aggType = AggregateType.NONE; - } else if (keysType.equals(KeysType.UNIQUE_KEYS) && isEnableMergeOnWrite) { - aggType = AggregateType.NONE; - } else if (!keysType.equals(KeysType.AGG_KEYS)) { - aggType = AggregateType.REPLACE; - } else { - throw new AnalysisException("should set aggregation type to non-key column when in aggregate key"); + if (keysType.equals(KeysType.AGG_KEYS)) { + if (aggType == null) { + throw new AnalysisException("complex type have to use aggregate function: " + name); + } } + isNullable = false; } - if (clusterKeySet.contains(name)) { + // check keys type + if (isKey || clusterKeySet.contains(name)) { checkKeyColumnType(isOlap); } if (aggType != null) { + if (isKey) { + throw new AnalysisException( + String.format("Key column %s can not set aggregation type", name)); + } // check if aggregate type is valid if (aggType != AggregateType.GENERIC && !aggType.checkCompatibility(type.toCatalogDataType().getPrimitiveType())) { 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 new file mode 100644 index 000000000000000..bd15ea84e4b3daf --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateIndexOp.java @@ -0,0 +1,107 @@ +// 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.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.TableName; +import org.apache.doris.catalog.Index; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.Maps; + +import java.util.Map; + +/** + * CreateIndexOp + */ +public class CreateIndexOp extends AlterTableOp { + // in which table the index on, only used when alter = false + private TableName tableName; + // index definition class + 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 CreateIndexOp(TableName tableName, IndexDefinition indexDef, boolean alter) { + super(AlterOpType.SCHEMA_CHANGE); + this.tableName = tableName; + this.indexDef = indexDef; + this.alter = alter; + } + + @Override + public Map getProperties() { + return Maps.newHashMap(); + } + + public Index getIndex() { + return index; + } + + public IndexDefinition getIndexDef() { + return indexDef; + } + + public boolean isAlter() { + return alter; + } + + public TableName getTableName() { + return tableName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (indexDef == null) { + throw new AnalysisException("index definition expected."); + } + indexDef.validate(); + index = indexDef.translateToCatalogStyle(); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new CreateIndexClause(tableName, indexDef.translateToLegacyIndexDef(), index, alter); + } + + @Override + public boolean allowOpMTMV() { + return true; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + if (alter) { + return indexDef.toSql(); + } else { + return "CREATE " + indexDef.toSql(tableName.toSql()); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index de5e188d5a65bf7..52f9cce8126f986 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -294,6 +294,7 @@ private void analyzeKeys() { || keyLength > FeConstants.shortkey_maxsize_bytes) { if (keys.isEmpty() && type.isStringLikeType()) { keys.add(column.getName()); + column.setIsKey(true); } break; } @@ -304,6 +305,7 @@ private void analyzeKeys() { break; } keys.add(column.getName()); + column.setIsKey(true); if (type.isVarcharType()) { break; } 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 e45e71bd1f997dd..07699bedb78771b 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 @@ -579,7 +579,6 @@ public void validate(ConnectContext ctx) { throw new AnalysisException(engineName + " catalog doesn't support column with 'NOT NULL'."); } columnDef.setIsKey(true); - columnDef.setAggType(AggregateType.NONE); } // TODO: support iceberg partition check if (engineName.equalsIgnoreCase(ENGINE_HIVE)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DefaultValue.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DefaultValue.java index e3f388d272deebc..7f5a55c94b842c1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DefaultValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DefaultValue.java @@ -118,4 +118,18 @@ public String getValue() { public DefaultValueExprDef getDefaultValueExprDef() { return defaultValueExprDef; } + + /** + * toSql + */ + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("DEFAULT "); + if (value != null) { + sb.append('"').append(value).append('"'); + } else { + sb.append("NULL"); + } + return sb.toString(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DistributionDescriptor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DistributionDescriptor.java index 3e4d007ff5b9132..d93fef7add77f40 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DistributionDescriptor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DistributionDescriptor.java @@ -115,4 +115,37 @@ public DistributionDesc translateToCatalogStyle() { public boolean inDistributionColumns(String columnName) { return cols != null && cols.contains(columnName); } + + /** + * toSql + */ + public String toSql() { + StringBuilder stringBuilder = new StringBuilder(); + if (isHash) { + stringBuilder.append("DISTRIBUTED BY HASH("); + int i = 0; + for (String columnName : cols) { + if (i != 0) { + stringBuilder.append(", "); + } + stringBuilder.append("`").append(columnName).append("`"); + i++; + } + stringBuilder.append(")\n"); + if (isAutoBucket) { + stringBuilder.append("BUCKETS AUTO"); + } else { + stringBuilder.append("BUCKETS ").append(bucketNum); + } + } else { + stringBuilder.append("DISTRIBUTED BY RANDOM\n") + .append("BUCKETS "); + if (isAutoBucket) { + stringBuilder.append("AUTO"); + } else { + stringBuilder.append(bucketNum); + } + } + return stringBuilder.toString(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropColumnOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropColumnOp.java new file mode 100644 index 000000000000000..18e86fe4fe2fae0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropColumnOp.java @@ -0,0 +1,105 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.DropColumnClause; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * DropColumnOp + */ +public class DropColumnOp extends AlterTableOp { + private String colName; + private String rollupName; + + private Map properties; + + /** + * DropColumnOp + */ + public DropColumnOp(String colName, String rollupName, Map properties) { + super(AlterOpType.SCHEMA_CHANGE); + this.colName = colName; + this.rollupName = rollupName; + this.properties = properties; + } + + public String getColName() { + return colName; + } + + public String getRollupName() { + return rollupName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(colName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_COLUMN_NAME, + colName, FeNameFormat.getColumnNameRegex()); + } + if (Strings.isNullOrEmpty(rollupName)) { + rollupName = null; + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new DropColumnClause(colName, rollupName, properties); + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return true; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("DROP COLUMN `").append(colName).append("`"); + if (rollupName != null) { + sb.append(" IN `").append(rollupName).append("`"); + } + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropIndexOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropIndexOp.java new file mode 100644 index 000000000000000..b4a54db0b9e1d17 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropIndexOp.java @@ -0,0 +1,102 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.DropIndexClause; +import org.apache.doris.analysis.TableName; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Map; + +/** + * DropIndexOp + */ +public class DropIndexOp extends AlterTableOp { + private final String indexName; + private final TableName tableName; + private boolean ifExists; + + private boolean alter; + + public DropIndexOp(String indexName, boolean ifExists, TableName tableName, boolean alter) { + super(AlterOpType.SCHEMA_CHANGE); + this.indexName = indexName; + this.ifExists = ifExists; + this.tableName = tableName; + this.alter = alter; + } + + public String getIndexName() { + return indexName; + } + + public TableName getTableName() { + return tableName; + } + + public boolean isSetIfExists() { + return ifExists; + } + + public boolean isAlter() { + return alter; + } + + @Override + public Map getProperties() { + return null; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (StringUtils.isEmpty(indexName)) { + throw new AnalysisException("index name is excepted"); + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new DropIndexClause(indexName, ifExists, tableName, alter); + } + + @Override + public boolean allowOpMTMV() { + return true; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append("DROP INDEX ").append(indexName); + if (!alter) { + stringBuilder.append(" ON ").append(tableName.toSql()); + } + return stringBuilder.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropPartitionFromIndexOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropPartitionFromIndexOp.java new file mode 100644 index 000000000000000..e8d4177abb606c8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropPartitionFromIndexOp.java @@ -0,0 +1,115 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.DropPartitionFromIndexClause; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * DropPartitionFromIndexOp + */ +public class DropPartitionFromIndexOp extends AlterTableOp { + private boolean ifExists; + private String partitionName; + // true if this is to drop a temp partition + private boolean isTempPartition; + private boolean forceDrop; + private String indexName; + + /** + * DropPartitionFromIndexOp + */ + public DropPartitionFromIndexOp(boolean ifExists, String partitionName, boolean isTempPartition, + boolean forceDrop, String indexName) { + super(AlterOpType.DROP_PARTITION); + this.ifExists = ifExists; + this.partitionName = partitionName; + this.isTempPartition = isTempPartition; + this.needTableStable = false; + this.forceDrop = forceDrop; + this.indexName = indexName; + } + + public boolean isSetIfExists() { + return ifExists; + } + + public String getPartitionName() { + return partitionName; + } + + public boolean isTempPartition() { + return isTempPartition; + } + + public boolean isForceDrop() { + return forceDrop; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(partitionName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_PARTITION_NAME, partitionName); + } + if (Strings.isNullOrEmpty(indexName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_NAME_FOR_INDEX, indexName); + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new DropPartitionFromIndexClause(ifExists, partitionName, isTempPartition, forceDrop, indexName); + } + + @Override + public Map getProperties() { + return null; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("DROP PARTITION " + partitionName); + sb.append(" FROM INDEX " + indexName); + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropPartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropPartitionOp.java new file mode 100644 index 000000000000000..e81bfb201b2b6aa --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropPartitionOp.java @@ -0,0 +1,108 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.DropPartitionClause; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * DropPartitionOp + */ +public class DropPartitionOp extends AlterTableOp { + private boolean ifExists; + private String partitionName; + // true if this is to drop a temp partition + private boolean isTempPartition; + private boolean forceDrop; + + /** + * DropPartitionOp + */ + public DropPartitionOp(boolean ifExists, String partitionName, boolean isTempPartition, boolean forceDrop) { + super(AlterOpType.DROP_PARTITION); + this.ifExists = ifExists; + this.partitionName = partitionName; + this.isTempPartition = isTempPartition; + this.needTableStable = false; + this.forceDrop = forceDrop; + } + + public boolean isSetIfExists() { + return ifExists; + } + + public String getPartitionName() { + return partitionName; + } + + public boolean isTempPartition() { + return isTempPartition; + } + + public boolean isForceDrop() { + return forceDrop; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(partitionName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_PARTITION_NAME, partitionName); + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new DropPartitionClause(ifExists, partitionName, isTempPartition, forceDrop); + } + + @Override + public Map getProperties() { + return null; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("DROP PARTITION " + partitionName); + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropRollupOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropRollupOp.java new file mode 100644 index 000000000000000..ef74d3bcc2a4848 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropRollupOp.java @@ -0,0 +1,88 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.DropRollupClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * DropRollupOp + */ +public class DropRollupOp extends AlterTableOp { + private final String rollupName; + private Map properties; + + public DropRollupOp(String rollupName, Map properties) { + super(AlterOpType.DROP_ROLLUP); + this.rollupName = rollupName; + this.properties = properties; + this.needTableStable = false; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(rollupName)) { + throw new AnalysisException("No rollup in delete rollup."); + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new DropRollupClause(rollupName, properties); + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public boolean allowOpMTMV() { + return true; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append("DROP ROLLUP "); + stringBuilder.append("`").append(rollupName).append("`"); + return stringBuilder.toString(); + } + + @Override + public String toString() { + return toSql(); + } + + public String getRollupName() { + return rollupName; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/EnableFeatureOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/EnableFeatureOp.java new file mode 100644 index 000000000000000..0b3e35ace80d78a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/EnableFeatureOp.java @@ -0,0 +1,122 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.EnableFeatureClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.PrintableMap; +import org.apache.doris.qe.ConnectContext; + +import java.util.Map; + +/** + * EnableFeatureOp + */ +public class EnableFeatureOp extends AlterTableOp { + /** + * Features + */ + public enum Features { + BATCH_DELETE, + SEQUENCE_LOAD, + UPDATE_FLEXIBLE_COLUMNS, + UNKNOWN + } + + private String featureName; + private boolean needSchemaChange; + private Features feature; + private Map properties; + + public EnableFeatureOp(String featureName) { + this(featureName, null); + } + + public EnableFeatureOp(String featureName, Map properties) { + super(AlterOpType.ENABLE_FEATURE); + this.featureName = featureName; + this.needSchemaChange = false; + this.properties = properties; + } + + public boolean needSchemaChange() { + return needSchemaChange; + } + + public Features getFeature() { + return feature; + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + switch (featureName.toUpperCase()) { + case "BATCH_DELETE": + this.needSchemaChange = true; + this.feature = Features.BATCH_DELETE; + break; + case "SEQUENCE_LOAD": + this.needSchemaChange = true; + this.feature = Features.SEQUENCE_LOAD; + if (properties == null || properties.isEmpty()) { + throw new AnalysisException("Properties is not set"); + } + break; + case "UPDATE_FLEXIBLE_COLUMNS": + this.needSchemaChange = true; + this.feature = Features.UPDATE_FLEXIBLE_COLUMNS; + break; + default: + throw new AnalysisException("unknown feature name: " + featureName); + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new EnableFeatureClause(featureName, properties); + } + + @Override + public boolean allowOpMTMV() { + return true; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("ENABLE FEATURE \"").append(featureName).append("\""); + if (properties != null && !properties.isEmpty()) { + sb.append(" WITH PROPERTIES ("); + sb.append(new PrintableMap(properties, "=", true, false)); + sb.append(")"); + } + return sb.toString(); + } +} 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 4d22e5af51c0eee..e8b582098e9f0e4 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 @@ -50,12 +50,15 @@ public class IndexDefinition { private Map properties = new HashMap<>(); private boolean isBuildDeferred = false; + private boolean ifNotExists = false; + /** * constructor for IndexDefinition */ - public IndexDefinition(String name, List cols, String indexTypeName, + public IndexDefinition(String name, boolean ifNotExists, List cols, String indexTypeName, Map properties, String comment) { this.name = name; + this.ifNotExists = ifNotExists; this.cols = Utils.copyRequiredList(cols); this.indexType = IndexType.INVERTED; if (indexTypeName != null) { @@ -217,4 +220,56 @@ public Index translateToCatalogStyle() { return new Index(Env.getCurrentEnv().getNextId(), name, cols, indexType, properties, comment, null); } + + public IndexDef translateToLegacyIndexDef() { + return new IndexDef(name, ifNotExists, cols, indexType, properties, comment); + } + + public String toSql() { + return toSql(null); + } + + /** + * toSql + */ + public String toSql(String tableName) { + StringBuilder sb = new StringBuilder("INDEX "); + sb.append(name); + if (tableName != null && !tableName.isEmpty()) { + sb.append(" ON ").append(tableName); + } + if (cols != null && cols.size() > 0) { + sb.append(" ("); + boolean first = true; + for (String col : cols) { + 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 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 '" + comment + "'"); + } + return sb.toString(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyColumnCommentOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyColumnCommentOp.java new file mode 100644 index 000000000000000..8553732f8dd5df2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyColumnCommentOp.java @@ -0,0 +1,92 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ModifyColumnCommentClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; +import com.google.common.collect.Maps; + +import java.util.Map; + +/** + * ModifyColumnCommentOp + */ +public class ModifyColumnCommentOp extends AlterTableOp { + private String colName; + private String comment; + + public ModifyColumnCommentOp(String colName, String comment) { + super(AlterOpType.MODIFY_COLUMN_COMMENT); + this.colName = colName; + this.comment = Strings.nullToEmpty(comment); + } + + public String getColName() { + return colName; + } + + public String getComment() { + return comment; + } + + @Override + public Map getProperties() { + return Maps.newHashMap(); + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(colName)) { + throw new AnalysisException("Empty column name"); + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ModifyColumnCommentClause(colName, comment); + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("MODIFY COLUMN COMMENT ").append(colName); + sb.append(" '").append(comment).append("'"); + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyColumnOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyColumnOp.java new file mode 100644 index 000000000000000..dee6872e4b28b62 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyColumnOp.java @@ -0,0 +1,156 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ColumnPosition; +import org.apache.doris.analysis.ModifyColumnClause; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; +import com.google.common.collect.Sets; + +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * ModifyColumnOp + */ +public class ModifyColumnOp extends AlterTableOp { + private ColumnDefinition columnDef; + private ColumnPosition colPos; + // which rollup is to be modify, if rollup is null, modify base table. + private String rollupName; + + private Map properties; + + // set in analyze + private Column column; + + public ModifyColumnOp(ColumnDefinition columnDef, ColumnPosition colPos, String rollup, + Map properties) { + super(AlterOpType.SCHEMA_CHANGE); + this.columnDef = columnDef; + this.colPos = colPos; + this.rollupName = rollup; + this.properties = properties; + } + + public Column getColumn() { + return column; + } + + public ColumnPosition getColPos() { + return colPos; + } + + public String getRollupName() { + return rollupName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (columnDef == null) { + throw new AnalysisException("No column definition in add column clause."); + } + boolean isOlap = false; + Table table = null; + Set keysSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + if (tableName != null) { + table = Env.getCurrentInternalCatalog().getDbOrDdlException(tableName.getDb()) + .getTableOrDdlException(tableName.getTbl()); + if (table instanceof OlapTable && ((OlapTable) table).getKeysType() == KeysType.AGG_KEYS + && columnDef.getAggType() == null) { + columnDef.setIsKey(true); + keysSet.add(columnDef.getName()); + } + if (table instanceof OlapTable) { + columnDef.setKeysType(((OlapTable) table).getKeysType()); + isOlap = true; + } + } + + boolean isEnableMergeOnWrite = false; + KeysType keysType = KeysType.DUP_KEYS; + Set clusterKeySet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + if (isOlap) { + OlapTable olapTable = (OlapTable) table; + isEnableMergeOnWrite = olapTable.getEnableUniqueKeyMergeOnWrite(); + keysType = olapTable.getKeysType(); + keysSet.addAll( + olapTable.getBaseSchemaKeyColumns().stream().map(Column::getName).collect(Collectors.toList())); + clusterKeySet.addAll(olapTable.getBaseSchema().stream().filter(Column::isClusterKey).map(Column::getName) + .collect(Collectors.toList())); + } + columnDef.validate(isOlap, keysSet, clusterKeySet, isEnableMergeOnWrite, keysType); + if (colPos != null) { + colPos.analyze(); + } + if (Strings.isNullOrEmpty(rollupName)) { + rollupName = null; + } + column = columnDef.translateToCatalogStyle(); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ModifyColumnClause(toSql(), column, colPos, rollupName, properties); + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return true; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("MODIFY COLUMN ").append(columnDef.toSql()); + if (colPos != null) { + sb.append(" ").append(colPos); + } + if (rollupName != null) { + sb.append(" IN `").append(rollupName).append("`"); + } + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyDistributionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyDistributionOp.java new file mode 100644 index 000000000000000..ab73f6b7e6b6682 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyDistributionOp.java @@ -0,0 +1,77 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ModifyDistributionClause; + +import java.util.HashMap; +import java.util.Map; + +/** + * ModifyDistributionOp + */ +public class ModifyDistributionOp extends AlterTableOp { + private DistributionDescriptor distributionDesc; + + public ModifyDistributionOp(DistributionDescriptor distributionDesc) { + super(AlterOpType.MODIFY_DISTRIBUTION); + this.distributionDesc = distributionDesc; + this.needTableStable = false; + } + + public DistributionDescriptor getDistributionDesc() { + return distributionDesc; + } + + @Override + public boolean allowOpMTMV() { + return true; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("MODIFY DISTRIBUTION "); + if (distributionDesc != null) { + sb.append(distributionDesc.toSql()); + } + return sb.toString(); + } + + @Override + public Map getProperties() { + return new HashMap<>(); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ModifyDistributionClause(distributionDesc.translateToCatalogStyle()); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyEngineOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyEngineOp.java new file mode 100644 index 000000000000000..1bbf71952cc76fc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyEngineOp.java @@ -0,0 +1,96 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ModifyEngineClause; +import org.apache.doris.catalog.OdbcTable; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * ModifyEngineOp + */ +public class ModifyEngineOp extends AlterTableOp { + private String engine; + private Map properties; + + public ModifyEngineOp(String engine, Map properties) { + super(AlterOpType.MODIFY_ENGINE); + this.engine = engine; + this.properties = properties; + } + + public String getEngine() { + return engine; + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(engine)) { + throw new AnalysisException("Engine name is missing"); + } + + if (!engine.equalsIgnoreCase(Table.TableType.ODBC.name())) { + throw new AnalysisException("Only support alter table engine from MySQL to ODBC"); + } + + if (properties == null || !properties.containsKey(OdbcTable.ODBC_DRIVER)) { + throw new AnalysisException("Need specify 'driver' property"); + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ModifyEngineClause(engine, properties); + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("MODIFY ENGINE TO ").append(engine); + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyPartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyPartitionOp.java new file mode 100644 index 000000000000000..32a83d1a88d0bd7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyPartitionOp.java @@ -0,0 +1,178 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ModifyPartitionClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.PrintableMap; +import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Joiner; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +/** + * ModifyPartitionOp + */ +public class ModifyPartitionOp extends AlterTableOp { + private List partitionNames; + private Map properties = Maps.newHashMap(); + private boolean isTempPartition; + private boolean needExpand; + + /** + * ModifyPartitionOp + */ + public ModifyPartitionOp(List partitionNames, Map properties, + boolean isTempPartition) { + super(AlterOpType.MODIFY_PARTITION); + this.partitionNames = partitionNames; + if (properties != null) { + this.properties.putAll(properties); + } + this.needExpand = false; + // ATTN: currently, modify partition only allow 3 kinds of operations: + // 1. modify replication num + // 2. modify data property + // 3. modify in memory + // And these 3 operations does not require table to be stable. + // If other kinds of operations be added later, "needTableStable" may be changed. + this.needTableStable = false; + this.isTempPartition = isTempPartition; + } + + // c'tor for 'Modify Partition(*)' clause + private ModifyPartitionOp(Map properties, boolean isTempPartition) { + super(AlterOpType.MODIFY_PARTITION); + this.partitionNames = Lists.newArrayList(); + if (properties != null) { + this.properties.putAll(properties); + } + this.needExpand = true; + this.needTableStable = false; + this.isTempPartition = isTempPartition; + } + + public List getPartitionNames() { + return partitionNames; + } + + public static ModifyPartitionOp createStarClause(Map properties, + boolean isTempPartition) { + return new ModifyPartitionOp(properties, isTempPartition); + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (partitionNames == null || (!needExpand && partitionNames.isEmpty())) { + throw new AnalysisException("Partition names is not set or empty"); + } + + if (partitionNames.stream().anyMatch(entity -> Strings.isNullOrEmpty(entity))) { + throw new AnalysisException("there are empty partition name"); + } + + if (properties == null || properties.isEmpty()) { + throw new AnalysisException("Properties is not set"); + } + + // check properties here + checkProperties(Maps.newHashMap(properties)); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ModifyPartitionClause(partitionNames, properties, isTempPartition, needExpand); + } + + // Check the following properties' legality before modifying partition. + // 1. replication_num or replication_allocation + // 2. storage_medium && storage_cooldown_time + // 3. in_memory + // 4. tablet type + private void checkProperties(Map properties) throws AnalysisException { + // 1. replica allocation + PropertyAnalyzer.analyzeReplicaAllocation(properties, ""); + + // 2. in memory + boolean isInMemory = PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_INMEMORY, + false); + if (isInMemory == true) { + throw new AnalysisException("Not support set 'in_memory'='true' now!"); + } + + // 3. tablet type + PropertyAnalyzer.analyzeTabletType(properties); + + // 4. mutable + PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_MUTABLE, true); + } + + @Override + public Map getProperties() { + return this.properties; + } + + public boolean isTempPartition() { + return isTempPartition; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("MODIFY PARTITION "); + if (isTempPartition) { + sb.append("TEMPORARY "); + } + sb.append("("); + if (needExpand) { + sb.append("*"); + } else { + sb.append(Joiner.on(", ").join(partitionNames)); + } + sb.append(")"); + sb.append(" SET ("); + sb.append(new PrintableMap(properties, "=", true, false)); + sb.append(")"); + + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTableCommentOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTableCommentOp.java new file mode 100644 index 000000000000000..6e20f2b32937d60 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTableCommentOp.java @@ -0,0 +1,76 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ModifyTableCommentClause; + +import com.google.common.base.Strings; +import com.google.common.collect.Maps; + +import java.util.Map; + +/** + * ModifyTableCommentOp + */ +public class ModifyTableCommentOp extends AlterTableOp { + private String comment; + + public ModifyTableCommentOp(String comment) { + super(AlterOpType.MODIFY_TABLE_COMMENT); + this.comment = Strings.nullToEmpty(comment); + } + + public String getComment() { + return comment; + } + + @Override + public Map getProperties() { + return Maps.newHashMap(); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ModifyTableCommentClause(comment); + } + + @Override + public boolean allowOpMTMV() { + return true; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("MODIFY COMMENT "); + sb.append("'").append(comment).append("'"); + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTablePropertiesOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTablePropertiesOp.java new file mode 100644 index 000000000000000..f6fa54e99b981a6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTablePropertiesOp.java @@ -0,0 +1,429 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ModifyTablePropertiesClause; +import org.apache.doris.catalog.DynamicPartitionProperty; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.ReplicaAllocation; +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableProperty; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.DynamicPartitionUtil; +import org.apache.doris.common.util.PrintableMap; +import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * ModifyTablePropertiesOp + */ +public class ModifyTablePropertiesOp extends AlterTableOp { + private Map properties; + + private String storagePolicy; + + private boolean isBeingSynced = false; + + public ModifyTablePropertiesOp(Map properties) { + super(AlterOpType.MODIFY_TABLE_PROPERTY); + this.properties = properties; + } + + public String getStoragePolicy() { + return this.storagePolicy; + } + + public void setStoragePolicy(String storagePolicy) { + this.storagePolicy = storagePolicy; + } + + public void setIsBeingSynced(boolean isBeingSynced) { + this.isBeingSynced = isBeingSynced; + } + + public boolean isBeingSynced() { + return isBeingSynced; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (properties == null || properties.isEmpty()) { + throw new AnalysisException("Properties is not set"); + } + + if (properties.size() != 1 + && !TableProperty.isSamePrefixProperties( + properties, DynamicPartitionProperty.DYNAMIC_PARTITION_PROPERTY_PREFIX) + && !TableProperty.isSamePrefixProperties(properties, PropertyAnalyzer.PROPERTIES_BINLOG_PREFIX)) { + throw new AnalysisException( + "Can only set one table property(without dynamic partition && binlog) at a time"); + } + + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_COLOCATE_WITH)) { + this.needTableStable = false; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_TYPE)) { + if (!properties.get(PropertyAnalyzer.PROPERTIES_STORAGE_TYPE).equalsIgnoreCase("column")) { + throw new AnalysisException("Can only change storage type to COLUMN"); + } + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_DISTRIBUTION_TYPE)) { + if (!properties.get(PropertyAnalyzer.PROPERTIES_DISTRIBUTION_TYPE).equalsIgnoreCase("random")) { + throw new AnalysisException("Can only change distribution type from HASH to RANDOM"); + } + this.needTableStable = false; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_SEND_CLEAR_ALTER_TASK)) { + if (!properties.get(PropertyAnalyzer.PROPERTIES_SEND_CLEAR_ALTER_TASK).equalsIgnoreCase("true")) { + throw new AnalysisException( + "Property " + PropertyAnalyzer.PROPERTIES_SEND_CLEAR_ALTER_TASK + " should be set to true"); + } + this.needTableStable = false; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BF_COLUMNS) + || properties.containsKey(PropertyAnalyzer.PROPERTIES_BF_FPP)) { + // do nothing, these 2 properties will be analyzed when creating alter job + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_FORMAT)) { + if (!properties.get(PropertyAnalyzer.PROPERTIES_STORAGE_FORMAT).equalsIgnoreCase("v2")) { + throw new AnalysisException( + "Property " + PropertyAnalyzer.PROPERTIES_STORAGE_FORMAT + " should be v2"); + } + } else if (DynamicPartitionUtil.checkDynamicPartitionPropertiesExist(properties)) { + // do nothing, dynamic properties will be analyzed in SchemaChangeHandler.process + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM) + || properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) { + ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, ""); + properties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, replicaAlloc.toCreateStmt()); + } else if (properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM) + || properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) { + ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "default"); + properties.put("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, + replicaAlloc.toCreateStmt()); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)) { + boolean isInMemory = Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY)); + if (isInMemory == true) { + throw new AnalysisException("Not support set 'in_memory'='true' now!"); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_TABLET_TYPE)) { + throw new AnalysisException("Alter tablet type not supported"); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_MIN_LOAD_REPLICA_NUM)) { + // do nothing, will be alter in Alter.processAlterOlapTable + this.needTableStable = false; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY)) { + this.needTableStable = false; + String storagePolicy = properties.getOrDefault(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY, ""); + if (!Strings.isNullOrEmpty(storagePolicy) + && properties.containsKey(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE)) { + throw new AnalysisException( + "Can not set UNIQUE KEY table that enables Merge-On-write" + + " with storage policy(" + storagePolicy + ")"); + } + setStoragePolicy(storagePolicy); + } else if (properties.containsKey(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE)) { + throw new AnalysisException("Can not change UNIQUE KEY to Merge-On-Write mode"); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_ENABLE_DUPLICATE_WITHOUT_KEYS_BY_DEFAULT)) { + throw new AnalysisException("Can not change enable_duplicate_without_keys_by_default"); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_ENABLE_LIGHT_SCHEMA_CHANGE)) { + // do nothing, will be alter in SchemaChangeHandler.updateTableProperties + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_IS_BEING_SYNCED)) { + this.needTableStable = false; + setIsBeingSynced(Boolean.parseBoolean(properties.getOrDefault( + PropertyAnalyzer.PROPERTIES_IS_BEING_SYNCED, "false"))); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_ENABLE) + || properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_TTL_SECONDS) + || properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_BYTES) + || properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_HISTORY_NUMS)) { + // do nothing, will be alter in SchemaChangeHandler.updateBinlogConfig + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_COMPACTION_POLICY)) { + String compactionPolicy = properties.getOrDefault(PropertyAnalyzer.PROPERTIES_COMPACTION_POLICY, ""); + if (compactionPolicy != null + && !compactionPolicy.equals(PropertyAnalyzer.TIME_SERIES_COMPACTION_POLICY) + && !compactionPolicy.equals(PropertyAnalyzer.SIZE_BASED_COMPACTION_POLICY)) { + throw new AnalysisException( + "Table compaction policy only support for " + PropertyAnalyzer.TIME_SERIES_COMPACTION_POLICY + + " or " + PropertyAnalyzer.SIZE_BASED_COMPACTION_POLICY); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_GOAL_SIZE_MBYTES)) { + long goalSizeMbytes; + String goalSizeMbytesStr = properties + .get(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_GOAL_SIZE_MBYTES); + try { + goalSizeMbytes = Long.parseLong(goalSizeMbytesStr); + if (goalSizeMbytes < 10) { + throw new AnalysisException("time_series_compaction_goal_size_mbytes can not be less than 10:" + + goalSizeMbytesStr); + } + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid time_series_compaction_goal_size_mbytes format: " + + goalSizeMbytesStr); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_FILE_COUNT_THRESHOLD)) { + long fileCountThreshold; + String fileCountThresholdStr = properties + .get(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_FILE_COUNT_THRESHOLD); + try { + fileCountThreshold = Long.parseLong(fileCountThresholdStr); + if (fileCountThreshold < 10) { + throw new AnalysisException("time_series_compaction_file_count_threshold can not be less than 10:" + + fileCountThresholdStr); + } + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid time_series_compaction_file_count_threshold format: " + + fileCountThresholdStr); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_TIME_THRESHOLD_SECONDS)) { + long timeThresholdSeconds; + String timeThresholdSecondsStr = properties + .get(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_TIME_THRESHOLD_SECONDS); + try { + timeThresholdSeconds = Long.parseLong(timeThresholdSecondsStr); + if (timeThresholdSeconds < 60) { + throw new AnalysisException("time_series_compaction_time_threshold_seconds can not be less than 60:" + + timeThresholdSecondsStr); + } + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid time_series_compaction_time_threshold_seconds format: " + + timeThresholdSecondsStr); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_EMPTY_ROWSETS_THRESHOLD)) { + long emptyRowsetsThreshold; + String emptyRowsetsThresholdStr = properties + .get(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_EMPTY_ROWSETS_THRESHOLD); + try { + emptyRowsetsThreshold = Long.parseLong(emptyRowsetsThresholdStr); + if (emptyRowsetsThreshold < 2) { + throw new AnalysisException("time_series_compaction_empty_rowsets_threshold can not be less than 2:" + + emptyRowsetsThresholdStr); + } + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid time_series_compaction_empty_rowsets_threshold format: " + + emptyRowsetsThresholdStr); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_LEVEL_THRESHOLD)) { + long levelThreshold; + String levelThresholdStr = properties + .get(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_LEVEL_THRESHOLD); + try { + levelThreshold = Long.parseLong(levelThresholdStr); + if (levelThreshold < 1 || levelThreshold > 2) { + throw new AnalysisException( + "time_series_compaction_level_threshold can not be less than 1 or greater than 2:" + + levelThresholdStr); + } + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid time_series_compaction_level_threshold format: " + + levelThresholdStr); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_SKIP_WRITE_INDEX_ON_LOAD)) { + if (properties.get(PropertyAnalyzer.PROPERTIES_SKIP_WRITE_INDEX_ON_LOAD).equalsIgnoreCase("true")) { + throw new AnalysisException( + "Property " + + PropertyAnalyzer.PROPERTIES_SKIP_WRITE_INDEX_ON_LOAD + " is forbidden now"); + } + if (!properties.get(PropertyAnalyzer.PROPERTIES_SKIP_WRITE_INDEX_ON_LOAD).equalsIgnoreCase("true") + && !properties.get(PropertyAnalyzer.PROPERTIES_SKIP_WRITE_INDEX_ON_LOAD) + .equalsIgnoreCase("false")) { + throw new AnalysisException( + "Property " + + PropertyAnalyzer.PROPERTIES_SKIP_WRITE_INDEX_ON_LOAD + + " should be set to true or false"); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_INVERTED_INDEX_STORAGE_FORMAT)) { + throw new AnalysisException( + "Property " + + PropertyAnalyzer.PROPERTIES_INVERTED_INDEX_STORAGE_FORMAT + " is not allowed to change"); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_ENABLE_SINGLE_REPLICA_COMPACTION)) { + if (!properties.get(PropertyAnalyzer.PROPERTIES_ENABLE_SINGLE_REPLICA_COMPACTION).equalsIgnoreCase("true") + && !properties.get(PropertyAnalyzer.PROPERTIES_ENABLE_SINGLE_REPLICA_COMPACTION) + .equalsIgnoreCase("false")) { + throw new AnalysisException( + "Property " + + PropertyAnalyzer.PROPERTIES_ENABLE_SINGLE_REPLICA_COMPACTION + + " should be set to true or false"); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_ENABLE_MOW_LIGHT_DELETE)) { + if (!properties.get(PropertyAnalyzer.PROPERTIES_ENABLE_MOW_LIGHT_DELETE) + .equalsIgnoreCase("true") + && !properties.get(PropertyAnalyzer.PROPERTIES_ENABLE_MOW_LIGHT_DELETE).equalsIgnoreCase("false")) { + throw new AnalysisException( + "Property " + + PropertyAnalyzer.PROPERTIES_ENABLE_MOW_LIGHT_DELETE + + " should be set to true or false"); + } + OlapTable table = null; + if (tableName != null) { + table = (OlapTable) (Env.getCurrentInternalCatalog().getDbOrAnalysisException(tableName.getDb()) + .getTableOrAnalysisException(tableName.getTbl())); + } + if (table == null || !table.getEnableUniqueKeyMergeOnWrite()) { + throw new AnalysisException( + "enable_mow_light_delete property is " + + "only supported for unique merge-on-write table"); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_DISABLE_AUTO_COMPACTION)) { + if (!properties.get(PropertyAnalyzer.PROPERTIES_DISABLE_AUTO_COMPACTION).equalsIgnoreCase("true") + && !properties.get(PropertyAnalyzer.PROPERTIES_DISABLE_AUTO_COMPACTION).equalsIgnoreCase("false")) { + throw new AnalysisException( + "Property " + + PropertyAnalyzer.PROPERTIES_DISABLE_AUTO_COMPACTION + + " should be set to true or false"); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_GROUP_COMMIT_INTERVAL_MS)) { + long groupCommitIntervalMs; + String groupCommitIntervalMsStr = properties.get(PropertyAnalyzer.PROPERTIES_GROUP_COMMIT_INTERVAL_MS); + try { + groupCommitIntervalMs = Long.parseLong(groupCommitIntervalMsStr); + if (groupCommitIntervalMs < 0) { + throw new AnalysisException("group_commit_interval_ms can not be less than 0:" + + groupCommitIntervalMsStr); + } + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid group_commit_interval_ms format: " + + groupCommitIntervalMsStr); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_GROUP_COMMIT_DATA_BYTES)) { + long groupCommitDataBytes; + String groupCommitDataBytesStr = properties.get(PropertyAnalyzer.PROPERTIES_GROUP_COMMIT_DATA_BYTES); + try { + groupCommitDataBytes = Long.parseLong(groupCommitDataBytesStr); + if (groupCommitDataBytes < 0) { + throw new AnalysisException("group_commit_data_bytes can not be less than 0:" + + groupCommitDataBytesStr); + } + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid group_commit_data_bytes format: " + + groupCommitDataBytesStr); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS)) { + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_VAULT_NAME)) { + throw new AnalysisException("You can not modify storage vault name"); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_VAULT_ID)) { + throw new AnalysisException("You can not modify storage vault id"); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_ESTIMATE_PARTITION_SIZE)) { + throw new AnalysisException("You can not modify estimate partition size"); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORE_ROW_COLUMN)) { + // do nothing, will be analyzed when creating alter job + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_ROW_STORE_COLUMNS)) { + // do nothing, will be analyzed when creating alter job + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY)) { + String analyzePolicy = properties.getOrDefault(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY, ""); + if (analyzePolicy != null + && !analyzePolicy.equals(PropertyAnalyzer.ENABLE_AUTO_ANALYZE_POLICY) + && !analyzePolicy.equals(PropertyAnalyzer.DISABLE_AUTO_ANALYZE_POLICY) + && !analyzePolicy.equals(PropertyAnalyzer.USE_CATALOG_AUTO_ANALYZE_POLICY)) { + throw new AnalysisException( + "Table auto analyze policy only support for " + PropertyAnalyzer.ENABLE_AUTO_ANALYZE_POLICY + + " or " + PropertyAnalyzer.DISABLE_AUTO_ANALYZE_POLICY + + " or " + PropertyAnalyzer.USE_CATALOG_AUTO_ANALYZE_POLICY); + } + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.ENABLE_UNIQUE_KEY_SKIP_BITMAP_COLUMN)) { + // do nothing, will be analyzed when creating alter job + } else { + throw new AnalysisException("Unknown table property: " + properties.keySet()); + } + analyzeForMTMV(); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ModifyTablePropertiesClause(properties, storagePolicy, isBeingSynced, needTableStable, opType); + } + + private void analyzeForMTMV() throws AnalysisException { + if (tableName != null) { + // Skip external catalog. + if (!(InternalCatalog.INTERNAL_CATALOG_NAME.equals(tableName.getCtl()))) { + return; + } + Table table = Env.getCurrentInternalCatalog().getDbOrAnalysisException(tableName.getDb()) + .getTableOrAnalysisException(tableName.getTbl()); + if (!(table instanceof MTMV)) { + return; + } + if (DynamicPartitionUtil.checkDynamicPartitionPropertiesExist(properties)) { + throw new AnalysisException("Not support dynamic partition properties on async materialized view"); + } + } + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public boolean allowOpMTMV() { + return true; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("PROPERTIES ("); + sb.append(new PrintableMap(properties, "=", true, false)); + sb.append(")"); + + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionDefinition.java index 8b012358217be75..d3decef5ae0aff8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionDefinition.java @@ -59,6 +59,7 @@ public abstract class PartitionDefinition { public PartitionDefinition(boolean ifNotExists, String partName) { this.ifNotExists = ifNotExists; this.partitionName = partName; + this.properties = Maps.newHashMap(); } public PartitionDefinition withProperties(Map properties) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionNamesInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionNamesInfo.java new file mode 100644 index 000000000000000..a31058c8271f0bd --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionNamesInfo.java @@ -0,0 +1,131 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.analysis.PartitionNames; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Joiner; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * PartitionNamesInfo + */ +public class PartitionNamesInfo { + // Default partition count to collect statistic for external table. + private static final long DEFAULT_PARTITION_COUNT = 100; + private final List partitionNames; + // true if these partitions are temp partitions + private final boolean isTemp; + private final boolean isStar; + private final long count; + + public PartitionNamesInfo(boolean isTemp, List partitionNames) { + this.partitionNames = partitionNames; + this.isTemp = isTemp; + this.isStar = false; + this.count = 0; + } + + public PartitionNamesInfo(PartitionNamesInfo other) { + this.partitionNames = Lists.newArrayList(other.partitionNames); + this.isTemp = other.isTemp; + this.isStar = other.isStar; + this.count = 0; + } + + public PartitionNamesInfo(boolean isStar) { + this.partitionNames = null; + this.isTemp = false; + this.isStar = isStar; + this.count = 0; + } + + public PartitionNamesInfo(long partitionCount) { + this.partitionNames = null; + this.isTemp = false; + this.isStar = false; + this.count = partitionCount; + } + + public List getPartitionNames() { + return partitionNames; + } + + public boolean isTemp() { + return isTemp; + } + + public boolean isStar() { + return isStar; + } + + public long getCount() { + return count; + } + + /** + * validate + */ + public void validate(ConnectContext ctx) throws UserException { + if (isStar && count > 0) { + throw new AnalysisException("All partition and partition count couldn't be set at the same time."); + } + if (isStar || count > 0) { + return; + } + if (partitionNames == null || partitionNames.isEmpty()) { + throw new AnalysisException("No partition specified in partition lists"); + } + // check if partition name is not empty string + if (partitionNames.stream().anyMatch(Strings::isNullOrEmpty)) { + throw new AnalysisException("there are empty partition name"); + } + } + + /** + * toSql + */ + public String toSql() { + if (partitionNames == null || partitionNames.isEmpty()) { + return ""; + } + StringBuilder sb = new StringBuilder(); + if (isTemp) { + sb.append("TEMPORARY "); + } + sb.append("PARTITIONS ("); + sb.append(Joiner.on(", ").join(partitionNames)); + sb.append(")"); + return sb.toString(); + } + + public PartitionNames translateToLegacyPartitionNames() { + return new PartitionNames(isTemp, partitionNames, isStar, count); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameColumnOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameColumnOp.java new file mode 100644 index 000000000000000..8bbdcfc723a1231 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameColumnOp.java @@ -0,0 +1,96 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ColumnRenameClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * RenameColumnOp + */ +public class RenameColumnOp extends AlterTableOp { + private String colName; + private String newColName; + + public RenameColumnOp(String colName, String newColName) { + super(AlterOpType.RENAME); + this.colName = colName; + this.newColName = newColName; + this.needTableStable = false; + } + + public String getColName() { + return colName; + } + + public String getNewColName() { + return newColName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(colName)) { + throw new AnalysisException("Column name is not set"); + } + + if (Strings.isNullOrEmpty(newColName)) { + throw new AnalysisException("New column name is not set"); + } + + FeNameFormat.checkColumnName(newColName); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ColumnRenameClause(colName, newColName); + } + + @Override + public Map getProperties() { + return null; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return true; + } + + @Override + public String toSql() { + return "RENAME COLUMN " + colName + " " + newColName; + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenamePartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenamePartitionOp.java new file mode 100644 index 000000000000000..692d83ccf7921a4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenamePartitionOp.java @@ -0,0 +1,96 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.PartitionRenameClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * RenamePartitionOp + */ +public class RenamePartitionOp extends AlterTableOp { + private String partitionName; + private String newPartitionName; + + public RenamePartitionOp(String partitionName, String newPartitionName) { + super(AlterOpType.RENAME); + this.partitionName = partitionName; + this.newPartitionName = newPartitionName; + this.needTableStable = false; + } + + public String getPartitionName() { + return partitionName; + } + + public String getNewPartitionName() { + return newPartitionName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(partitionName)) { + throw new AnalysisException("Partition name is not set"); + } + + if (Strings.isNullOrEmpty(newPartitionName)) { + throw new AnalysisException("New partition name is not set"); + } + + FeNameFormat.checkPartitionName(newPartitionName); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new PartitionRenameClause(partitionName, newPartitionName); + } + + @Override + public Map getProperties() { + return null; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + return "RENAME PARTITION " + partitionName + " " + newPartitionName; + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameRollupOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameRollupOp.java new file mode 100644 index 000000000000000..37605f1700af73d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameRollupOp.java @@ -0,0 +1,96 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.RollupRenameClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * RenameRollupOp + */ +public class RenameRollupOp extends AlterTableOp { + private String rollupName; + private String newRollupName; + + public RenameRollupOp(String rollupName, String newRollupName) { + super(AlterOpType.RENAME); + this.rollupName = rollupName; + this.newRollupName = newRollupName; + this.needTableStable = false; + } + + public String getRollupName() { + return rollupName; + } + + public String getNewRollupName() { + return newRollupName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(rollupName)) { + throw new AnalysisException("Rollup name is not set"); + } + + if (Strings.isNullOrEmpty(newRollupName)) { + throw new AnalysisException("New rollup name is not set"); + } + + FeNameFormat.checkTableName(newRollupName); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new RollupRenameClause(rollupName, newRollupName); + } + + @Override + public Map getProperties() { + return null; + } + + @Override + public boolean allowOpMTMV() { + return true; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + return "RENAME ROLLUP " + rollupName + " " + newRollupName; + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameTableOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameTableOp.java new file mode 100644 index 000000000000000..7cf4feae6412a98 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameTableOp.java @@ -0,0 +1,85 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +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.TableRenameClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * TableRenameOp + */ +public class RenameTableOp extends AlterTableOp { + private String newTableName; + + public RenameTableOp(String newTableName) { + super(AlterOpType.RENAME); + this.newTableName = newTableName; + } + + public String getNewTableName() { + return newTableName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(newTableName)) { + throw new AnalysisException("New Table name is not set"); + } + + FeNameFormat.checkTableName(newTableName); + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new TableRenameClause(newTableName); + } + + @Override + public Map getProperties() { + return null; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return true; + } + + @Override + public String toSql() { + return "RENAME " + newTableName; + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReorderColumnsOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReorderColumnsOp.java new file mode 100644 index 000000000000000..02c2d942a0a34f8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReorderColumnsOp.java @@ -0,0 +1,112 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ReorderColumnsClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.List; +import java.util.Map; + +/** + * ReorderColumnsOp + */ +public class ReorderColumnsOp extends AlterTableOp { + private List columnsByPos; + private String rollupName; + private Map properties; + + public ReorderColumnsOp(List cols, String rollup, Map properties) { + super(AlterOpType.SCHEMA_CHANGE); + this.columnsByPos = cols; + this.rollupName = rollup; + this.properties = properties; + } + + public List getColumnsByPos() { + return columnsByPos; + } + + public String getRollupName() { + return rollupName; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (columnsByPos == null || columnsByPos.isEmpty()) { + throw new AnalysisException("No column in reorder columns clause."); + } + for (String col : columnsByPos) { + if (Strings.isNullOrEmpty(col)) { + throw new AnalysisException("Empty column in reorder columns."); + } + } + if (Strings.isNullOrEmpty(rollupName)) { + rollupName = null; + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ReorderColumnsClause(columnsByPos, rollupName, properties); + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("ORDER BY "); + int idx = 0; + for (String col : columnsByPos) { + if (idx != 0) { + sb.append(", "); + } + sb.append("`").append(col).append('`'); + idx++; + } + if (rollupName != null) { + sb.append(" IN `").append(rollupName).append("`"); + } + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplacePartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplacePartitionOp.java new file mode 100644 index 000000000000000..9ac4eec1ac2d433 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplacePartitionOp.java @@ -0,0 +1,175 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ReplacePartitionClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Joiner; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +/** + * ReplacePartitionOp + */ +public class ReplacePartitionOp extends AlterTableOp { + private PartitionNamesInfo partitionNames; + private PartitionNamesInfo tempPartitionNames; + private Map properties = Maps.newHashMap(); + + // "isStrictMode" is got from property "strict_range", and default is true. + // If true, when replacing partition, the range of partitions must same as the range of temp partitions. + private boolean isStrictRange; + + // "useTempPartitionName" is got from property "use_temp_partition_name", and default is false. + // If false, after replacing, the replaced partition's name will remain unchanged. + // Otherwise, the replaced partition's name will be the temp partitions name. + // This parameter is valid only when the number of partitions is the same as the number of temp partitions. + // For example: + // 1. REPLACE PARTITION (p1, p2, p3) WITH TEMPORARY PARTITION(tp1, tp2) + // PROPERTIES("use_temp_partition_name" = "false"); + // "use_temp_partition_name" will take no effect after replacing, + // and the partition names will be "tp1" and "tp2". + // + // 2. REPLACE PARTITION (p1, p2) WITH TEMPORARY PARTITION(tp1, tp2) PROPERTIES("use_temp_partition_name" = "false"); + // alter replacing, the partition names will be "p1" and "p2". + // but if "use_temp_partition_name" is true, the partition names will be "tp1" and "tp2". + private boolean useTempPartitionName; + + // The replaced partitions will be moved to recycle bin when "forceDropNormalPartition" is false, + // and instead, these partitions will be deleted directly. + private boolean forceDropOldPartition; + + /** + * ReplacePartitionOp + */ + public ReplacePartitionOp(PartitionNamesInfo partitionNames, PartitionNamesInfo tempPartitionNames, + boolean isForce, Map properties) { + super(AlterOpType.REPLACE_PARTITION); + this.partitionNames = partitionNames; + this.tempPartitionNames = tempPartitionNames; + this.needTableStable = false; + this.forceDropOldPartition = isForce; + this.properties = properties; + + // ATTN: During ReplacePartitionClause.analyze(), the default value of isStrictRange is true. + // However, ReplacePartitionClause instances constructed by internal code do not call analyze(), + // so their isStrictRange value is incorrect (e.g., INSERT INTO ... OVERWRITE). + // + // Considering this, we should handle the relevant properties when constructing. + this.isStrictRange = getBoolProperty(properties, PropertyAnalyzer.PROPERTIES_STRICT_RANGE, true); + this.useTempPartitionName = getBoolProperty( + properties, PropertyAnalyzer.PROPERTIES_USE_TEMP_PARTITION_NAME, false); + } + + public List getPartitionNames() { + return partitionNames.getPartitionNames(); + } + + public List getTempPartitionNames() { + return tempPartitionNames.getPartitionNames(); + } + + public boolean isStrictRange() { + return isStrictRange; + } + + public boolean useTempPartitionName() { + return useTempPartitionName; + } + + public boolean isForceDropOldPartition() { + return forceDropOldPartition; + } + + @SuppressWarnings("checkstyle:LineLength") + @Override + public void validate(ConnectContext ctx) throws UserException { + if (partitionNames == null || tempPartitionNames == null) { + throw new AnalysisException("No partition specified"); + } + + partitionNames.validate(ctx); + tempPartitionNames.validate(ctx); + + if (partitionNames.isTemp() || !tempPartitionNames.isTemp()) { + throw new AnalysisException("Only support replace partitions with temp partitions"); + } + + this.isStrictRange = PropertyAnalyzer.analyzeBooleanProp( + properties, PropertyAnalyzer.PROPERTIES_STRICT_RANGE, true); + this.useTempPartitionName = PropertyAnalyzer.analyzeBooleanProp(properties, + PropertyAnalyzer.PROPERTIES_USE_TEMP_PARTITION_NAME, false); + + if (properties != null && !properties.isEmpty()) { + throw new AnalysisException("Unknown properties: " + properties.keySet()); + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ReplacePartitionClause(partitionNames.translateToLegacyPartitionNames(), + tempPartitionNames.translateToLegacyPartitionNames(), + forceDropOldPartition, properties, isStrictRange, useTempPartitionName); + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return false; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("REPLACE PARTITION("); + sb.append(Joiner.on(", ").join(partitionNames.getPartitionNames())).append(")"); + sb.append(" WITH TEMPORARY PARTITION("); + sb.append(Joiner.on(", ").join(tempPartitionNames.getPartitionNames())).append(")"); + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } + + public static boolean getBoolProperty(Map properties, String propKey, boolean defaultVal) { + if (properties != null && properties.containsKey(propKey)) { + String val = properties.get(propKey); + return Boolean.parseBoolean(val); + } + return defaultVal; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplaceTableOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplaceTableOp.java new file mode 100644 index 000000000000000..59f4b7be517ea87 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplaceTableOp.java @@ -0,0 +1,106 @@ +// 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.nereids.trees.plans.commands.info; + +import org.apache.doris.alter.AlterOpType; +import org.apache.doris.analysis.AlterTableClause; +import org.apache.doris.analysis.ReplaceTableClause; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * ReplaceTableOp + */ +public class ReplaceTableOp extends AlterTableOp { + private final String tblName; + private Map properties; + + private final boolean isForce; + + // parsed from properties. + // if false, after replace, there will be only one table exist with. + // if true, the new table and the old table will be exchanged. + // default is true. + private boolean swapTable; + + public ReplaceTableOp(String tblName, Map properties, boolean isForce) { + super(AlterOpType.REPLACE_TABLE); + this.tblName = tblName; + this.properties = properties; + this.isForce = isForce; + } + + public String getTblName() { + return tblName; + } + + public boolean isSwapTable() { + return swapTable; + } + + @Override + public void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(tblName)) { + throw new AnalysisException("No table specified"); + } + + this.swapTable = PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_SWAP_TABLE, true); + + if (properties != null && !properties.isEmpty()) { + throw new AnalysisException("Unknown properties: " + properties.keySet()); + } + } + + @Override + public AlterTableClause translateToLegacyAlterClause() { + return new ReplaceTableClause(tblName, properties, isForce, swapTable); + } + + @Override + public Map getProperties() { + return this.properties; + } + + @Override + public boolean allowOpMTMV() { + return false; + } + + @Override + public boolean needChangeMTMVState() { + return true; + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("REPLACE WITH TABLE ").append(tblName); + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index b38684582a1a748..6511c0cca1a3392 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -20,6 +20,7 @@ import org.apache.doris.nereids.trees.plans.commands.AddConstraintCommand; import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterTableCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; @@ -251,6 +252,10 @@ default R visitSetDefaultStorageVault(SetDefaultStorageVaultCommand setDefaultSt default R visitShowLastInsertCommand(ShowLastInsertCommand showLastInsertCommand, C context) { return visitCommand(showLastInsertCommand, context); } + + default R visitAlterTableCommand(AlterTableCommand alterTableCommand, C context) { + return visitCommand(alterTableCommand, context); + } default R visitShowVariablesCommand(ShowVariablesCommand showVariablesCommand, C context) { return visitCommand(showVariablesCommand, context); diff --git a/regression-test/suites/rollup_p0/test_create_mv_and_mtmv.groovy b/regression-test/suites/rollup_p0/test_create_mv_and_mtmv.groovy index a5446966bc4ef3d..0b1922620566fa4 100644 --- a/regression-test/suites/rollup_p0/test_create_mv_and_mtmv.groovy +++ b/regression-test/suites/rollup_p0/test_create_mv_and_mtmv.groovy @@ -21,6 +21,8 @@ suite("test_create_mv_and_mtmt") { def tableName = "test_create_mv_and_mtmt_advertiser_view_record" def mvName = "test_create_mv_and_mtmt_advertiser_uv" def mtmvName = "test_create_mv_and_mtmt_advertiser_uv_mtmv" + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ DROP MATERIALIZED VIEW ${mtmvName} """ sql """ CREATE TABLE ${tableName}( time date not null, diff --git a/regression-test/suites/schema_change_p0/test_agg_schema_key_add.groovy b/regression-test/suites/schema_change_p0/test_agg_schema_key_add.groovy index 715bbd2a296f882..b54c0711ba98ef2 100644 --- a/regression-test/suites/schema_change_p0/test_agg_schema_key_add.groovy +++ b/regression-test/suites/schema_change_p0/test_agg_schema_key_add.groovy @@ -52,7 +52,7 @@ suite("test_agg_schema_key_add", "p0") { def getTableStatusSql = " SHOW ALTER TABLE COLUMN WHERE IndexName='${tbName1}' ORDER BY createtime DESC LIMIT 1 " def errorMessage //Test the AGGREGATE model by adding a key column with VARCHAR - errorMessage = "errCode = 2, detailMessage = Key column can not set aggregation type: province" + errorMessage = "errCode = 2, detailMessage = Key column province can not set aggregation type" expectException({ sql initTable sql initTableData @@ -66,7 +66,7 @@ suite("test_agg_schema_key_add", "p0") { //Test the AGGREGATE model by adding a key column with BOOLEAN - errorMessage = "errCode = 2, detailMessage = Key column can not set aggregation type: special_area" + errorMessage = "errCode = 2, detailMessage = Key column special_area can not set aggregation type" expectException({ sql initTable sql initTableData @@ -80,7 +80,7 @@ suite("test_agg_schema_key_add", "p0") { //Test the AGGREGATE model by adding a key column with TINYINT - errorMessage = "errCode = 2, detailMessage = Key column can not set aggregation type: special_area" + errorMessage = "errCode = 2, detailMessage = Key column special_area can not set aggregation type" expectException({ sql initTable sql initTableData @@ -94,7 +94,7 @@ suite("test_agg_schema_key_add", "p0") { //Test the AGGREGATE model by adding a key column with SMALLINT - errorMessage = "errCode = 2, detailMessage = Key column can not set aggregation type: area_num" + errorMessage = "errCode = 2, detailMessage = Key column area_num can not set aggregation type" expectException({ sql initTable sql initTableData @@ -446,7 +446,7 @@ suite("test_agg_schema_key_add", "p0") { //TODO Test the AGGREGATE model by adding a key column with STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[comment]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[comment]" expectException({ sql initTable sql initTableData @@ -489,7 +489,7 @@ suite("test_agg_schema_key_add", "p0") { //TODO Test the AGGREGATE model by adding a column with JSON type none default value - errorMessage = "errCode = 2, detailMessage = JSONB or VARIANT type should not be used in key column[j]." + errorMessage = "errCode = 2, detailMessage = JsonType type should not be used in key column[j]." expectException({ sql initTable sql initTableData @@ -503,7 +503,7 @@ suite("test_agg_schema_key_add", "p0") { //TODO Test the AGGREGATE model by adding a key column with JSON - errorMessage = "errCode = 2, detailMessage = JSONB or VARIANT type should not be used in key column[j]." + errorMessage = "errCode = 2, detailMessage = JsonType type should not be used in key column[j]." expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_agg_schema_key_change_modify.groovy b/regression-test/suites/schema_change_p0/test_agg_schema_key_change_modify.groovy index 70e2c2c242c0cf7..73052855dc08c62 100644 --- a/regression-test/suites/schema_change_p0/test_agg_schema_key_change_modify.groovy +++ b/regression-test/suites/schema_change_p0/test_agg_schema_key_change_modify.groovy @@ -128,7 +128,7 @@ suite("test_agg_schema_key_change_modify","p0") { //TODO Test the agg model by modify a key type from BOOLEAN to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL128" expectException({ sql initTable sql initTableData @@ -156,7 +156,7 @@ suite("test_agg_schema_key_change_modify","p0") { },errorMessage) //TODO Test the agg model by modify a key type from BOOLEAN to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_teacher]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_teacher]" expectException({ sql initTable sql initTableData @@ -403,7 +403,7 @@ suite("test_agg_schema_key_change_modify","p0") { //TODO Test the agg model by modify a key type from TINYINT to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL128" expectException({ sql initTable sql initTableData @@ -442,7 +442,7 @@ suite("test_agg_schema_key_change_modify","p0") { sql """ DROP TABLE IF EXISTS ${tbName1} """ //Test the agg model by modify a key type from TINYINT to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_student]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_student]" expectException({ sql initTable sql initTableData @@ -645,7 +645,7 @@ suite("test_agg_schema_key_change_modify","p0") { //TODO Test the agg model by modify a key type from SMALLINT to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL128" expectException({ sql initTable sql initTableData @@ -684,7 +684,7 @@ suite("test_agg_schema_key_change_modify","p0") { sql """ DROP TABLE IF EXISTS ${tbName1} """ //Test the agg model by modify a key type from SMALLINT to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[car_number]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[car_number]" expectException({ sql initTable sql initTableData @@ -903,7 +903,7 @@ suite("test_agg_schema_key_change_modify","p0") { }, errorMessage) //Test the agg model by modify a key type from INT to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]" expectException({ sql initTable sql initTableData @@ -1095,7 +1095,7 @@ suite("test_agg_schema_key_change_modify","p0") { }, errorMessage) //Test the agg model by modify a key type from BIGINT to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]" expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_agg_schema_key_change_modify1.groovy b/regression-test/suites/schema_change_p0/test_agg_schema_key_change_modify1.groovy index 78d203c4f738d6c..a7ec40a31f9c662 100644 --- a/regression-test/suites/schema_change_p0/test_agg_schema_key_change_modify1.groovy +++ b/regression-test/suites/schema_change_p0/test_agg_schema_key_change_modify1.groovy @@ -268,7 +268,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from LARGEINT to STRING //Test the agg model by modify a key type from LARGEINT to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]" expectException({ sql initTable sql initTableData @@ -842,7 +842,7 @@ suite("test_agg_schema_key_change_modify1","p0") { sql """ DROP TABLE IF EXISTS ${tbName1} """ //Test the agg model by modify a key type from DECIMAL to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[rice]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[rice]" expectException({ sql initTable sql initTableData @@ -974,7 +974,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from DATE to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1002,7 +1002,7 @@ suite("test_agg_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the agg model by modify a key type from DATE to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -1148,7 +1148,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from DATETIME to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1176,7 +1176,7 @@ suite("test_agg_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the agg model by modify a key type from DATETIME to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -1322,7 +1322,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from CHAR to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1350,7 +1350,7 @@ suite("test_agg_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the agg model by modify a key type from CHAR to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]" expectException({ sql initTable sql initTableData @@ -1495,7 +1495,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from VARCHAR to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1523,7 +1523,7 @@ suite("test_agg_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the agg model by modify a key type from VARCHAR to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]" expectException({ sql initTable sql initTableData @@ -1672,7 +1672,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from DATE to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1700,7 +1700,7 @@ suite("test_agg_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the agg model by modify a key type from DATE to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -1758,7 +1758,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from DATE to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1786,7 +1786,7 @@ suite("test_agg_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the agg model by modify a key type from DATE to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -1934,7 +1934,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from DATEV2 to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1962,7 +1962,7 @@ suite("test_agg_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the agg model by modify a key type from DATEV2 to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -2020,7 +2020,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from DATEV2 to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2048,7 +2048,7 @@ suite("test_agg_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the agg model by modify a key type from DATEV2 to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -2194,7 +2194,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from DATETIMEV2 to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2222,7 +2222,7 @@ suite("test_agg_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the agg model by modify a key type from DATETIMEV2 to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -2366,7 +2366,7 @@ suite("test_agg_schema_key_change_modify1","p0") { //TODO Test the agg model by modify a key type from DATETIME to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2394,7 +2394,7 @@ suite("test_agg_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the agg model by modify a key type from DATETIME to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_agg_schema_value_add.groovy b/regression-test/suites/schema_change_p0/test_agg_schema_value_add.groovy index fc2864114ed4739..6068f906e5cfb1b 100644 --- a/regression-test/suites/schema_change_p0/test_agg_schema_value_add.groovy +++ b/regression-test/suites/schema_change_p0/test_agg_schema_value_add.groovy @@ -553,7 +553,7 @@ suite("test_agg_schema_value_add", "p0") { //Test the AGGREGATE model by adding a value column with STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[comment]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[comment]" expectException({ sql initTable sql initTableData @@ -611,7 +611,7 @@ suite("test_agg_schema_value_add", "p0") { //Test the AGGREGATE model by adding a value column with JSON - errorMessage = "errCode = 2, detailMessage = JSONB or VARIANT type should not be used in key column[j]." + errorMessage = "errCode = 2, detailMessage = JsonType type should not be used in key column[j]." expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_agg_schema_value_modify.groovy b/regression-test/suites/schema_change_p0/test_agg_schema_value_modify.groovy index 4d10211a523c4b8..31fead3ee35ad0c 100644 --- a/regression-test/suites/schema_change_p0/test_agg_schema_value_modify.groovy +++ b/regression-test/suites/schema_change_p0/test_agg_schema_value_modify.groovy @@ -142,7 +142,7 @@ suite("test_agg_schema_value_modify","p0") { //TODO Test the agg model by modify a value type from BOOLEAN to DECIMAL - errorMessagge="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL32" + errorMessagge="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL128" expectException({ sql initTable sql initTableData @@ -505,8 +505,8 @@ suite("test_agg_schema_value_modify","p0") { checkTableData("${tbName1}","${tbName2}","is_student") sql """ DROP TABLE IF EXISTS ${tbName1} """ - //TODO Test the agg model by modify a value type from TINYINT to DECIMAL32 - errorMessagge="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL32" + //TODO Test the agg model by modify a value type from TINYINT to DECIMAL128 + errorMessagge="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL128" expectException({ sql initTable sql initTableData @@ -898,7 +898,7 @@ suite("test_agg_schema_value_modify","p0") { //TODO Test the AGGREGATE model by modify a value type from SMALLINT to DECIMAL - errorMessagge = "errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL32" + errorMessagge = "errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL128" expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_alter_table_modify_column.groovy b/regression-test/suites/schema_change_p0/test_alter_table_modify_column.groovy index 3920086bf6c041a..e6c189b0baa4215 100644 --- a/regression-test/suites/schema_change_p0/test_alter_table_modify_column.groovy +++ b/regression-test/suites/schema_change_p0/test_alter_table_modify_column.groovy @@ -99,7 +99,7 @@ suite("test_alter_table_modify_column") { test { sql """alter table ${aggTableName} modify COLUMN siteid INT key SUM DEFAULT '0';""" // check exception message contains - exception "Key column can not set aggregation type" + exception "Key column siteid can not set aggregation type" } test { diff --git a/regression-test/suites/schema_change_p0/test_dup_schema_key_add.groovy b/regression-test/suites/schema_change_p0/test_dup_schema_key_add.groovy index 8641aaea3eadafd..bcc5583a6fc56b5 100644 --- a/regression-test/suites/schema_change_p0/test_dup_schema_key_add.groovy +++ b/regression-test/suites/schema_change_p0/test_dup_schema_key_add.groovy @@ -543,7 +543,7 @@ suite("test_dup_schema_key_add","p0") { //TODO Test the duplicate model by adding a key column with STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[comment]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[comment]" expectException({ sql initTable sql initTableData @@ -587,7 +587,7 @@ suite("test_dup_schema_key_add","p0") { //TODO Test the duplicate model by adding a column with JSON type none default value - errorMessage="errCode = 2, detailMessage = JSONB or VARIANT type column default value just support null" + errorMessage="errCode = 2, detailMessage = Json or Variant type column default value just support null" expectException({ sql initTable sql initTableData @@ -601,7 +601,7 @@ suite("test_dup_schema_key_add","p0") { //TODO Test the duplicate model by adding a key column with JSON - errorMessage="errCode = 2, detailMessage = JSONB or VARIANT type should not be used in key column[j]." + errorMessage="errCode = 2, detailMessage = JsonType type should not be used in key column[j]." expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_dup_schema_key_change_modify.groovy b/regression-test/suites/schema_change_p0/test_dup_schema_key_change_modify.groovy index ab2e5b1671ddc3a..b90af52ee91a0ff 100644 --- a/regression-test/suites/schema_change_p0/test_dup_schema_key_change_modify.groovy +++ b/regression-test/suites/schema_change_p0/test_dup_schema_key_change_modify.groovy @@ -128,7 +128,7 @@ suite("test_dup_schema_key_change_modify","p0") { //TODO Test the dup model by modify a key type from BOOLEAN to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL128" expectException({ sql initTable sql initTableData @@ -156,7 +156,7 @@ suite("test_dup_schema_key_change_modify","p0") { },errorMessage) //TODO Test the dup model by modify a key type from BOOLEAN to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_teacher]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_teacher]" expectException({ sql initTable sql initTableData @@ -403,7 +403,7 @@ suite("test_dup_schema_key_change_modify","p0") { //TODO Test the dup model by modify a key type from TINYINT to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL128" expectException({ sql initTable sql initTableData @@ -442,7 +442,7 @@ suite("test_dup_schema_key_change_modify","p0") { sql """ DROP TABLE IF EXISTS ${tbName1} """ //Test the dup model by modify a key type from TINYINT to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_student]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_student]" expectException({ sql initTable sql initTableData @@ -645,7 +645,7 @@ suite("test_dup_schema_key_change_modify","p0") { //TODO Test the dup model by modify a key type from SMALLINT to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL128" expectException({ sql initTable sql initTableData @@ -684,7 +684,7 @@ suite("test_dup_schema_key_change_modify","p0") { sql """ DROP TABLE IF EXISTS ${tbName1} """ //Test the dup model by modify a key type from SMALLINT to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[car_number]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[car_number]" expectException({ sql initTable sql initTableData @@ -903,7 +903,7 @@ suite("test_dup_schema_key_change_modify","p0") { }, errorMessage) //Test the dup model by modify a key type from INT to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]" expectException({ sql initTable sql initTableData @@ -1095,7 +1095,7 @@ suite("test_dup_schema_key_change_modify","p0") { }, errorMessage) //Test the dup model by modify a key type from BIGINT to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]" expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_dup_schema_key_change_modify1.groovy b/regression-test/suites/schema_change_p0/test_dup_schema_key_change_modify1.groovy index 74a5c1f86d8bbf8..7361bdd7b6b6cb5 100644 --- a/regression-test/suites/schema_change_p0/test_dup_schema_key_change_modify1.groovy +++ b/regression-test/suites/schema_change_p0/test_dup_schema_key_change_modify1.groovy @@ -267,7 +267,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from LARGEINT to STRING //Test the dup model by modify a key type from LARGEINT to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]" expectException({ sql initTable sql initTableData @@ -841,7 +841,7 @@ suite("test_dup_schema_key_change_modify1","p0") { sql """ DROP TABLE IF EXISTS ${tbName1} """ //Test the dup model by modify a key type from DECIMAL to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[rice]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[rice]" expectException({ sql initTable sql initTableData @@ -973,7 +973,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from DATE to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1001,7 +1001,7 @@ suite("test_dup_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the dup model by modify a key type from DATE to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -1147,7 +1147,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from DATETIME to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1175,7 +1175,7 @@ suite("test_dup_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the dup model by modify a key type from DATETIME to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -1321,7 +1321,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from CHAR to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1349,7 +1349,7 @@ suite("test_dup_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the dup model by modify a key type from CHAR to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]" expectException({ sql initTable sql initTableData @@ -1494,7 +1494,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from VARCHAR to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1522,7 +1522,7 @@ suite("test_dup_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the dup model by modify a key type from VARCHAR to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]" expectException({ sql initTable sql initTableData @@ -1671,7 +1671,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from DATE to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1699,7 +1699,7 @@ suite("test_dup_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the dup model by modify a key type from DATE to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -1757,7 +1757,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from DATE to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1785,7 +1785,7 @@ suite("test_dup_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the dup model by modify a key type from DATE to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -1933,7 +1933,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from DATEV2 to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1961,7 +1961,7 @@ suite("test_dup_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the dup model by modify a key type from DATEV2 to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -2019,7 +2019,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from DATEV2 to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2047,7 +2047,7 @@ suite("test_dup_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the dup model by modify a key type from DATEV2 to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -2193,7 +2193,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from DATETIMEV2 to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2221,7 +2221,7 @@ suite("test_dup_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the dup model by modify a key type from DATETIMEV2 to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -2365,7 +2365,7 @@ suite("test_dup_schema_key_change_modify1","p0") { //TODO Test the dup model by modify a key type from DATETIME to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2393,7 +2393,7 @@ suite("test_dup_schema_key_change_modify1","p0") { },errorMessage) //TODO Test the dup model by modify a key type from DATETIME to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_dup_schema_value_modify.groovy b/regression-test/suites/schema_change_p0/test_dup_schema_value_modify.groovy index e63bb55dfc187be..8a0d2fdfd3d158d 100644 --- a/regression-test/suites/schema_change_p0/test_dup_schema_value_modify.groovy +++ b/regression-test/suites/schema_change_p0/test_dup_schema_value_modify.groovy @@ -141,7 +141,7 @@ suite("test_dup_schema_value_modify","p0") { //TODO Test the dup model by modify a value type from BOOLEAN to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL128" expectException({ sql initTable sql initTableData @@ -473,8 +473,8 @@ suite("test_dup_schema_value_modify","p0") { checkTableData("${tbName1}","${tbName2}","is_student") sql """ DROP TABLE IF EXISTS ${tbName1} """ - //TODO Test the dup model by modify a value type from TINYINT to DECIMAL32 - errorMessage="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL32" + //TODO Test the dup model by modify a value type from TINYINT to DECIMAL128 + errorMessage="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL128" expectException({ sql initTable sql initTableData @@ -836,7 +836,7 @@ suite("test_dup_schema_value_modify","p0") { //TODO Test the unique model by modify a value type from SMALLINT to DECIMAL - errorMessage = "errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL32" + errorMessage = "errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL128" expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy b/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy index 6c6660d10551c4e..1f30e3d79fc310f 100644 --- a/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy +++ b/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy @@ -147,7 +147,7 @@ suite("test_schema_change_duplicate", "p0") { sql """ SHOW ALTER TABLE COLUMN WHERE TableName='${tableName3}' ORDER BY createtime DESC LIMIT 1 """ time 600 } - + def max_try_num = 60 int val = 100000 + max_try_num sql """ insert into ${tableName3} values (${val}, 2, 3, 4, 5, 6.6, 1.7, 8.8, 'a', 'b', 'c', '2021-10-30', '2021-10-30 00:00:00', 9527) """ diff --git a/regression-test/suites/schema_change_p0/test_unique_schema_key_change_add.groovy b/regression-test/suites/schema_change_p0/test_unique_schema_key_change_add.groovy index eaf2834019fda58..7a39ab2b11dd2b5 100644 --- a/regression-test/suites/schema_change_p0/test_unique_schema_key_change_add.groovy +++ b/regression-test/suites/schema_change_p0/test_unique_schema_key_change_add.groovy @@ -212,7 +212,7 @@ suite("test_unique_schema_key_change_add","p0") { //TODO Test the unique model by adding a key column with STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[comment]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[comment]" expectException({ sql initTable sql initTableData @@ -256,7 +256,7 @@ suite("test_unique_schema_key_change_add","p0") { //TODO Test the unique model by adding a column with JSON type none default value - errorMessage="errCode = 2, detailMessage = JSONB or VARIANT type column default value just support null" + errorMessage="errCode = 2, detailMessage = Json or Variant type column default value just support null" expectException({ sql initTable sql initTableData @@ -270,7 +270,7 @@ suite("test_unique_schema_key_change_add","p0") { //TODO Test the unique model by adding a key column with JSON - errorMessage="errCode = 2, detailMessage = JSONB or VARIANT type should not be used in key column[j]." + errorMessage="errCode = 2, detailMessage = JsonType type should not be used in key column[j]." expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_unique_schema_key_change_modify.groovy b/regression-test/suites/schema_change_p0/test_unique_schema_key_change_modify.groovy index cf43e8387733fd9..d8410cb57c30fe3 100644 --- a/regression-test/suites/schema_change_p0/test_unique_schema_key_change_modify.groovy +++ b/regression-test/suites/schema_change_p0/test_unique_schema_key_change_modify.groovy @@ -129,7 +129,7 @@ suite("test_unique_schema_key_change_modify","p0") { //TODO Test the unique model by modify a key type from BOOLEAN to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL128" expectException({ sql initTable sql initTableData @@ -157,7 +157,7 @@ suite("test_unique_schema_key_change_modify","p0") { },errorMessage) //TODO Test the unique model by modify a key type from BOOLEAN to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_teacher]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_teacher]" expectException({ sql initTable sql initTableData @@ -303,7 +303,7 @@ suite("test_unique_schema_key_change_modify","p0") { //TODO Test the unique model by modify a key type from TINYINT to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL128" expectException({ sql initTable sql initTableData @@ -344,7 +344,7 @@ suite("test_unique_schema_key_change_modify","p0") { //Test the unique model by modify a key type from TINYINT to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_student]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[is_student]" expectException({ sql initTable sql initTableData @@ -481,7 +481,7 @@ suite("test_unique_schema_key_change_modify","p0") { //TODO Test the unique model by modify a key type from SMALLINT to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL128" expectException({ sql initTable sql initTableData @@ -522,7 +522,7 @@ suite("test_unique_schema_key_change_modify","p0") { //Test the unique model by modify a key type from SMALLINT to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[car_number]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[car_number]" expectException({ sql initTable sql initTableData @@ -706,7 +706,7 @@ suite("test_unique_schema_key_change_modify","p0") { }, errorMessage) //Test the unique model by modify a key type from INT to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]" expectException({ sql initTable sql initTableData @@ -897,7 +897,7 @@ suite("test_unique_schema_key_change_modify","p0") { }, errorMessage) //Test the unique model by modify a key type from BIGINT to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]" expectException({ sql initTable sql initTableData @@ -1127,7 +1127,7 @@ suite("test_unique_schema_key_change_modify","p0") { //TODO Test the unique model by modify a key type from LARGEINT to STRING //Test the unique model by modify a key type from LARGEINT to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[sn_number]" expectException({ sql initTable sql initTableData @@ -1705,7 +1705,7 @@ suite("test_unique_schema_key_change_modify","p0") { sql """ DROP TABLE IF EXISTS ${tbName} """ //Test the unique model by modify a key type from DECIMAL to STRING - errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[rice]." + errorMessage = "errCode = 2, detailMessage = String Type should not be used in key column[rice]" expectException({ sql initTable sql initTableData @@ -1838,7 +1838,7 @@ suite("test_unique_schema_key_change_modify","p0") { //TODO Test the unique model by modify a key type from DATE to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -1866,7 +1866,7 @@ suite("test_unique_schema_key_change_modify","p0") { },errorMessage) //TODO Test the unique model by modify a key type from DATE to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -2013,7 +2013,7 @@ suite("test_unique_schema_key_change_modify","p0") { //TODO Test the unique model by modify a key type from DATETIME to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2041,7 +2041,7 @@ suite("test_unique_schema_key_change_modify","p0") { },errorMessage) //TODO Test the unique model by modify a key type from DATETIME to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -2186,7 +2186,7 @@ suite("test_unique_schema_key_change_modify","p0") { //TODO Test the unique model by modify a key type from CHAR to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2214,7 +2214,7 @@ suite("test_unique_schema_key_change_modify","p0") { },errorMessage) //TODO Test the unique model by modify a key type from CHAR to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]" expectException({ sql initTable sql initTableData @@ -2360,7 +2360,7 @@ suite("test_unique_schema_key_change_modify","p0") { //TODO Test the unique model by modify a key type from VARCHAR to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change VARCHAR to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2388,7 +2388,7 @@ suite("test_unique_schema_key_change_modify","p0") { },errorMessage) //TODO Test the unique model by modify a key type from VARCHAR to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[username]" expectException({ sql initTable sql initTableData @@ -2534,7 +2534,7 @@ suite("test_unique_schema_key_change_modify","p0") { //TODO Test the unique model by modify a key type from DATEV2 to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2562,7 +2562,7 @@ suite("test_unique_schema_key_change_modify","p0") { },errorMessage) //TODO Test the unique model by modify a key type from DATEV2 to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData @@ -2707,7 +2707,7 @@ suite("test_unique_schema_key_change_modify","p0") { //TODO Test the unique model by modify a key type from DATETIMEV2 to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change DATEV2 to DECIMAL128" expectException({ sql initTable sql initTableData @@ -2735,7 +2735,7 @@ suite("test_unique_schema_key_change_modify","p0") { },errorMessage) //TODO Test the unique model by modify a key type from DATETIMEV2 to STRING - errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]." + errorMessage="errCode = 2, detailMessage = String Type should not be used in key column[login_time]" expectException({ sql initTable sql initTableData diff --git a/regression-test/suites/schema_change_p0/test_unique_schema_value_modify.groovy b/regression-test/suites/schema_change_p0/test_unique_schema_value_modify.groovy index c8574bd02902644..30f11b17e916ec2 100644 --- a/regression-test/suites/schema_change_p0/test_unique_schema_value_modify.groovy +++ b/regression-test/suites/schema_change_p0/test_unique_schema_value_modify.groovy @@ -144,7 +144,7 @@ suite("test_unique_schema_value_modify","p0") { //TODO Test the unique model by modify a value type from BOOLEAN to DECIMAL - errorMessage="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL32" + errorMessage="errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL128" expectException({ sql initTable sql initTableData @@ -344,8 +344,8 @@ suite("test_unique_schema_value_modify","p0") { }, insertSql, true,"${tbName}") - //TODO Test the unique model by modify a value type from TINYINT to DECIMAL32 - errorMessage="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL32" + //TODO Test the unique model by modify a value type from TINYINT to DECIMAL128 + errorMessage="errCode = 2, detailMessage = Can not change TINYINT to DECIMAL128" expectException({ sql initTable sql initTableData @@ -508,7 +508,7 @@ suite("test_unique_schema_value_modify","p0") { //TODO Test the unique model by modify a value type from SMALLINT to DECIMAL - errorMessage = "errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL32" + errorMessage = "errCode = 2, detailMessage = Can not change SMALLINT to DECIMAL128" expectException({ sql initTable sql initTableData