diff --git a/be/src/vec/exec/scan/vmeta_scanner.cpp b/be/src/vec/exec/scan/vmeta_scanner.cpp index 66d4138d48f540..3a3473fa1643e8 100644 --- a/be/src/vec/exec/scan/vmeta_scanner.cpp +++ b/be/src/vec/exec/scan/vmeta_scanner.cpp @@ -231,6 +231,9 @@ Status VMetaScanner::_fetch_metadata(const TMetaScanRange& meta_scan_range) { case TMetadataType::CATALOGS: RETURN_IF_ERROR(_build_catalogs_metadata_request(meta_scan_range, &request)); break; + case TMetadataType::MATERIALIZED_VIEWS: + RETURN_IF_ERROR(_build_materialized_views_metadata_request(meta_scan_range, &request)); + break; case TMetadataType::QUERIES: RETURN_IF_ERROR(_build_queries_metadata_request(meta_scan_range, &request)); break; @@ -375,6 +378,27 @@ Status VMetaScanner::_build_catalogs_metadata_request(const TMetaScanRange& meta return Status::OK(); } +Status VMetaScanner::_build_materialized_views_metadata_request( + const TMetaScanRange& meta_scan_range, TFetchSchemaTableDataRequest* request) { + VLOG_CRITICAL << "VMetaScanner::_build_materialized_views_metadata_request"; + if (!meta_scan_range.__isset.materialized_views_params) { + return Status::InternalError( + "Can not find TMaterializedViewsMetadataParams from meta_scan_range."); + } + + // create request + request->__set_schema_table_name(TSchemaTableName::METADATA_TABLE); + + // create TMetadataTableRequestParams + TMetadataTableRequestParams metadata_table_params; + metadata_table_params.__set_metadata_type(TMetadataType::MATERIALIZED_VIEWS); + metadata_table_params.__set_materialized_views_metadata_params( + meta_scan_range.materialized_views_params); + + request->__set_metada_table_params(metadata_table_params); + return Status::OK(); +} + Status VMetaScanner::_build_queries_metadata_request(const TMetaScanRange& meta_scan_range, TFetchSchemaTableDataRequest* request) { VLOG_CRITICAL << "VMetaScanner::_build_queries_metadata_request"; diff --git a/be/src/vec/exec/scan/vmeta_scanner.h b/be/src/vec/exec/scan/vmeta_scanner.h index 6c364b94fe2d27..e6eb32e81b263a 100644 --- a/be/src/vec/exec/scan/vmeta_scanner.h +++ b/be/src/vec/exec/scan/vmeta_scanner.h @@ -81,6 +81,8 @@ class VMetaScanner : public VScanner { TFetchSchemaTableDataRequest* request); Status _build_catalogs_metadata_request(const TMetaScanRange& meta_scan_range, TFetchSchemaTableDataRequest* request); + Status _build_materialized_views_metadata_request(const TMetaScanRange& meta_scan_range, + TFetchSchemaTableDataRequest* request); Status _build_queries_metadata_request(const TMetaScanRange& meta_scan_range, TFetchSchemaTableDataRequest* request); bool _meta_eos; diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 424897a5c86d83..0f55208fd3b156 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -1581,15 +1581,16 @@ public class Config extends ConfigBase { @ConfField(description = {"任务堆积时用于存放定时任务的队列大小", "The number of timer jobs that can be queued."}) public static int job_dispatch_timer_job_queue_size = 1024; - /** - * The number of threads used to consume insert tasks. - * if you have a lot of insert jobs,and the average execution frequency is relatively high you need to increase - * this value or increase the number of {@code @job_insert_task_queue_size} - * The value should be greater than 0, if it is 0 or <=0, set it to 5 - */ - @ConfField(description = {"用于执行 Insert 任务的线程数", "The number of threads used to consume insert tasks."}) + @ConfField(description = {"用于执行 Insert 任务的线程数,值应该大于0,否则默认为5", + "The number of threads used to consume Insert tasks, " + + "the value should be greater than 0, if it is <=0, default is 5."}) public static int job_insert_task_consumer_thread_num = 10; + @ConfField(description = {"用于执行 MTMV 任务的线程数,值应该大于0,否则默认为5", + "The number of threads used to consume mtmv tasks, " + + "the value should be greater than 0, if it is <=0, default is 5."}) + public static int job_mtmv_task_consumer_thread_num = 10; + /*---------------------- JOB CONFIG END------------------------*/ /** * The number of async tasks that can be queued. @See TaskDisruptor diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index ee65d926e85bc2..9f182168509aee 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -2595,16 +2595,28 @@ resume_job_stmt ::= show_job_stmt ::= KW_SHOW KW_JOBS {: - RESULT = new ShowJobStmt(null,null,null); + RESULT = new ShowJobStmt(null,null); :} - | KW_SHOW KW_JOB KW_FOR job_label:jobLabel + | KW_SHOW KW_MTMV KW_JOBS + {: + RESULT = new ShowJobStmt(null,org.apache.doris.job.common.JobType.MTMV); + :} + | KW_SHOW KW_MTMV KW_JOB KW_FOR job_label:jobLabel {: - RESULT = new ShowJobStmt(null,jobLabel,null); + RESULT = new ShowJobStmt(jobLabel,org.apache.doris.job.common.JobType.MTMV); :} + | KW_SHOW KW_JOB KW_FOR job_label:jobLabel + {: + RESULT = new ShowJobStmt(jobLabel,org.apache.doris.job.common.JobType.INSERT); + :} | KW_SHOW KW_JOB KW_TASKS KW_FOR job_label:jobLabel {: - RESULT = new ShowJobTaskStmt(null,jobLabel); + RESULT = new ShowJobTaskStmt(jobLabel,org.apache.doris.job.common.JobType.INSERT); :} + | KW_SHOW KW_MTMV KW_JOB KW_TASKS KW_FOR job_label:jobLabel + {: + RESULT = new ShowJobTaskStmt(jobLabel,org.apache.doris.job.common.JobType.MTMV); + :} ; pause_job_stmt ::= KW_PAUSE KW_JOB KW_FOR job_label:jobLabel @@ -2978,10 +2990,6 @@ drop_stmt ::= {: RESULT = new AlterTableStmt(tableName, Lists.newArrayList(new DropIndexClause(indexName, ifExists, tableName, false))); :} - | KW_DROP KW_MATERIALIZED KW_VIEW opt_if_exists:ifExists table_name:mvName - {: - RESULT = new DropMaterializedViewStmt(ifExists, mvName); - :} | KW_DROP KW_MATERIALIZED KW_VIEW opt_if_exists:ifExists ident:mvName KW_ON table_name:tableName {: RESULT = new DropMaterializedViewStmt(ifExists, mvName, tableName); 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 fdc27bb62d1485..1cc679740e164c 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 @@ -28,7 +28,6 @@ import org.apache.doris.analysis.DropMaterializedViewStmt; import org.apache.doris.analysis.DropPartitionClause; import org.apache.doris.analysis.DropPartitionFromIndexClause; -import org.apache.doris.analysis.DropTableStmt; import org.apache.doris.analysis.ModifyColumnCommentClause; import org.apache.doris.analysis.ModifyDistributionClause; import org.apache.doris.analysis.ModifyEngineClause; @@ -45,6 +44,7 @@ import org.apache.doris.catalog.DataProperty; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.MysqlTable; import org.apache.doris.catalog.OdbcTable; import org.apache.doris.catalog.OlapTable; @@ -62,6 +62,8 @@ import org.apache.doris.common.util.DynamicPartitionUtil; import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; +import org.apache.doris.persist.AlterMTMV; import org.apache.doris.persist.AlterViewInfo; import org.apache.doris.persist.BatchModifyPartitionsInfo; import org.apache.doris.persist.ModifyCommentOperationLog; @@ -120,26 +122,13 @@ public void processCreateMaterializedView(CreateMaterializedViewStmt stmt) } public void processDropMaterializedView(DropMaterializedViewStmt stmt) throws DdlException, MetaNotFoundException { - if (!stmt.isForMTMV() && stmt.getTableName() == null) { - throw new DdlException("Drop materialized view without table name is unsupported : " + stmt.toSql()); - } - - // drop materialized view - if (!stmt.isForMTMV()) { - TableName tableName = stmt.getTableName(); - - // check db - String dbName = tableName.getDb(); - Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName); + TableName tableName = stmt.getTableName(); + String dbName = tableName.getDb(); + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName); - String name = tableName.getTbl(); - OlapTable olapTable = (OlapTable) db.getTableOrMetaException(name, TableType.OLAP); - ((MaterializedViewHandler) materializedViewHandler).processDropMaterializedView(stmt, db, olapTable); - } else { - DropTableStmt dropTableStmt = new DropTableStmt(stmt.isIfExists(), stmt.getMTMVName(), false); - dropTableStmt.setMaterializedView(true); - Env.getCurrentInternalCatalog().dropTable(dropTableStmt); - } + String name = tableName.getTbl(); + OlapTable olapTable = (OlapTable) db.getTableOrMetaException(name, TableType.OLAP); + ((MaterializedViewHandler) materializedViewHandler).processDropMaterializedView(stmt, db, olapTable); } private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, List alterClauses, @@ -155,6 +144,10 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, AlterOperations currentAlterOps = new AlterOperations(); currentAlterOps.checkConflict(alterClauses); + if (olapTable instanceof MTMV) { + currentAlterOps.checkMTMVAllow(alterClauses); + } + // check cluster capacity and db quota, only need to check once. if (currentAlterOps.needCheckCapacity()) { Env.getCurrentSystemInfo().checkAvailableCapacity(); @@ -201,6 +194,8 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, } 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, clusterName, db, olapTable); + // if base table schemaChanged, need change mtmv status + Env.getCurrentEnv().getMtmvService().alterTable(olapTable); } else if (currentAlterOps.hasRollupOp()) { materializedViewHandler.process(alterClauses, clusterName, db, olapTable); } else if (currentAlterOps.hasPartitionOp()) { @@ -256,6 +251,7 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, } } else if (currentAlterOps.hasRenameOp()) { processRename(db, olapTable, alterClauses); + Env.getCurrentEnv().getMtmvService().alterTable(olapTable); } else if (currentAlterOps.hasReplaceTableOp()) { processReplaceTable(db, olapTable, alterClauses); } else if (currentAlterOps.contains(AlterOpType.MODIFY_TABLE_PROPERTY_SYNC)) { @@ -877,4 +873,45 @@ public AlterHandler getMaterializedViewHandler() { public AlterHandler getClusterHandler() { return clusterHandler; } + + public void processAlterMTMV(AlterMTMV alterMTMV, boolean isReplay) { + TableNameInfo tbl = alterMTMV.getMvName(); + MTMV mtmv = null; + try { + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(tbl.getDb()); + mtmv = (MTMV) db.getTableOrMetaException(tbl.getTbl(), TableType.MATERIALIZED_VIEW); + + mtmv.writeLock(); + switch (alterMTMV.getOpType()) { + case ALTER_REFRESH_INFO: + mtmv.alterRefreshInfo(alterMTMV.getRefreshInfo()); + break; + case ALTER_STATUS: + mtmv.alterStatus(alterMTMV.getStatus()); + break; + case ALTER_PROPERTY: + mtmv.alterMvProperties(alterMTMV.getMvProperties()); + break; + case ADD_TASK: + mtmv.addTaskResult(alterMTMV.getTask(), alterMTMV.getRelation()); + Env.getCurrentEnv().getMtmvService() + .refreshComplete(mtmv, alterMTMV.getRelation(), alterMTMV.getTask()); + break; + default: + throw new RuntimeException("Unknown type value: " + alterMTMV.getOpType()); + } + // 4. log it and replay it in the follower + if (!isReplay) { + Env.getCurrentEnv().getMtmvService().alterMTMV(mtmv, alterMTMV); + Env.getCurrentEnv().getEditLog().logAlterMTMV(alterMTMV); + } + } catch (UserException e) { + // if MTMV has been dropped, ignore this exception + LOG.warn(e); + } finally { + if (mtmv != null) { + mtmv.writeUnlock(); + } + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java index 9563eaa89470a2..270132c1e1798b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java @@ -71,4 +71,9 @@ public boolean needCheckCapacity() { return this == ADD_ROLLUP || this == SCHEMA_CHANGE || this == ADD_PARTITION || this == ENABLE_FEATURE; } + public boolean mtmvAllowOp() { + return this == MODIFY_TABLE_PROPERTY || this == MODIFY_DISTRIBUTION || this == MODIFY_TABLE_COMMENT + || this == ADD_PARTITION || this == DROP_PARTITION || this == REPLACE_PARTITION + || this == MODIFY_PARTITION; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java index 5742c631de28c1..a6fefb119c87bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java @@ -48,6 +48,14 @@ public void checkConflict(List alterClauses) throws DdlException { } } + public void checkMTMVAllow(List alterClauses) throws DdlException { + for (AlterClause alterClause : alterClauses) { + if (!alterClause.getOpType().mtmvAllowOp()) { + throw new DdlException("Alter operation " + alterClause.getOpType() + " Not allowed to MTMV"); + } + } + } + // some operations take up disk space. so we need to check the disk capacity before processing. // return true if we see these kind of operations. public boolean needCheckCapacity() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java index d2aed10983faaf..a1c008fb95eede 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java @@ -83,8 +83,6 @@ public class CreateJobStmt extends DdlStmt { private final String comment; private JobExecuteType executeType; - private String timezone = TimeUtils.DEFAULT_TIME_ZONE; - private static final ImmutableSet> supportStmtSuperClass = new ImmutableSet.Builder>().add(InsertStmt.class) .add(UpdateStmt.class).build(); @@ -114,7 +112,7 @@ public void analyze(Analyzer analyzer) throws UserException { Env.getCurrentInternalCatalog().getDbOrAnalysisException(dbName); analyzerSqlStmt(); // check its insert stmt,currently only support insert stmt - //todo used InsertIntoCommand if job is InsertJob + //todo when support other stmt,need to check stmt type and generate jobInstance InsertJob job = new InsertJob(); JobExecutionConfiguration jobExecutionConfiguration = new JobExecutionConfiguration(); jobExecutionConfiguration.setExecuteType(executeType); @@ -144,10 +142,12 @@ public void analyze(Analyzer analyzer) throws UserException { job.setJobName(labelName.getLabelName()); job.setCreateUser(ConnectContext.get().getCurrentUserIdentity()); job.setJobStatus(JobStatus.RUNNING); - job.checkJobParams(); + job.setJobId(Env.getCurrentEnv().getNextId()); String originStmt = getOrigStmt().originStmt; String executeSql = parseExecuteSql(originStmt); job.setExecuteSql(executeSql); + + //job.checkJobParams(); jobInstance = job; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateMTMVStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateMTMVStmt.java new file mode 100644 index 00000000000000..bb2efdd6281503 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateMTMVStmt.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.analysis; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Index; +import org.apache.doris.mtmv.EnvInfo; +import org.apache.doris.mtmv.MTMVRefreshInfo; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class CreateMTMVStmt extends CreateTableStmt { + private final MTMVRefreshInfo refreshInfo; + private final String querySql; + private final EnvInfo envInfo; + private Map mvProperties; + + public CreateMTMVStmt(boolean ifNotExists, TableName mvName, List columns, + MTMVRefreshInfo refreshInfo, KeysDesc keyDesc, DistributionDesc distributionDesc, + Map properties, Map mvProperties, String querySql, String comment, + EnvInfo envInfo) { + super(ifNotExists, false, mvName, columns, new ArrayList(), DEFAULT_ENGINE_NAME, keyDesc, null, + distributionDesc, properties, null, comment, null, null); + this.refreshInfo = refreshInfo; + this.querySql = querySql; + this.envInfo = envInfo; + this.mvProperties = mvProperties; + } + + public MTMVRefreshInfo getRefreshInfo() { + return refreshInfo; + } + + public String getQuerySql() { + return querySql; + } + + public EnvInfo getEnvInfo() { + return envInfo; + } + + public Map getMvProperties() { + return mvProperties; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java index aecee1465d56ce..a2ae5547baf993 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java @@ -67,7 +67,7 @@ public class CreateTableStmt extends DdlStmt { protected static final String DEFAULT_ENGINE_NAME = "olap"; - private boolean ifNotExists; + protected boolean ifNotExists; private boolean isExternal; protected TableName tableName; protected List columnDefs; @@ -244,7 +244,7 @@ public CreateTableStmt(boolean ifNotExists, this.extProperties = extProperties; this.columnDefs = Lists.newArrayList(); this.comment = Strings.nullToEmpty(comment); - this.rollupAlterClauseList = rollupAlterClauseList; + this.rollupAlterClauseList = (rollupAlterClauseList == null) ? Lists.newArrayList() : rollupAlterClauseList; } public void addColumnDef(ColumnDef columnDef) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropMaterializedViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropMaterializedViewStmt.java index 6c8e58b259f4a6..68cd29688d508c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropMaterializedViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropMaterializedViewStmt.java @@ -41,8 +41,6 @@ public class DropMaterializedViewStmt extends DdlStmt { private String mvName; private TableName tableName; - - private TableName mtmvName; private boolean ifExists; public DropMaterializedViewStmt(boolean ifExists, String mvName, TableName tableName) { @@ -51,59 +49,31 @@ public DropMaterializedViewStmt(boolean ifExists, String mvName, TableName table this.ifExists = ifExists; } - public DropMaterializedViewStmt(boolean ifExists, TableName mvName) { - this.mtmvName = mvName; - this.ifExists = ifExists; - this.tableName = null; - } - public String getMvName() { - if (tableName != null) { - return mvName; - } else if (mtmvName != null) { - return mtmvName.toString(); - } else { - return null; - } + return mvName; } public TableName getTableName() { return tableName; } - public TableName getMTMVName() { - return mtmvName; - } - public boolean isIfExists() { return ifExists; } - public boolean isForMTMV() { - return mtmvName != null; - } - @Override public void analyze(Analyzer analyzer) throws UserException { - if (!isForMTMV()) { - if (Strings.isNullOrEmpty(mvName)) { - throw new AnalysisException("The materialized name could not be empty or null."); - } - tableName.analyze(analyzer); - // disallow external catalog - Util.prohibitExternalCatalog(tableName.getCtl(), this.getClass().getSimpleName()); - - // check access - if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), tableName.getDb(), - tableName.getTbl(), PrivPredicate.DROP)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "DROP"); - } - } else { - mtmvName.analyze(analyzer); - if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), mtmvName.getDb(), - mtmvName.getTbl(), PrivPredicate.DROP)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "DROP"); - } + if (Strings.isNullOrEmpty(mvName)) { + throw new AnalysisException("The materialized name could not be empty or null."); + } + tableName.analyze(analyzer); + // disallow external catalog + Util.prohibitExternalCatalog(tableName.getCtl(), this.getClass().getSimpleName()); + + // check access + if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), tableName.getDb(), + tableName.getTbl(), PrivPredicate.DROP)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "DROP"); } } @@ -114,12 +84,8 @@ public String toSql() { if (ifExists) { stringBuilder.append("IF EXISTS "); } - if (mtmvName != null) { - stringBuilder.append(mtmvName.toSql()); - } else { - stringBuilder.append("`").append(mvName).append("` "); - stringBuilder.append("ON ").append(tableName.toSql()); - } + stringBuilder.append("`").append(mvName).append("` "); + stringBuilder.append("ON ").append(tableName.toSql()); return stringBuilder.toString(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobStmt.java index dc4e96e05f01bf..59c5fc2fc47066 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobStmt.java @@ -19,18 +19,16 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.ScalarType; -import org.apache.doris.cluster.ClusterNamespace; 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.job.common.JobType; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; -import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import lombok.Getter; +import java.util.Arrays; import java.util.List; /** @@ -63,24 +61,30 @@ public class ShowJobStmt extends ShowStmt { @Getter private String dbFullName; // optional - private String jobCategoryName; // optional + @Getter + private JobType jobType; // optional + + /** + * Supported job types, if we want to support more job types, we need to add them here. + */ + @Getter + private List jobTypes = Arrays.asList(JobType.INSERT); // optional @Getter private String name; // optional @Getter private String pattern; // optional - public ShowJobStmt(String category, LabelName labelName, String pattern) { + public ShowJobStmt(LabelName labelName, JobType jobType) { this.labelName = labelName; - this.pattern = pattern; - this.jobCategoryName = category; + this.jobType = jobType; + this.name = labelName == null ? null : labelName.getLabelName(); } @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); checkAuth(); - checkLabelName(analyzer); } private void checkAuth() throws AnalysisException { @@ -90,19 +94,6 @@ private void checkAuth() throws AnalysisException { } } - private void checkLabelName(Analyzer analyzer) throws AnalysisException { - String dbName = labelName == null ? null : labelName.getDbName(); - if (Strings.isNullOrEmpty(dbName)) { - dbFullName = analyzer.getContext().getDatabase(); - if (Strings.isNullOrEmpty(dbFullName)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); - } - } else { - dbFullName = ClusterNamespace.getFullName(getClusterName(), dbName); - } - name = labelName == null ? null : labelName.getLabelName(); - } - public static List getTitleNames() { return TITLE_NAMES; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java index 8d5c2b61db2fbf..c5688f22b52a2f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java @@ -19,14 +19,10 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.ScalarType; -import org.apache.doris.cluster.ClusterNamespace; -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.job.common.JobType; import org.apache.doris.qe.ShowResultSetMetaData; -import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import lombok.Getter; @@ -49,43 +45,27 @@ public class ShowJobTaskStmt extends ShowStmt { .add("ExecuteSql") .add("Result") .add("ErrorMsg") - .add("TaskType") .build(); @Getter private final LabelName labelName; - - @Getter - private String dbFullName; // optional @Getter private String name; // optional - public ShowJobTaskStmt(String category, LabelName labelName) { + @Getter + JobType jobType; + + public ShowJobTaskStmt(LabelName labelName, JobType jobType) { this.labelName = labelName; + this.jobType = jobType; + this.name = labelName == null ? null : labelName.getLabelName(); } @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); CreateJobStmt.checkAuth(); - checkLabelName(analyzer); - } - - private void checkLabelName(Analyzer analyzer) throws AnalysisException { - String dbName = labelName == null ? null : labelName.getDbName(); - if (Strings.isNullOrEmpty(dbName)) { - dbFullName = analyzer.getContext().getDatabase(); - if (Strings.isNullOrEmpty(dbFullName)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); - } - } else { - dbFullName = ClusterNamespace.getFullName(getClusterName(), dbName); - } - if (null == labelName) { - throw new AnalysisException("Job name is null"); - } - name = labelName.getLabelName(); } public static List getTitleNames() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 62d2e0bd6d6e56..5965ec99650cb4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -396,6 +396,8 @@ public Pair createTableWithLock( } if (table.getType() == TableType.ELASTICSEARCH) { Env.getCurrentEnv().getEsRepository().registerTable((EsTable) table); + } else if (table.getType() == TableType.MATERIALIZED_VIEW) { + Env.getCurrentEnv().getMtmvService().registerMTMV((MTMV) table, id); } } return Pair.of(result, isTableExist); @@ -628,6 +630,9 @@ public void readFields(DataInput in) throws IOException { for (int i = 0; i < numTables; ++i) { Table table = Table.read(in); table.setQualifiedDbName(fullQualifiedName); + if (table instanceof MTMV) { + Env.getCurrentEnv().getMtmvService().registerMTMV((MTMV) table, id); + } String tableName = table.getName(); nameToTable.put(tableName, table); idToTable.put(table.getId(), table); 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 941e24599b7e67..a756dcd4e6ca51 100755 --- 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 @@ -148,6 +148,7 @@ import org.apache.doris.httpv2.meta.MetaBaseAction; import org.apache.doris.httpv2.rest.RestApiStatusCode; import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.extensions.mtmv.MTMVTask; import org.apache.doris.job.manager.JobManager; import org.apache.doris.journal.JournalCursor; import org.apache.doris.journal.JournalEntity; @@ -175,9 +176,17 @@ import org.apache.doris.master.PartitionInMemoryInfoCollector; import org.apache.doris.meta.MetaContext; import org.apache.doris.metric.MetricRepo; +import org.apache.doris.mtmv.MTMVAlterOpType; +import org.apache.doris.mtmv.MTMVRelation; +import org.apache.doris.mtmv.MTMVService; +import org.apache.doris.mtmv.MTMVStatus; import org.apache.doris.mysql.privilege.AccessControllerManager; import org.apache.doris.mysql.privilege.Auth; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVPropertyInfo; +import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVRefreshInfo; +import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; +import org.apache.doris.persist.AlterMTMV; import org.apache.doris.persist.AutoIncrementIdUpdateLog; import org.apache.doris.persist.BackendReplicasInfo; import org.apache.doris.persist.BackendTabletsInfo; @@ -494,6 +503,8 @@ public class Env { private TopicPublisherThread topicPublisherThread; + private MTMVService mtmvService; + public List getFrontendInfos() { List res = new ArrayList<>(); @@ -720,6 +731,7 @@ private Env(boolean isCheckpointCatalog) { this.queryCancelWorker = new QueryCancelWorker(systemInfo); this.topicPublisherThread = new TopicPublisherThread( "TopicPublisher", Config.publish_topic_info_interval_ms, systemInfo); + this.mtmvService = new MTMVService(); } public static void destroyCheckpoint() { @@ -775,6 +787,10 @@ public AccessControllerManager getAccessManager() { return accessManager; } + public MTMVService getMtmvService() { + return mtmvService; + } + public TabletScheduler getTabletScheduler() { return tabletScheduler; } @@ -2950,48 +2966,47 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis || table.getType() == TableType.HIVE || table.getType() == TableType.JDBC) { sb.append("EXTERNAL "); } - sb.append(table.getType() != TableType.MATERIALIZED_VIEW ? "TABLE " : "MATERIALIZED VIEW "); + sb.append("TABLE "); if (!Strings.isNullOrEmpty(dbName)) { sb.append("`").append(dbName).append("`."); } sb.append("`").append(table.getName()).append("`"); - if (table.getType() != TableType.MATERIALIZED_VIEW) { - sb.append(" (\n"); - int idx = 0; - List columns; - // when 'create table B like A', always return schema of A without hidden columns - if (getDdlForLike) { - columns = table.getBaseSchema(false); + + sb.append(" (\n"); + int idx = 0; + List columns; + // when 'create table B like A', always return schema of A without hidden columns + if (getDdlForLike) { + columns = table.getBaseSchema(false); + } else { + columns = table.getBaseSchema(); + } + for (Column column : columns) { + if (idx++ != 0) { + sb.append(",\n"); + } + // There MUST BE 2 space in front of each column description line + // sqlalchemy requires this to parse SHOW CREATE TABLE stmt. + if (table.getType() == TableType.OLAP) { + sb.append(" ").append( + column.toSql(((OlapTable) table).getKeysType() == KeysType.UNIQUE_KEYS, true)); } else { - columns = table.getBaseSchema(); + sb.append(" ").append(column.toSql()); } - for (Column column : columns) { - if (idx++ != 0) { + } + if (table.getType() == TableType.OLAP) { + OlapTable olapTable = (OlapTable) table; + if (CollectionUtils.isNotEmpty(olapTable.getIndexes())) { + for (Index index : olapTable.getIndexes()) { sb.append(",\n"); - } - // There MUST BE 2 space in front of each column description line - // sqlalchemy requires this to parse SHOW CREATE TABLE stmt. - if (table.getType() == TableType.OLAP) { - sb.append(" ").append( - column.toSql(((OlapTable) table).getKeysType() == KeysType.UNIQUE_KEYS, true)); - } else { - sb.append(" ").append(column.toSql()); - } - } - if (table.getType() == TableType.OLAP) { - OlapTable olapTable = (OlapTable) table; - if (CollectionUtils.isNotEmpty(olapTable.getIndexes())) { - for (Index index : olapTable.getIndexes()) { - sb.append(",\n"); - sb.append(" ").append(index.toSql()); - } + sb.append(" ").append(index.toSql()); } } - sb.append("\n) ENGINE="); - sb.append(table.getType().name()); } + sb.append("\n) ENGINE="); + sb.append(table.getType().name()); if (table.getType() == TableType.OLAP || table.getType() == TableType.MATERIALIZED_VIEW) { OlapTable olapTable = (OlapTable) table; @@ -4061,8 +4076,12 @@ public void cancelBackup(CancelBackupStmt stmt) throws DdlException { getBackupHandler().cancel(stmt); } - // entry of rename table operation public void renameTable(Database db, Table table, TableRenameClause tableRenameClause) throws DdlException { + renameTable(db, table, tableRenameClause.getNewTableName()); + } + + // entry of rename table operation + public void renameTable(Database db, Table table, String newTableName) throws DdlException { db.writeLockOrDdlException(); try { table.writeLockOrDdlException(); @@ -4073,7 +4092,6 @@ public void renameTable(Database db, Table table, TableRenameClause tableRenameC } String oldTableName = table.getName(); - String newTableName = tableRenameClause.getNewTableName(); if (Env.isStoredTableNamesLowerCase() && !Strings.isNullOrEmpty(newTableName)) { newTableName = newTableName.toLowerCase(); } @@ -5247,7 +5265,8 @@ public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempP long dbId = replaceTempPartitionLog.getDbId(); long tableId = replaceTempPartitionLog.getTblId(); Database db = getInternalCatalog().getDbOrMetaException(dbId); - OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP); + OlapTable olapTable = (OlapTable) db + .getTableOrMetaException(tableId, Lists.newArrayList(TableType.OLAP, TableType.MATERIALIZED_VIEW)); olapTable.writeLock(); try { olapTable.replaceTempPartitions(replaceTempPartitionLog.getPartitions(), @@ -5762,4 +5781,28 @@ public ColumnIdFlushDaemon getColumnIdFlusher() { public StatisticsAutoCollector getStatisticsAutoCollector() { return statisticsAutoCollector; } + + public void alterMTMVRefreshInfo(AlterMTMVRefreshInfo info) { + AlterMTMV alter = new AlterMTMV(info.getMvName(), info.getRefreshInfo(), MTMVAlterOpType.ALTER_REFRESH_INFO); + this.alter.processAlterMTMV(alter, false); + } + + public void alterMTMVProperty(AlterMTMVPropertyInfo info) { + AlterMTMV alter = new AlterMTMV(info.getMvName(), MTMVAlterOpType.ALTER_PROPERTY); + alter.setMvProperties(info.getProperties()); + this.alter.processAlterMTMV(alter, false); + } + + public void alterMTMVStatus(TableNameInfo mvName, MTMVStatus status) { + AlterMTMV alter = new AlterMTMV(mvName, MTMVAlterOpType.ALTER_STATUS); + alter.setStatus(status); + this.alter.processAlterMTMV(alter, false); + } + + public void addMTMVTaskResult(TableNameInfo mvName, MTMVTask task, MTMVRelation relation) { + AlterMTMV alter = new AlterMTMV(mvName, MTMVAlterOpType.ADD_TASK); + alter.setTask(task); + alter.setRelation(relation); + this.alter.processAlterMTMV(alter, false); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index e463c3dfe700e7..0c59f56db5c831 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -17,16 +17,185 @@ package org.apache.doris.catalog; +import org.apache.doris.catalog.OlapTableFactory.MTMVParams; import org.apache.doris.common.io.Text; +import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.job.common.TaskStatus; +import org.apache.doris.job.extensions.mtmv.MTMVTask; +import org.apache.doris.mtmv.EnvInfo; +import org.apache.doris.mtmv.MTMVJobInfo; +import org.apache.doris.mtmv.MTMVJobManager; +import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVRefreshState; +import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVState; +import org.apache.doris.mtmv.MTMVRefreshInfo; +import org.apache.doris.mtmv.MTMVRelation; +import org.apache.doris.mtmv.MTMVStatus; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.DataInput; +import java.io.DataOutput; import java.io.IOException; +import java.util.Map; +import java.util.concurrent.locks.ReentrantReadWriteLock; + public class MTMV extends OlapTable { + private static final Logger LOG = LogManager.getLogger(MTMV.class); + private ReentrantReadWriteLock mvRwLock; + + @SerializedName("ri") + private MTMVRefreshInfo refreshInfo; + @SerializedName("qs") + private String querySql; + @SerializedName("s") + private MTMVStatus status; + @SerializedName("ei") + private EnvInfo envInfo; + @SerializedName("ji") + private MTMVJobInfo jobInfo; + @SerializedName("mp") + private Map mvProperties; + @SerializedName("r") + private MTMVRelation relation; + + // For deserialization + public MTMV() { + type = TableType.MATERIALIZED_VIEW; + mvRwLock = new ReentrantReadWriteLock(true); + } + + MTMV(MTMVParams params) { + super( + params.tableId, + params.tableName, + params.schema, + params.keysType, + params.partitionInfo, + params.distributionInfo + ); + this.type = TableType.MATERIALIZED_VIEW; + this.querySql = params.querySql; + this.refreshInfo = params.refreshInfo; + this.envInfo = params.envInfo; + this.status = new MTMVStatus(); + this.jobInfo = new MTMVJobInfo(MTMVJobManager.MTMV_JOB_PREFIX + params.tableId); + this.mvProperties = params.mvProperties; + mvRwLock = new ReentrantReadWriteLock(true); + } + + public MTMVRefreshInfo getRefreshInfo() { + return refreshInfo; + } + + public String getQuerySql() { + return querySql; + } + + public MTMVStatus getStatus() { + try { + readMvLock(); + return status; + } finally { + readMvUnlock(); + } + } + + public EnvInfo getEnvInfo() { + return envInfo; + } + + public MTMVJobInfo getJobInfo() { + return jobInfo; + } + + public MTMVRelation getRelation() { + return relation; + } + + public MTMVRefreshInfo alterRefreshInfo(MTMVRefreshInfo newRefreshInfo) { + return refreshInfo.updateNotNull(newRefreshInfo); + } + + public MTMVStatus alterStatus(MTMVStatus newStatus) { + try { + writeMvLock(); + return this.status.updateNotNull(newStatus); + } finally { + writeMvUnlock(); + } + } + + public void addTaskResult(MTMVTask task, MTMVRelation relation) { + try { + writeMvLock(); + if (task.getStatus() == TaskStatus.SUCCESS) { + this.status.setState(MTMVState.NORMAL); + this.status.setSchemaChangeDetail(null); + this.status.setRefreshState(MTMVRefreshState.SUCCESS); + this.relation = relation; + } else { + this.status.setRefreshState(MTMVRefreshState.FAIL); + } + this.jobInfo.addHistoryTask(task); + } finally { + writeMvUnlock(); + } + } + + public Map alterMvProperties(Map mvProperties) { + this.mvProperties.putAll(mvProperties); + return this.mvProperties; + } + + public long getGracePeriod() { + if (mvProperties.containsKey(PropertyAnalyzer.PROPERTIES_GRACE_PERIOD)) { + return Long.parseLong(mvProperties.get(PropertyAnalyzer.PROPERTIES_GRACE_PERIOD)); + } else { + return 0L; + } + } + + public Map getMvProperties() { + return mvProperties; + } + + public void readMvLock() { + this.mvRwLock.readLock().lock(); + } + + public void readMvUnlock() { + this.mvRwLock.readLock().unlock(); + } + + public void writeMvLock() { + this.mvRwLock.writeLock().lock(); + } + + public void writeMvUnlock() { + this.mvRwLock.writeLock().unlock(); + } + + @Override + public void write(DataOutput out) throws IOException { + super.write(out); + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } @Override public void readFields(DataInput in) throws IOException { super.readFields(in); - Text.readString(in); + MTMV materializedView = GsonUtils.GSON.fromJson(Text.readString(in), this.getClass()); + refreshInfo = materializedView.refreshInfo; + querySql = materializedView.querySql; + status = materializedView.status; + envInfo = materializedView.envInfo; + jobInfo = materializedView.jobInfo; + mvProperties = materializedView.mvProperties; + relation = materializedView.relation; } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTableFactory.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTableFactory.java index b17b58abb53f1f..af87ec47991732 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTableFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTableFactory.java @@ -17,13 +17,17 @@ package org.apache.doris.catalog; +import org.apache.doris.analysis.CreateMTMVStmt; import org.apache.doris.analysis.CreateTableStmt; import org.apache.doris.analysis.DdlStmt; import org.apache.doris.catalog.TableIf.TableType; +import org.apache.doris.mtmv.EnvInfo; +import org.apache.doris.mtmv.MTMVRefreshInfo; import com.google.common.base.Preconditions; import java.util.List; +import java.util.Map; public class OlapTableFactory { @@ -40,10 +44,19 @@ public static class OlapTableParams extends BuildParams { public TableIndexes indexes; } + public static class MTMVParams extends BuildParams { + public MTMVRefreshInfo refreshInfo; + public EnvInfo envInfo; + public String querySql; + public Map mvProperties; + } + private BuildParams params; public static TableType getTableType(DdlStmt stmt) { - if (stmt instanceof CreateTableStmt) { + if (stmt instanceof CreateMTMVStmt) { + return TableType.MATERIALIZED_VIEW; + } else if (stmt instanceof CreateTableStmt) { return TableType.OLAP; } else { throw new IllegalArgumentException("Invalid DDL statement: " + stmt.toSql()); @@ -51,22 +64,28 @@ public static TableType getTableType(DdlStmt stmt) { } public OlapTableFactory init(TableType type) { - params = new OlapTableParams(); + params = (type == TableType.OLAP) ? new OlapTableParams() : new MTMVParams(); return this; } public Table build() { Preconditions.checkNotNull(params, "The factory isn't initialized."); - OlapTableParams olapTableParams = (OlapTableParams) params; - return new OlapTable( - olapTableParams.tableId, - olapTableParams.tableName, - olapTableParams.schema, - olapTableParams.keysType, - olapTableParams.partitionInfo, - olapTableParams.distributionInfo, - olapTableParams.indexes - ); + + if (params instanceof OlapTableParams) { + OlapTableParams olapTableParams = (OlapTableParams) params; + return new OlapTable( + olapTableParams.tableId, + olapTableParams.tableName, + olapTableParams.schema, + olapTableParams.keysType, + olapTableParams.partitionInfo, + olapTableParams.distributionInfo, + olapTableParams.indexes + ); + } else { + MTMVParams mtmvParams = (MTMVParams) params; + return new MTMV(mtmvParams); + } } public OlapTableFactory withTableId(long tableId) { @@ -107,8 +126,49 @@ public OlapTableFactory withIndexes(TableIndexes indexes) { return this; } + public OlapTableFactory withQuerySql(String querySql) { + Preconditions.checkState(params instanceof MTMVParams, "Invalid argument for " + + params.getClass().getSimpleName()); + MTMVParams mtmvParams = (MTMVParams) params; + mtmvParams.querySql = querySql; + return this; + } + + public OlapTableFactory withMvProperties(Map mvProperties) { + Preconditions.checkState(params instanceof MTMVParams, "Invalid argument for " + + params.getClass().getSimpleName()); + MTMVParams mtmvParams = (MTMVParams) params; + mtmvParams.mvProperties = mvProperties; + return this; + } + + private OlapTableFactory withRefreshInfo(MTMVRefreshInfo refreshInfo) { + Preconditions.checkState(params instanceof MTMVParams, "Invalid argument for " + + params.getClass().getSimpleName()); + MTMVParams mtmvParams = (MTMVParams) params; + mtmvParams.refreshInfo = refreshInfo; + return this; + } + + private OlapTableFactory withEnvInfo(EnvInfo envInfo) { + Preconditions.checkState(params instanceof MTMVParams, "Invalid argument for " + + params.getClass().getSimpleName()); + MTMVParams mtmvParams = (MTMVParams) params; + mtmvParams.envInfo = envInfo; + return this; + } + public OlapTableFactory withExtraParams(DdlStmt stmt) { - CreateTableStmt createOlapTableStmt = (CreateTableStmt) stmt; - return withIndexes(new TableIndexes(createOlapTableStmt.getIndexes())); + boolean isMaterializedView = stmt instanceof CreateMTMVStmt; + if (!isMaterializedView) { + CreateTableStmt createOlapTableStmt = (CreateTableStmt) stmt; + return withIndexes(new TableIndexes(createOlapTableStmt.getIndexes())); + } else { + CreateMTMVStmt createMTMVStmt = (CreateMTMVStmt) stmt; + return withRefreshInfo(createMTMVStmt.getRefreshInfo()) + .withQuerySql(createMTMVStmt.getQuerySql()) + .withMvProperties(createMTMVStmt.getMvProperties()) + .withEnvInfo(createMTMVStmt.getEnvInfo()); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java index 46e1ad04c2be5c..6fb93756f65b6e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java @@ -157,6 +157,7 @@ public class PropertyAnalyzer { public static final String PROPERTIES_ENABLE_DUPLICATE_WITHOUT_KEYS_BY_DEFAULT = "enable_duplicate_without_keys_by_default"; + public static final String PROPERTIES_GRACE_PERIOD = "grace_period"; // For unique key data model, the feature Merge-on-Write will leverage a primary // key index and a delete-bitmap to mark duplicate keys as deleted in load stage, // which can avoid the merging cost in read stage, and accelerate the aggregation diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index 6ab7dc2588a05d..7b85244da7553c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -75,6 +75,7 @@ import org.apache.doris.catalog.JdbcTable; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.ListPartitionItem; +import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.MaterializedIndex; import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.MaterializedIndex.IndexState; @@ -884,6 +885,12 @@ public void dropTable(DropTableStmt stmt) throws DdlException { } } + if (!stmt.isMaterializedView() && table instanceof MTMV) { + ErrorReport.reportDdlException(ErrorCode.ERR_WRONG_OBJECT, dbName, tableName, "TABLE"); + } else if (stmt.isMaterializedView() && !(table instanceof MTMV)) { + ErrorReport.reportDdlException(ErrorCode.ERR_WRONG_OBJECT, dbName, tableName, "MTMV"); + } + if (!stmt.isForceDrop()) { if (Env.getCurrentGlobalTransactionMgr().existCommittedTxns(db.getId(), table.getId(), null)) { throw new DdlException( @@ -917,6 +924,12 @@ public void dropTable(DropTableStmt stmt) throws DdlException { Env.getCurrentEnv().getQueryStats().clear(Env.getCurrentEnv().getCurrentCatalog().getId(), db.getId(), table.getId()); Env.getCurrentEnv().getAnalysisManager().removeTableStats(table.getId()); + if (table.getType() == TableType.MATERIALIZED_VIEW) { + Env.getCurrentEnv().getMtmvService().dropMTMV((MTMV) table); + } + Env.getCurrentEnv().getMtmvService().dropTable(table); + } catch (UserException e) { + throw new DdlException(e.getMessage(), e); } finally { db.writeUnlock(); } @@ -934,6 +947,8 @@ public boolean unprotectDropTable(Database db, Table table, boolean isForceDrop, } else if (table.getType() == TableType.ICEBERG) { // drop Iceberg database table creation record icebergTableCreationRecordMgr.deregisterTable(db, (IcebergTable) table); + } else if (table.getType() == TableType.MATERIALIZED_VIEW) { + Env.getCurrentEnv().getMtmvService().deregisterMTMV((MTMV) table); } db.dropTable(table.getName()); @@ -1320,7 +1335,7 @@ public void addPartitionLike(Database db, String tableName, AddPartitionLikeClau try { Table table = db.getTableOrDdlException(tableName); - if (table.getType() != TableType.OLAP) { + if (table.getType() != TableType.OLAP && table.getType() != TableType.MATERIALIZED_VIEW) { throw new DdlException("Only support create partition from a OLAP table"); } @@ -1663,7 +1678,8 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa public void replayAddPartition(PartitionPersistInfo info) throws MetaNotFoundException { Database db = (Database) getDbOrMetaException(info.getDbId()); - OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), TableType.OLAP); + OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), + Lists.newArrayList(TableType.OLAP, TableType.MATERIALIZED_VIEW)); olapTable.writeLock(); try { Partition partition = info.getPartition(); @@ -2589,6 +2605,9 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep throw e; } + if (olapTable instanceof MTMV) { + Env.getCurrentEnv().getMtmvService().createMTMV((MTMV) olapTable); + } } private void createMysqlTable(Database db, CreateTableStmt stmt) throws DdlException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index 6e2498cf013fe4..908676df5ba8fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -18,25 +18,29 @@ package org.apache.doris.job.base; import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.job.common.JobStatus; -import org.apache.doris.job.common.JobType; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.exception.JobException; -import org.apache.doris.job.extensions.insert.InsertJob; import org.apache.doris.job.task.AbstractTask; -import org.apache.doris.job.task.Task; +import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.qe.ShowResultSetMetaData; +import com.google.common.collect.ImmutableList; import com.google.gson.annotations.SerializedName; import lombok.Data; import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.RandomUtils; import java.io.DataInput; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.List; @Data @@ -72,16 +76,30 @@ public abstract class AbstractJob implements Job, Wri private List runningTasks = new ArrayList<>(); @Override - public void cancel() throws JobException { + public void cancelAllTasks() throws JobException { if (CollectionUtils.isEmpty(runningTasks)) { return; } - runningTasks.forEach(Task::cancel); - + for (T task : runningTasks) { + task.cancel(); + } } + private static final ImmutableList TITLE_NAMES = + new ImmutableList.Builder() + .add("Id") + .add("Name") + .add("Definer") + .add("ExecuteType") + .add("RecurringStrategy") + .add("Status") + .add("ExecuteSql") + .add("CreateTime") + .add("Comment") + .build(); + @Override - public void cancel(long taskId) throws JobException { + public void cancelTaskById(long taskId) throws JobException { if (CollectionUtils.isEmpty(runningTasks)) { throw new JobException("no running task"); } @@ -89,13 +107,17 @@ public void cancel(long taskId) throws JobException { .orElseThrow(() -> new JobException("no task id:" + taskId)).cancel(); } - public void initTasks(List tasks) { + public void initTasks(List tasks) { tasks.forEach(task -> { task.setJobId(jobId); - task.setTaskId(Env.getCurrentEnv().getNextId()); + task.setTaskId(getNextId()); task.setCreateTimeMs(System.currentTimeMillis()); task.setStatus(TaskStatus.PENDING); }); + if (CollectionUtils.isEmpty(getRunningTasks())) { + setRunningTasks(new ArrayList<>()); + } + getRunningTasks().addAll((Collection) tasks); } public void checkJobParams() { @@ -105,7 +127,7 @@ public void checkJobParams() { if (null == jobConfig) { throw new IllegalArgumentException("jobConfig cannot be null"); } - jobConfig.checkParams(createTimeMs); + jobConfig.checkParams(); checkJobParamsInternal(); } @@ -132,23 +154,16 @@ public void updateJobStatus(JobStatus newJobStatus) { protected abstract void checkJobParamsInternal(); public static AbstractJob readFields(DataInput in) throws IOException { - // todo use RuntimeTypeAdapterFactory of Gson to do the serde - JobType jobType = JobType.valueOf(Text.readString(in)); - switch (jobType) { - case INSERT: - return InsertJob.readFields(in); - case MTMV: - // return MTMVJob.readFields(in); - break; - default: - throw new IllegalArgumentException("unknown job type"); - } - throw new IllegalArgumentException("unknown job type"); + String jsonJob = Text.readString(in); + AbstractJob job = GsonUtils.GSON.fromJson(jsonJob, AbstractJob.class); + job.setRunningTasks(new ArrayList<>()); + return job; } @Override public void onTaskFail(T task) { updateJobStatusIfEnd(); + runningTasks.remove(task); } @Override @@ -166,16 +181,14 @@ private void updateJobStatusIfEnd() { switch (executeType) { case ONE_TIME: case INSTANT: - jobStatus = JobStatus.FINISHED; - Env.getCurrentEnv().getJobManager().getJob(jobId).updateJobStatus(jobStatus); + Env.getCurrentEnv().getJobManager().getJob(jobId).updateJobStatus(JobStatus.FINISHED); break; case RECURRING: TimerDefinition timerDefinition = getJobConfig().getTimerDefinition(); if (null != timerDefinition.getEndTimeMs() && timerDefinition.getEndTimeMs() < System.currentTimeMillis() + timerDefinition.getIntervalUnit().getIntervalMs(timerDefinition.getInterval())) { - jobStatus = JobStatus.FINISHED; - Env.getCurrentEnv().getJobManager().getJob(jobId).updateJobStatus(jobStatus); + Env.getCurrentEnv().getJobManager().getJob(jobId).updateJobStatus(JobStatus.FINISHED); } break; default: @@ -202,4 +215,23 @@ public List getCommonShowInfo() { commonShowInfo.add(comment); return commonShowInfo; } + + @Override + public List getShowInfo() { + return getCommonShowInfo(); + } + + @Override + public ShowResultSetMetaData getJobMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + + for (String title : TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(30))); + } + return builder.build(); + } + + private static long getNextId() { + return System.nanoTime() + RandomUtils.nextInt(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/Job.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/Job.java index 25e93928d8251f..fef447d160f189 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/Job.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/Job.java @@ -50,7 +50,7 @@ public interface Job { * @throws JobException If the task is not in the running state, it may have already * finished and cannot be cancelled. */ - void cancel(long taskId) throws JobException; + void cancelTaskById(long taskId) throws JobException; /** * Checks if the job is ready for scheduling. @@ -95,7 +95,7 @@ public interface Job { * * @throws JobException If cancelling a running task fails. */ - void cancel() throws JobException; + void cancelAllTasks() throws JobException; /** * Notifies the job when a task execution fails. @@ -112,9 +112,8 @@ public interface Job { void onTaskSuccess(T task); /** - * Notifies the job when a task execution is cancelled. - * - * @param task The cancelled task. + * get the job's show info, which is used to sql show the job information + * @return List job common show info */ - void onTaskCancel(T task); + List getShowInfo(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java index ead943d4bf1eb7..16b9dd24281eec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java @@ -38,15 +38,18 @@ public class JobExecutionConfiguration { @SerializedName(value = "ec") private JobExecuteType executeType; + @Getter + @Setter + private boolean immediate = false; + /** * Maximum number of concurrent tasks, <= 0 means no limit * if the number of tasks exceeds the limit, the task will be delayed execution * todo: implement this later, we need to consider concurrency strategies */ - @SerializedName(value = "maxConcurrentTaskNum") private Integer maxConcurrentTaskNum; - public void checkParams(Long createTimeMs) { + public void checkParams() { if (executeType == null) { throw new IllegalArgumentException("executeType cannot be null"); } @@ -55,7 +58,7 @@ public void checkParams(Long createTimeMs) { return; } - checkTimerDefinition(createTimeMs); + checkTimerDefinition(immediate); if (executeType == JobExecuteType.ONE_TIME) { validateStartTimeMs(); @@ -77,12 +80,12 @@ public void checkParams(Long createTimeMs) { } } - private void checkTimerDefinition(long createTimeMs) { + private void checkTimerDefinition(boolean immediate) { if (timerDefinition == null) { throw new IllegalArgumentException( "timerDefinition cannot be null when executeType is not instant or manual"); } - timerDefinition.checkParams(createTimeMs); + timerDefinition.checkParams(immediate); } private void validateStartTimeMs() { @@ -130,9 +133,14 @@ public List getTriggerDelayTimes(Long currentTimeMs, Long startTimeMs, Lon && timerDefinition.getEndTimeMs() < startTimeMs) { return delayTimeSeconds; } + long intervalValue = timerDefinition.getIntervalUnit().getIntervalMs(timerDefinition.getInterval()); + long jobStartTimeMs = timerDefinition.getStartTimeMs(); + if (isImmediate()) { + jobStartTimeMs += intervalValue; + } - return getExecutionDelaySeconds(startTimeMs, endTimeMs, timerDefinition.getStartTimeMs(), - timerDefinition.getIntervalUnit().getIntervalMs(timerDefinition.getInterval()), currentTimeMs); + return getExecutionDelaySeconds(startTimeMs, endTimeMs, jobStartTimeMs, + intervalValue, currentTimeMs); } return delayTimeSeconds; @@ -160,7 +168,6 @@ private List getExecutionDelaySeconds(long windowStartTimeMs, long windowE long firstTriggerTime = windowStartTimeMs + (intervalMs - ((windowStartTimeMs - startTimeMs) % intervalMs)) % intervalMs; - if (firstTriggerTime < currentTimeMs) { firstTriggerTime += intervalMs; } @@ -173,6 +180,7 @@ private List getExecutionDelaySeconds(long windowStartTimeMs, long windowE for (long triggerTime = firstTriggerTime; triggerTime <= windowEndTimeMs; triggerTime += intervalMs) { if (triggerTime >= currentTimeMs && (null == timerDefinition.getEndTimeMs() || triggerTime < timerDefinition.getEndTimeMs())) { + timerDefinition.setLatestSchedulerTimeMs(triggerTime); timestamps.add(queryDelayTimeSecond(currentTimeMs, triggerTime)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/TimerDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/TimerDefinition.java index 153bc2c43c711f..978538e607dbcf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/TimerDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/TimerDefinition.java @@ -38,12 +38,18 @@ public class TimerDefinition { private Long latestSchedulerTimeMs; - public void checkParams(Long createTimeMs) { + public void checkParams(boolean immediate) { if (null != startTimeMs && startTimeMs < System.currentTimeMillis()) { throw new IllegalArgumentException("startTimeMs must be greater than current time"); } + if (null != startTimeMs && immediate) { + throw new IllegalArgumentException("startTimeMs must be null when immediate is true"); + } + if (null == startTimeMs && immediate) { + startTimeMs = System.currentTimeMillis(); + } if (null == startTimeMs) { - startTimeMs = createTimeMs + intervalUnit.getIntervalMs(interval); + startTimeMs = System.currentTimeMillis() + intervalUnit.getIntervalMs(interval); } if (null != endTimeMs && endTimeMs < startTimeMs) { throw new IllegalArgumentException("end time cannot be less than start time"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java index 70a558009dbf58..eb5e9499e38d4e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java @@ -17,53 +17,93 @@ package org.apache.doris.job.extensions.insert; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.io.Text; import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.common.JobType; import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; +import org.apache.doris.load.loadv2.LoadJob; +import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand; import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; import com.google.gson.annotations.SerializedName; import lombok.Data; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; -import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; @Data +@Slf4j public class InsertJob extends AbstractJob { - @SerializedName(value = "labelPrefix") + @SerializedName(value = "lp") String labelPrefix; + InsertIntoTableCommand command; + + StmtExecutor stmtExecutor; + + ConnectContext ctx; + + @SerializedName("tis") + ConcurrentLinkedQueue taskIdList; + + // max save task num, do we need to config it? + private static final int MAX_SAVE_TASK_NUM = 50; + @Override public List createTasks(TaskType taskType) { - InsertTask task = new InsertTask(null, null, null, null, null); + InsertTask task = new InsertTask(null, getCurrentDbName(), getExecuteSql(), getCreateUser()); task.setJobId(getJobId()); task.setTaskType(taskType); task.setTaskId(Env.getCurrentEnv().getNextId()); ArrayList tasks = new ArrayList<>(); tasks.add(task); super.initTasks(tasks); - getRunningTasks().addAll(tasks); + addNewTask(task.getTaskId()); return tasks; } + public void addNewTask(long id) { + + if (CollectionUtils.isEmpty(taskIdList)) { + taskIdList = new ConcurrentLinkedQueue<>(); + Env.getCurrentEnv().getEditLog().logUpdateJob(this); + taskIdList.add(id); + return; + } + taskIdList.add(id); + if (taskIdList.size() >= MAX_SAVE_TASK_NUM) { + taskIdList.poll(); + } + Env.getCurrentEnv().getEditLog().logUpdateJob(this); + } + @Override - public void cancel(long taskId) throws JobException { - super.cancel(); + public void cancelTaskById(long taskId) throws JobException { + super.cancelTaskById(taskId); } @Override - public void cancel() throws JobException { - super.cancel(); + public void cancelAllTasks() throws JobException { + super.cancelAllTasks(); } @Override @@ -74,16 +114,41 @@ public boolean isReadyForScheduling() { @Override protected void checkJobParamsInternal() { - - } - - public static InsertJob readFields(DataInput in) throws IOException { - return GsonUtils.GSON.fromJson(Text.readString(in), InsertJob.class); + if (command == null && StringUtils.isBlank(getExecuteSql())) { + throw new IllegalArgumentException("command or sql is null,must be set"); + } + if (null != command && !getJobConfig().getExecuteType().equals(JobExecuteType.INSTANT)) { + throw new IllegalArgumentException("command must be null when executeType is not instant"); + } } @Override public List queryTasks() { - return null; + if (CollectionUtils.isEmpty(taskIdList)) { + return new ArrayList<>(); + } + //TODO it's will be refactor, we will storage task info in job inner and query from it + List taskIdList = new ArrayList<>(this.taskIdList); + Collections.reverse(taskIdList); + List loadJobs = Env.getCurrentEnv().getLoadManager().queryLoadJobsByJobIds(taskIdList); + if (CollectionUtils.isEmpty(loadJobs)) { + return new ArrayList<>(); + } + List tasks = new ArrayList<>(); + loadJobs.forEach(loadJob -> { + InsertTask task; + try { + task = new InsertTask(loadJob.getLabel(), loadJob.getDb().getFullName(), null, getCreateUser()); + } catch (MetaNotFoundException e) { + log.warn("load job not found,job id is {}", loadJob.getId()); + return; + } + task.setJobId(getJobId()); + task.setTaskId(loadJob.getId()); + task.setLoadJob(loadJob); + tasks.add(task); + }); + return tasks; } @Override @@ -93,12 +158,12 @@ public JobType getJobType() { @Override public ShowResultSetMetaData getJobMetaData() { - return null; + return super.getJobMetaData(); } @Override public ShowResultSetMetaData getTaskMetaData() { - return null; + return TASK_META_DATA; } @Override @@ -108,18 +173,32 @@ public void onTaskFail(InsertTask task) { @Override public void onTaskSuccess(InsertTask task) { - getRunningTasks().remove(task); + super.onTaskSuccess(task); } @Override - public void onTaskCancel(InsertTask task) { - getRunningTasks().remove(task); + public List getShowInfo() { + return super.getCommonShowInfo(); } - @Override public void write(DataOutput out) throws IOException { - Text.writeString(out, JobType.INSERT.name()); Text.writeString(out, GsonUtils.GSON.toJson(this)); } + + private static final ShowResultSetMetaData TASK_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("TaskId", ScalarType.createVarchar(20))) + .addColumn(new Column("Label", ScalarType.createVarchar(20))) + .addColumn(new Column("Status", ScalarType.createVarchar(20))) + .addColumn(new Column("EtlInfo", ScalarType.createVarchar(20))) + .addColumn(new Column("TaskInfo", ScalarType.createVarchar(20))) + .addColumn(new Column("ErrorMsg", ScalarType.createVarchar(20))) + + .addColumn(new Column("CreateTimeMs", ScalarType.createVarchar(20))) + .addColumn(new Column("FinishTimeMs", ScalarType.createVarchar(20))) + .addColumn(new Column("TrackingUrl", ScalarType.createVarchar(20))) + .addColumn(new Column("LoadStatistic", ScalarType.createVarchar(20))) + .addColumn(new Column("User", ScalarType.createVarchar(20))) + .build(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java index 9802ebc55d0d4f..38b3969d7d8c10 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -17,14 +17,32 @@ package org.apache.doris.job.extensions.insert; +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.job.exception.JobException; import org.apache.doris.job.task.AbstractTask; -import org.apache.doris.load.FailMsg; import org.apache.doris.load.loadv2.LoadJob; +import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TUniqueId; +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import lombok.Getter; +import lombok.Setter; import lombok.extern.slf4j.Slf4j; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; + /** * todo implement this later */ @@ -34,44 +52,140 @@ public class InsertTask extends AbstractTask { private String labelName; private InsertIntoTableCommand command; - private LoadJob.LoadStatistic statistic; - private FailMsg failMsg; - private InsertIntoState insertIntoState; + private StmtExecutor stmtExecutor; + + private ConnectContext ctx; + + private String sql; + + private String currentDb; + + private UserIdentity userIdentity; + + private AtomicBoolean isCanceled = new AtomicBoolean(false); + + private AtomicBoolean isFinished = new AtomicBoolean(false); + + + @Getter + @Setter + private LoadJob loadJob; + @Override - public void before() { + public void before() throws JobException { + if (isCanceled.get()) { + throw new JobException("Export executor has been canceled, task id: {}", getTaskId()); + } + ctx = new ConnectContext(); + ctx.setEnv(Env.getCurrentEnv()); + ctx.setCluster(SystemInfoService.DEFAULT_CLUSTER); + ctx.setQualifiedUser(userIdentity.getQualifiedUser()); + ctx.setCurrentUserIdentity(userIdentity); + ctx.getState().reset(); + ctx.setThreadLocalInfo(); + ctx.setDatabase(currentDb); + TUniqueId queryId = generateQueryId(UUID.randomUUID().toString()); + ctx.getSessionVariable().enableFallbackToOriginalPlanner = false; + stmtExecutor = new StmtExecutor(ctx, (String) null); + ctx.setQueryId(queryId); + NereidsParser parser = new NereidsParser(); + this.command = (InsertIntoTableCommand) parser.parseSingle(sql); + this.command.setLabelName(Optional.of(getJobId() + "_" + getTaskId())); + this.command.setJobId(getTaskId()); + super.before(); + + } + + protected TUniqueId generateQueryId(String taskIdString) { + UUID taskId = UUID.fromString(taskIdString); + return new TUniqueId(taskId.getMostSignificantBits(), taskId.getLeastSignificantBits()); } - public InsertTask(String labelName, InsertIntoTableCommand command, LoadJob.LoadStatistic statistic, - FailMsg failMsg, InsertIntoState insertIntoState) { + public InsertTask(String labelName, String currentDb, String sql, UserIdentity userIdentity) { this.labelName = labelName; - this.command = command; - this.statistic = statistic; - this.failMsg = failMsg; - this.insertIntoState = insertIntoState; + this.sql = sql; + this.currentDb = currentDb; + this.userIdentity = userIdentity; + } @Override - public void run() { - //just for test - log.info(getJobId() + "InsertTask run" + TimeUtils.longToTimeString(System.currentTimeMillis())); + public void run() throws JobException { + try { + command.run(ctx, stmtExecutor); + } catch (Exception e) { + throw new JobException(e); + } } @Override - public void onFail() { + public void onFail() throws JobException { + isFinished.set(true); super.onFail(); } @Override - public void onSuccess() { + public void onSuccess() throws JobException { + isFinished.set(true); super.onSuccess(); } @Override - public void cancel() { + public void cancel() throws JobException { + if (isFinished.get() || isCanceled.get()) { + return; + } + isCanceled.getAndSet(true); + if (null != stmtExecutor) { + stmtExecutor.cancel(); + } super.cancel(); } + @Override + public List getShowInfo() { + if (null == loadJob) { + return new ArrayList<>(); + } + List jobInfo = Lists.newArrayList(); + // jobId + jobInfo.add(String.valueOf(loadJob.getId())); + // label + jobInfo.add(loadJob.getLabel()); + // state + jobInfo.add(loadJob.getState().name()); + + // etl info + if (loadJob.getLoadingStatus().getCounters().isEmpty()) { + jobInfo.add(FeConstants.null_string); + } else { + jobInfo.add(Joiner.on("; ").withKeyValueSeparator("=").join(loadJob.getLoadingStatus().getCounters())); + } + + // task info + jobInfo.add("cluster:" + loadJob.getResourceName() + "; timeout(s):" + loadJob.getTimeout() + + "; max_filter_ratio:" + loadJob.getMaxFilterRatio() + "; priority:" + loadJob.getPriority()); + // error msg + if (loadJob.getFailMsg() == null) { + jobInfo.add(FeConstants.null_string); + } else { + jobInfo.add("type:" + loadJob.getFailMsg().getCancelType() + "; msg:" + loadJob.getFailMsg().getMsg()); + } + + // create time + jobInfo.add(TimeUtils.longToTimeString(loadJob.getCreateTimestamp())); + + // load end time + jobInfo.add(TimeUtils.longToTimeString(loadJob.getFinishTimestamp())); + // tracking url + jobInfo.add(loadJob.getLoadingStatus().getTrackingUrl()); + jobInfo.add(loadJob.getLoadStatistic().toJson()); + // user + jobInfo.add(loadJob.getUserInfo().getQualifiedUser()); + return jobInfo; + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVJob.java new file mode 100644 index 00000000000000..c08a96ee6eb39f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVJob.java @@ -0,0 +1,152 @@ +// 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.job.extensions.mtmv; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.catalog.TableIf.TableType; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.common.JobType; +import org.apache.doris.job.common.TaskType; +import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.qe.ShowResultSetMetaData; + +import com.google.common.collect.Lists; +import com.google.gson.annotations.SerializedName; +import org.apache.commons.collections.CollectionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class MTMVJob extends AbstractJob { + private static final Logger LOG = LogManager.getLogger(MTMVJob.class); + private static final ShowResultSetMetaData JOB_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("JobId", ScalarType.createVarchar(20))) + .addColumn(new Column("JobName", ScalarType.createVarchar(20))) + .addColumn(new Column("ExecuteType", ScalarType.createVarchar(20))) + .addColumn(new Column("RecurringStrategy", ScalarType.createVarchar(20))) + .addColumn(new Column("JobStatus", ScalarType.createVarchar(20))) + .addColumn(new Column("CreateTime", ScalarType.createVarchar(20))) + .addColumn(new Column("Comment", ScalarType.createVarchar(20))) + .build(); + private static final ShowResultSetMetaData TASK_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("JobId", ScalarType.createVarchar(20))) + .addColumn(new Column("TaskId", ScalarType.createVarchar(20))) + .addColumn(new Column("Status", ScalarType.createVarchar(20))) + .addColumn(new Column("CreateTime", ScalarType.createVarchar(20))) + .addColumn(new Column("StartTime", ScalarType.createVarchar(20))) + .addColumn(new Column("FinishTime", ScalarType.createVarchar(20))) + .addColumn(new Column("DurationMs", ScalarType.createVarchar(20))) + .addColumn(new Column("ExecuteSql", ScalarType.createVarchar(20))) + .build(); + + @SerializedName(value = "di") + private long dbId; + @SerializedName(value = "mi") + private long mtmvId; + + public MTMVJob(long dbId, long mtmvId) { + this.dbId = dbId; + this.mtmvId = mtmvId; + super.setCreateTimeMs(System.currentTimeMillis()); + } + + @Override + protected void checkJobParamsInternal() { + + } + + @Override + public List createTasks(TaskType taskType) { + MTMVTask task = new MTMVTask(dbId, mtmvId); + task.setTaskType(taskType); + ArrayList tasks = new ArrayList<>(); + tasks.add(task); + super.initTasks(tasks); + return tasks; + } + + @Override + public boolean isReadyForScheduling() { + return CollectionUtils.isEmpty(getRunningTasks()); + } + + @Override + public ShowResultSetMetaData getJobMetaData() { + return JOB_META_DATA; + } + + @Override + public ShowResultSetMetaData getTaskMetaData() { + return TASK_META_DATA; + } + + @Override + public JobType getJobType() { + return JobType.MTMV; + } + + @Override + public List queryTasks() { + MTMV mtmv = null; + try { + mtmv = getMTMV(); + } catch (DdlException | MetaNotFoundException e) { + LOG.warn("get mtmv failed", e); + return Lists.newArrayList(); + } + return mtmv.getJobInfo().getHistoryTasks(); + } + + @Override + public List getShowInfo() { + List data = Lists.newArrayList(); + data.add(String.valueOf(super.getJobId())); + data.add(super.getJobName()); + data.add(super.getJobConfig().getExecuteType().name()); + data.add(super.getJobConfig().convertRecurringStrategyToString()); + data.add(super.getJobStatus().name()); + data.add(TimeUtils.longToTimeString(super.getCreateTimeMs())); + data.add(super.getComment()); + return data; + } + + private MTMV getMTMV() throws DdlException, MetaNotFoundException { + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbId); + return (MTMV) db.getTableOrMetaException(mtmvId, TableType.MATERIALIZED_VIEW); + } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java new file mode 100644 index 00000000000000..f4d922442891f5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -0,0 +1,169 @@ +// 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.job.extensions.mtmv; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.TableIf.TableType; +import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.mtmv.MTMVCacheManager; +import org.apache.doris.mtmv.MTMVRelation; +import org.apache.doris.mysql.privilege.Auth; +import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.Lists; +import com.google.gson.annotations.SerializedName; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.UUID; + +public class MTMVTask extends AbstractTask { + private static final Logger LOG = LogManager.getLogger(MTMVTask.class); + public static final Long MAX_HISTORY_TASKS_NUM = 100L; + + @SerializedName(value = "di") + private long dbId; + @SerializedName(value = "mi") + private long mtmvId; + @SerializedName("sql") + private String sql; + + private MTMV mtmv; + private MTMVRelation relation; + + public MTMVTask(long dbId, long mtmvId) { + this.dbId = dbId; + this.mtmvId = mtmvId; + } + + @Override + public void run() throws JobException { + try { + ConnectContext ctx = createContext(); + TUniqueId queryId = generateQueryId(); + // Every time a task is run, the relation is regenerated because baseTables and baseViews may change, + // such as deleting a table and creating a view with the same name + relation = MTMVCacheManager.generateMTMVRelation(mtmv, ctx); + StmtExecutor executor = new StmtExecutor(ctx, sql); + executor.execute(queryId); + } catch (Throwable e) { + LOG.warn(e); + throw new JobException(e); + } + } + + @Override + public void onFail() throws JobException { + super.onFail(); + after(); + } + + @Override + public void onSuccess() throws JobException { + super.onSuccess(); + after(); + } + + @Override + public void cancel() throws JobException { + super.cancel(); + after(); + } + + @Override + public void before() throws JobException { + super.before(); + try { + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbId); + mtmv = (MTMV) db.getTableOrMetaException(mtmvId, TableType.MATERIALIZED_VIEW); + sql = generateSql(mtmv); + } catch (UserException e) { + LOG.warn(e); + throw new JobException(e); + } + } + + @Override + public List getShowInfo() { + List data = Lists.newArrayList(); + data.add(super.getJobId() + ""); + data.add(super.getTaskId() + ""); + data.add(super.getStatus() + ""); + data.add(TimeUtils.longToTimeString(super.getCreateTimeMs())); + data.add(TimeUtils.longToTimeString(super.getStartTimeMs())); + data.add(TimeUtils.longToTimeString(super.getFinishTimeMs())); + data.add(String.valueOf(super.getFinishTimeMs() - super.getStartTimeMs())); + data.add(sql); + return data; + } + + private static String generateSql(MTMV mtmv) { + StringBuilder builder = new StringBuilder(); + builder.append("INSERT OVERWRITE TABLE "); + builder.append(mtmv.getDatabase().getCatalog().getName()); + builder.append("."); + builder.append(ClusterNamespace.getNameFromFullName(mtmv.getQualifiedDbName())); + builder.append("."); + builder.append(mtmv.getName()); + builder.append(" "); + builder.append(mtmv.getQuerySql()); + return builder.toString(); + } + + private ConnectContext createContext() throws AnalysisException { + ConnectContext ctx = new ConnectContext(); + ctx.setEnv(Env.getCurrentEnv()); + ctx.setCluster(SystemInfoService.DEFAULT_CLUSTER); + ctx.setQualifiedUser(Auth.ADMIN_USER); + ctx.setCurrentUserIdentity(UserIdentity.ADMIN); + ctx.getState().reset(); + ctx.setThreadLocalInfo(); + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr() + .getCatalogOrAnalysisException(mtmv.getEnvInfo().getCtlId()); + ctx.changeDefaultCatalog(catalog.getName()); + ctx.setDatabase(catalog.getDbOrAnalysisException(mtmv.getEnvInfo().getDbId()).getFullName()); + ctx.getSessionVariable().enableFallbackToOriginalPlanner = false; + return ctx; + } + + private TUniqueId generateQueryId() { + UUID taskId = UUID.randomUUID(); + return new TUniqueId(taskId.getMostSignificantBits(), taskId.getLeastSignificantBits()); + } + + private void after() { + Env.getCurrentEnv() + .addMTMVTaskResult(new TableNameInfo(mtmv.getQualifiedDbName(), mtmv.getName()), this, relation); + mtmv = null; + relation = null; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index 0df87a845173f2..c7d04cdd282129 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -17,6 +17,7 @@ package org.apache.doris.job.manager; +import org.apache.doris.catalog.Env; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.LogBuilder; import org.apache.doris.common.util.LogKey; @@ -53,21 +54,19 @@ public void start() { public void registerJob(T job) throws JobException { job.checkJobParams(); - checkJobNameExist(job.getJobName(), job.getJobType(), job.getCurrentDbName()); + checkJobNameExist(job.getJobName(), job.getJobType()); if (jobMap.get(job.getJobId()) != null) { throw new JobException("job id exist,jobId:" + job.getJobId()); } - //Env.getCurrentEnv().getEditLog().logCreateJob(job); - //check name exist + Env.getCurrentEnv().getEditLog().logCreateJob(job); jobMap.put(job.getJobId(), job); //check its need to scheduler jobScheduler.scheduleOneJob(job); } - private void checkJobNameExist(String jobName, JobType type, String currentDbName) throws JobException { - if (jobMap.values().stream().anyMatch(a -> a.getJobName().equals(jobName) && a.getJobType().equals(type) - && (null == a.getCurrentDbName() || a.getCurrentDbName().equals(currentDbName)))) { + private void checkJobNameExist(String jobName, JobType type) throws JobException { + if (jobMap.values().stream().anyMatch(a -> a.getJobName().equals(jobName) && a.getJobType().equals(type))) { throw new JobException("job name exist,jobName:" + jobName); } } @@ -75,15 +74,14 @@ private void checkJobNameExist(String jobName, JobType type, String currentDbNam public void unregisterJob(Long jobId) throws JobException { checkJobExist(jobId); jobMap.get(jobId).setJobStatus(JobStatus.STOPPED); - jobMap.get(jobId).cancel(); - //Env.getCurrentEnv().getEditLog().logDeleteJob(jobMap.get(jobId)); + jobMap.get(jobId).cancelAllTasks(); + Env.getCurrentEnv().getEditLog().logDeleteJob(jobMap.get(jobId)); jobMap.remove(jobId); } - public void unregisterJob(String currentDbName, String jobName) throws JobException { + public void unregisterJob(String jobName, JobType jobType) throws JobException { for (T a : jobMap.values()) { - if (a.getJobName().equals(jobName) && (null != a.getCurrentDbName() - && a.getCurrentDbName().equals(currentDbName)) && a.getJobType().equals(JobType.INSERT)) { + if (a.getJobName().equals(jobName) && a.getJobType().equals(jobType)) { try { unregisterJob(a.getJobId()); } catch (JobException e) { @@ -97,13 +95,12 @@ public void unregisterJob(String currentDbName, String jobName) throws JobExcept public void alterJobStatus(Long jobId, JobStatus status) throws JobException { checkJobExist(jobId); jobMap.get(jobId).updateJobStatus(status); - //Env.getCurrentEnv().getEditLog().logUpdateJob(jobMap.get(jobId)); + Env.getCurrentEnv().getEditLog().logUpdateJob(jobMap.get(jobId)); } - public void alterJobStatus(String currentDbName, String jobName, JobStatus jobStatus) throws JobException { + public void alterJobStatus(String jobName, JobStatus jobStatus, JobType jobType) throws JobException { for (T a : jobMap.values()) { - if (a.getJobName().equals(jobName) && (null != a.getCurrentDbName() - && a.getCurrentDbName().equals(currentDbName)) && JobType.INSERT.equals(a.getJobType())) { + if (a.getJobName().equals(jobName) && jobType.equals(a.getJobType())) { try { alterJobStatus(a.getJobId(), jobStatus); } catch (JobException e) { @@ -124,19 +121,37 @@ public List queryJobs(JobType type) { .collect(java.util.stream.Collectors.toList()); } - public List queryJobs(String currentDb, String jobName) { + public List queryJobs(JobType jobType, String jobName) { //only query insert job,we just provide insert job - return jobMap.values().stream().filter(a -> checkItsMatch(currentDb, jobName, a)) + return jobMap.values().stream().filter(a -> checkItsMatch(jobType, jobName, a)) .collect(Collectors.toList()); } - private boolean checkItsMatch(String currentDb, String jobName, T job) { + /** + * query jobs by job type + * @param jobTypes @JobType + * @return List job list + */ + public List queryJobs(List jobTypes) { + return jobMap.values().stream().filter(a -> checkItsMatch(jobTypes, a)) + .collect(Collectors.toList()); + } + + private boolean checkItsMatch(JobType jobType, String jobName, T job) { + if (null == jobType) { + throw new IllegalArgumentException("jobType cannot be null"); + } if (StringUtils.isBlank(jobName)) { - return job.getJobType().equals(JobType.INSERT) && (null != job.getCurrentDbName() - && job.getCurrentDbName().equals(currentDb)); + return job.getJobType().equals(jobType); + } + return job.getJobType().equals(jobType) && job.getJobName().equals(jobName); + } + + private boolean checkItsMatch(List jobTypes, T job) { + if (null == jobTypes) { + throw new IllegalArgumentException("jobType cannot be null"); } - return job.getJobType().equals(JobType.INSERT) && (null != job.getCurrentDbName() - && job.getCurrentDbName().equals(currentDb)) && job.getJobName().equals(jobName); + return jobTypes.contains(job.getJobType()); } public List queryTasks(Long jobId) throws JobException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java index 1becb12d70b2ca..97a6f94099ffdb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java @@ -28,6 +28,7 @@ import org.apache.doris.job.executor.DefaultTaskExecutorHandler; import org.apache.doris.job.executor.DispatchTaskHandler; import org.apache.doris.job.extensions.insert.InsertTask; +import org.apache.doris.job.extensions.mtmv.MTMVTask; import org.apache.doris.job.task.AbstractTask; import com.lmax.disruptor.BlockingWaitStrategy; @@ -58,13 +59,18 @@ public class TaskDisruptorGroupManager { ? Config.job_dispatch_timer_job_thread_num : DEFAULT_CONSUMER_THREAD_NUM; private static final int DISPATCH_INSERT_THREAD_NUM = Config.job_insert_task_consumer_thread_num > 0 - ? Config.job_insert_task_consumer_thread_num : DEFAULT_RING_BUFFER_SIZE; + ? Config.job_insert_task_consumer_thread_num : DEFAULT_CONSUMER_THREAD_NUM; + + private static final int DISPATCH_MTMV_THREAD_NUM = Config.job_mtmv_task_consumer_thread_num > 0 + ? Config.job_mtmv_task_consumer_thread_num : DEFAULT_CONSUMER_THREAD_NUM; private static final int DISPATCH_INSERT_TASK_QUEUE_SIZE = DEFAULT_RING_BUFFER_SIZE; + private static final int DISPATCH_MTMV_TASK_QUEUE_SIZE = DEFAULT_RING_BUFFER_SIZE; public void init() { registerInsertDisruptor(); + registerMTMVDisruptor(); //when all task queue is ready, dispatch task to registered task executor registerDispatchDisruptor(); } @@ -100,6 +106,23 @@ private void registerInsertDisruptor() { disruptorMap.put(JobType.INSERT, insertDisruptor); } + private void registerMTMVDisruptor() { + EventFactory> mtmvEventFactory = ExecuteTaskEvent.factory(); + ThreadFactory mtmvTaskThreadFactory = new CustomThreadFactory("mtmv-task-execute"); + WorkHandler[] insertTaskExecutorHandlers = new WorkHandler[DISPATCH_MTMV_THREAD_NUM]; + for (int i = 0; i < DISPATCH_MTMV_THREAD_NUM; i++) { + insertTaskExecutorHandlers[i] = new DefaultTaskExecutorHandler(); + } + EventTranslatorVararg> eventTranslator = + (event, sequence, args) -> { + event.setTask((MTMVTask) args[0]); + event.setJobConfig((JobExecutionConfiguration) args[1]); + }; + TaskDisruptor mtmvDisruptor = new TaskDisruptor<>(mtmvEventFactory, DISPATCH_MTMV_TASK_QUEUE_SIZE, + mtmvTaskThreadFactory, new BlockingWaitStrategy(), insertTaskExecutorHandlers, eventTranslator); + disruptorMap.put(JobType.MTMV, mtmvDisruptor); + } + public void dispatchTimerJob(AbstractJob job) { dispatchDisruptor.publishEvent(job); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java index a9cedc999475f4..c701c9fd1f851b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java @@ -53,7 +53,7 @@ public class JobScheduler> implements Closeable { private long latestBatchSchedulerTimerTaskTimeMs = 0L; - private static final long BATCH_SCHEDULER_INTERVAL_SECONDS = 60; + private static final long BATCH_SCHEDULER_INTERVAL_SECONDS = 600; private static final int HASHED_WHEEL_TIMER_TICKS_PER_WHEEL = 660; @@ -104,13 +104,22 @@ public void scheduleOneJob(T job) throws JobException { if (!job.getJobConfig().checkIsTimerJob()) { //manual job will not scheduler if (JobExecuteType.MANUAL.equals(job.getJobConfig().getExecuteType())) { + if (job.getJobConfig().isImmediate()) { + schedulerInstantJob(job, TaskType.MANUAL); + } return; } + //todo skip streaming job,improve in the future if (JobExecuteType.INSTANT.equals(job.getJobConfig().getExecuteType())) { schedulerInstantJob(job, TaskType.SCHEDULED); } } + //RECURRING job and immediate is true + if (job.getJobConfig().isImmediate()) { + job.getJobConfig().getTimerDefinition().setLatestSchedulerTimeMs(System.currentTimeMillis()); + schedulerInstantJob(job, TaskType.SCHEDULED); + } //if it's timer job and trigger last window already start, we will scheduler it immediately cycleTimerJobScheduler(job); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java index 7fc666339af895..4580456928f342 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java @@ -21,7 +21,9 @@ import org.apache.doris.job.base.Job; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.common.TaskType; +import org.apache.doris.job.exception.JobException; +import com.google.gson.annotations.SerializedName; import lombok.Data; import lombok.extern.slf4j.Slf4j; @@ -29,31 +31,38 @@ @Slf4j public abstract class AbstractTask implements Task { + @SerializedName(value = "jid") private Long jobId; - + @SerializedName(value = "tid") private Long taskId; + @SerializedName(value = "st") private TaskStatus status; - + @SerializedName(value = "ctm") private Long createTimeMs; - + @SerializedName(value = "stm") private Long startTimeMs; - + @SerializedName(value = "ftm") private Long finishTimeMs; + @SerializedName(value = "tt") private TaskType taskType; @Override public void onFail(String msg) { + status = TaskStatus.FAILD; if (!isCallable()) { return; } Env.getCurrentEnv().getJobManager().getJob(jobId).onTaskFail(this); - status = TaskStatus.FAILD; } @Override - public void onFail() { + public void onFail() throws JobException { + if (TaskStatus.CANCEL.equals(status)) { + return; + } + status = TaskStatus.FAILD; setFinishTimeMs(System.currentTimeMillis()); if (!isCallable()) { return; @@ -73,7 +82,7 @@ private boolean isCallable() { } @Override - public void onSuccess() { + public void onSuccess() throws JobException { status = TaskStatus.SUCCESS; setFinishTimeMs(System.currentTimeMillis()); if (!isCallable()) { @@ -88,17 +97,17 @@ public void onSuccess() { } @Override - public void cancel() { + public void cancel() throws JobException { status = TaskStatus.CANCEL; } @Override - public void before() { + public void before() throws JobException { status = TaskStatus.RUNNING; setStartTimeMs(System.currentTimeMillis()); } - public void runTask() { + public void runTask() throws JobException { try { before(); run(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/task/Task.java b/fe/fe-core/src/main/java/org/apache/doris/job/task/Task.java index 350c562f7e882f..3f61ce60c700f7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/task/Task.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/task/Task.java @@ -17,6 +17,10 @@ package org.apache.doris.job.task; +import org.apache.doris.job.exception.JobException; + +import java.util.List; + /** * The Task interface represents a task that can be executed and managed by a scheduler. * All extension tasks must implement this interface. @@ -29,19 +33,19 @@ public interface Task { * This method is called before the task is executed. * Implementations can use this method to perform any necessary setup or initialization. */ - void before(); + void before() throws JobException; /** * This method contains the main logic of the task. * Implementations should define the specific actions to be performed by the task. */ - void run(); + void run() throws JobException; /** * This method is called when the task fails to execute successfully. * Implementations can use this method to handle any failure scenarios. */ - void onFail(); + void onFail() throws JobException; /** * This method is called when the task fails to execute successfully, with an additional error message. @@ -55,12 +59,17 @@ public interface Task { * This method is called when the task executes successfully. * Implementations can use this method to handle successful execution scenarios. */ - void onSuccess(); + void onSuccess() throws JobException; /** * This method is called to cancel the execution of the task. * Implementations should define the necessary steps to cancel the task. */ - void cancel(); + void cancel() throws JobException; + /** + * get the job's show info, which is used to sql show the task information + * @return List task common show info + */ + List getShowInfo(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index 162e8b3cc882c4..b0749db1fe14ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -59,6 +59,7 @@ import org.apache.doris.mysql.privilege.UserPropertyInfo; import org.apache.doris.persist.AlterDatabasePropertyInfo; import org.apache.doris.persist.AlterLightSchemaChangeInfo; +import org.apache.doris.persist.AlterMTMV; import org.apache.doris.persist.AlterRoutineLoadJobOperationLog; import org.apache.doris.persist.AlterUserOperationLog; import org.apache.doris.persist.AlterViewInfo; @@ -870,6 +871,11 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } + case OperationType.OP_ALTER_MTMV: { + data = AlterMTMV.read(in); + isRead = true; + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/EtlJobType.java b/fe/fe-core/src/main/java/org/apache/doris/load/EtlJobType.java index d00dc21b846fa7..2770738d3d7553 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/EtlJobType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/EtlJobType.java @@ -25,5 +25,7 @@ public enum EtlJobType { DELETE, SPARK, LOCAL_FILE, + // create by job scheduler,inner use + INSERT_JOB, UNKNOWN } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java index 821878584921d6..fd416b8e449415 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java @@ -404,7 +404,7 @@ public void afterVisible(TransactionState txnState, boolean txnOperated) { } @Override - protected String getResourceName() { + public String getResourceName() { StorageBackend.StorageType storageType = brokerDesc.getStorageType(); if (storageType == StorageBackend.StorageType.BROKER) { return brokerDesc.getName(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/InsertLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/InsertLoadJob.java index 62b108d7f496b0..19814036ec40a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/InsertLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/InsertLoadJob.java @@ -73,6 +73,28 @@ public InsertLoadJob(String label, long transactionId, long dbId, long tableId, this.userInfo = userInfo; } + public InsertLoadJob(String label, long transactionId, long dbId, long tableId, + long createTimestamp, String failMsg, String trackingUrl, + UserIdentity userInfo, Long jobId) throws MetaNotFoundException { + super(EtlJobType.INSERT_JOB, dbId, label, jobId); + this.tableId = tableId; + this.transactionId = transactionId; + this.createTimestamp = createTimestamp; + this.loadStartTimestamp = createTimestamp; + this.finishTimestamp = System.currentTimeMillis(); + if (Strings.isNullOrEmpty(failMsg)) { + this.state = JobState.FINISHED; + this.progress = 100; + } else { + this.state = JobState.CANCELLED; + this.failMsg = new FailMsg(CancelType.LOAD_RUN_FAIL, failMsg); + this.progress = 0; + } + this.authorizationInfo = gatherAuthInfo(); + this.loadingStatus.setTrackingUrl(trackingUrl); + this.userInfo = userInfo; + } + public AuthorizationInfo gatherAuthInfo() throws MetaNotFoundException { Database database = Env.getCurrentInternalCatalog().getDbOrMetaException(dbId); return new AuthorizationInfo(database.getFullName(), getTableNames()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java index 2c70883d8f1dbe..bd8d09d5c608ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java @@ -253,6 +253,13 @@ public LoadJob(EtlJobType jobType, long dbId, String label) { this.label = label; } + public LoadJob(EtlJobType jobType, long dbId, String label, long jobId) { + this(jobType); + this.id = jobId; + this.dbId = dbId; + this.label = label; + } + protected void readLock() { lock.readLock().lock(); } @@ -837,7 +844,7 @@ public String errorTabletsToJson() { return gson.toJson(map); } - protected String getResourceName() { + public String getResourceName() { return "N/A"; } @@ -864,7 +871,7 @@ public static LoadJob read(DataInput in) throws IOException { job = new BrokerLoadJob(); } else if (type == EtlJobType.SPARK) { job = new SparkLoadJob(); - } else if (type == EtlJobType.INSERT) { + } else if (type == EtlJobType.INSERT || type == EtlJobType.INSERT_JOB) { job = new InsertLoadJob(); } else if (type == EtlJobType.MINI) { job = new MiniLoadJob(); @@ -1194,7 +1201,7 @@ public static LoadJobStateUpdateInfo read(DataInput in) throws IOException { } // unit: second - protected long getTimeout() { + public long getTimeout() { return (long) jobProperties.get(LoadStmt.TIMEOUT_PROPERTY); } @@ -1206,7 +1213,7 @@ protected long getExecMemLimit() { return (long) jobProperties.get(LoadStmt.EXEC_MEM_LIMIT); } - protected double getMaxFilterRatio() { + public double getMaxFilterRatio() { return (double) jobProperties.get(LoadStmt.MAX_FILTER_RATIO_PROPERTY); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java index 009c5144fad29e..3eacb9f66c492e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java @@ -56,6 +56,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -213,8 +214,8 @@ private void addLoadJob(LoadJob loadJob) { * Record finished load job by editLog. **/ public void recordFinishedLoadJob(String label, long transactionId, String dbName, long tableId, EtlJobType jobType, - long createTimestamp, String failMsg, String trackingUrl, - UserIdentity userInfo) throws MetaNotFoundException { + long createTimestamp, String failMsg, String trackingUrl, + UserIdentity userInfo, long jobId) throws MetaNotFoundException { // get db id Database db = Env.getCurrentInternalCatalog().getDbOrMetaException(dbName); @@ -225,6 +226,10 @@ public void recordFinishedLoadJob(String label, long transactionId, String dbNam loadJob = new InsertLoadJob(label, transactionId, db.getId(), tableId, createTimestamp, failMsg, trackingUrl, userInfo); break; + case INSERT_JOB: + loadJob = new InsertLoadJob(label, transactionId, db.getId(), tableId, createTimestamp, failMsg, + trackingUrl, userInfo, jobId); + break; default: return; } @@ -504,16 +509,16 @@ public List> getCreateLoadStmt(long dbId, String label) throw /** * This method will return the jobs info which can meet the condition of input param. * - * @param dbId used to filter jobs which belong to this db - * @param labelValue used to filter jobs which's label is or like labelValue. + * @param dbId used to filter jobs which belong to this db + * @param labelValue used to filter jobs which's label is or like labelValue. * @param accurateMatch true: filter jobs which's label is labelValue. false: filter jobs which's label like itself. - * @param statesValue used to filter jobs which's state within the statesValue set. + * @param statesValue used to filter jobs which's state within the statesValue set. * @return The result is the list of jobInfo. * JobInfo is a list which includes the comparable object: jobId, label, state etc. * The result is unordered. */ public List> getLoadJobInfosByDb(long dbId, String labelValue, boolean accurateMatch, - Set statesValue) throws AnalysisException { + Set statesValue) throws AnalysisException { LinkedList> loadJobInfos = new LinkedList>(); if (!dbIdToLabelToLoadJobs.containsKey(dbId)) { return loadJobInfos; @@ -641,6 +646,19 @@ public LoadJob getLoadJob(long jobId) { return idToLoadJob.get(jobId); } + public List queryLoadJobsByJobIds(List jobIds) { + if (CollectionUtils.isEmpty(jobIds)) { + return new ArrayList<>(); + } + List jobs = new ArrayList<>(); + jobIds.forEach(id -> { + if (null != idToLoadJob.get(id)) { + jobs.add(idToLoadJob.get(id)); + } + }); + return jobs; + } + public void prepareJobs() { analyzeLoadJobs(); submitJobs(); @@ -797,7 +815,7 @@ private void writeUnlock() { * Init. **/ public void initJobProgress(Long jobId, TUniqueId loadId, Set fragmentIds, - List relatedBackendIds) { + List relatedBackendIds) { LoadJob job = idToLoadJob.get(jobId); if (job != null) { job.initLoadProgress(loadId, fragmentIds, relatedBackendIds); @@ -808,7 +826,7 @@ public void initJobProgress(Long jobId, TUniqueId loadId, Set fragmen * Update. **/ public void updateJobProgress(Long jobId, Long beId, TUniqueId loadId, TUniqueId fragmentId, long scannedRows, - long scannedBytes, boolean isDone) { + long scannedBytes, boolean isDone) { LoadJob job = idToLoadJob.get(jobId); if (job != null) { job.updateProgress(beId, loadId, fragmentId, scannedRows, scannedBytes, isDone); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java index 09a85d3dffe692..69e1420ff34dc5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java @@ -743,7 +743,7 @@ public void cancelJob(FailMsg failMsg) throws DdlException { } @Override - protected String getResourceName() { + public String getResourceName() { return sparkResource.getName(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java new file mode 100644 index 00000000000000..575552cb61eea2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java @@ -0,0 +1,91 @@ +// 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.mtmv; + +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.InternalCatalog; + +import com.google.common.base.Objects; +import com.google.gson.annotations.SerializedName; + +public class BaseTableInfo { + @SerializedName("ti") + private Long tableId; + @SerializedName("di") + private Long dbId; + @SerializedName("ci") + private Long ctlId; + + public BaseTableInfo(Long tableId, Long dbId) { + this.tableId = java.util.Objects.requireNonNull(tableId, "tableId is null"); + this.dbId = java.util.Objects.requireNonNull(dbId, "dbId is null"); + this.ctlId = InternalCatalog.INTERNAL_CATALOG_ID; + } + + public BaseTableInfo(TableIf table) { + DatabaseIf database = table.getDatabase(); + java.util.Objects.requireNonNull(database, "database is null"); + CatalogIf catalog = database.getCatalog(); + java.util.Objects.requireNonNull(database, "catalog is null"); + this.tableId = table.getId(); + this.dbId = database.getId(); + this.ctlId = catalog.getId(); + } + + public Long getTableId() { + return tableId; + } + + public Long getDbId() { + return dbId; + } + + public Long getCtlId() { + return ctlId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BaseTableInfo that = (BaseTableInfo) o; + return Objects.equal(tableId, that.tableId) + && Objects.equal(dbId, that.dbId) + && Objects.equal(ctlId, that.ctlId); + } + + @Override + public int hashCode() { + return Objects.hashCode(tableId, dbId, ctlId); + } + + @Override + public String toString() { + return "BaseTableInfo{" + + "tableId=" + tableId + + ", dbId=" + dbId + + ", ctlId=" + ctlId + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/EnvInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/EnvInfo.java index c5c92a15db90ce..97ad491cb317b7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/EnvInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/EnvInfo.java @@ -23,21 +23,29 @@ * EnvInfo */ public class EnvInfo { - @SerializedName("cn") - private String ctlName; - @SerializedName("dn") - private String dbName; + @SerializedName("ci") + private long ctlId; + @SerializedName("di") + private long dbId; - public EnvInfo(String ctlName, String dbName) { - this.ctlName = ctlName; - this.dbName = dbName; + public EnvInfo(long ctlId, long dbId) { + this.ctlId = ctlId; + this.dbId = dbId; } - public String getCtlName() { - return ctlName; + public long getCtlId() { + return ctlId; } - public String getDbName() { - return dbName; + public long getDbId() { + return dbId; + } + + @Override + public String toString() { + return "EnvInfo{" + + "ctlId='" + ctlId + '\'' + + ", dbId='" + dbId + '\'' + + '}'; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAlterOpType.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAlterOpType.java new file mode 100644 index 00000000000000..99f056d4224ba7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAlterOpType.java @@ -0,0 +1,25 @@ +// 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.mtmv; + +public enum MTMVAlterOpType { + ALTER_REFRESH_INFO, + ALTER_STATUS, + ALTER_PROPERTY, + ADD_TASK; +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCacheManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCacheManager.java new file mode 100644 index 00000000000000..847f85b9074995 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCacheManager.java @@ -0,0 +1,312 @@ +// 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.mtmv; + +import org.apache.doris.analysis.StatementBase; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.TableIf.TableType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.job.common.TaskStatus; +import org.apache.doris.job.extensions.mtmv.MTMVTask; +import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVRefreshState; +import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVState; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.exceptions.ParseException; +import org.apache.doris.nereids.glue.LogicalPlanAdapter; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; +import org.apache.doris.nereids.trees.plans.commands.info.RefreshMTMVInfo; +import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.visitor.TableCollector; +import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; +import org.apache.doris.persist.AlterMTMV; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import org.apache.commons.collections.CollectionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * when do some operation, do something about cache + */ +public class MTMVCacheManager implements MTMVHookService { + private static final Logger LOG = LogManager.getLogger(MTMVCacheManager.class); + private Map> tableMTMVs = Maps.newConcurrentMap(); + + public Set getMtmvsByBaseTable(BaseTableInfo table) { + return tableMTMVs.get(table); + } + + public boolean isAvailableMTMV(MTMV mtmv, ConnectContext ctx) throws AnalysisException, DdlException { + // check session variable if enable rewrite + if (!ctx.getSessionVariable().isEnableMvRewrite()) { + return false; + } + MTMVRelation mtmvRelation = mtmv.getRelation(); + if (mtmvRelation == null) { + return false; + } + // chaek mv is normal + if (!(mtmv.getStatus().getState() == MTMVState.NORMAL + && mtmv.getStatus().getRefreshState() == MTMVRefreshState.SUCCESS)) { + return false; + } + // check external table + boolean containsExternalTable = containsExternalTable(mtmvRelation.getBaseTables()); + if (containsExternalTable) { + return ctx.getSessionVariable().isEnableExternalMvRewrite(); + } + // check gracePeriod + Long gracePeriod = mtmv.getGracePeriod(); + // do not care data is delayed + if (gracePeriod < 0) { + return true; + } + // compare with base table + Long mtmvLastTime = getTableLastVisibleVersionTime(mtmv); + Long maxAvailableTime = mtmvLastTime + gracePeriod; + for (BaseTableInfo baseTableInfo : mtmvRelation.getBaseTables()) { + long tableLastVisibleVersionTime = getTableLastVisibleVersionTime(baseTableInfo); + if (tableLastVisibleVersionTime > maxAvailableTime) { + return false; + } + } + return true; + } + + private long getTableLastVisibleVersionTime(BaseTableInfo baseTableInfo) throws AnalysisException, DdlException { + Table table = Env.getCurrentEnv().getInternalCatalog() + .getDbOrAnalysisException(baseTableInfo.getDbId()) + .getTableOrDdlException(baseTableInfo.getTableId(), TableType.OLAP); + return getTableLastVisibleVersionTime((OlapTable) table); + } + + private long getTableLastVisibleVersionTime(OlapTable table) { + long result = 0L; + long visibleVersionTime; + for (Partition partition : table.getAllPartitions()) { + visibleVersionTime = partition.getVisibleVersionTime(); + if (visibleVersionTime > result) { + result = visibleVersionTime; + } + } + return result; + } + + private boolean containsExternalTable(Set baseTableInfos) { + for (BaseTableInfo baseTableInfo : baseTableInfos) { + if (InternalCatalog.INTERNAL_CATALOG_ID != baseTableInfo.getCtlId()) { + return true; + } + } + return false; + } + + public static MTMVRelation generateMTMVRelation(MTMV mtmv, ConnectContext ctx) { + Plan plan = getPlanBySql(mtmv.getQuerySql(), ctx); + return new MTMVRelation(getBaseTables(plan), getBaseViews(plan)); + } + + private static Set getBaseTables(Plan plan) { + TableCollectorContext collectorContext = + new TableCollector.TableCollectorContext( + Sets.newHashSet(TableType.MATERIALIZED_VIEW, TableType.OLAP)); + plan.accept(TableCollector.INSTANCE, collectorContext); + List collectedTables = collectorContext.getCollectedTables(); + return transferTableIfToInfo(collectedTables); + } + + private static Set getBaseViews(Plan plan) { + TableCollectorContext collectorContext = + new TableCollector.TableCollectorContext( + Sets.newHashSet(TableType.VIEW)); + plan.accept(TableCollector.INSTANCE, collectorContext); + List collectedTables = collectorContext.getCollectedTables(); + return transferTableIfToInfo(collectedTables); + } + + private static Set transferTableIfToInfo(List tables) { + Set result = Sets.newHashSet(); + for (TableIf table : tables) { + result.add(new BaseTableInfo(table)); + } + return result; + } + + private static Plan getPlanBySql(String querySql, ConnectContext ctx) { + List statements; + try { + statements = new NereidsParser().parseSQL(querySql); + } catch (Exception e) { + throw new ParseException("Nereids parse failed. " + e.getMessage()); + } + StatementBase parsedStmt = statements.get(0); + LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStmt).getLogicalPlan(); + NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); + return planner.plan(logicalPlan, PhysicalProperties.ANY, ExplainLevel.NONE); + } + + private Set getOrCreateMTMVs(BaseTableInfo baseTableInfo) { + if (!tableMTMVs.containsKey(baseTableInfo)) { + tableMTMVs.put(baseTableInfo, Sets.newConcurrentHashSet()); + } + return tableMTMVs.get(baseTableInfo); + } + + private void refreshMTMVCache(MTMVRelation relation, BaseTableInfo mtmvInfo) { + LOG.info("refreshMTMVCache,relation: {}, mtmvInfo: {}", relation, mtmvInfo); + removeMTMV(mtmvInfo); + addMTMV(relation, mtmvInfo); + } + + private void addMTMV(MTMVRelation relation, BaseTableInfo mtmvInfo) { + if (relation == null) { + return; + } + addMTMVTables(relation.getBaseTables(), mtmvInfo); + addMTMVTables(relation.getBaseViews(), mtmvInfo); + } + + private void addMTMVTables(Set baseTables, BaseTableInfo mtmvInfo) { + if (CollectionUtils.isEmpty(baseTables)) { + return; + } + for (BaseTableInfo baseTableInfo : baseTables) { + getOrCreateMTMVs(baseTableInfo).add(mtmvInfo); + } + } + + private void removeMTMV(BaseTableInfo mtmvInfo) { + for (Set sets : tableMTMVs.values()) { + sets.remove(mtmvInfo); + } + } + + @Override + public void createMTMV(MTMV mtmv) throws DdlException { + + } + + @Override + public void dropMTMV(MTMV mtmv) throws DdlException { + + } + + /** + * modify `tableMTMVs` by MTMVRelation + * @param mtmv + * @param dbId + */ + @Override + public void registerMTMV(MTMV mtmv, Long dbId) { + refreshMTMVCache(mtmv.getRelation(), new BaseTableInfo(mtmv.getId(), dbId)); + } + + /** + * remove cache of mtmv + * @param mtmv + */ + @Override + public void deregisterMTMV(MTMV mtmv) { + removeMTMV(new BaseTableInfo(mtmv)); + } + + @Override + public void alterMTMV(MTMV mtmv, AlterMTMV alterMTMV) throws DdlException { + + } + + @Override + public void refreshMTMV(RefreshMTMVInfo info) throws DdlException, MetaNotFoundException { + + } + + /** + * modify `tableMTMVs` by MTMVRelation + * @param mtmv + * @param relation + * @param task + */ + @Override + public void refreshComplete(MTMV mtmv, MTMVRelation relation, MTMVTask task) { + if (task.getStatus() == TaskStatus.SUCCESS) { + Objects.requireNonNull(relation); + refreshMTMVCache(relation, new BaseTableInfo(mtmv)); + } + } + + /** + * update mtmv status to `SCHEMA_CHANGE` + * @param table + */ + @Override + public void dropTable(Table table) { + processBaseTableChange(table, "The base table has been deleted:"); + } + + /** + * update mtmv status to `SCHEMA_CHANGE` + * @param table + */ + @Override + public void alterTable(Table table) { + processBaseTableChange(table, "The base table has been updated:"); + } + + private void processBaseTableChange(Table table, String msgPrefix) { + BaseTableInfo baseTableInfo = new BaseTableInfo(table); + Set mtmvsByBaseTable = getMtmvsByBaseTable(baseTableInfo); + if (CollectionUtils.isEmpty(mtmvsByBaseTable)) { + return; + } + for (BaseTableInfo mtmvInfo : mtmvsByBaseTable) { + Table mtmv = null; + try { + mtmv = Env.getCurrentEnv().getInternalCatalog() + .getDbOrAnalysisException(mtmvInfo.getDbId()) + .getTableOrAnalysisException(mtmvInfo.getTableId()); + } catch (AnalysisException e) { + LOG.warn(e); + continue; + } + TableNameInfo tableNameInfo = new TableNameInfo(mtmv.getQualifiedDbName(), + mtmv.getName()); + MTMVStatus status = new MTMVStatus(MTMVState.SCHEMA_CHANGE, + msgPrefix + baseTableInfo); + Env.getCurrentEnv().alterMTMVStatus(tableNameInfo, status); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVHookService.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVHookService.java new file mode 100644 index 00000000000000..63ae15fef1ba71 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVHookService.java @@ -0,0 +1,104 @@ +// 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.mtmv; + +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.job.extensions.mtmv.MTMVTask; +import org.apache.doris.nereids.trees.plans.commands.info.RefreshMTMVInfo; +import org.apache.doris.persist.AlterMTMV; + +/** + * Contains all operations that affect the mtmv + */ +public interface MTMVHookService { + /** + * triggered when create mtmv, only once + * + * @param mtmv + * @throws DdlException + */ + void createMTMV(MTMV mtmv) throws DdlException; + + /** + * triggered when drop mtmv, only once + * + * @param mtmv + * @throws DdlException + */ + void dropMTMV(MTMV mtmv) throws DdlException; + + /** + * triggered when playing `create mtmv` logs + * When triggered, db has not completed playback yet, so use dbId as param + * + * @param mtmv + * @param dbId + */ + void registerMTMV(MTMV mtmv, Long dbId); + + /** + * triggered when playing `drop mtmv` logs + * + * @param mtmv + */ + void deregisterMTMV(MTMV mtmv); + + /** + * triggered when alter mtmv, only once + * + * @param mtmv + * @param alterMTMV + * @throws DdlException + */ + void alterMTMV(MTMV mtmv, AlterMTMV alterMTMV) throws DdlException; + + /** + * triggered when refresh mtmv + * + * @param info + * @throws DdlException + * @throws MetaNotFoundException + */ + void refreshMTMV(RefreshMTMVInfo info) throws DdlException, MetaNotFoundException; + + /** + * triggered when mtmv task finish + * + * @param mtmv + * @param relation + * @param task + */ + void refreshComplete(MTMV mtmv, MTMVRelation relation, MTMVTask task); + + /** + * Triggered when baseTable is dropped + * + * @param table + */ + void dropTable(Table table); + + /** + * Triggered when baseTable is altered + * + * @param table + */ + void alterTable(Table table); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobInfo.java new file mode 100644 index 00000000000000..f253c708e909f4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobInfo.java @@ -0,0 +1,63 @@ +// 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.mtmv; + +import org.apache.doris.job.extensions.mtmv.MTMVTask; + +import com.google.common.collect.Lists; +import com.google.gson.annotations.SerializedName; + +import java.util.LinkedList; + +/** + * MTMVJobInfo + */ +public class MTMVJobInfo { + @SerializedName("jobName") + private String jobName; + @SerializedName("ht") + private LinkedList historyTasks; + + public MTMVJobInfo(String jobName) { + this.jobName = jobName; + historyTasks = Lists.newLinkedList(); + } + + public String getJobName() { + return jobName; + } + + public void addHistoryTask(MTMVTask task) { + historyTasks.add(task); + if (historyTasks.size() > MTMVTask.MAX_HISTORY_TASKS_NUM) { + historyTasks.removeFirst(); + } + } + + public LinkedList getHistoryTasks() { + return historyTasks; + } + + @Override + public String toString() { + return "MTMVJobInfo{" + + "jobName='" + jobName + '\'' + + ", historyTasks=" + historyTasks + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java new file mode 100644 index 00000000000000..dbd534e47c4dd7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java @@ -0,0 +1,195 @@ +// 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.mtmv; + +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf.TableType; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.job.base.JobExecuteType; +import org.apache.doris.job.base.JobExecutionConfiguration; +import org.apache.doris.job.base.TimerDefinition; +import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.common.JobType; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.mtmv.MTMVJob; +import org.apache.doris.job.extensions.mtmv.MTMVTask; +import org.apache.doris.mtmv.MTMVRefreshEnum.BuildMode; +import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshTrigger; +import org.apache.doris.nereids.trees.plans.commands.info.RefreshMTMVInfo; +import org.apache.doris.persist.AlterMTMV; +import org.apache.doris.qe.ConnectContext; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + +import java.util.List; + +/** + * when do some operation, do something about job + */ +public class MTMVJobManager implements MTMVHookService { + public static final String MTMV_JOB_PREFIX = "mtmv_"; + + /** + * create MTMVJob + * @param mtmv + * @throws DdlException + */ + @Override + public void createMTMV(MTMV mtmv) throws DdlException { + MTMVJob job = new MTMVJob(mtmv.getDatabase().getId(), mtmv.getId()); + job.setJobId(Env.getCurrentEnv().getNextId()); + job.setJobName(mtmv.getJobInfo().getJobName()); + job.setComment(mtmv.getName()); + job.setCreateUser(ConnectContext.get().getCurrentUserIdentity()); + job.setJobStatus(JobStatus.RUNNING); + job.setJobConfig(getJobConfig(mtmv)); + try { + Env.getCurrentEnv().getJobManager().registerJob(job); + } catch (JobException e) { + throw new DdlException(e.getMessage(), e); + } + } + + private JobExecutionConfiguration getJobConfig(MTMV mtmv) { + JobExecutionConfiguration jobExecutionConfiguration = new JobExecutionConfiguration(); + if (mtmv.getRefreshInfo().getRefreshTriggerInfo().getRefreshTrigger() + .equals(RefreshTrigger.SCHEDULE)) { + setScheduleJobConfig(jobExecutionConfiguration, mtmv); + } else if (mtmv.getRefreshInfo().getRefreshTriggerInfo().getRefreshTrigger() + .equals(RefreshTrigger.MANUAL)) { + setManualJobConfig(jobExecutionConfiguration, mtmv); + } + return jobExecutionConfiguration; + } + + private void setManualJobConfig(JobExecutionConfiguration jobExecutionConfiguration, MTMV mtmv) { + jobExecutionConfiguration.setExecuteType(JobExecuteType.MANUAL); + if (mtmv.getRefreshInfo().getBuildMode().equals(BuildMode.IMMEDIATE)) { + jobExecutionConfiguration.setImmediate(true); + } else { + jobExecutionConfiguration.setImmediate(false); + } + } + + private void setScheduleJobConfig(JobExecutionConfiguration jobExecutionConfiguration, MTMV mtmv) { + jobExecutionConfiguration.setExecuteType(JobExecuteType.RECURRING); + TimerDefinition timerDefinition = new TimerDefinition(); + timerDefinition + .setInterval(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getInterval()); + timerDefinition + .setIntervalUnit(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getTimeUnit()); + if (mtmv.getRefreshInfo().getBuildMode().equals(BuildMode.IMMEDIATE)) { + jobExecutionConfiguration.setImmediate(true); + } else if (mtmv.getRefreshInfo().getBuildMode().equals(BuildMode.DEFERRED) && !StringUtils + .isEmpty(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getStartTime())) { + timerDefinition.setStartTimeMs(TimeUtils.timeStringToLong( + mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getStartTime())); + } + + jobExecutionConfiguration.setTimerDefinition(timerDefinition); + } + + /** + * drop MTMVJob + * @param mtmv + * @throws DdlException + */ + @Override + public void dropMTMV(MTMV mtmv) throws DdlException { + List jobs = Env.getCurrentEnv().getJobManager() + .queryJobs(JobType.MTMV, mtmv.getJobInfo().getJobName()); + if (!CollectionUtils.isEmpty(jobs)) { + try { + Env.getCurrentEnv().getJobManager() + .unregisterJob(jobs.get(0).getJobId()); + } catch (JobException e) { + e.printStackTrace(); + throw new DdlException(e.getMessage()); + } + } + } + + @Override + public void registerMTMV(MTMV mtmv, Long dbId) { + + } + + @Override + public void deregisterMTMV(MTMV mtmv) { + + } + + /** + * drop MTMVJob and then create MTMVJob + * @param mtmv + * @param alterMTMV + * @throws DdlException + */ + @Override + public void alterMTMV(MTMV mtmv, AlterMTMV alterMTMV) throws DdlException { + if (alterMTMV.isNeedRebuildJob()) { + dropMTMV(mtmv); + createMTMV(mtmv); + } + } + + /** + * trigger MTMVJob + * @param info + * @throws DdlException + * @throws MetaNotFoundException + */ + @Override + public void refreshMTMV(RefreshMTMVInfo info) throws DdlException, MetaNotFoundException { + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(info.getMvName().getDb()); + MTMV mtmv = (MTMV) db.getTableOrMetaException(info.getMvName().getTbl(), TableType.MATERIALIZED_VIEW); + List jobs = Env.getCurrentEnv().getJobManager() + .queryJobs(JobType.MTMV, mtmv.getJobInfo().getJobName()); + if (CollectionUtils.isEmpty(jobs) || jobs.size() != 1) { + throw new DdlException("jobs not normal,should have one job,but job num is: " + jobs.size()); + } + try { + Env.getCurrentEnv().getJobManager().triggerJob(jobs.get(0).getJobId()); + } catch (JobException e) { + e.printStackTrace(); + throw new DdlException(e.getMessage()); + } + } + + @Override + public void refreshComplete(MTMV mtmv, MTMVRelation relation, MTMVTask task) { + + } + + @Override + public void dropTable(Table table) { + + } + + @Override + public void alterTable(Table table) { + + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshEnum.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshEnum.java index 6f14a2ba014e7b..ed2f0f709f49fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshEnum.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshEnum.java @@ -59,7 +59,6 @@ public enum MTMVState { */ public enum MTMVRefreshState { INIT, - REFRESHING, FAIL, SUCCESS } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelation.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelation.java new file mode 100644 index 00000000000000..d8f4d5be627cdb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelation.java @@ -0,0 +1,54 @@ +// 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.mtmv; + +import com.google.gson.annotations.SerializedName; + +import java.util.Set; + +public class MTMVRelation { + // if mtmv query sql is `select * from view1`; + // and `view1` query sql is `select * from table1 join table2` + // then baseTables will include: `table1` and `table2` + // baseViews will include `view1` + @SerializedName("bt") + private Set baseTables; + @SerializedName("bv") + private Set baseViews; + + public MTMVRelation(Set baseTables, Set baseViews) { + this.baseTables = baseTables; + this.baseViews = baseViews; + } + + public Set getBaseTables() { + return baseTables; + } + + public Set getBaseViews() { + return baseViews; + } + + @Override + public String toString() { + return "MTMVRelation{" + + "baseTables=" + baseTables + + ", baseViews=" + baseViews + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVService.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVService.java new file mode 100644 index 00000000000000..e83ca4e9496332 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVService.java @@ -0,0 +1,136 @@ +// 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.mtmv; + +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.job.extensions.mtmv.MTMVTask; +import org.apache.doris.nereids.trees.plans.commands.info.RefreshMTMVInfo; +import org.apache.doris.persist.AlterMTMV; + +import com.google.common.collect.Maps; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Map; +import java.util.Objects; + +public class MTMVService { + private static final Logger LOG = LogManager.getLogger(MTMVService.class); + + private Map hooks = Maps.newConcurrentMap(); + private MTMVCacheManager cacheManager = new MTMVCacheManager(); + private MTMVJobManager jobManager = new MTMVJobManager(); + + public MTMVService() { + registerHook("MTMVJobManager", jobManager); + registerHook("MTMVCacheManager", cacheManager); + } + + public MTMVCacheManager getCacheManager() { + return cacheManager; + } + + public void registerHook(String name, MTMVHookService mtmvHookService) { + Objects.requireNonNull(name); + Objects.requireNonNull(mtmvHookService); + hooks.put(name, mtmvHookService); + LOG.info("registerHook: " + name); + } + + public void deregisterHook(String name) { + hooks.remove(name); + LOG.info("deregisterHook: " + name); + } + + public void registerMTMV(MTMV mtmv, Long dbId) { + Objects.requireNonNull(mtmv); + LOG.info("registerMTMV: " + mtmv.getName()); + for (MTMVHookService mtmvHookService : hooks.values()) { + mtmvHookService.registerMTMV(mtmv, dbId); + } + } + + public void deregisterMTMV(MTMV mtmv) { + Objects.requireNonNull(mtmv); + LOG.info("deregisterMTMV: " + mtmv.getName()); + for (MTMVHookService mtmvHookService : hooks.values()) { + mtmvHookService.deregisterMTMV(mtmv); + } + } + + public void createMTMV(MTMV mtmv) throws DdlException { + Objects.requireNonNull(mtmv); + LOG.info("createMTMV: " + mtmv.getName()); + for (MTMVHookService mtmvHookService : hooks.values()) { + mtmvHookService.createMTMV(mtmv); + } + } + + public void dropMTMV(MTMV mtmv) throws DdlException { + Objects.requireNonNull(mtmv); + LOG.info("dropMTMV: " + mtmv.getName()); + for (MTMVHookService mtmvHookService : hooks.values()) { + mtmvHookService.dropMTMV(mtmv); + } + } + + public void alterMTMV(MTMV mtmv, AlterMTMV alterMTMV) throws DdlException { + Objects.requireNonNull(mtmv); + Objects.requireNonNull(alterMTMV); + LOG.info("alterMTMV, mtmvName: {}, AlterMTMV: {}", mtmv.getName(), alterMTMV); + for (MTMVHookService mtmvHookService : hooks.values()) { + mtmvHookService.alterMTMV(mtmv, alterMTMV); + } + } + + public void refreshMTMV(RefreshMTMVInfo info) throws DdlException, MetaNotFoundException { + Objects.requireNonNull(info); + LOG.info("refreshMTMV, RefreshMTMVInfo: {}", info); + for (MTMVHookService mtmvHookService : hooks.values()) { + mtmvHookService.refreshMTMV(info); + } + } + + public void dropTable(Table table) { + Objects.requireNonNull(table); + LOG.info("dropTable, tableName: {}", table.getName()); + for (MTMVHookService mtmvHookService : hooks.values()) { + mtmvHookService.dropTable(table); + } + } + + public void alterTable(Table table) { + Objects.requireNonNull(table); + LOG.info("alterTable, tableName: {}", table.getName()); + for (MTMVHookService mtmvHookService : hooks.values()) { + mtmvHookService.alterTable(table); + } + } + + public void refreshComplete(MTMV mtmv, MTMVRelation cache, MTMVTask task) { + Objects.requireNonNull(mtmv); + Objects.requireNonNull(task); + LOG.info("refreshComplete: " + mtmv.getName()); + for (MTMVHookService mtmvHookService : hooks.values()) { + mtmvHookService.refreshComplete(mtmv, cache, task); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVStatus.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVStatus.java new file mode 100644 index 00000000000000..ea05dbc86730e8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVStatus.java @@ -0,0 +1,97 @@ +// 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.mtmv; + +import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVRefreshState; +import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVState; + +import com.google.gson.annotations.SerializedName; + +import java.util.Objects; + +public class MTMVStatus { + @SerializedName("state") + private MTMVState state; + @SerializedName("schemaChangeDetail") + private String schemaChangeDetail; + @SerializedName("refreshState") + private MTMVRefreshState refreshState; + + public MTMVStatus() { + this.state = MTMVState.INIT; + this.refreshState = MTMVRefreshState.INIT; + } + + public MTMVStatus(MTMVState state, String schemaChangeDetail) { + this.state = state; + this.schemaChangeDetail = schemaChangeDetail; + } + + public MTMVStatus(MTMVRefreshState refreshState) { + this.refreshState = refreshState; + } + + public MTMVState getState() { + return state; + } + + public String getSchemaChangeDetail() { + return schemaChangeDetail; + } + + public MTMVRefreshState getRefreshState() { + return refreshState; + } + + public void setState(MTMVState state) { + this.state = state; + } + + public void setSchemaChangeDetail(String schemaChangeDetail) { + this.schemaChangeDetail = schemaChangeDetail; + } + + public void setRefreshState(MTMVRefreshState refreshState) { + this.refreshState = refreshState; + } + + public MTMVStatus updateNotNull(MTMVStatus status) { + Objects.requireNonNull(status); + if (status.getState() != null) { + this.state = status.getState(); + if (this.state == MTMVState.SCHEMA_CHANGE) { + this.schemaChangeDetail = status.getSchemaChangeDetail(); + } else { + this.schemaChangeDetail = null; + } + } + if (status.getRefreshState() != null) { + this.refreshState = status.getRefreshState(); + } + return this; + } + + @Override + public String toString() { + return "MTMVStatus{" + + "state=" + state + + ", schemaChangeDetail='" + schemaChangeDetail + '\'' + + ", refreshState=" + refreshState + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateMTMVCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateMTMVCommand.java index 76a646fc819f92..c246991e61876b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateMTMVCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateMTMVCommand.java @@ -17,7 +17,7 @@ package org.apache.doris.nereids.trees.plans.commands; -import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.catalog.Env; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -48,7 +48,7 @@ public CreateMTMVCommand(CreateMTMVInfo createMTMVInfo) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { createMTMVInfo.analyze(ctx); - throw new AnalysisException("current not support."); + Env.getCurrentEnv().createTable(createMTMVInfo.translateToLegacyStmt()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropMTMVCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropMTMVCommand.java index 0a6494edeac586..a0d614b163c740 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropMTMVCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropMTMVCommand.java @@ -17,7 +17,7 @@ package org.apache.doris.nereids.trees.plans.commands; -import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.catalog.Env; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.commands.info.DropMTMVInfo; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -40,7 +40,7 @@ public DropMTMVCommand(DropMTMVInfo dropMTMVInfo) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { dropMTMVInfo.analyze(ctx); - throw new AnalysisException("current not support."); + Env.getCurrentEnv().dropTable(dropMTMVInfo.translateToLegacyStmt()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java index 45411204524193..ceb2ad9270a452 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java @@ -66,6 +66,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Lists; +import lombok.Setter; import org.apache.commons.collections.CollectionUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -95,11 +96,19 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync, public static final Logger LOG = LogManager.getLogger(InsertIntoTableCommand.class); private final LogicalPlan logicalQuery; - private final Optional labelName; + + @Setter + private Optional labelName; private final boolean isOverwrite; private NereidsPlanner planner; private boolean isTxnBegin = false; + /** + * When source it's from job scheduler,it will be set. + */ + @Setter + private long jobId; + /** * constructor */ @@ -204,7 +213,7 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { LOG.info("Nereids start to execute the insert command, query id: {}, txn id: {}", ctx.queryId(), txn.getTxnId()); - txn.executeInsertIntoTableCommand(executor); + txn.executeInsertIntoTableCommand(executor, jobId); if (ctx.getState().getStateType() == MysqlStateType.ERR) { try { String errMsg = Strings.emptyToNull(ctx.getState().getErrorMessage()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/RefreshMTMVCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/RefreshMTMVCommand.java index 352801f5d83a46..982e8d862573de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/RefreshMTMVCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/RefreshMTMVCommand.java @@ -17,7 +17,7 @@ package org.apache.doris.nereids.trees.plans.commands; -import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.catalog.Env; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.commands.info.RefreshMTMVInfo; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -40,7 +40,7 @@ public RefreshMTMVCommand(RefreshMTMVInfo refreshMTMVInfo) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { refreshMTMVInfo.analyze(ctx); - throw new AnalysisException("current not support."); + Env.getCurrentEnv().getMtmvService().refreshMTMV(refreshMTMVInfo); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVPropertyInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVPropertyInfo.java index a7ddfb8f83c544..5f9be47fc70c3c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVPropertyInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVPropertyInfo.java @@ -17,7 +17,9 @@ package org.apache.doris.nereids.trees.plans.commands.info; +import org.apache.doris.catalog.Env; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.qe.ConnectContext; @@ -40,11 +42,29 @@ public AlterMTMVPropertyInfo(TableNameInfo mvName, Map propertie public void analyze(ConnectContext ctx) throws AnalysisException { super.analyze(ctx); + analyzeProperties(); } @Override public void run() throws UserException { - throw new org.apache.doris.nereids.exceptions.AnalysisException("current not support."); + Env.getCurrentEnv().alterMTMVProperty(this); + } + + private void analyzeProperties() { + for (String key : properties.keySet()) { + if (PropertyAnalyzer.PROPERTIES_GRACE_PERIOD.equals(key)) { + String gracePeriod = properties.get(PropertyAnalyzer.PROPERTIES_GRACE_PERIOD); + try { + Long.parseLong(gracePeriod); + } catch (NumberFormatException e) { + throw new org.apache.doris.nereids.exceptions.AnalysisException( + "valid grace_period: " + properties.get(PropertyAnalyzer.PROPERTIES_GRACE_PERIOD)); + } + } else { + throw new org.apache.doris.nereids.exceptions.AnalysisException("illegal key:" + key); + } + } + } public Map getProperties() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVRefreshInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVRefreshInfo.java index b54bba1ece35dd..25bb78f2fc2130 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVRefreshInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVRefreshInfo.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.commands.info; +import org.apache.doris.catalog.Env; import org.apache.doris.common.UserException; import org.apache.doris.mtmv.MTMVRefreshInfo; import org.apache.doris.nereids.exceptions.AnalysisException; @@ -45,7 +46,7 @@ public void analyze(ConnectContext ctx) throws AnalysisException { @Override public void run() throws UserException { - throw new AnalysisException("current not support."); + Env.getCurrentEnv().alterMTMVRefreshInfo(this); } public MTMVRefreshInfo getRefreshInfo() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVRenameInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVRenameInfo.java index c9e09e29187438..a93c53087d3cac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVRenameInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterMTMVRenameInfo.java @@ -17,6 +17,9 @@ package org.apache.doris.nereids.trees.plans.commands.info; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.Table; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeNameFormat; import org.apache.doris.nereids.exceptions.AnalysisException; @@ -40,6 +43,7 @@ public AlterMTMVRenameInfo(TableNameInfo mvName, String newName) { /** * analyze + * * @param ctx ctx * @throws AnalysisException AnalysisException */ @@ -54,6 +58,8 @@ public void analyze(ConnectContext ctx) throws AnalysisException { @Override public void run() throws DdlException { - throw new AnalysisException("current not support."); + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(mvName.getDb()); + Table table = db.getTableOrDdlException(mvName.getTbl()); + Env.getCurrentEnv().renameTable(db, table, newName); } } 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 fba382e9140318..89b8f5f9bb06fd 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 @@ -79,6 +79,10 @@ public ColumnDefinition(String name, DataType type, boolean isNullable) { this(name, type, false, null, isNullable, Optional.empty(), ""); } + public ColumnDefinition(String name, DataType type, boolean isNullable, String comment) { + this(name, type, false, null, isNullable, Optional.empty(), comment); + } + public String getName() { return name; } 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 1b255adc528ccc..98e610aab9e79a 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 @@ -17,18 +17,48 @@ package org.apache.doris.nereids.trees.plans.commands.info; +import org.apache.doris.analysis.CreateMTMVStmt; +import org.apache.doris.analysis.KeysDesc; +import org.apache.doris.analysis.TableName; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.TableIf.TableType; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.mtmv.EnvInfo; import org.apache.doris.mtmv.MTMVRefreshInfo; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.TreeNode; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; +import org.apache.doris.nereids.trees.plans.visitor.NondeterministicFunctionCollector; +import org.apache.doris.nereids.trees.plans.visitor.TableCollector; +import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; import org.apache.doris.nereids.util.Utils; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import org.apache.commons.collections.CollectionUtils; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; /** * MTMV info in creating MTMV. @@ -69,14 +99,133 @@ public CreateMTMVInfo(boolean ifNotExists, TableNameInfo mvName, this.refreshInfo = Objects.requireNonNull(refreshInfo, "require refreshInfo object"); this.simpleColumnDefinitions = Objects .requireNonNull(simpleColumnDefinitions, "require simpleColumnDefinitions object"); - this.envInfo = new EnvInfo(ConnectContext.get().getDefaultCatalog(), ConnectContext.get().getDatabase()); + this.envInfo = new EnvInfo(ConnectContext.get().getCurrentCatalog().getId(), + ConnectContext.get().getCurrentDbId()); } /** * analyze create table info */ public void analyze(ConnectContext ctx) { + // analyze table name + mvName.analyze(ctx); + if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), mvName.getDb(), + mvName.getTbl(), PrivPredicate.CREATE)) { + String message = ErrorCode.ERR_TABLEACCESS_DENIED_ERROR.formatErrorMsg("CREATE", + ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), + mvName.getDb() + ": " + mvName.getTbl()); + throw new AnalysisException(message); + } + analyzeProperties(); + analyzeQuery(ctx); + // analyze column + final boolean finalEnableMergeOnWrite = false; + Set keysSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + keysSet.addAll(keys); + columns.forEach(c -> c.validate(keysSet, finalEnableMergeOnWrite, KeysType.DUP_KEYS)); + if (distribution == null) { + throw new AnalysisException("Create MTMV should contain distribution desc"); + } + + if (properties == null) { + properties = Maps.newHashMap(); + } + + // analyze distribute + Map columnMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER); + columns.forEach(c -> columnMap.put(c.getName(), c)); + distribution.updateCols(columns.get(0).getName()); + distribution.validate(columnMap, KeysType.DUP_KEYS); + refreshInfo.validate(); + + analyzeProperties(); + } + + private void analyzeProperties() { + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_GRACE_PERIOD)) { + String gracePeriod = properties.get(PropertyAnalyzer.PROPERTIES_GRACE_PERIOD); + try { + Long.parseLong(gracePeriod); + } catch (NumberFormatException e) { + throw new AnalysisException( + "valid grace_period: " + properties.get(PropertyAnalyzer.PROPERTIES_GRACE_PERIOD)); + } + mvProperties.put(PropertyAnalyzer.PROPERTIES_GRACE_PERIOD, gracePeriod); + properties.remove(PropertyAnalyzer.PROPERTIES_GRACE_PERIOD); + } + } + + /** + * analyzeQuery + */ + public void analyzeQuery(ConnectContext ctx) { + // create table as select + NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); + Plan plan = planner.plan(logicalQuery, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); + analyzeBaseTables(plan); + analyzeExpressions((PhysicalPlan) plan); + getColumns(plan); + } + + private void analyzeBaseTables(Plan plan) { + TableCollectorContext collectorContext = + new TableCollector.TableCollectorContext(Sets.newHashSet(TableType.MATERIALIZED_VIEW)); + plan.accept(TableCollector.INSTANCE, collectorContext); + List collectedTables = collectorContext.getCollectedTables(); + if (!CollectionUtils.isEmpty(collectedTables)) { + throw new AnalysisException("can not contain MATERIALIZED_VIEW"); + } + } + + private void analyzeExpressions(PhysicalPlan plan) { + List> functionCollectResult = new ArrayList<>(); + plan.accept(NondeterministicFunctionCollector.INSTANCE, functionCollectResult); + if (!CollectionUtils.isEmpty(functionCollectResult)) { + throw new AnalysisException("can not contain invalid expression"); + } + } + + private void getColumns(Plan plan) { + List slots = plan.getOutput(); + if (slots.isEmpty()) { + throw new AnalysisException("table should contain at least one column"); + } + if (!CollectionUtils.isEmpty(simpleColumnDefinitions) && simpleColumnDefinitions.size() != slots.size()) { + throw new AnalysisException("simpleColumnDefinitions size is not equal to the query's"); + } + Set colNames = Sets.newHashSet(); + for (int i = 0; i < slots.size(); i++) { + String colName = CollectionUtils.isEmpty(simpleColumnDefinitions) ? slots.get(i).getName() + : simpleColumnDefinitions.get(i).getName(); + try { + FeNameFormat.checkColumnName(colName); + } catch (org.apache.doris.common.AnalysisException e) { + throw new AnalysisException(e.getMessage()); + } + if (colNames.contains(colName)) { + throw new AnalysisException("repeat cols:" + colName); + } else { + colNames.add(colName); + } + columns.add(new ColumnDefinition( + colName, slots.get(i).getDataType(), true, + CollectionUtils.isEmpty(simpleColumnDefinitions) ? null + : simpleColumnDefinitions.get(i).getComment())); + } + } + + /** + * translate to catalog CreateMultiTableMaterializedViewStmt + */ + public CreateMTMVStmt translateToLegacyStmt() { + TableName tableName = mvName.transferToTableName(); + KeysDesc keysDesc = new KeysDesc(KeysType.DUP_KEYS, keys); + List catalogColumns = columns.stream() + .map(ColumnDefinition::translateToCatalogStyle) + .collect(Collectors.toList()); + return new CreateMTMVStmt(ifNotExists, tableName, catalogColumns, refreshInfo, keysDesc, + distribution.translateToCatalogStyle(), properties, mvProperties, querySql, comment, envInfo); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropMTMVInfo.java index 958d6315c39dd8..650df2c96c37e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropMTMVInfo.java @@ -17,6 +17,12 @@ package org.apache.doris.nereids.trees.plans.commands.info; +import org.apache.doris.analysis.DropTableStmt; +import org.apache.doris.analysis.TableName; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.qe.ConnectContext; import java.util.Objects; @@ -39,6 +45,14 @@ public DropMTMVInfo(TableNameInfo mvName, boolean ifExists) { * @param ctx ConnectContext */ public void analyze(ConnectContext ctx) { + mvName.analyze(ctx); + if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), mvName.getDb(), + mvName.getTbl(), PrivPredicate.DROP)) { + String message = ErrorCode.ERR_TABLEACCESS_DENIED_ERROR.formatErrorMsg("DROP", + ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), + mvName.getDb() + ": " + mvName.getTbl()); + throw new AnalysisException(message); + } } /** @@ -49,4 +63,14 @@ public void analyze(ConnectContext ctx) { public TableNameInfo getMvName() { return mvName; } + + /** + * translate to catalog DropTableStmt + */ + public DropTableStmt translateToLegacyStmt() { + TableName tableName = mvName.transferToTableName(); + DropTableStmt dropTableStmt = new DropTableStmt(ifExists, tableName, false); + dropTableStmt.setMaterializedView(true); + return dropTableStmt; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RefreshMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RefreshMTMVInfo.java index 8439ed2ad0acfb..422b9697c7c559 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RefreshMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RefreshMTMVInfo.java @@ -17,6 +17,10 @@ package org.apache.doris.nereids.trees.plans.commands.info; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.qe.ConnectContext; import java.util.Objects; @@ -36,6 +40,14 @@ public RefreshMTMVInfo(TableNameInfo mvName) { * @param ctx ConnectContext */ public void analyze(ConnectContext ctx) { + mvName.analyze(ctx); + if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), mvName.getDb(), + mvName.getTbl(), PrivPredicate.CREATE)) { + String message = ErrorCode.ERR_TABLEACCESS_DENIED_ERROR.formatErrorMsg("CREATE", + ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), + mvName.getDb() + ": " + mvName.getTbl()); + throw new AnalysisException(message); + } } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/txn/Transaction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/txn/Transaction.java index 48f80ad907a1b4..9e8663fa1a723a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/txn/Transaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/txn/Transaction.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.txn; import org.apache.doris.analysis.StatementBase; +import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Table; @@ -92,7 +93,7 @@ public long getTxnId() { /** * execute insert txn for insert into select command. */ - public void executeInsertIntoTableCommand(StmtExecutor executor) { + public void executeInsertIntoTableCommand(StmtExecutor executor, long jobId) { LOG.info("Do insert [{}] with query id: {}", labelName, DebugUtil.printId(ctx.queryId())); Throwable throwable = null; @@ -200,11 +201,23 @@ public void executeInsertIntoTableCommand(StmtExecutor executor) { try { // the statement parsed by Nereids is saved at executor::parsedStmt. StatementBase statement = executor.getParsedStmt(); + UserIdentity userIdentity; + //if we use job scheduler, parse statement will not set user identity,so we need to get it from context + if (null == statement) { + userIdentity = ctx.getCurrentUserIdentity(); + } else { + userIdentity = statement.getUserInfo(); + } + EtlJobType etlJobType = EtlJobType.INSERT; + if (0 != jobId) { + etlJobType = EtlJobType.INSERT_JOB; + } + ctx.getEnv().getLoadManager() .recordFinishedLoadJob(labelName, txnId, database.getFullName(), table.getId(), - EtlJobType.INSERT, createAt, throwable == null ? "" : throwable.getMessage(), - coordinator.getTrackingUrl(), statement.getUserInfo()); + etlJobType, createAt, throwable == null ? "" : throwable.getMessage(), + coordinator.getTrackingUrl(), userIdentity, jobId); } catch (MetaNotFoundException e) { LOG.warn("Record info of insert load with error {}", e.getMessage(), e); errMsg = "Record info of insert load with error " + e.getMessage(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/AlterMTMV.java b/fe/fe-core/src/main/java/org/apache/doris/persist/AlterMTMV.java new file mode 100644 index 00000000000000..d2084ff6fa3df7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/AlterMTMV.java @@ -0,0 +1,150 @@ +// 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.persist; + +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.job.extensions.mtmv.MTMVTask; +import org.apache.doris.mtmv.MTMVAlterOpType; +import org.apache.doris.mtmv.MTMVRefreshInfo; +import org.apache.doris.mtmv.MTMVRelation; +import org.apache.doris.mtmv.MTMVStatus; +import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + +public class AlterMTMV implements Writable { + @SerializedName("ot") + private MTMVAlterOpType opType; + @SerializedName("mn") + private TableNameInfo mvName; + @SerializedName("ri") + private MTMVRefreshInfo refreshInfo; + @SerializedName("s") + private MTMVStatus status; + @SerializedName("nrj") + private boolean needRebuildJob = false; + @SerializedName("mp") + private Map mvProperties; + @SerializedName("t") + private MTMVTask task; + @SerializedName("r") + private MTMVRelation relation; + + public AlterMTMV(TableNameInfo mvName, MTMVRefreshInfo refreshInfo, MTMVAlterOpType opType) { + this.mvName = Objects.requireNonNull(mvName, "require mvName object"); + this.refreshInfo = Objects.requireNonNull(refreshInfo, "require refreshInfo object"); + this.opType = Objects.requireNonNull(opType, "require opType object"); + this.needRebuildJob = true; + } + + public AlterMTMV(TableNameInfo mvName, MTMVAlterOpType opType) { + this.mvName = Objects.requireNonNull(mvName, "require mvName object"); + this.opType = Objects.requireNonNull(opType, "require opType object"); + } + + public TableNameInfo getMvName() { + return mvName; + } + + public MTMVStatus getStatus() { + return status; + } + + public boolean isNeedRebuildJob() { + return needRebuildJob; + } + + public MTMVRefreshInfo getRefreshInfo() { + return refreshInfo; + } + + public void setMvName(TableNameInfo mvName) { + this.mvName = mvName; + } + + public void setRefreshInfo(MTMVRefreshInfo refreshInfo) { + this.refreshInfo = refreshInfo; + } + + public void setStatus(MTMVStatus status) { + this.status = status; + } + + public void setMvProperties(Map mvProperties) { + this.mvProperties = mvProperties; + } + + public Map getMvProperties() { + return mvProperties; + } + + public void setNeedRebuildJob(boolean needRebuildJob) { + this.needRebuildJob = needRebuildJob; + } + + public MTMVAlterOpType getOpType() { + return opType; + } + + public MTMVTask getTask() { + return task; + } + + public void setTask(MTMVTask task) { + this.task = task; + } + + public MTMVRelation getRelation() { + return relation; + } + + public void setRelation(MTMVRelation relation) { + this.relation = relation; + } + + @Override + public String toString() { + return "AlterMTMV{" + + "mvName=" + mvName + + ", refreshInfo=" + refreshInfo + + ", status=" + status + + ", needRebuildJob=" + needRebuildJob + + ", mvProperties=" + mvProperties + + ", task=" + task + + ", relation=" + relation + + '}'; + } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } + + public static AlterMTMV read(DataInput in) throws IOException { + return GsonUtils.GSON.fromJson(Text.readString(in), AlterMTMV.class); + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index f34d0c2c69d38c..c5da3e600e5e08 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -1111,6 +1111,11 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { env.getAnalysisManager().replayTableStatsDeletion((TableStatsDeletionLog) journal.getData()); break; } + case OperationType.OP_ALTER_MTMV: { + final AlterMTMV alterMtmv = (AlterMTMV) journal.getData(); + env.getAlterInstance().processAlterMTMV(alterMtmv, true); + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); @@ -1936,4 +1941,8 @@ public void logAutoJob(AnalysisInfo analysisInfo) { public void logDeleteTableStats(TableStatsDeletionLog log) { logEdit(OperationType.OP_DELETE_TABLE_STATS, log); } + + public void logAlterMTMV(AlterMTMV log) { + logEdit(OperationType.OP_ALTER_MTMV, log); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java index 6484a9a041f810..0769cba0d81e11 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java @@ -346,6 +346,8 @@ public class OperationType { public static final short OP_ALTER_CATALOG_COMMENT = 458; + public static final short OP_ALTER_MTMV = 459; + /** * Get opcode name by op code. **/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java index f467128780114e..5e4792cf4f0495 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java @@ -76,6 +76,7 @@ import org.apache.doris.datasource.paimon.PaimonHMSExternalCatalog; import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.extensions.insert.InsertJob; +import org.apache.doris.job.extensions.mtmv.MTMVJob; import org.apache.doris.load.loadv2.LoadJob.LoadJobStateUpdateInfo; import org.apache.doris.load.loadv2.SparkLoadJob.SparkLoadJobStateUpdateInfo; import org.apache.doris.load.routineload.AbstractDataSourceProperties; @@ -218,9 +219,9 @@ public class GsonUtils { AbstractDataSourceProperties.class, "clazz") .registerSubtype(KafkaDataSourceProperties.class, KafkaDataSourceProperties.class.getSimpleName()); private static RuntimeTypeAdapterFactory jobExecutorRuntimeTypeAdapterFactory = - RuntimeTypeAdapterFactory.of( - AbstractJob.class, "clazz") - .registerSubtype(InsertJob.class, InsertJob.class.getSimpleName()); + RuntimeTypeAdapterFactory.of(AbstractJob.class, "clazz") + .registerSubtype(InsertJob.class, InsertJob.class.getSimpleName()) + .registerSubtype(MTMVJob.class, MTMVJob.class.getSimpleName()); private static RuntimeTypeAdapterFactory dbTypeAdapterFactory = RuntimeTypeAdapterFactory.of( DatabaseIf.class, "clazz") diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java index 2013216742b655..5a2bae1ee05d99 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java @@ -122,6 +122,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.util.ProfileManager; import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.common.JobType; import org.apache.doris.load.sync.SyncJobManager; import org.apache.doris.persist.CleanQueryStatsInfo; import org.apache.doris.statistics.StatisticsRepository; @@ -184,16 +185,32 @@ public static void execute(Env env, DdlStmt ddlStmt) throws Exception { } else if (ddlStmt instanceof AlterRoutineLoadStmt) { env.getRoutineLoadManager().alterRoutineLoadJob((AlterRoutineLoadStmt) ddlStmt); } else if (ddlStmt instanceof CreateJobStmt) { - env.getJobManager().registerJob(((CreateJobStmt) ddlStmt).getJobInstance()); + try { + env.getJobManager().registerJob(((CreateJobStmt) ddlStmt).getJobInstance()); + } catch (Exception e) { + throw new DdlException(e.getMessage()); + } } else if (ddlStmt instanceof StopJobStmt) { StopJobStmt stmt = (StopJobStmt) ddlStmt; - env.getJobManager().unregisterJob(stmt.getDbFullName(), stmt.getName()); + try { + env.getJobManager().unregisterJob(stmt.getName(), JobType.INSERT); + } catch (Exception e) { + throw new DdlException(e.getMessage()); + } } else if (ddlStmt instanceof PauseJobStmt) { PauseJobStmt stmt = (PauseJobStmt) ddlStmt; - env.getJobManager().alterJobStatus(stmt.getDbFullName(), stmt.getName(), JobStatus.PAUSED); + try { + env.getJobManager().alterJobStatus(stmt.getName(), JobStatus.PAUSED, JobType.INSERT); + } catch (Exception e) { + throw new DdlException(e.getMessage()); + } } else if (ddlStmt instanceof ResumeJobStmt) { ResumeJobStmt stmt = (ResumeJobStmt) ddlStmt; - env.getJobManager().alterJobStatus(stmt.getDbFullName(), stmt.getName(), JobStatus.RUNNING); + try { + env.getJobManager().alterJobStatus(stmt.getName(), JobStatus.RUNNING, JobType.INSERT); + } catch (Exception e) { + throw new DdlException(e.getMessage()); + } } else if (ddlStmt instanceof CreateUserStmt) { CreateUserStmt stmt = (CreateUserStmt) ddlStmt; env.getAuth().createUser(stmt); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 1fcaa56ddf0dc6..3adfbc0d1a6aff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -438,6 +438,8 @@ public class SessionVariable implements Serializable, Writable { public static final String ENABLE_DECIMAL256 = "enable_decimal256"; + public static final String ENABLE_EXTERNAL_MV_REWRITE = "enable_external_mv_rewrite"; + public static final String ENABLE_MV_REWRITE = "enable_mv_rewrite"; public static final String STATS_INSERT_MERGE_ITEM_COUNT = "stats_insert_merge_item_count"; public static final String HUGE_TABLE_DEFAULT_SAMPLE_ROWS = "huge_table_default_sample_rows"; @@ -766,6 +768,11 @@ public class SessionVariable implements Serializable, Writable { @VariableMgr.VarAttr(name = TRIM_TAILING_SPACES_FOR_EXTERNAL_TABLE_QUERY, needForward = true) public boolean trimTailingSpacesForExternalTableQuery = false; + @VariableMgr.VarAttr(name = ENABLE_EXTERNAL_MV_REWRITE, needForward = true) + public boolean enableExternalMvRewrite = false; + + @VariableMgr.VarAttr(name = ENABLE_MV_REWRITE, needForward = true) + public boolean enableMvRewrite = false; // the maximum size in bytes for a table that will be broadcast to all be nodes // when performing a join, By setting this value to -1 broadcasting can be disabled. @@ -2421,6 +2428,22 @@ public void setMaxTableCountUseCascadesJoinReorder(int maxTableCountUseCascadesJ : maxTableCountUseCascadesJoinReorder; } + public boolean isEnableExternalMvRewrite() { + return enableExternalMvRewrite; + } + + public void setEnableExternalMvRewrite(boolean enableExternalMvRewrite) { + this.enableExternalMvRewrite = enableExternalMvRewrite; + } + + public boolean isEnableMvRewrite() { + return enableMvRewrite; + } + + public void setEnableMvRewrite(boolean enableMvRewrite) { + this.enableMvRewrite = enableMvRewrite; + } + public boolean isShowUserDefaultRole() { return showUserDefaultRole; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index 8f92115695e3de..9a77e615c43262 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -180,6 +180,7 @@ import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.HMSExternalCatalog; import org.apache.doris.external.iceberg.IcebergTableCreationRecord; +import org.apache.doris.job.task.AbstractTask; import org.apache.doris.load.DeleteHandler; import org.apache.doris.load.ExportJobState; import org.apache.doris.load.ExportMgr; @@ -428,7 +429,7 @@ public ShowResultSet execute() throws AnalysisException { } else if (stmt instanceof ShowJobStmt) { handleShowJob(); } else if (stmt instanceof ShowJobTaskStmt) { - //handleShowJobTask(); + handleShowJobTask(); } else if (stmt instanceof ShowConvertLSCStmt) { handleShowConvertLSC(); } else { @@ -1417,36 +1418,39 @@ private void handleShowLoadWarningsFromURL(ShowLoadWarningsStmt showWarningsStmt resultSet = new ShowResultSet(showWarningsStmt.getMetaData(), rows); } - /*private void handleShowJobTask() { + private void handleShowJobTask() { ShowJobTaskStmt showJobTaskStmt = (ShowJobTaskStmt) stmt; List> rows = Lists.newArrayList(); - List jobs = Env.getCurrentEnv().getJobRegister() - .getJobs(showJobTaskStmt.getDbFullName(), showJobTaskStmt.getName(), showJobTaskStmt.getJobCategory(), - null); + List jobs = Env.getCurrentEnv().getJobManager() + .queryJobs(showJobTaskStmt.getJobType(), showJobTaskStmt.getName()); if (CollectionUtils.isEmpty(jobs)) { resultSet = new ShowResultSet(showJobTaskStmt.getMetaData(), rows); return; } - Job job = jobs.get(0); - long jobId = job.getJobId(); - List jobTasks = Env.getCurrentEnv().getJobTaskManager().getJobTasks(jobId); + org.apache.doris.job.base.AbstractJob job = jobs.get(0); + List jobTasks = job.queryTasks(); if (CollectionUtils.isEmpty(jobTasks)) { - resultSet = new ShowResultSet(showJobTaskStmt.getMetaData(), rows); + resultSet = new ShowResultSet(job.getTaskMetaData(), rows); return; } - for (JobTask jobTask : jobTasks) { - rows.add(jobTask.getShowInfo(job.getJobName())); + for (AbstractTask jobTask : jobTasks) { + rows.add(jobTask.getShowInfo()); } - resultSet = new ShowResultSet(showJobTaskStmt.getMetaData(), rows); - }*/ + resultSet = new ShowResultSet(job.getTaskMetaData(), rows); + } private void handleShowJob() throws AnalysisException { ShowJobStmt showJobStmt = (ShowJobStmt) stmt; List> rows = Lists.newArrayList(); // if job exists List jobList; - jobList = Env.getCurrentEnv().getJobManager() - .queryJobs(showJobStmt.getDbFullName(), showJobStmt.getName()); + if (null == showJobStmt.getJobType()) { + jobList = Env.getCurrentEnv().getJobManager() + .queryJobs(showJobStmt.getJobTypes()); + } else { + jobList = Env.getCurrentEnv().getJobManager() + .queryJobs(showJobStmt.getJobType(), showJobStmt.getName()); + } if (jobList.isEmpty()) { resultSet = new ShowResultSet(showJobStmt.getMetaData(), rows); @@ -1456,9 +1460,9 @@ private void handleShowJob() throws AnalysisException { // check auth for (org.apache.doris.job.base.AbstractJob job : jobList) { - rows.add(job.getCommonShowInfo()); + rows.add(job.getShowInfo()); } - resultSet = new ShowResultSet(showJobStmt.getMetaData(), rows); + resultSet = new ShowResultSet(jobList.get(0).getJobMetaData(), rows); } private void handleShowRoutineLoad() throws AnalysisException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index 770c876ed2a013..243cead688c95f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -2058,7 +2058,7 @@ private void handleInsertStmt() throws Exception { .recordFinishedLoadJob(label, txnId, insertStmt.getDbName(), insertStmt.getTargetTable().getId(), EtlJobType.INSERT, createTime, throwable == null ? "" : throwable.getMessage(), - coord.getTrackingUrl(), insertStmt.getUserInfo()); + coord.getTrackingUrl(), insertStmt.getUserInfo(), 0L); } catch (MetaNotFoundException e) { LOG.warn("Record info of insert load with error {}", e.getMessage(), e); errMsg = "Record info of insert load with error " + e.getMessage(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MaterializedViewsTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MaterializedViewsTableValuedFunction.java new file mode 100644 index 00000000000000..a6384cfd6834ee --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MaterializedViewsTableValuedFunction.java @@ -0,0 +1,117 @@ +// 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.tablefunction; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TMaterializedViewsMetadataParams; +import org.apache.doris.thrift.TMetaScanRange; +import org.apache.doris.thrift.TMetadataType; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +/** + * The Implement of table valued function + * mtmvs("database" = "db1"). + */ +public class MaterializedViewsTableValuedFunction extends MetadataTableValuedFunction { + public static final String NAME = "mtmvs"; + private static final String DB = "database"; + + private static final ImmutableSet PROPERTIES_SET = ImmutableSet.of(DB); + + private static final ImmutableList SCHEMA = ImmutableList.of( + new Column("Id", ScalarType.createType(PrimitiveType.BIGINT)), + new Column("Name", ScalarType.createStringType()), + new Column("JobName", ScalarType.createStringType()), + new Column("State", ScalarType.createStringType()), + new Column("SchemaChangeDetail", ScalarType.createStringType()), + new Column("RefreshState", ScalarType.createStringType()), + new Column("RefreshInfo", ScalarType.createStringType()), + new Column("QuerySql", ScalarType.createStringType()), + new Column("EnvInfo", ScalarType.createStringType()), + new Column("MvProperties", ScalarType.createStringType())); + + private static final ImmutableMap COLUMN_TO_INDEX; + + static { + ImmutableMap.Builder builder = new ImmutableMap.Builder(); + for (int i = 0; i < SCHEMA.size(); i++) { + builder.put(SCHEMA.get(i).getName().toLowerCase(), i); + } + COLUMN_TO_INDEX = builder.build(); + } + + public static Integer getColumnIndexFromColumnName(String columnName) { + return COLUMN_TO_INDEX.get(columnName.toLowerCase()); + } + + private final String databaseName; + + public MaterializedViewsTableValuedFunction(Map params) throws AnalysisException { + Map validParams = Maps.newHashMap(); + for (String key : params.keySet()) { + if (!PROPERTIES_SET.contains(key.toLowerCase())) { + throw new AnalysisException("'" + key + "' is invalid property"); + } + // check ctl, db, tbl + validParams.put(key.toLowerCase(), params.get(key)); + } + String dbName = validParams.get(DB); + if (dbName == null) { + throw new AnalysisException("Invalid mtmv metadata query"); + } + this.databaseName = ClusterNamespace.getFullName(SystemInfoService.DEFAULT_CLUSTER, dbName); + } + + @Override + public TMetadataType getMetadataType() { + return TMetadataType.MATERIALIZED_VIEWS; + } + + @Override + public TMetaScanRange getMetaScanRange() { + TMetaScanRange metaScanRange = new TMetaScanRange(); + metaScanRange.setMetadataType(TMetadataType.MATERIALIZED_VIEWS); + TMaterializedViewsMetadataParams mtmvParam = new TMaterializedViewsMetadataParams(); + mtmvParam.setDatabase(databaseName); + metaScanRange.setMaterializedViewsParams(mtmvParam); + return metaScanRange; + } + + @Override + public String getTableName() { + return "MaterializedViewsTableValuedFunction"; + } + + @Override + public List getTableColumns() throws AnalysisException { + return SCHEMA; + } +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index c38744ba35e1b7..00b5180fdd71ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -18,6 +18,8 @@ package org.apache.doris.tablefunction; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ClientPool; import org.apache.doris.common.Pair; @@ -26,6 +28,7 @@ import org.apache.doris.common.util.NetUtils; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.planner.external.iceberg.IcebergMetadataCache; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.QueryDetail; @@ -39,6 +42,7 @@ import org.apache.doris.thrift.TFetchSchemaTableDataResult; import org.apache.doris.thrift.TIcebergMetadataParams; import org.apache.doris.thrift.TIcebergQueryType; +import org.apache.doris.thrift.TMaterializedViewsMetadataParams; import org.apache.doris.thrift.TMetadataTableRequestParams; import org.apache.doris.thrift.TMetadataType; import org.apache.doris.thrift.TNetworkAddress; @@ -93,6 +97,9 @@ public static TFetchSchemaTableDataResult getMetadataTable(TFetchSchemaTableData case CATALOGS: result = catalogsMetadataResult(params); break; + case MATERIALIZED_VIEWS: + result = mtmvMetadataResult(params); + break; case QUERIES: result = queriesMetadataResult(params, request); break; @@ -476,5 +483,45 @@ private static long convertToDateTimeV2( return (long) microsecond | (long) second << 20 | (long) minute << 26 | (long) hour << 32 | (long) day << 37 | (long) month << 42 | (long) year << 46; } + + private static TFetchSchemaTableDataResult mtmvMetadataResult(TMetadataTableRequestParams params) { + if (!params.isSetMaterializedViewsMetadataParams()) { + return errorResult("MaterializedViews metadata params is not set."); + } + + TMaterializedViewsMetadataParams mtmvMetadataParams = params.getMaterializedViewsMetadataParams(); + String dbName = mtmvMetadataParams.getDatabase(); + List dataBatch = Lists.newArrayList(); + TFetchSchemaTableDataResult result = new TFetchSchemaTableDataResult(); + List tables; + try { + tables = Env.getCurrentEnv().getCatalogMgr() + .getCatalogOrAnalysisException(InternalCatalog.INTERNAL_CATALOG_NAME) + .getDbOrAnalysisException(dbName).getTables(); + } catch (AnalysisException e) { + return errorResult(e.getMessage()); + } + + for (Table table : tables) { + if (table instanceof MTMV) { + MTMV mv = (MTMV) table; + TRow trow = new TRow(); + trow.addToColumnValue(new TCell().setLongVal(mv.getId())); + trow.addToColumnValue(new TCell().setStringVal(mv.getName())); + trow.addToColumnValue(new TCell().setStringVal(mv.getJobInfo().getJobName())); + trow.addToColumnValue(new TCell().setStringVal(mv.getStatus().getState().name())); + trow.addToColumnValue(new TCell().setStringVal(mv.getStatus().getSchemaChangeDetail())); + trow.addToColumnValue(new TCell().setStringVal(mv.getStatus().getRefreshState().name())); + trow.addToColumnValue(new TCell().setStringVal(mv.getRefreshInfo().toString())); + trow.addToColumnValue(new TCell().setStringVal(mv.getQuerySql())); + trow.addToColumnValue(new TCell().setStringVal(mv.getEnvInfo().toString())); + trow.addToColumnValue(new TCell().setStringVal(mv.getMvProperties().toString())); + dataBatch.add(trow); + } + } + result.setDataBatch(dataBatch); + result.setStatus(new TStatus(TStatusCode.OK)); + return result; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java index cc84a29f5e9191..1cea2ce048bf84 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java @@ -41,6 +41,8 @@ public static Integer getColumnIndexFromColumnName(TMetadataType type, String co return WorkloadGroupsTableValuedFunction.getColumnIndexFromColumnName(columnName); case CATALOGS: return CatalogsTableValuedFunction.getColumnIndexFromColumnName(columnName); + case MATERIALIZED_VIEWS: + return MaterializedViewsTableValuedFunction.getColumnIndexFromColumnName(columnName); case QUERIES: return QueriesTableValuedFunction.getColumnIndexFromColumnName(columnName); default: diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java index 90472aa48455db..012f163440f15a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java @@ -66,6 +66,8 @@ public static TableValuedFunctionIf getTableFunction(String funcName, Map> showTaskMetaResult = sql_meta(showTasks) + logger.info("showTaskMetaResult: " + showTaskMetaResult.toString()) + int index = showTaskMetaResult.indexOf(['Status', 'CHAR']) + logger.info("index: " + index) + String status = "PENDING" + List> result + long startTime = System.currentTimeMillis() + long timeoutTimestamp = startTime + 5 * 60 * 1000 // 5 min + do { + result = sql(showTasks) + logger.info("result: " + result.toString()) + if (!result.isEmpty()) { + status = result.last().get(index) + } + logger.info("The state of ${showTasks} is ${status}") + Thread.sleep(1000); + } while (timeoutTimestamp > System.currentTimeMillis() && (status == 'PENDING' || status == 'RUNNING')) + if (status != "SUCCESS") { + logger.info("status is not success") + } + Assert.assertEquals("SUCCESS", status) + } + + String getJobName(String dbName, String mtmvName) { + String showMTMV = "select * from mtmvs('database'='${dbName}') where Name = '${mtmvName}'"; + logger.info(showMTMV) + List> showTaskMetaResult = sql_meta(showMTMV) + logger.info("showTaskMetaResult: " + showTaskMetaResult.toString()) + int index = showTaskMetaResult.indexOf(['JobName', 'TINYTEXT']) + logger.info("index: " + index) + List> result = sql(showMTMV) + logger.info("result: " + result.toString()) + if (result.isEmpty()) { + Assert.fail(); + } + return result.last().get(index); + } } diff --git a/regression-test/suites/job_p0/test_base_insert_job.groovy b/regression-test/suites/job_p0/test_base_insert_job.groovy new file mode 100644 index 00000000000000..76d38460ffb7de --- /dev/null +++ b/regression-test/suites/job_p0/test_base_insert_job.groovy @@ -0,0 +1,111 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.time.Instant; +import java.time.ZoneId; + +suite("test_base_insert_job") { + def tableName = "t_test_base_insert_job" + def jobName = "insert_recovery_test_base_insert_job" + sql """drop table if exists `${tableName}` force""" + sql """ + STOP JOB for ${jobName} + """ + + sql """ + CREATE TABLE IF NOT EXISTS `${tableName}` + ( + `timestamp` DATE NOT NULL COMMENT "['0000-01-01', '9999-12-31']", + `type` TINYINT NOT NULL COMMENT "[-128, 127]", + `user_id` BIGINT COMMENT "[-9223372036854775808, 9223372036854775807]" + ) + DUPLICATE KEY(`timestamp`, `type`) + DISTRIBUTED BY HASH(`type`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """ + CREATE JOB ${jobName} ON SCHEDULER every 1 second comment 'test' DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); + """ + Thread.sleep(2500) + def jobs = sql """select * from ${tableName}""" + println jobs + assert 3>=jobs.size() >= (2 as Boolean) //at least 2 records, some times 3 records + sql """ + STOP JOB for ${jobName} + """ + sql """drop table if exists `${tableName}` force """ + sql """ + CREATE TABLE IF NOT EXISTS `${tableName}` + ( + `timestamp` DATE NOT NULL COMMENT "['0000-01-01', '9999-12-31']", + `type` TINYINT NOT NULL COMMENT "[-128, 127]", + `user_id` BIGINT COMMENT "[-9223372036854775808, 9223372036854775807]" + ) + DUPLICATE KEY(`timestamp`, `type`) + DISTRIBUTED BY HASH(`type`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + def currentMs=System.currentTimeMillis()+1000; + def dateTime = LocalDateTime.ofInstant(Instant.ofEpochMilli(currentMs), ZoneId.systemDefault()); + + def formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + def startTime= dateTime.format(formatter); + sql """ + CREATE JOB ${jobName} ON SCHEDULER at '${startTime}' comment 'test' DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); + """ + + Thread.sleep(2500) + + def datas = sql """select * from ${tableName}""" + println datas + //assert datas.size() == 1 + try{ + sql """ + CREATE JOB ${jobName} ON SCHEDULER at '${startTime}' comment 'test' DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); + """ + } catch (Exception e) { + assert true + } + sql """ + STOP JOB for test_one_time_error_starts + """ + try{ + sql """ + CREATE JOB test_one_time_error_starts ON SCHEDULER at '2023-11-13 14:18:07' comment 'test' DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); + """ + } catch (Exception e) { + assert true + } + sql """ + STOP JOB for test_error_starts + """ + try{ + sql """ + CREATE JOB test_error_starts ON SCHEDULER every 1 second ends '2023-11-13 14:18:07' comment 'test' DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); + """ + } catch (Exception e) { + assert true + } + + +} diff --git a/regression-test/suites/mtmv_p0/test_base_mtmv.groovy b/regression-test/suites/mtmv_p0/test_base_mtmv.groovy new file mode 100644 index 00000000000000..62fd1cbbc9ca83 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_base_mtmv.groovy @@ -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. + +suite("test_base_mtmv") { + def tableName = "t_test_base_mtmv_user" + def mvName = "multi_mv_test_base_mtmv" + def dbName = "regression_test_mtmv_p0" + sql """drop table if exists `${tableName}`""" + + sql """ + CREATE TABLE IF NOT EXISTS `${tableName}` ( + event_day DATE, + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + INSERT INTO ${tableName} VALUES("2022-10-26",1,"clz"),("2022-10-28",2,"zhangsang"),("2022-10-29",3,"lisi"); + """ + + sql """drop materialized view if exists ${mvName};""" + + // IMMEDIATE MANUAL + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${tableName}; + """ + def jobName = getJobName("regression_test_mtmv_p0", mvName); + order_qt_status "select Name,State,RefreshState from mtmvs('database'='${dbName}') where Name='${mvName}'" + sql """ + REFRESH MATERIALIZED VIEW ${mvName} + """ + waitingMTMVTaskFinished(jobName) + order_qt_status "select Name,State,RefreshState from mtmvs('database'='${dbName}') where Name='${mvName}'" + + // alter table + sql """ + alter table ${tableName} add COLUMN new_col INT AFTER username; + """ + order_qt_status "select Name,State,RefreshState from mtmvs('database'='${dbName}') where Name='${mvName}'" + sql """ + alter table ${tableName} drop COLUMN new_col; + """ + sql """ + REFRESH MATERIALIZED VIEW ${mvName} + """ + waitingMTMVTaskFinished(jobName) + order_qt_status "select Name,State,RefreshState from mtmvs('database'='${dbName}') where Name='${mvName}'" + + // drop table + sql """ + drop table ${tableName} + """ + order_qt_status "select Name,State,RefreshState from mtmvs('database'='${dbName}') where Name='${mvName}'" + sql """ + CREATE TABLE IF NOT EXISTS `${tableName}` ( + event_day DATE, + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + REFRESH MATERIALIZED VIEW ${mvName} + """ + waitingMTMVTaskFinished(jobName) + order_qt_status "select Name,State,RefreshState from mtmvs('database'='${dbName}') where Name='${mvName}'" + sql """ + DROP MATERIALIZED VIEW ${mvName} + """ +} diff --git a/regression-test/suites/mtmv_p0/test_build_mtmv.groovy b/regression-test/suites/mtmv_p0/test_build_mtmv.groovy new file mode 100644 index 00000000000000..a08c49a8b98790 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_build_mtmv.groovy @@ -0,0 +1,279 @@ +// 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. + +suite("test_build_mtmv") { + def tableName = "t_test_create_mtmv_user" + def tableNamePv = "t_test_create_mtmv_user_pv" + def mvName = "multi_mv_test_create_mtmv" + def mvNameRenamed = "multi_mv_test_create_mtmv_renamed" + + sql """drop table if exists `${tableName}`""" + sql """drop table if exists `${tableNamePv}`""" + + sql """ + CREATE TABLE IF NOT EXISTS `${tableName}` ( + event_day DATE, + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + INSERT INTO ${tableName} VALUES("2022-10-26",1,"clz"),("2022-10-28",2,"zhangsang"),("2022-10-29",3,"lisi"); + """ + sql """ + create table IF NOT EXISTS ${tableNamePv}( + event_day DATE, + id BIGINT, + pv BIGINT + ) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + INSERT INTO ${tableNamePv} VALUES("2022-10-26",1,200),("2022-10-28",2,200),("2022-10-28",3,300); + """ + + sql """drop materialized view if exists ${mvName};""" + sql """drop materialized view if exists ${mvNameRenamed};""" + + // IMMEDIATE MANUAL + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + def jobName = getJobName("regression_test_mtmv_p0", mvName); + println jobName + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + sql """ + DROP MATERIALIZED VIEW ${mvName} + """ + + // IMMEDIATE schedule interval + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 SECOND + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + jobName = getJobName("regression_test_mtmv_p0", mvName); + println jobName + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + sql """ + DROP MATERIALIZED VIEW ${mvName} + """ + + // IMMEDIATE schedule interval and start + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 SECOND STARTS "2023-12-13 21:07:09" + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + jobName = getJobName("regression_test_mtmv_p0", mvName); + println jobName + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + sql """ + DROP MATERIALIZED VIEW ${mvName} + """ + + // DEFERRED MANUAL + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + sql """ + REFRESH MATERIALIZED VIEW ${mvName} + """ + jobName = getJobName("regression_test_mtmv_p0", mvName); + println jobName + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + sql """ + DROP MATERIALIZED VIEW ${mvName} + """ + + // DEFERRED schedule interval + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 SECOND + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + jobName = getJobName("regression_test_mtmv_p0", mvName); + println jobName + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + sql """ + DROP MATERIALIZED VIEW ${mvName} + """ + + // DEFERRED schedule interval and start + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 SECOND STARTS "2023-12-13 21:07:09" + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + jobName = getJobName("regression_test_mtmv_p0", mvName); + println jobName + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + sql """ + DROP MATERIALIZED VIEW ${mvName} + """ + + // random + try { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 SECOND STARTS "2023-12-13 21:07:09" + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT random() as dd, ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + Assert.fail(); + } catch (Exception e) { + log.info(e.getMessage()) + } + + // repeat cols + try { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 SECOND STARTS "2023-12-13 21:07:09" + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT ${tableName}.username, ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + Assert.fail(); + } catch (Exception e) { + log.info(e.getMessage()) + } + + // alter + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + jobName = getJobName("regression_test_mtmv_p0", mvName); + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + + // alter refreshMethod + sql """ + alter MATERIALIZED VIEW ${mvName} REFRESH COMPLETE; + """ + jobName = getJobName("regression_test_mtmv_p0", mvName); + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + + // alter refreshTrigger + sql """ + alter MATERIALIZED VIEW ${mvName} REFRESH ON MANUAL; + """ + jobName = getJobName("regression_test_mtmv_p0", mvName); + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + + // alter refreshMethod refreshTrigger + sql """ + alter MATERIALIZED VIEW ${mvName} REFRESH COMPLETE ON MANUAL; + """ + jobName = getJobName("regression_test_mtmv_p0", mvName); + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + + // alter rename + sql """ + alter Materialized View ${mvName} rename ${mvNameRenamed}; + """ + jobName = getJobName("regression_test_mtmv_p0", mvNameRenamed); + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvNameRenamed}" + + sql """ + DROP MATERIALIZED VIEW ${mvNameRenamed} + """ + + // drop + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + jobName = getJobName("regression_test_mtmv_p0", mvName); + waitingMTMVTaskFinished(jobName) + order_qt_select "SELECT * FROM ${mvName}" + + // test use drop table + try { + sql """ + drop table ${mvName}; + """ + Assert.fail(); + } catch (Exception e) { + log.info(e.getMessage()) + } + + // test use drop mv + + sql """ + DROP MATERIALIZED VIEW ${mvName} + """ + def jobs = sql """show mtmv job for ${jobName}""" + println jobs + assertEquals(jobs.size(), 0); + def tasks = sql """show mtmv job tasks for ${jobName}""" + println tasks + assertEquals(tasks.size(), 0); + +}