Skip to content

Commit

Permalink
[feature](nereids)support alter table command
Browse files Browse the repository at this point in the history
  • Loading branch information
starocean999 committed Nov 15, 2024
1 parent 84aa726 commit f0fb46d
Show file tree
Hide file tree
Showing 66 changed files with 4,742 additions and 188 deletions.
34 changes: 17 additions & 17 deletions fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
130 changes: 126 additions & 4 deletions fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -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<AlterClause> 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<AlterClause> 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);

Expand Down Expand Up @@ -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()) {
Expand Down Expand Up @@ -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<AlterClause> alterClauses = new ArrayList<>();
alterClauses.addAll(command.getOps());
processAlterExternalTableInternal(alterClauses, externalTable, db);
}

private void processAlterExternalTableInternal(List<AlterClause> alterClauses, Table externalTable, Database db)
throws UserException {
// check conflict alter ops first
List<AlterClause> alterClauses = stmt.getOps();
AlterOperations currentAlterOps = new AlterOperations();
currentAlterOps.checkConflict(alterClauses);
if (currentAlterOps.hasRenameOp()) {
Expand Down Expand Up @@ -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<AlterClause> 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<String, String> properties = clause.getProperties();
List<String> 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<String, String> 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<AlterClause> alterClauses)
throws UserException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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<String, String> 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) {
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
*/
public class AddColumnsClause extends AlterTableClause {
private List<ColumnDef> columnDefs;
private String sql;
private String rollupName;

private Map<String, String> properties;
Expand All @@ -47,6 +48,15 @@ public AddColumnsClause(List<ColumnDef> columnDefs, String rollupName, Map<Strin
this.properties = properties;
}

// for nereids
public AddColumnsClause(String sql, List<Column> columns, String rollupName, Map<String, String> properties) {
super(AlterOpType.SCHEMA_CHANGE);
this.sql = sql;
this.columns = columns;
this.rollupName = rollupName;
this.properties = properties;
}

public List<Column> getColumns() {
return columns;
}
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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<String, String> getProperties() {
return Maps.newHashMap();
Expand Down
Loading

0 comments on commit f0fb46d

Please sign in to comment.