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 c40e4c4bec32fc..e2861310faa592 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 @@ -1613,7 +1613,7 @@ public class Config extends ConfigBase { "This parameter controls the time interval for automatic collection jobs to check the health of table" + "statistics and trigger automatic collection" }) - public static int auto_check_statistics_in_minutes = 5; + public static int auto_check_statistics_in_minutes = 1; /** * If set to TRUE, the compaction slower replica will be skipped when select get queryable replicas diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 4efd1cf61d4ded..fc0b116bbea967 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -4568,6 +4568,10 @@ show_param ::= {: RESULT = new ShowAnalyzeStmt(tbl, parser.where, true); :} + | KW_AUTO KW_JOBS opt_table_name:tbl opt_wild_where + {: + RESULT = new ShowAutoAnalyzeJobsStmt(tbl, parser.where); + :} | KW_ANALYZE KW_TASK KW_STATUS INTEGER_LITERAL:jobId {: RESULT = new ShowAnalyzeTaskStatus(jobId); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeProperties.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeProperties.java index 94083989ca7b83..f78c63ebea1a83 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeProperties.java @@ -44,6 +44,7 @@ public class AnalyzeProperties { public static final String PROPERTY_PERIOD_SECONDS = "period.seconds"; public static final String PROPERTY_FORCE_FULL = "force.full"; public static final String PROPERTY_PARTITION_COLUMN_FROM_SQL = "partition.column.from.sql"; + public static final String PROPERTY_USE_AUTO_ANALYZER = "use.auto.analyzer"; public static final AnalyzeProperties DEFAULT_PROP = new AnalyzeProperties(new HashMap() { { @@ -72,6 +73,7 @@ public class AnalyzeProperties { .add(PROPERTY_PERIOD_CRON) .add(PROPERTY_FORCE_FULL) .add(PROPERTY_PARTITION_COLUMN_FROM_SQL) + .add(PROPERTY_USE_AUTO_ANALYZER) .build(); public AnalyzeProperties(Map properties) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeStmt.java index 9ccfd956ca5d84..f660d6eeb3c6b5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeStmt.java @@ -62,6 +62,7 @@ public class ShowAnalyzeStmt extends ShowStmt { .add("schedule_type") .add("start_time") .add("end_time") + .add("priority") .build(); private long jobId; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java new file mode 100644 index 00000000000000..560387fa5bc11c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java @@ -0,0 +1,210 @@ +// 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.Env; +import org.apache.doris.catalog.ScalarType; +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.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.statistics.JobPriority; + +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; + +/** + * ShowAutoAnalyzeJobsStmt is used to show pending auto analysis jobs. + * syntax: + * SHOW AUTO ANALYZE JOBS + * [TABLE] + * [ + * WHERE + * [PRIORITY = ["HIGH"|"MID"|"LOW"]] + * ] + */ +public class ShowAutoAnalyzeJobsStmt extends ShowStmt { + private static final String PRIORITY = "priority"; + private static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("catalog_name") + .add("db_name") + .add("tbl_name") + .add("col_list") + .add("priority") + .build(); + + private final TableName tableName; + private final Expr whereClause; + + public ShowAutoAnalyzeJobsStmt(TableName tableName, Expr whereClause) { + this.tableName = tableName; + this.whereClause = whereClause; + } + + // extract from predicate + private String jobPriority; + + public String getPriority() { + Preconditions.checkArgument(isAnalyzed(), + "The stateValue must be obtained after the parsing is complete"); + return jobPriority; + } + + public Expr getWhereClause() { + Preconditions.checkArgument(isAnalyzed(), + "The whereClause must be obtained after the parsing is complete"); + return whereClause; + } + + @Override + public void analyze(Analyzer analyzer) throws UserException { + if (!ConnectContext.get().getSessionVariable().enableStats) { + throw new UserException("Analyze function is forbidden, you should add `enable_stats=true`" + + "in your FE conf file"); + } + super.analyze(analyzer); + if (tableName != null) { + tableName.analyze(analyzer); + String catalogName = tableName.getCtl(); + String dbName = tableName.getDb(); + String tblName = tableName.getTbl(); + checkShowAnalyzePriv(catalogName, dbName, tblName); + } + + // analyze where clause if not null + if (whereClause != null) { + analyzeSubPredicate(whereClause); + } + } + + @Override + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(128))); + } + return builder.build(); + } + + @Override + public RedirectStatus getRedirectStatus() { + return RedirectStatus.FORWARD_NO_SYNC; + } + + private void checkShowAnalyzePriv(String catalogName, String dbName, String tblName) throws AnalysisException { + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), catalogName, dbName, tblName, PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException( + ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, + "SHOW ANALYZE", + ConnectContext.get().getQualifiedUser(), + ConnectContext.get().getRemoteIP(), + dbName + ": " + tblName); + } + } + + private void analyzeSubPredicate(Expr subExpr) throws AnalysisException { + if (subExpr == null) { + return; + } + + boolean valid = true; + + CHECK: { + if (subExpr instanceof BinaryPredicate) { + BinaryPredicate binaryPredicate = (BinaryPredicate) subExpr; + if (binaryPredicate.getOp() != BinaryPredicate.Operator.EQ) { + valid = false; + break CHECK; + } + } else { + valid = false; + break CHECK; + } + + // left child + if (!(subExpr.getChild(0) instanceof SlotRef)) { + valid = false; + break CHECK; + } + String leftKey = ((SlotRef) subExpr.getChild(0)).getColumnName(); + if (!PRIORITY.equalsIgnoreCase(leftKey)) { + valid = false; + break CHECK; + } + + // right child + if (!(subExpr.getChild(1) instanceof StringLiteral)) { + valid = false; + break CHECK; + } + + String value = subExpr.getChild(1).getStringValue(); + if (Strings.isNullOrEmpty(value)) { + valid = false; + break CHECK; + } + + jobPriority = value.toUpperCase(); + try { + JobPriority.valueOf(jobPriority); + } catch (Exception e) { + valid = false; + } + } + + if (!valid) { + throw new AnalysisException("Where clause should looks like: " + + "PRIORITY = \"HIGH|MID|LOW\""); + } + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("SHOW AUTO ANALYZE"); + + if (tableName != null) { + sb.append(" "); + sb.append(tableName.toSql()); + } + + if (whereClause != null) { + sb.append(" "); + sb.append("WHERE"); + sb.append(" "); + sb.append(whereClause.toSql()); + } + + return sb.toString(); + } + + @Override + public String toString() { + return toSql(); + } + + public TableName getTableName() { + return tableName; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java index 18bb916b8bdfce..cfe2d426f7b166 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java @@ -62,6 +62,8 @@ public class ShowColumnStatsStmt extends ShowStmt { .add("trigger") .add("query_times") .add("updated_time") + .add("update_rows") + .add("last_analyze_row_count") .build(); private final TableName tableName; @@ -162,6 +164,8 @@ public ShowResultSet constructResultSet(List, ColumnSt row.add(String.valueOf(colStatsMeta == null ? "N/A" : colStatsMeta.jobType)); row.add(String.valueOf(colStatsMeta == null ? "N/A" : colStatsMeta.queriedTimes)); row.add(String.valueOf(p.second.updatedTime)); + row.add(String.valueOf(colStatsMeta == null ? "N/A" : colStatsMeta.updatedRows)); + row.add(String.valueOf(colStatsMeta == null ? "N/A" : colStatsMeta.rowCount)); result.add(row); }); return new ShowResultSet(getMetaData(), result); 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 0ad32f76566768..275c4d1ff42171 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 @@ -244,9 +244,11 @@ import org.apache.doris.service.ExecuteEnv; import org.apache.doris.service.FrontendOptions; import org.apache.doris.statistics.AnalysisManager; +import org.apache.doris.statistics.FollowerColumnSender; import org.apache.doris.statistics.StatisticsAutoCollector; import org.apache.doris.statistics.StatisticsCache; import org.apache.doris.statistics.StatisticsCleaner; +import org.apache.doris.statistics.StatisticsJobAppender; import org.apache.doris.statistics.query.QueryStats; import org.apache.doris.system.Backend; import org.apache.doris.system.Frontend; @@ -524,6 +526,10 @@ public class Env { private StatisticsAutoCollector statisticsAutoCollector; + private StatisticsJobAppender statisticsJobAppender; + + private FollowerColumnSender followerColumnSender; + private HiveTransactionMgr hiveTransactionMgr; private TopicPublisherThread topicPublisherThread; @@ -756,6 +762,7 @@ public Env(boolean isCheckpointCatalog) { this.analysisManager = new AnalysisManager(); this.statisticsCleaner = new StatisticsCleaner(); this.statisticsAutoCollector = new StatisticsAutoCollector(); + this.statisticsJobAppender = new StatisticsJobAppender(); this.globalFunctionMgr = new GlobalFunctionMgr(); this.workloadGroupMgr = new WorkloadGroupMgr(); this.workloadSchedPolicyMgr = new WorkloadSchedPolicyMgr(); @@ -1058,13 +1065,6 @@ public void initialize(String[] args) throws Exception { // If not using bdb, we need to notify the FE type transfer manually. notifyNewFETypeTransfer(FrontendNodeType.MASTER); } - if (statisticsCleaner != null) { - statisticsCleaner.start(); - } - if (statisticsAutoCollector != null) { - statisticsAutoCollector.start(); - } - queryCancelWorker.start(); } @@ -1715,6 +1715,10 @@ protected void startMasterOnlyDaemonThreads() { topicPublisherThread.addToTopicPublisherList(wpPublisher); topicPublisherThread.start(); + // auto analyze related threads. + statisticsCleaner.start(); + statisticsAutoCollector.start(); + statisticsJobAppender.start(); } // start threads that should run on all FE @@ -1777,6 +1781,11 @@ private void transferToNonMaster(FrontendNodeType newType) { if (analysisManager != null) { analysisManager.getStatisticsCache().preHeat(); } + + if (followerColumnSender == null) { + followerColumnSender = new FollowerColumnSender(); + followerColumnSender.start(); + } } // Set global variable 'lower_case_table_names' only when the cluster is initialized. @@ -6113,6 +6122,10 @@ public NereidsSqlCacheManager getSqlCacheManager() { return sqlCacheManager; } + public StatisticsJobAppender getStatisticsJobAppender() { + return statisticsJobAppender; + } + public void alterMTMVRefreshInfo(AlterMTMVRefreshInfo info) { AlterMTMV alter = new AlterMTMV(info.getMvName(), info.getRefreshInfo(), MTMVAlterOpType.ALTER_REFRESH_INFO); this.alter.processAlterMTMV(alter, false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 3932496f661c5a..a4c79d91890c1b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -64,7 +64,6 @@ import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.HistogramTask; import org.apache.doris.statistics.OlapAnalysisTask; -import org.apache.doris.statistics.TableStatsMeta; import org.apache.doris.statistics.util.StatisticsUtil; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; @@ -802,9 +801,20 @@ public List getSchemaByIndexId(Long indexId, boolean full) { } @Override - public List getSchemaAllIndexes(boolean full) { + public Set getSchemaAllIndexes(boolean full) { + Set columns = Sets.newHashSet(); + for (Long indexId : indexIdToMeta.keySet()) { + columns.addAll(getSchemaByIndexId(indexId, full)); + } + return columns; + } + + public List getMvColumns(boolean full) { List columns = Lists.newArrayList(); for (Long indexId : indexIdToMeta.keySet()) { + if (indexId == baseIndexId) { + continue; + } columns.addAll(getSchemaByIndexId(indexId, full)); } return columns; @@ -1323,29 +1333,9 @@ public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { } } - public boolean needReAnalyzeTable(TableStatsMeta tblStats) { - if (tblStats == null) { - return true; - } - if (!tblStats.analyzeColumns().containsAll(getColumnIndexPairs(getSchemaAllIndexes(false) - .stream() - .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) - .map(Column::getName) - .collect(Collectors.toSet())))) { - return true; - } - long rowCount = getRowCount(); - if (rowCount > 0 && tblStats.rowCount == 0) { - return true; - } - long updateRows = tblStats.updatedRows.get(); - int tblHealth = StatisticsUtil.getTableHealth(rowCount, updateRows); - return tblHealth < StatisticsUtil.getTableStatsHealthThreshold(); - } - @Override - public List> getColumnIndexPairs(Set columns) { - List> ret = Lists.newArrayList(); + public Set> getColumnIndexPairs(Set columns) { + Set> ret = Sets.newHashSet(); // Check the schema of all indexes for each given column name, // If the column name exists in the index, add the pair to return list. for (String column : columns) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index 136a0e04f2c253..9de006cebb5c46 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -34,13 +34,13 @@ import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ColumnStatistic; -import org.apache.doris.statistics.TableStatsMeta; import org.apache.doris.thrift.TTableDescriptor; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.google.gson.annotations.SerializedName; import org.apache.commons.lang3.NotImplementedException; import org.apache.commons.lang3.StringUtils; @@ -391,11 +391,6 @@ public List getBaseSchema() { return getBaseSchema(Util.showHiddenColumns()); } - @Override - public List getSchemaAllIndexes(boolean full) { - return getBaseSchema(); - } - public List getBaseSchema(boolean full) { if (full) { return fullSchema; @@ -623,11 +618,6 @@ public Optional getColumnStatistic(String colName) { public void analyze(String dbName) {} - @Override - public boolean needReAnalyzeTable(TableStatsMeta tblStats) { - return true; - } - @Override public List getChunkSizes() { throw new NotImplementedException("getChunkSized not implemented"); @@ -639,8 +629,8 @@ public long fetchRowCount() { } @Override - public List> getColumnIndexPairs(Set columns) { - return Lists.newArrayList(); + public Set> getColumnIndexPairs(Set columns) { + return Sets.newHashSet(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java index 1ba8ee30766667..c5039660e6eeae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java @@ -31,7 +31,6 @@ import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ColumnStatistic; -import org.apache.doris.statistics.TableStatsMeta; import org.apache.doris.thrift.TTableDescriptor; import com.google.common.collect.ImmutableList; @@ -120,7 +119,11 @@ default boolean tryWriteLockIfExist(long timeout, TimeUnit unit) { List getBaseSchema(); - List getSchemaAllIndexes(boolean full); + default Set getSchemaAllIndexes(boolean full) { + Set ret = Sets.newHashSet(); + ret.addAll(getBaseSchema()); + return ret; + } default List getBaseSchemaOrEmpty() { try { @@ -188,13 +191,11 @@ default long getRowCountForNereids() { Optional getColumnStatistic(String colName); - boolean needReAnalyzeTable(TableStatsMeta tblStats); - /** * @param columns Set of column names. - * @return List of pairs. Each pair is . For external table, index name is table name. + * @return Set of pairs. Each pair is . For external table, index name is table name. */ - List> getColumnIndexPairs(Set columns); + Set> getColumnIndexPairs(Set columns); // Get all the chunk sizes of this table. Now, only HMS external table implemented this interface. // For HMS external table, the return result is a list of all the files' size. diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index 952b5c64cf8fd5..11226fc9d78aa9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -33,11 +33,10 @@ import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ColumnStatistic; -import org.apache.doris.statistics.TableStatsMeta; import org.apache.doris.statistics.util.StatisticsUtil; import org.apache.doris.thrift.TTableDescriptor; -import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.google.gson.annotations.SerializedName; import lombok.Getter; import org.apache.commons.lang3.NotImplementedException; @@ -51,7 +50,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.stream.Collectors; /** * External table represent tables that are not self-managed by Doris. @@ -151,11 +149,6 @@ public List getBaseSchema() { return getFullSchema(); } - @Override - public List getSchemaAllIndexes(boolean full) { - return getBaseSchema(); - } - @Override public List getBaseSchema(boolean full) { return getFullSchema(); @@ -331,25 +324,8 @@ public void gsonPostProcess() throws IOException { } @Override - public boolean needReAnalyzeTable(TableStatsMeta tblStats) { - if (tblStats == null) { - return true; - } - if (!tblStats.analyzeColumns().containsAll(getColumnIndexPairs( - getBaseSchema() - .stream() - .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) - .map(Column::getName) - .collect(Collectors.toSet())))) { - return true; - } - return System.currentTimeMillis() - - tblStats.updatedTime > StatisticsUtil.getExternalTableAutoAnalyzeIntervalInMillis(); - } - - @Override - public List> getColumnIndexPairs(Set columns) { - List> ret = Lists.newArrayList(); + public Set> getColumnIndexPairs(Set columns) { + Set> ret = Sets.newHashSet(); for (String column : columns) { Column col = getColumn(column); if (col == null || StatisticsUtil.isUnsupportedType(col.getType())) { 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 10983a955b79c1..6f5c1ca2bbdfe1 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 @@ -3202,7 +3202,6 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti rowsToTruncate += partition.getBaseIndex().getRowCount(); } } else { - rowsToTruncate = olapTable.getRowCount(); for (Partition partition : olapTable.getPartitions()) { // If need absolutely correct, should check running txn here. // But if the txn is in prepare state, cann't known which partitions had load data. @@ -3211,6 +3210,7 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti } origPartitions.put(partition.getName(), partition.getId()); partitionsDistributionInfo.put(partition.getId(), partition.getDistributionInfo()); + rowsToTruncate += partition.getBaseIndex().getRowCount(); } } // if table currently has no partitions, this sql like empty command and do nothing, should return directly. @@ -3371,10 +3371,8 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti if (truncateEntireTable) { // Drop the whole table stats after truncate the entire table Env.getCurrentEnv().getAnalysisManager().dropStats(olapTable); - } else { - // Update the updated rows in table stats after truncate some partitions. - Env.getCurrentEnv().getAnalysisManager().updateUpdatedRows(updateRecords); } + Env.getCurrentEnv().getAnalysisManager().updateUpdatedRows(updateRecords); LOG.info("finished to truncate table {}, partitions: {}", tblRef.getName().toSql(), tblRef.getPartitionNames()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index c4e67d6bc1260f..51d2f4f44d0d55 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -32,6 +32,7 @@ import org.apache.doris.nereids.rules.expression.ExpressionNormalization; import org.apache.doris.nereids.rules.expression.ExpressionNormalizationAndOptimization; import org.apache.doris.nereids.rules.expression.ExpressionRewrite; +import org.apache.doris.nereids.rules.expression.QueryColumnCollector; import org.apache.doris.nereids.rules.rewrite.AddDefaultLimit; import org.apache.doris.nereids.rules.rewrite.AdjustConjunctsReturnType; import org.apache.doris.nereids.rules.rewrite.AdjustNullable; @@ -417,7 +418,8 @@ public class Rewriter extends AbstractBatchJobExecutor { new CollectFilterAboveConsumer(), new CollectProjectAboveConsumer() ) - ) + ), + topic("Collect used column", custom(RuleType.COLLECT_COLUMNS, QueryColumnCollector::new)) ); private static final List WHOLE_TREE_REWRITE_JOBS diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index 10004953cdf195..f1a797f4e2bc75 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -311,6 +311,7 @@ public enum RuleType { LEADING_JOIN(RuleTypeClass.REWRITE), REWRITE_SENTINEL(RuleTypeClass.REWRITE), + COLLECT_COLUMNS(RuleTypeClass.REWRITE), // topn opts DEFER_MATERIALIZE_TOP_N_RESULT(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/QueryColumnCollector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/QueryColumnCollector.java new file mode 100644 index 00000000000000..ebf361de1d3a9a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/QueryColumnCollector.java @@ -0,0 +1,215 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.expression; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.rules.expression.QueryColumnCollector.CollectorContext; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalHaving; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.statistics.AnalysisManager; +import org.apache.doris.statistics.util.StatisticsUtil; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Used to collect query column. + */ +public class QueryColumnCollector extends DefaultPlanRewriter implements CustomRewriter { + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + ConnectContext connectContext = ConnectContext.get(); + if (connectContext != null && connectContext.getSessionVariable().internalSession) { + return plan; + } + CollectorContext context = new CollectorContext(); + plan.accept(this, context); + if (StatisticsUtil.enableAutoAnalyze()) { + context.midPriority.removeAll(context.highPriority); + AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); + analysisManager.updateHighPriorityColumn(context.highPriority); + analysisManager.updateMidPriorityColumn(context.midPriority); + } + return plan; + } + + /** + * Context. + */ + public static class CollectorContext { + public Map projects = new HashMap<>(); + + public Set highPriority = new HashSet<>(); + + public Set midPriority = new HashSet<>(); + } + + @Override + public Plan visitLogicalProject(LogicalProject project, CollectorContext context) { + project.child().accept(this, context); + List projects = project.getOutputs(); + List slots = project.computeOutput(); + for (int i = 0; i < slots.size(); i++) { + context.projects.put(slots.get(i), projects.get(i)); + } + if (project.child() instanceof LogicalCatalogRelation + || project.child() instanceof LogicalFilter + && ((LogicalFilter) project.child()).child() instanceof LogicalCatalogRelation) { + Set allUsed = project.getExpressions() + .stream().flatMap(e -> e.>collect(n -> n instanceof SlotReference).stream()) + .collect(Collectors.toSet()); + LogicalCatalogRelation scan = project.child() instanceof LogicalCatalogRelation + ? (LogicalCatalogRelation) project.child() + : (LogicalCatalogRelation) project.child().child(0); + List outputOfScan = scan.getOutput(); + for (Slot slot : outputOfScan) { + if (!allUsed.contains(slot)) { + context.midPriority.remove(slot); + } + } + } + return project; + } + + @Override + public Plan visitLogicalJoin(LogicalJoin join, CollectorContext context) { + join.child(0).accept(this, context); + join.child(1).accept(this, context); + context.highPriority.addAll( + (join.isMarkJoin() ? join.getLeftConditionSlot() : join.getConditionSlot()) + .stream().flatMap(s -> backtrace(s, context).stream()) + .collect(Collectors.toSet()) + ); + return join; + } + + @Override + public Plan visitLogicalAggregate(LogicalAggregate aggregate, CollectorContext context) { + aggregate.child(0).accept(this, context); + context.highPriority.addAll(aggregate.getGroupByExpressions() + .stream() + .flatMap(e -> e.>collect(n -> n instanceof SlotReference).stream()) + .flatMap(s -> backtrace(s, context).stream()) + .collect(Collectors.toSet())); + return aggregate; + } + + @Override + public Plan visitLogicalHaving(LogicalHaving having, CollectorContext context) { + having.child(0).accept(this, context); + context.highPriority.addAll( + having.getExpressions().stream() + .flatMap(e -> e.>collect(n -> n instanceof SlotReference).stream()) + .flatMap(s -> backtrace(s, context).stream()) + .collect(Collectors.toSet())); + return having; + } + + @Override + public Plan visitLogicalOlapScan(LogicalOlapScan olapScan, CollectorContext context) { + List slots = olapScan.getOutput(); + context.midPriority.addAll(slots); + return olapScan; + } + + @Override + public Plan visitLogicalFileScan(LogicalFileScan fileScan, CollectorContext context) { + List slots = fileScan.getOutput(); + context.midPriority.addAll(slots); + return fileScan; + } + + @Override + public Plan visitLogicalFilter(LogicalFilter filter, CollectorContext context) { + filter.child(0).accept(this, context); + context.highPriority.addAll(filter + .getExpressions() + .stream() + .flatMap(e -> e.>collect(n -> n instanceof SlotReference).stream()) + .flatMap(s -> backtrace(s, context).stream()) + .collect(Collectors.toSet())); + return filter; + } + + @Override + public Plan visitLogicalWindow(LogicalWindow window, CollectorContext context) { + window.child(0).accept(this, context); + context.highPriority.addAll(window + .getWindowExpressions() + .stream() + .flatMap(e -> e.>collect(n -> n instanceof SlotReference).stream()) + .flatMap(s -> backtrace(s, context).stream()) + .collect(Collectors.toSet())); + return window; + } + + private Set backtrace(Slot slot, CollectorContext context) { + return backtrace(slot, new HashSet<>(), context); + } + + private Set backtrace(Slot slot, Set path, CollectorContext context) { + if (path.contains(slot)) { + return Collections.emptySet(); + } + path.add(slot); + if (slot instanceof SlotReference) { + SlotReference slotReference = (SlotReference) slot; + Optional col = slotReference.getColumn(); + Optional table = slotReference.getTable(); + if (col.isPresent() && table.isPresent()) { + return Collections.singleton(slot); + } + } + NamedExpression namedExpression = context.projects.get(slot); + if (namedExpression == null) { + return Collections.emptySet(); + } + Set slotReferences + = namedExpression.>collect(n -> n instanceof SlotReference); + Set refCol = new HashSet<>(); + for (SlotReference slotReference : slotReferences) { + refCol.addAll(backtrace(slotReference, path, context)); + } + return refCol; + } + +} 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 2037f5fa1efb27..4152a6b77b45a6 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 @@ -480,6 +480,8 @@ public class SessionVariable implements Serializable, Writable { public static final String FORCE_SAMPLE_ANALYZE = "force_sample_analyze"; + public static final String ENABLE_AUTO_ANALYZE_INTERNAL_CATALOG = "enable_auto_analyze_internal_catalog"; + public static final String AUTO_ANALYZE_TABLE_WIDTH_THRESHOLD = "auto_analyze_table_width_threshold"; public static final String FASTER_FLOAT_CONVERT = "faster_float_convert"; @@ -1551,6 +1553,11 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { flag = VariableMgr.GLOBAL) public boolean forceSampleAnalyze = Config.force_sample_analyze; + @VariableMgr.VarAttr(name = ENABLE_AUTO_ANALYZE_INTERNAL_CATALOG, + description = {"临时参数,收否自动收集所有内表", "Temp variable, enable to auto collect all OlapTable."}, + flag = VariableMgr.GLOBAL) + public boolean enableAutoAnalyzeInternalCatalog = true; + @VariableMgr.VarAttr(name = AUTO_ANALYZE_TABLE_WIDTH_THRESHOLD, description = {"参与自动收集的最大表宽度,列数多于这个参数的表不参与自动收集", "Maximum table width to enable auto analyze, " 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 f56a9993459428..e6fbef298890b8 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 @@ -30,6 +30,7 @@ import org.apache.doris.analysis.ShowAnalyzeStmt; import org.apache.doris.analysis.ShowAnalyzeTaskStatus; import org.apache.doris.analysis.ShowAuthorStmt; +import org.apache.doris.analysis.ShowAutoAnalyzeJobsStmt; import org.apache.doris.analysis.ShowBackendsStmt; import org.apache.doris.analysis.ShowBackupStmt; import org.apache.doris.analysis.ShowBrokerStmt; @@ -213,6 +214,7 @@ import org.apache.doris.qe.help.HelpTopic; import org.apache.doris.rpc.RpcException; import org.apache.doris.statistics.AnalysisInfo; +import org.apache.doris.statistics.AutoAnalysisPendingJob; import org.apache.doris.statistics.ColumnStatistic; import org.apache.doris.statistics.Histogram; import org.apache.doris.statistics.ResultRow; @@ -455,6 +457,8 @@ public ShowResultSet execute() throws AnalysisException { handleShowCreateCatalog(); } else if (stmt instanceof ShowAnalyzeStmt) { handleShowAnalyze(); + } else if (stmt instanceof ShowAutoAnalyzeJobsStmt) { + handleShowAutoAnalyzePendingJobs(); } else if (stmt instanceof ShowTabletsBelongStmt) { handleShowTabletsBelong(); } else if (stmt instanceof AdminCopyTabletStmt) { @@ -2855,6 +2859,7 @@ private void handleShowAnalyze() { java.time.ZoneId.systemDefault()); row.add(startTime.format(formatter)); row.add(endTime.format(formatter)); + row.add(analysisInfo.priority.name()); resultRows.add(row); } catch (Exception e) { LOG.warn("Failed to get analyze info for table {}.{}.{}, reason: {}", @@ -2865,6 +2870,35 @@ private void handleShowAnalyze() { resultSet = new ShowResultSet(showStmt.getMetaData(), resultRows); } + private void handleShowAutoAnalyzePendingJobs() { + ShowAutoAnalyzeJobsStmt showStmt = (ShowAutoAnalyzeJobsStmt) stmt; + List jobs = Env.getCurrentEnv().getAnalysisManager().showAutoPendingJobs(showStmt); + List> resultRows = Lists.newArrayList(); + for (AutoAnalysisPendingJob job : jobs) { + try { + List row = new ArrayList<>(); + CatalogIf> c = StatisticsUtil.findCatalog(job.catalogName); + row.add(c.getName()); + Optional> databaseIf = c.getDb(job.dbName); + row.add(databaseIf.isPresent() ? databaseIf.get().getFullName() : "DB may get deleted"); + if (databaseIf.isPresent()) { + Optional table = databaseIf.get().getTable(job.tableName); + row.add(table.isPresent() ? table.get().getName() : "Table may get deleted"); + } else { + row.add("DB may get deleted"); + } + row.add(job.getColumnNames()); + row.add(String.valueOf(job.priority)); + resultRows.add(row); + } catch (Exception e) { + LOG.warn("Failed to get pending jobs for table {}.{}.{}, reason: {}", + job.catalogName, job.dbName, job.tableName, e.getMessage()); + continue; + } + } + resultSet = new ShowResultSet(showStmt.getMetaData(), resultRows); + } + private void handleShowTabletsBelong() { ShowTabletsBelongStmt showStmt = (ShowTabletsBelongStmt) stmt; List> rows = new ArrayList<>(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index d5f0a5aafd7939..9dc4028203d91d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -225,6 +225,7 @@ import org.apache.doris.thrift.TStreamLoadMultiTablePutResult; import org.apache.doris.thrift.TStreamLoadPutRequest; import org.apache.doris.thrift.TStreamLoadPutResult; +import org.apache.doris.thrift.TSyncQueryColumns; import org.apache.doris.thrift.TTableIndexQueryStats; import org.apache.doris.thrift.TTableMetadataNameIds; import org.apache.doris.thrift.TTableQueryStats; @@ -3775,4 +3776,11 @@ public TShowUserResult showUser(TShowUserRequest request) { result.setUserinfoList(userInfo); return result; } + + public TStatus syncQueryColumns(TSyncQueryColumns request) throws TException { + Env.getCurrentEnv().getAnalysisManager().mergeFollowerQueryColumns(request.highPriorityColumns, + request.midPriorityColumns); + return new TStatus(TStatusCode.OK); + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java index c167db2228d8cc..e0fd91d1100354 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java @@ -96,7 +96,7 @@ public enum ScheduleType { public final long tblId; // Pair - public final List> jobColumns; + public final Set> jobColumns; public final Set partitionNames; @@ -188,8 +188,11 @@ public enum ScheduleType { @SerializedName("endTime") public long endTime; - @SerializedName("emptyJob") - public final boolean emptyJob; + @SerializedName("rowCount") + public final long rowCount; + + @SerializedName("updateRows") + public final long updateRows; /** * * Used to store the newest partition version of tbl when creating this job. @@ -197,16 +200,21 @@ public enum ScheduleType { */ public final long tblUpdateTime; + @SerializedName("userInject") public final boolean userInject; + @SerializedName("priority") + public final JobPriority priority; + public AnalysisInfo(long jobId, long taskId, List taskIds, long catalogId, long dbId, long tblId, - List> jobColumns, Set partitionNames, String colName, Long indexId, + Set> jobColumns, Set partitionNames, String colName, Long indexId, JobType jobType, AnalysisMode analysisMode, AnalysisMethod analysisMethod, AnalysisType analysisType, int samplePercent, long sampleRows, int maxBucketNum, long periodTimeInMs, String message, long lastExecTimeInMs, long timeCostInMs, AnalysisState state, ScheduleType scheduleType, boolean isExternalTableLevelTask, boolean partitionOnly, boolean samplingPartition, boolean isAllPartition, long partitionCount, CronExpression cronExpression, boolean forceFull, - boolean usingSqlForPartitionColumn, long tblUpdateTime, boolean emptyJob, boolean userInject) { + boolean usingSqlForPartitionColumn, long tblUpdateTime, long rowCount, boolean userInject, + long updateRows, JobPriority priority) { this.jobId = jobId; this.taskId = taskId; this.taskIds = taskIds; @@ -242,8 +250,10 @@ public AnalysisInfo(long jobId, long taskId, List taskIds, long catalogId, this.forceFull = forceFull; this.usingSqlForPartitionColumn = usingSqlForPartitionColumn; this.tblUpdateTime = tblUpdateTime; - this.emptyJob = emptyJob; + this.rowCount = rowCount; this.userInject = userInject; + this.updateRows = updateRows; + this.priority = priority; } @Override @@ -285,7 +295,10 @@ public String toString() { } sj.add("forceFull: " + forceFull); sj.add("usingSqlForPartitionColumn: " + usingSqlForPartitionColumn); - sj.add("emptyJob: " + emptyJob); + sj.add("rowCount: " + rowCount); + sj.add("userInject: " + userInject); + sj.add("updateRows: " + updateRows); + sj.add("priority: " + priority.name()); return sj.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java index 00cf9f7b1bc560..83da112d33a366 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java @@ -36,7 +36,7 @@ public class AnalysisInfoBuilder { private long catalogId; private long dbId; private long tblId; - private List> jobColumns; + private Set> jobColumns; private Set partitionNames; private String colName; private long indexId = -1L; @@ -62,8 +62,10 @@ public class AnalysisInfoBuilder { private boolean forceFull; private boolean usingSqlForPartitionColumn; private long tblUpdateTime; - private boolean emptyJob; + private long rowCount; private boolean userInject; + private long updateRows; + private JobPriority priority; public AnalysisInfoBuilder() { } @@ -101,8 +103,10 @@ public AnalysisInfoBuilder(AnalysisInfo info) { forceFull = info.forceFull; usingSqlForPartitionColumn = info.usingSqlForPartitionColumn; tblUpdateTime = info.tblUpdateTime; - emptyJob = info.emptyJob; + rowCount = info.rowCount; userInject = info.userInject; + updateRows = info.updateRows; + priority = info.priority; } public AnalysisInfoBuilder setJobId(long jobId) { @@ -135,7 +139,7 @@ public AnalysisInfoBuilder setTblId(long tblId) { return this; } - public AnalysisInfoBuilder setJobColumns(List> jobColumns) { + public AnalysisInfoBuilder setJobColumns(Set> jobColumns) { this.jobColumns = jobColumns; return this; } @@ -265,8 +269,8 @@ public AnalysisInfoBuilder setTblUpdateTime(long tblUpdateTime) { return this; } - public AnalysisInfoBuilder setEmptyJob(boolean emptyJob) { - this.emptyJob = emptyJob; + public AnalysisInfoBuilder setRowCount(long rowCount) { + this.rowCount = rowCount; return this; } @@ -275,12 +279,23 @@ public AnalysisInfoBuilder setUserInject(boolean userInject) { return this; } + public AnalysisInfoBuilder setUpdateRows(long updateRows) { + this.updateRows = updateRows; + return this; + } + + public AnalysisInfoBuilder setPriority(JobPriority priority) { + this.priority = priority; + return this; + } + public AnalysisInfo build() { return new AnalysisInfo(jobId, taskId, taskIds, catalogId, dbId, tblId, jobColumns, partitionNames, colName, indexId, jobType, analysisMode, analysisMethod, analysisType, samplePercent, sampleRows, maxBucketNum, periodTimeInMs, message, lastExecTimeInMs, timeCostInMs, state, scheduleType, externalTableLevelTask, partitionOnly, samplingPartition, isAllPartition, partitionCount, - cronExpression, forceFull, usingSqlForPartitionColumn, tblUpdateTime, emptyJob, userInject); + cronExpression, forceFull, usingSqlForPartitionColumn, tblUpdateTime, rowCount, userInject, updateRows, + priority); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java index 5fd5e43be53f2b..0bc0a437898c71 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java @@ -84,14 +84,12 @@ public synchronized void rowCountDone(BaseAnalysisTask task) { protected void markOneTaskDone() { if (queryingTask.isEmpty()) { try { - writeBuf(); - updateTaskState(AnalysisState.FINISHED, "Cost time in sec: " - + (System.currentTimeMillis() - start) / 1000); + flushBuffer(); } finally { deregisterJob(); } } else if (buf.size() >= StatisticsUtil.getInsertMergeCount()) { - writeBuf(); + flushBuffer(); } } @@ -115,7 +113,7 @@ public void updateTaskState(AnalysisState state, String msg) { } } - protected void writeBuf() { + protected void flushBuffer() { if (killed) { return; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java index a2b30cce2ac5f4..03314fe7748a13 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java @@ -25,6 +25,7 @@ import org.apache.doris.analysis.DropStatsStmt; import org.apache.doris.analysis.KillAnalysisJobStmt; import org.apache.doris.analysis.ShowAnalyzeStmt; +import org.apache.doris.analysis.ShowAutoAnalyzeJobsStmt; import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.DatabaseIf; @@ -48,6 +49,8 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.persist.AnalyzeDeletionLog; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ConnectContext; @@ -63,6 +66,7 @@ import org.apache.doris.system.Frontend; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TInvalidateFollowerStatsCacheRequest; +import org.apache.doris.thrift.TQueryColumn; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; @@ -83,15 +87,19 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.NavigableMap; +import java.util.Objects; import java.util.Optional; +import java.util.Queue; import java.util.Set; import java.util.StringJoiner; import java.util.TreeMap; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; @@ -104,6 +112,14 @@ public class AnalysisManager implements Writable { private static final Logger LOG = LogManager.getLogger(AnalysisManager.class); + public static final int COLUMN_QUEUE_SIZE = 1000; + public final Queue highPriorityColumns = new ArrayBlockingQueue<>(COLUMN_QUEUE_SIZE); + public final Queue midPriorityColumns = new ArrayBlockingQueue<>(COLUMN_QUEUE_SIZE); + // Map>> + public final Map>> highPriorityJobs = new LinkedHashMap<>(); + public final Map>> midPriorityJobs = new LinkedHashMap<>(); + public final Map>> lowPriorityJobs = new LinkedHashMap<>(); + // Tracking running manually submitted async tasks, keep in mem only protected final ConcurrentMap> analysisJobIdToTaskMap = new ConcurrentHashMap<>(); @@ -154,13 +170,8 @@ public void createAnalyze(AnalyzeStmt analyzeStmt, boolean proxy) throws DdlExce } } - public void createAnalysisJobs(AnalyzeDBStmt analyzeDBStmt, boolean proxy) throws DdlException, AnalysisException { + public void createAnalysisJobs(AnalyzeDBStmt analyzeDBStmt, boolean proxy) throws AnalysisException { DatabaseIf db = analyzeDBStmt.getDb(); - // Using auto analyzer if user specifies. - if (analyzeDBStmt.getAnalyzeProperties().getProperties().containsKey("use.auto.analyzer")) { - Env.getCurrentEnv().getStatisticsAutoCollector().analyzeDb(db); - return; - } List analysisInfos = buildAnalysisInfosForDB(db, analyzeDBStmt.getAnalyzeProperties()); if (!analyzeDBStmt.isSync()) { sendJobId(analysisInfos, proxy); @@ -178,9 +189,8 @@ public List buildAnalysisInfosForDB(DatabaseIf db, Analyz if (table instanceof View) { continue; } - TableName tableName = new TableName(db.getCatalog().getName(), db.getFullName(), - table.getName()); - // columnNames null means to add all visitable columns. + TableName tableName = new TableName(db.getCatalog().getName(), db.getFullName(), table.getName()); + // columnNames null means to add all visible columns. // Will get all the visible columns in analyzeTblStmt.check() AnalyzeTblStmt analyzeTblStmt = new AnalyzeTblStmt(analyzeProperties, tableName, null, db.getId(), table); @@ -208,6 +218,13 @@ public List buildAnalysisInfosForDB(DatabaseIf db, Analyz // Each analyze stmt corresponding to an analysis job. public void createAnalysisJob(AnalyzeTblStmt stmt, boolean proxy) throws DdlException { + // Using auto analyzer if user specifies. + if ("true".equalsIgnoreCase(stmt.getAnalyzeProperties().getProperties().get("use.auto.analyzer"))) { + Env.getCurrentEnv().getStatisticsAutoCollector() + .processOneJob(stmt.getTable(), + stmt.getTable().getColumnIndexPairs(stmt.getColumnNames()), JobPriority.HIGH); + return; + } AnalysisInfo jobInfo = buildAndAssignJob(stmt); if (jobInfo == null) { return; @@ -219,8 +236,9 @@ public void createAnalysisJob(AnalyzeTblStmt stmt, boolean proxy) throws DdlExce @VisibleForTesting protected AnalysisInfo buildAndAssignJob(AnalyzeTblStmt stmt) throws DdlException { AnalysisInfo jobInfo = buildAnalysisJobInfo(stmt); - if (jobInfo.jobColumns.isEmpty()) { + if (jobInfo.jobColumns == null || jobInfo.jobColumns.isEmpty()) { // No statistics need to be collected or updated + LOG.info("Job columns are empty, skip analyze table {}", stmt.getTblName().toString()); return null; } // Only OlapTable and Hive HMSExternalTable support sample analyze. @@ -295,7 +313,7 @@ private void sendJobId(List analysisInfos, boolean proxy) { // Make sure colName of job has all the column as this AnalyzeStmt specified, no matter whether it will be analyzed // or not. @VisibleForTesting - public AnalysisInfo buildAnalysisJobInfo(AnalyzeTblStmt stmt) throws DdlException { + public AnalysisInfo buildAnalysisJobInfo(AnalyzeTblStmt stmt) { AnalysisInfoBuilder infoBuilder = new AnalysisInfoBuilder(); long jobId = Env.getCurrentEnv().getNextId(); TableIf table = stmt.getTable(); @@ -329,7 +347,6 @@ public AnalysisInfo buildAnalysisJobInfo(AnalyzeTblStmt stmt) throws DdlExceptio infoBuilder.setAnalysisMode(analysisMode); infoBuilder.setAnalysisMethod(analysisMethod); infoBuilder.setScheduleType(scheduleType); - infoBuilder.setLastExecTimeInMs(0); infoBuilder.setCronExpression(cronExpression); infoBuilder.setForceFull(stmt.forceFull()); infoBuilder.setUsingSqlForPartitionColumn(stmt.usingSqlForPartitionColumn()); @@ -346,7 +363,7 @@ public AnalysisInfo buildAnalysisJobInfo(AnalyzeTblStmt stmt) throws DdlExceptio long periodTimeInMs = stmt.getPeriodTimeInMs(); infoBuilder.setPeriodTimeInMs(periodTimeInMs); - List> jobColumns = table.getColumnIndexPairs(columnNames); + Set> jobColumns = table.getColumnIndexPairs(columnNames); infoBuilder.setJobColumns(jobColumns); StringJoiner stringJoiner = new StringJoiner(",", "[", "]"); for (Pair pair : jobColumns) { @@ -355,8 +372,10 @@ public AnalysisInfo buildAnalysisJobInfo(AnalyzeTblStmt stmt) throws DdlExceptio infoBuilder.setColName(stringJoiner.toString()); infoBuilder.setTaskIds(Lists.newArrayList()); infoBuilder.setTblUpdateTime(table.getUpdateTime()); - infoBuilder.setEmptyJob(table instanceof OlapTable && table.getRowCount() == 0 - && analysisMethod.equals(AnalysisMethod.SAMPLE)); + infoBuilder.setRowCount(StatisticsUtil.isEmptyTable(table, analysisMethod) ? 0 : table.getRowCount()); + TableStatsMeta tableStatsStatus = findTableStatsStatus(table.getId()); + infoBuilder.setUpdateRows(tableStatsStatus == null ? 0 : tableStatsStatus.updatedRows.get()); + infoBuilder.setPriority(JobPriority.MANUAL); return infoBuilder.build(); } @@ -372,7 +391,7 @@ public void recordAnalysisJob(AnalysisInfo jobInfo) { public void createTaskForEachColumns(AnalysisInfo jobInfo, Map analysisTasks, boolean isSync) throws DdlException { - List> jobColumns = jobInfo.jobColumns; + Set> jobColumns = jobInfo.jobColumns; TableIf table = jobInfo.getTable(); for (Pair pair : jobColumns) { AnalysisInfoBuilder colTaskInfoBuilder = new AnalysisInfoBuilder(jobInfo); @@ -505,7 +524,7 @@ public void updateTableStats(AnalysisInfo jobInfo) { } TableStatsMeta tableStats = findTableStatsStatus(tbl.getId()); if (tableStats == null) { - updateTableStatsStatus(new TableStatsMeta(jobInfo.emptyJob ? 0 : tbl.getRowCount(), jobInfo, tbl)); + updateTableStatsStatus(new TableStatsMeta(jobInfo.rowCount, jobInfo, tbl)); } else { tableStats.update(jobInfo, tbl); logCreateTableStats(tableStats); @@ -529,6 +548,39 @@ public void updateTableStatsForAlterStats(AnalysisInfo jobInfo, TableIf tbl) { } } + public List showAutoPendingJobs(ShowAutoAnalyzeJobsStmt stmt) { + TableName tblName = stmt.getTableName(); + String priority = stmt.getPriority(); + List result = Lists.newArrayList(); + if (priority == null || priority.isEmpty()) { + result.addAll(getPendingJobs(highPriorityJobs, JobPriority.HIGH, tblName)); + result.addAll(getPendingJobs(midPriorityJobs, JobPriority.MID, tblName)); + result.addAll(getPendingJobs(lowPriorityJobs, JobPriority.LOW, tblName)); + } else if (priority.equals(JobPriority.HIGH.name())) { + result.addAll(getPendingJobs(highPriorityJobs, JobPriority.HIGH, tblName)); + } else if (priority.equals(JobPriority.MID.name())) { + result.addAll(getPendingJobs(midPriorityJobs, JobPriority.MID, tblName)); + } else if (priority.equals(JobPriority.LOW.name())) { + result.addAll(getPendingJobs(lowPriorityJobs, JobPriority.LOW, tblName)); + } + return result; + } + + protected List getPendingJobs(Map>> jobMap, + JobPriority priority, TableName tblName) { + List result = Lists.newArrayList(); + synchronized (jobMap) { + for (Entry>> entry : jobMap.entrySet()) { + TableName table = entry.getKey(); + if (tblName == null || tblName.equals(table)) { + result.add(new AutoAnalysisPendingJob(table.getCtl(), + table.getDb(), table.getTbl(), entry.getValue(), priority)); + } + } + } + return result; + } + public List showAnalysisJob(ShowAnalyzeStmt stmt) { return findShowAnalyzeResult(stmt); } @@ -555,7 +607,7 @@ private List findShowAnalyzeResult(ShowAnalyzeStmt stmt) { public String getJobProgress(long jobId) { List tasks = findTasksByTaskIds(jobId); - if (tasks == null) { + if (tasks == null || tasks.isEmpty()) { return "N/A"; } int finished = 0; @@ -674,6 +726,7 @@ public void invalidateLocalStats(long catalogId, long dbId, long tableId, } tableStats.updatedTime = 0; tableStats.userInjected = false; + tableStats.rowCount = table.getRowCount(); } public void invalidateRemoteStats(long catalogId, long dbId, long tableId, @@ -751,7 +804,7 @@ private BaseAnalysisTask createTask(AnalysisInfo analysisInfo) throws DdlExcepti analysisInfo.dbId, analysisInfo.tblId); return table.createAnalysisTask(analysisInfo); } catch (Throwable t) { - LOG.warn("Failed to find table", t); + LOG.warn("Failed to create task.", t); throw new DdlException("Failed to create task", t); } } @@ -854,7 +907,7 @@ public List findTasks(long jobId) { public List findTasksByTaskIds(long jobId) { AnalysisInfo jobInfo = analysisJobInfoMap.get(jobId); if (jobInfo != null && jobInfo.taskIds != null) { - return jobInfo.taskIds.stream().map(analysisTaskInfoMap::get).filter(i -> i != null) + return jobInfo.taskIds.stream().map(analysisTaskInfoMap::get).filter(Objects::nonNull) .collect(Collectors.toList()); } return null; @@ -871,7 +924,7 @@ public void removeAll(List analysisInfos) { public void dropAnalyzeJob(DropAnalyzeJobStmt analyzeJobStmt) throws DdlException { AnalysisInfo jobInfo = analysisJobInfoMap.get(analyzeJobStmt.getJobId()); if (jobInfo == null) { - throw new DdlException(String.format("Analyze job [%d] not exists", jobInfo.jobId)); + throw new DdlException(String.format("Analyze job [%d] not exists", analyzeJobStmt.getJobId())); } checkPriv(jobInfo); long jobId = analyzeJobStmt.getJobId(); @@ -911,15 +964,12 @@ public static boolean needAbandon(AnalysisInfo analysisInfo) { if (analysisInfo == null) { return true; } - if (analysisInfo.scheduleType == null || analysisInfo.scheduleType == null || analysisInfo.jobType == null) { - return true; - } - if ((AnalysisState.PENDING.equals(analysisInfo.state) || AnalysisState.RUNNING.equals(analysisInfo.state)) - && ScheduleType.ONCE.equals(analysisInfo.scheduleType) - && JobType.MANUAL.equals(analysisInfo.jobType)) { + if (analysisInfo.scheduleType == null || analysisInfo.jobType == null) { return true; } - return false; + return (AnalysisState.PENDING.equals(analysisInfo.state) || AnalysisState.RUNNING.equals(analysisInfo.state)) + && ScheduleType.ONCE.equals(analysisInfo.scheduleType) + && JobType.MANUAL.equals(analysisInfo.jobType); } private static void readIdToTblStats(DataInput in, Map map) throws IOException { @@ -1075,17 +1125,66 @@ public void removeJob(long id) { /** * Only OlapTable and Hive HMSExternalTable can sample for now. - * @param table + * @param table Table to check * @return Return true if the given table can do sample analyze. False otherwise. */ public boolean canSample(TableIf table) { if (table instanceof OlapTable) { return true; } - if (table instanceof HMSExternalTable - && ((HMSExternalTable) table).getDlaType().equals(HMSExternalTable.DLAType.HIVE)) { - return true; + return table instanceof HMSExternalTable + && ((HMSExternalTable) table).getDlaType().equals(HMSExternalTable.DLAType.HIVE); + } + + + public void updateHighPriorityColumn(Set slotReferences) { + updateColumn(slotReferences, highPriorityColumns); + } + + public void updateMidPriorityColumn(Collection slotReferences) { + updateColumn(slotReferences, midPriorityColumns); + } + + protected void updateColumn(Collection slotReferences, Queue queue) { + for (Slot s : slotReferences) { + if (!(s instanceof SlotReference)) { + return; + } + Optional optionalColumn = ((SlotReference) s).getColumn(); + Optional optionalTable = ((SlotReference) s).getTable(); + if (optionalColumn.isPresent() && optionalTable.isPresent() + && !StatisticsUtil.isUnsupportedType(optionalColumn.get().getType())) { + TableIf table = optionalTable.get(); + DatabaseIf database = table.getDatabase(); + if (database != null) { + CatalogIf catalog = database.getCatalog(); + if (catalog != null) { + queue.offer(new QueryColumn(catalog.getId(), database.getId(), + table.getId(), optionalColumn.get().getName())); + if (LOG.isDebugEnabled()) { + LOG.debug("Offer column " + table.getName() + "(" + table.getId() + ")." + + optionalColumn.get().getName()); + } + } + } + } + } + } + + public void mergeFollowerQueryColumns(Collection highColumns, + Collection midColumns) { + LOG.info("Received {} high columns and {} mid columns", highColumns.size(), midColumns.size()); + for (TQueryColumn c : highColumns) { + if (!highPriorityColumns.offer(new QueryColumn(Long.parseLong(c.catalogId), Long.parseLong(c.dbId), + Long.parseLong(c.tblId), c.colName))) { + break; + } + } + for (TQueryColumn c : midColumns) { + if (!midPriorityColumns.offer(new QueryColumn(Long.parseLong(c.catalogId), Long.parseLong(c.dbId), + Long.parseLong(c.tblId), c.colName))) { + break; + } } - return false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskExecutor.java index 3bdccaca047954..d787794534a7c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskExecutor.java @@ -27,6 +27,7 @@ import java.util.Comparator; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; @@ -52,7 +53,7 @@ public AnalysisTaskExecutor(int simultaneouslyRunningTaskNum, int taskQueueSize) simultaneouslyRunningTaskNum, simultaneouslyRunningTaskNum, 0, TimeUnit.DAYS, new LinkedBlockingQueue<>(taskQueueSize), - new BlockedPolicy("Analysis Job Executor", Integer.MAX_VALUE), + new BlockedPolicy("Analysis Job Executor Block Policy", Integer.MAX_VALUE), "Analysis Job Executor", true); cancelExpiredTask(); } else { @@ -88,9 +89,9 @@ protected void tryToCancel() { } } - public void submitTask(BaseAnalysisTask task) { + public Future submitTask(BaseAnalysisTask task) { AnalysisTaskWrapper taskWrapper = new AnalysisTaskWrapper(this, task); - executors.submit(taskWrapper); + return executors.submit(taskWrapper); } public void putJob(AnalysisTaskWrapper wrapper) throws Exception { diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AutoAnalysisPendingJob.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AutoAnalysisPendingJob.java new file mode 100644 index 00000000000000..e349e4fcb3f2e8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AutoAnalysisPendingJob.java @@ -0,0 +1,52 @@ +// 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.statistics; + +import org.apache.doris.common.Pair; + +import java.util.Set; +import java.util.StringJoiner; + +public class AutoAnalysisPendingJob { + + public final String catalogName; + public final String dbName; + public final String tableName; + public final Set> columns; + public final JobPriority priority; + + public AutoAnalysisPendingJob(String catalogName, String dbName, String tableName, + Set> columns, JobPriority priority) { + this.catalogName = catalogName; + this.dbName = dbName; + this.tableName = tableName; + this.columns = columns; + this.priority = priority; + } + + public String getColumnNames() { + if (columns == null) { + return ""; + } + StringJoiner stringJoiner = new StringJoiner(","); + for (Pair col : columns) { + stringJoiner.add(col.toString()); + } + return stringJoiner.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java index f871e8761a5e55..d35e45987b8501 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java @@ -39,7 +39,6 @@ import java.text.MessageFormat; import java.util.Collections; -import java.util.concurrent.TimeUnit; public abstract class BaseAnalysisTask { @@ -48,7 +47,7 @@ public abstract class BaseAnalysisTask { public static final long LIMIT_SIZE = 1024 * 1024 * 1024; // 1GB public static final double LIMIT_FACTOR = 1.2; - protected static final String COLLECT_COL_STATISTICS = + protected static final String FULL_ANALYZE_TEMPLATE = "SELECT CONCAT(${tblId}, '-', ${idxId}, '-', '${colId}') AS `id`, " + " ${catalogId} AS `catalog_id`, " + " ${dbId} AS `db_id`, " @@ -194,9 +193,9 @@ protected void init(AnalysisInfo info) { } } - public void execute() { + public void execute() throws Exception { prepareExecution(); - executeWithRetry(); + doExecute(); afterExecution(); } @@ -204,29 +203,6 @@ protected void prepareExecution() { setTaskStateToRunning(); } - protected void executeWithRetry() { - int retriedTimes = 0; - while (retriedTimes < StatisticConstants.ANALYZE_TASK_RETRY_TIMES) { - if (killed) { - break; - } - try { - doExecute(); - break; - } catch (Throwable t) { - if (killed) { - throw new RuntimeException(t); - } - LOG.warn("Failed to execute analysis task, retried times: {}", retriedTimes++, t); - if (retriedTimes >= StatisticConstants.ANALYZE_TASK_RETRY_TIMES) { - job.taskFailed(this, t.getMessage()); - throw new RuntimeException(t); - } - StatisticsUtil.sleep(TimeUnit.SECONDS.toMillis(2 ^ retriedTimes) * 10); - } - } - } - public abstract void doExecute() throws Exception; protected void afterExecution() {} @@ -284,9 +260,8 @@ protected String getNdvFunction(String totalRows) { // (https://github.com/postgres/postgres/blob/master/src/backend/commands/analyze.c) // (http://citeseerx.ist.psu.edu/viewdoc/download?doi=10.1.1.93.8637&rep=rep1&type=pdf) // sample_row * count_distinct / ( sample_row - once_count + once_count * sample_row / total_row) - String fn = MessageFormat.format("{0} * {1} / ({0} - {2} + {2} * {0} / {3})", sampleRows, + return MessageFormat.format("{0} * {1} / ({0} - {2} + {2} * {0} / {3})", sampleRows, countDistinct, onceCount, totalRows); - return fn; } // Max value is not accurate while sample, so set it to NULL to avoid optimizer generate bad plan. @@ -336,6 +311,9 @@ protected void runQuery(String sql) { Env.getCurrentEnv().getStatisticsCache().syncColStats(colStatsData); queryId = DebugUtil.printId(stmtExecutor.getContext().queryId()); job.appendBuf(this, Collections.singletonList(colStatsData)); + } catch (Exception e) { + LOG.warn("Failed to execute sql {}", sql); + throw e; } finally { if (LOG.isDebugEnabled()) { LOG.debug("End cost time in millisec: " + (System.currentTimeMillis() - startTime) diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsMeta.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsMeta.java index 445641b2505610..7e317d67bd740f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsMeta.java @@ -43,16 +43,20 @@ public class ColStatsMeta { @SerializedName("trigger") public JobType jobType; - public ColStatsMeta(long updatedTime, AnalysisMethod analysisMethod, - AnalysisType analysisType, JobType jobType, long queriedTimes) { + @SerializedName("updatedRows") + public long updatedRows; + + @SerializedName("rowCount") + public long rowCount; + + public ColStatsMeta(long updatedTime, AnalysisMethod analysisMethod, AnalysisType analysisType, JobType jobType, + long queriedTimes, long rowCount, long updatedRows) { this.updatedTime = updatedTime; this.analysisMethod = analysisMethod; this.analysisType = analysisType; this.jobType = jobType; this.queriedTimes.addAndGet(queriedTimes); - } - - public void clear() { - updatedTime = 0; + this.updatedRows = updatedRows; + this.rowCount = rowCount; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java index 287941be526635..7d3c9af254800d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java @@ -73,6 +73,8 @@ protected void setTable(ExternalTable table) { */ private void getTableStats() { Map params = buildStatsParams(null); + Pair sampleInfo = getSampleInfo(); + params.put("scaleFactor", String.valueOf(sampleInfo.first)); List columnResult = StatisticsUtil.execStatisticQuery(new StringSubstitutor(params) .replace(ANALYZE_TABLE_COUNT_TEMPLATE)); @@ -98,7 +100,7 @@ protected void getColumnStats() throws Exception { if (LOG.isDebugEnabled()) { LOG.debug("Will do full collection for column {}", col.getName()); } - sb.append(COLLECT_COL_STATISTICS); + sb.append(FULL_ANALYZE_TEMPLATE); } else { // Do sample analyze if (LOG.isDebugEnabled()) { @@ -254,9 +256,6 @@ protected boolean needLimit(long sizeToRead, double factor) { } target = columnSize * tableSample.getSampleValue(); } - if (sizeToRead > LIMIT_SIZE && sizeToRead > target * LIMIT_FACTOR) { - return true; - } - return false; + return sizeToRead > LIMIT_SIZE && sizeToRead > target * LIMIT_FACTOR; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/FollowerColumnSender.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/FollowerColumnSender.java new file mode 100644 index 00000000000000..0e66c7f8a75be7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/FollowerColumnSender.java @@ -0,0 +1,151 @@ +// 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.statistics; + +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.ClientPool; +import org.apache.doris.common.Pair; +import org.apache.doris.common.util.MasterDaemon; +import org.apache.doris.ha.FrontendNodeType; +import org.apache.doris.statistics.util.StatisticsUtil; +import org.apache.doris.system.Frontend; +import org.apache.doris.thrift.FrontendService; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TQueryColumn; +import org.apache.doris.thrift.TSyncQueryColumns; + +import com.google.common.collect.Sets; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Queue; +import java.util.Set; + +public class FollowerColumnSender extends MasterDaemon { + + private static final Logger LOG = LogManager.getLogger(FollowerColumnSender.class); + + public static final long INTERVAL = 60000; + + public FollowerColumnSender() { + super("Follower Column Sender", INTERVAL); + } + + @Override + protected void runAfterCatalogReady() { + if (!StatisticsUtil.enableAutoAnalyze()) { + return; + } + if (Env.getCurrentEnv().isMaster()) { + return; + } + if (Env.isCheckpointThread()) { + return; + } + send(); + } + + protected void send() { + if (Env.getCurrentEnv().isMaster()) { + return; + } + Env currentEnv = Env.getCurrentEnv(); + AnalysisManager analysisManager = currentEnv.getAnalysisManager(); + if (analysisManager.highPriorityColumns.isEmpty() && analysisManager.midPriorityColumns.isEmpty()) { + return; + } + Set highs = getNeedAnalyzeColumns(analysisManager.highPriorityColumns); + Set mids = getNeedAnalyzeColumns(analysisManager.midPriorityColumns); + mids.removeAll(highs); + TSyncQueryColumns queryColumns = new TSyncQueryColumns(); + queryColumns.highPriorityColumns = new ArrayList<>(highs); + queryColumns.midPriorityColumns = new ArrayList<>(mids); + Frontend master = null; + try { + InetSocketAddress masterAddress = currentEnv.getHaProtocol().getLeader(); + for (Frontend fe : currentEnv.getFrontends(FrontendNodeType.FOLLOWER)) { + InetSocketAddress socketAddress = new InetSocketAddress(fe.getHost(), fe.getEditLogPort()); + if (socketAddress.equals(masterAddress)) { + master = fe; + break; + } + } + } catch (Exception e) { + LOG.warn("Failed to find master FE.", e); + return; + } + + if (master == null) { + LOG.warn("No master found in cluster."); + return; + } + TNetworkAddress address = new TNetworkAddress(master.getHost(), master.getRpcPort()); + FrontendService.Client client = null; + try { + client = ClientPool.frontendPool.borrowObject(address); + client.syncQueryColumns(queryColumns); + LOG.info("Send {} high priority columns and {} mid priority columns to master.", + highs.size(), mids.size()); + } catch (Throwable t) { + LOG.warn("Failed to sync stats to master: {}", address, t); + } finally { + if (client != null) { + ClientPool.frontendPool.returnObject(address, client); + } + } + } + + protected Set getNeedAnalyzeColumns(Queue columnQueue) { + Set ret = Sets.newHashSet(); + TableIf table; + int size = columnQueue.size(); + for (int i = 0; i < size; i++) { + QueryColumn column = columnQueue.poll(); + if (column == null) { + continue; + } + try { + table = StatisticsUtil.findTable(column.catalogId, column.dbId, column.tblId); + } catch (Exception e) { + LOG.warn("Failed to find table for column {}", column.colName, e); + continue; + } + if (StatisticsUtil.isUnsupportedType(table.getColumn(column.colName).getType())) { + continue; + } + Set> columnIndexPairs = table.getColumnIndexPairs( + Collections.singleton(column.colName)); + for (Pair pair : columnIndexPairs) { + if (StatisticsUtil.needAnalyzeColumn(table, pair)) { + ret.add(column.toThrift()); + break; + } + } + } + return ret; + } + + protected List convertSetToList(Set set) { + return new ArrayList<>(set); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java index 60da8f4d2a0803..26ef561ddf6a27 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java @@ -79,11 +79,6 @@ public void doExecute() throws Exception { tbl.getDatabase().getCatalog().getId(), tbl.getDatabase().getId(), tbl.getId(), -1, col.getName()); } - @Override - protected void afterExecution() { - // DO NOTHING - } - private String getSampleRateFunction() { if (info.analysisMethod == AnalysisMethod.FULL) { return "0"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/JobPriority.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/JobPriority.java new file mode 100644 index 00000000000000..c3656b929279e6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/JobPriority.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.statistics; + +public enum JobPriority { + HIGH, + MID, + LOW, + MANUAL; +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java index 60bfcab6157377..ce7982d4f1ad58 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java @@ -37,7 +37,6 @@ import java.security.SecureRandom; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -65,11 +64,11 @@ public OlapAnalysisTask(AnalysisInfo info) { } public void doExecute() throws Exception { - List> columnList = info.jobColumns; - if (StatisticsUtil.isEmptyTable(tbl, info.analysisMethod) || columnList == null || columnList.isEmpty()) { + // For empty table, write empty result directly, no need to run SQL to collect stats. + if (info.rowCount == 0 && tableSample != null) { StatsId statsId = new StatsId(concatColumnStatsId(), info.catalogId, info.dbId, info.tblId, info.indexId, info.colName, null); - job.appendBuf(this, Arrays.asList(new ColStatsData(statsId))); + job.appendBuf(this, Collections.singletonList(new ColStatsData(statsId))); return; } if (tableSample != null) { @@ -84,7 +83,7 @@ public void doExecute() throws Exception { * 2. estimate partition stats * 3. insert col stats and partition stats */ - protected void doSample() throws Exception { + protected void doSample() { if (LOG.isDebugEnabled()) { LOG.debug("Will do sample collection for column {}", col.getName()); } @@ -209,7 +208,7 @@ protected ResultRow collectBasicStat(AutoCloseConnectContext context) { * 2. insert partition in batch * 3. calculate column stats based on partition stats */ - protected void doFull() throws Exception { + protected void doFull() { if (LOG.isDebugEnabled()) { LOG.debug("Will do full collection for column {}", col.getName()); } @@ -228,8 +227,7 @@ protected void doFull() throws Exception { params.put("tblName", String.valueOf(tbl.getName())); params.put("index", getIndex()); StringSubstitutor stringSubstitutor = new StringSubstitutor(params); - String collectColStats = stringSubstitutor.replace(COLLECT_COL_STATISTICS); - runQuery(collectColStats); + runQuery(stringSubstitutor.replace(FULL_ANALYZE_TEMPLATE)); } protected String getIndex() { @@ -316,10 +314,7 @@ protected boolean needLimit() { return false; } // Partition column need to scan tablets from all partitions. - if (tbl.isPartitionColumn(col.getName())) { - return false; - } - return true; + return !tbl.isPartitionColumn(col.getName()); } /** @@ -382,12 +377,6 @@ protected boolean isSingleUniqueKey() { } protected String concatColumnStatsId() { - StringBuilder stringBuilder = new StringBuilder(); - stringBuilder.append(info.tblId); - stringBuilder.append("-"); - stringBuilder.append(info.indexId); - stringBuilder.append("-"); - stringBuilder.append(info.colName); - return stringBuilder.toString(); + return info.tblId + "-" + info.indexId + "-" + info.colName; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/QueryColumn.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/QueryColumn.java new file mode 100644 index 00000000000000..df91ea7f4c0582 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/QueryColumn.java @@ -0,0 +1,66 @@ +// 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.statistics; + +import org.apache.doris.thrift.TQueryColumn; + +import java.util.Objects; + +public class QueryColumn { + + public final long catalogId; + public final long dbId; + public final long tblId; + public final String colName; + + public QueryColumn(long catalogId, long dbId, long tblId, String colName) { + this.catalogId = catalogId; + this.dbId = dbId; + this.tblId = tblId; + this.colName = colName; + } + + @Override + public int hashCode() { + return Objects.hash(catalogId, dbId, tblId, colName); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + if (!(other instanceof QueryColumn)) { + return false; + } + QueryColumn otherCriticalColumn = (QueryColumn) other; + return this.catalogId == otherCriticalColumn.catalogId + && this.dbId == otherCriticalColumn.dbId + && this.tblId == otherCriticalColumn.tblId + && this.colName.equals(otherCriticalColumn.colName); + } + + public TQueryColumn toThrift() { + TQueryColumn tQueryColumn = new TQueryColumn(); + tQueryColumn.catalogId = String.valueOf(catalogId); + tQueryColumn.dbId = String.valueOf(dbId); + tQueryColumn.tblId = String.valueOf(tblId); + tQueryColumn.colName = colName; + return tQueryColumn; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java index 74c7bd7c9db127..a5bd18946e8376 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java @@ -64,8 +64,6 @@ public class StatisticConstants { public static List SYSTEM_DBS = new ArrayList<>(); - public static int ANALYZE_TASK_RETRY_TIMES = 5; - public static final String DB_NAME = FeConstants.INTERNAL_DB_NAME; public static final String FULL_QUALIFIED_STATS_TBL_NAME = InternalCatalog.INTERNAL_CATALOG_NAME @@ -95,7 +93,7 @@ public class StatisticConstants { public static final int ANALYZE_TIMEOUT_IN_SEC = 43200; - public static final int TASK_QUEUE_CAP = 10; + public static final int TASK_QUEUE_CAP = 1; public static final int AUTO_ANALYZE_TABLE_WIDTH_THRESHOLD = 100; diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java index 9ca971845b7e64..479610ccea25ba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java @@ -17,17 +17,16 @@ package org.apache.doris.statistics; +import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.Pair; +import org.apache.doris.common.util.MasterDaemon; import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.statistics.AnalysisInfo.AnalysisMethod; import org.apache.doris.statistics.AnalysisInfo.JobType; @@ -40,200 +39,199 @@ import java.time.LocalTime; import java.util.ArrayList; -import java.util.List; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; import java.util.Set; import java.util.StringJoiner; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -public class StatisticsAutoCollector extends StatisticsCollector { +public class StatisticsAutoCollector extends MasterDaemon { private static final Logger LOG = LogManager.getLogger(StatisticsAutoCollector.class); + protected final AnalysisTaskExecutor analysisTaskExecutor; + public StatisticsAutoCollector() { - super("Automatic Analyzer", - TimeUnit.MINUTES.toMillis(Config.auto_check_statistics_in_minutes), - new AnalysisTaskExecutor(Config.auto_analyze_simultaneously_running_task_num, - StatisticConstants.TASK_QUEUE_CAP)); + super("Automatic Analyzer", TimeUnit.MINUTES.toMillis(Config.auto_check_statistics_in_minutes)); + this.analysisTaskExecutor = new AnalysisTaskExecutor(Config.auto_analyze_simultaneously_running_task_num, + StatisticConstants.TASK_QUEUE_CAP); } @Override - protected void collect() { - if (canCollect()) { - analyzeAll(); + protected void runAfterCatalogReady() { + if (!Env.getCurrentEnv().isMaster()) { + return; } + if (!StatisticsUtil.statsTblAvailable()) { + LOG.info("Stats table not available, skip"); + return; + } + if (Env.isCheckpointThread()) { + return; + } + collect(); } - protected boolean canCollect() { - return StatisticsUtil.enableAutoAnalyze() - && StatisticsUtil.inAnalyzeTime(LocalTime.now(TimeUtils.getTimeZone().toZoneId())); - } - - protected void analyzeAll() { - List catalogs = getCatalogsInOrder(); - for (CatalogIf ctl : catalogs) { - if (!canCollect()) { - analysisTaskExecutor.clear(); + protected void collect() { + while (canCollect()) { + Pair>>, JobPriority> job = getJob(); + if (job == null) { + // No more job to process, break and sleep. break; } - if (!ctl.enableAutoAnalyze()) { - continue; - } - List dbs = getDatabasesInOrder(ctl); - for (DatabaseIf databaseIf : dbs) { - if (!canCollect()) { - analysisTaskExecutor.clear(); - break; - } - if (StatisticConstants.SYSTEM_DBS.contains(databaseIf.getFullName())) { - continue; - } - try { - analyzeDb(databaseIf); - } catch (Throwable t) { - LOG.warn("Failed to analyze database {}.{}", ctl.getName(), databaseIf.getFullName(), t); + try { + TableName tblName = job.first.getKey(); + TableIf table = StatisticsUtil.findTable(tblName.getCtl(), tblName.getDb(), tblName.getTbl()); + if (!supportAutoAnalyze(table)) { continue; } + processOneJob(table, job.first.getValue(), job.second); + } catch (Exception e) { + LOG.warn("Failed to analyze table {} with columns [{}]", job.first.getKey().getTbl(), + job.first.getValue().stream().map(Pair::toString).collect(Collectors.joining(",")), e); } } } - public List getCatalogsInOrder() { - return Env.getCurrentEnv().getCatalogMgr().getCopyOfCatalog().stream() - .sorted((c1, c2) -> (int) (c1.getId() - c2.getId())).collect(Collectors.toList()); - } - - public List> getDatabasesInOrder(CatalogIf catalog) { - return catalog.getAllDbs().stream() - .sorted((d1, d2) -> (int) (d1.getId() - d2.getId())).collect(Collectors.toList()); + protected boolean canCollect() { + return StatisticsUtil.enableAutoAnalyze() + && StatisticsUtil.inAnalyzeTime(LocalTime.now(TimeUtils.getTimeZone().toZoneId())); } - public List getTablesInOrder(DatabaseIf db) { - return db.getTables().stream() - .sorted((t1, t2) -> (int) (t1.getId() - t2.getId())).collect(Collectors.toList()); + protected Pair>>, JobPriority> getJob() { + AnalysisManager manager = Env.getServingEnv().getAnalysisManager(); + Optional>>> job = fetchJobFromMap(manager.highPriorityJobs); + if (job.isPresent()) { + return Pair.of(job.get(), JobPriority.HIGH); + } + job = fetchJobFromMap(manager.midPriorityJobs); + if (job.isPresent()) { + return Pair.of(job.get(), JobPriority.MID); + } + job = fetchJobFromMap(manager.lowPriorityJobs); + return job.map(entry -> Pair.of(entry, JobPriority.LOW)).orElse(null); } - public void analyzeDb(DatabaseIf databaseIf) throws DdlException { - List analysisInfos = constructAnalysisInfo(databaseIf); - for (AnalysisInfo analysisInfo : analysisInfos) { - try { - if (!canCollect()) { - analysisTaskExecutor.clear(); - break; - } - analysisInfo = getNeedAnalyzeColumns(analysisInfo); - if (analysisInfo == null) { - continue; - } - createSystemAnalysisJob(analysisInfo); - } catch (Throwable t) { - analysisInfo.message = t.getMessage(); - LOG.warn("Failed to auto analyze table {}.{}, reason {}", - databaseIf.getFullName(), analysisInfo.tblId, analysisInfo.message, t); - continue; - } + protected Optional>>> fetchJobFromMap( + Map>> jobMap) { + synchronized (jobMap) { + Optional>>> first = jobMap.entrySet().stream().findFirst(); + first.ifPresent(entry -> jobMap.remove(entry.getKey())); + return first; } } - protected List constructAnalysisInfo(DatabaseIf db) { - List analysisInfos = new ArrayList<>(); - for (TableIf table : getTablesInOrder(db)) { - try { - if (skip(table)) { - continue; - } - createAnalyzeJobForTbl(db, analysisInfos, table); - } catch (Throwable t) { - LOG.warn("Failed to analyze table {}.{}.{}", - db.getCatalog().getName(), db.getFullName(), table.getName(), t); - continue; + protected void processOneJob(TableIf table, Set> columns, + JobPriority priority) throws DdlException { + // appendMvColumn(table, columns); + appendPartitionColumns(table, columns); + columns = columns.stream().filter(c -> StatisticsUtil.needAnalyzeColumn(table, c)).collect(Collectors.toSet()); + if (columns.isEmpty()) { + return; + } + AnalysisInfo analyzeJob = createAnalyzeJobForTbl(table, columns, priority); + LOG.debug("Auto analyze job : {}", analyzeJob.toString()); + try { + executeSystemAnalysisJob(analyzeJob); + } catch (Exception e) { + StringJoiner stringJoiner = new StringJoiner(",", "[", "]"); + for (Pair pair : columns) { + stringJoiner.add(pair.toString()); } + LOG.warn("Fail to auto analyze table {}, columns [{}]", table.getName(), stringJoiner.toString()); } - return analysisInfos; } - // return true if skip auto analyze this time. - protected boolean skip(TableIf table) { - if (!(table instanceof OlapTable || table instanceof HMSExternalTable)) { - return true; + protected void appendPartitionColumns(TableIf table, Set> columns) throws DdlException { + if (!(table instanceof OlapTable)) { + return; } - // For now, only support Hive HMS table auto collection. - if (table instanceof HMSExternalTable - && !((HMSExternalTable) table).getDlaType().equals(HMSExternalTable.DLAType.HIVE)) { - return true; + AnalysisManager manager = Env.getServingEnv().getAnalysisManager(); + TableStatsMeta tableStatsStatus = manager.findTableStatsStatus(table.getId()); + if (tableStatsStatus != null && tableStatsStatus.newPartitionLoaded.get()) { + OlapTable olapTable = (OlapTable) table; + columns.addAll(olapTable.getColumnIndexPairs(olapTable.getPartitionColumnNames())); } - if (table.getDataSize(true) < StatisticsUtil.getHugeTableLowerBoundSizeInBytes() * 5) { - return false; + } + + protected void appendMvColumn(TableIf table, Set columns) { + if (!(table instanceof OlapTable)) { + return; } - TableStatsMeta tableStats = Env.getCurrentEnv().getAnalysisManager().findTableStatsStatus(table.getId()); - // means it's never got analyzed or new partition loaded data. - if (tableStats == null || tableStats.newPartitionLoaded.get()) { + OlapTable olapTable = (OlapTable) table; + Set mvColumns = olapTable.getMvColumns(false).stream().map(Column::getName).collect(Collectors.toSet()); + columns.addAll(mvColumns); + } + + protected boolean supportAutoAnalyze(TableIf tableIf) { + if (tableIf == null) { return false; } - if (tableStats.userInjected) { - return true; - } - return System.currentTimeMillis() - - tableStats.updatedTime < StatisticsUtil.getHugeTableAutoAnalyzeIntervalInMillis(); + return tableIf instanceof OlapTable + || tableIf instanceof HMSExternalTable + && ((HMSExternalTable) tableIf).getDlaType().equals(HMSExternalTable.DLAType.HIVE); } - protected void createAnalyzeJobForTbl(DatabaseIf db, - List analysisInfos, TableIf table) { + protected AnalysisInfo createAnalyzeJobForTbl( + TableIf table, Set> jobColumns, JobPriority priority) { AnalysisMethod analysisMethod = table.getDataSize(true) >= StatisticsUtil.getHugeTableLowerBoundSizeInBytes() ? AnalysisMethod.SAMPLE : AnalysisMethod.FULL; - AnalysisInfo jobInfo = new AnalysisInfoBuilder() + AnalysisManager manager = Env.getServingEnv().getAnalysisManager(); + TableStatsMeta tableStatsStatus = manager.findTableStatsStatus(table.getId()); + long rowCount = StatisticsUtil.isEmptyTable(table, analysisMethod) ? 0 : table.getRowCount(); + StringJoiner stringJoiner = new StringJoiner(",", "[", "]"); + for (Pair pair : jobColumns) { + stringJoiner.add(pair.toString()); + } + return new AnalysisInfoBuilder() .setJobId(Env.getCurrentEnv().getNextId()) - .setCatalogId(db.getCatalog().getId()) - .setDBId(db.getId()) + .setCatalogId(table.getDatabase().getCatalog().getId()) + .setDBId(table.getDatabase().getId()) .setTblId(table.getId()) - .setColName(null) + .setColName(stringJoiner.toString()) + .setJobColumns(jobColumns) .setAnalysisType(AnalysisInfo.AnalysisType.FUNDAMENTALS) .setAnalysisMode(AnalysisInfo.AnalysisMode.INCREMENTAL) .setAnalysisMethod(analysisMethod) .setSampleRows(analysisMethod.equals(AnalysisMethod.SAMPLE) - ? StatisticsUtil.getHugeTableSampleRows() : -1) + ? StatisticsUtil.getHugeTableSampleRows() : -1) .setScheduleType(ScheduleType.AUTOMATIC) .setState(AnalysisState.PENDING) .setTaskIds(new ArrayList<>()) .setLastExecTimeInMs(System.currentTimeMillis()) .setJobType(JobType.SYSTEM) .setTblUpdateTime(table.getUpdateTime()) - .setEmptyJob(table instanceof OlapTable && table.getRowCount() == 0 - && analysisMethod.equals(AnalysisMethod.SAMPLE)) + .setRowCount(rowCount) + .setUpdateRows(tableStatsStatus == null ? 0 : tableStatsStatus.updatedRows.get()) + .setPriority(priority) .build(); - analysisInfos.add(jobInfo); } + // Analysis job created by the system @VisibleForTesting - protected AnalysisInfo getNeedAnalyzeColumns(AnalysisInfo jobInfo) { - TableIf table = StatisticsUtil.findTable(jobInfo.catalogId, jobInfo.dbId, jobInfo.tblId); - // Skip tables that are too wide. - if (table.getBaseSchema().size() > StatisticsUtil.getAutoAnalyzeTableWidthThreshold()) { - return null; - } - - AnalysisManager analysisManager = Env.getServingEnv().getAnalysisManager(); - TableStatsMeta tblStats = analysisManager.findTableStatsStatus(table.getId()); - - List> needRunColumns = null; - if (table.needReAnalyzeTable(tblStats)) { - needRunColumns = table.getColumnIndexPairs(table.getSchemaAllIndexes(false) - .stream().map(Column::getName).collect(Collectors.toSet())); - } else if (table instanceof OlapTable && tblStats.newPartitionLoaded.get()) { - OlapTable olapTable = (OlapTable) table; - Set partitionNames = olapTable.getAllPartitions().stream() - .map(Partition::getName).collect(Collectors.toSet()); - needRunColumns = olapTable.getColumnIndexPairs(partitionNames); + protected void executeSystemAnalysisJob(AnalysisInfo jobInfo) + throws DdlException, ExecutionException, InterruptedException { + Map analysisTasks = new HashMap<>(); + AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); + analysisManager.createTaskForEachColumns(jobInfo, analysisTasks, false); + if (StatisticsUtil.isExternalTable(jobInfo.catalogId, jobInfo.dbId, jobInfo.tblId) + && jobInfo.priority.equals(JobPriority.LOW)) { + analysisManager.createTableLevelTaskForExternalTable(jobInfo, analysisTasks, false); } - - if (needRunColumns == null || needRunColumns.isEmpty()) { - return null; + Env.getCurrentEnv().getAnalysisManager().constructJob(jobInfo, analysisTasks.values()); + Env.getCurrentEnv().getAnalysisManager().registerSysJob(jobInfo, analysisTasks); + Future[] futures = new Future[analysisTasks.values().size()]; + int i = 0; + for (BaseAnalysisTask task : analysisTasks.values()) { + futures[i++] = analysisTaskExecutor.submitTask(task); } - StringJoiner stringJoiner = new StringJoiner(",", "[", "]"); - for (Pair pair : needRunColumns) { - stringJoiner.add(pair.toString()); + for (Future future : futures) { + future.get(); } - return new AnalysisInfoBuilder(jobInfo) - .setColName(stringJoiner.toString()).setJobColumns(needRunColumns).build(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCollector.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCollector.java deleted file mode 100644 index ec187fe893af49..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCollector.java +++ /dev/null @@ -1,79 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.statistics; - -import org.apache.doris.catalog.Env; -import org.apache.doris.common.DdlException; -import org.apache.doris.common.util.MasterDaemon; -import org.apache.doris.statistics.util.StatisticsUtil; - -import org.apache.hudi.common.util.VisibleForTesting; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.HashMap; -import java.util.Map; - -public abstract class StatisticsCollector extends MasterDaemon { - - private static final Logger LOG = LogManager.getLogger(StatisticsCollector.class); - - protected final AnalysisTaskExecutor analysisTaskExecutor; - - public StatisticsCollector(String name, long intervalMs, AnalysisTaskExecutor analysisTaskExecutor) { - super(name, intervalMs); - this.analysisTaskExecutor = analysisTaskExecutor; - } - - @Override - protected void runAfterCatalogReady() { - if (!Env.getCurrentEnv().isMaster()) { - return; - } - if (!StatisticsUtil.statsTblAvailable()) { - LOG.info("Stats table not available, skip"); - return; - } - if (Env.isCheckpointThread()) { - return; - } - collect(); - } - - protected abstract void collect(); - - // Analysis job created by the system - @VisibleForTesting - protected void createSystemAnalysisJob(AnalysisInfo jobInfo) - throws DdlException { - if (jobInfo.jobColumns.isEmpty()) { - // No statistics need to be collected or updated - return; - } - Map analysisTasks = new HashMap<>(); - AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); - analysisManager.createTaskForEachColumns(jobInfo, analysisTasks, false); - if (StatisticsUtil.isExternalTable(jobInfo.catalogId, jobInfo.dbId, jobInfo.tblId)) { - analysisManager.createTableLevelTaskForExternalTable(jobInfo, analysisTasks, false); - } - Env.getCurrentEnv().getAnalysisManager().constructJob(jobInfo, analysisTasks.values()); - Env.getCurrentEnv().getAnalysisManager().registerSysJob(jobInfo, analysisTasks); - analysisTasks.values().forEach(analysisTaskExecutor::submitTask); - } - -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsJobAppender.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsJobAppender.java new file mode 100644 index 00000000000000..74484a06afa99f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsJobAppender.java @@ -0,0 +1,204 @@ +// 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.statistics; + +import org.apache.doris.analysis.TableName; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.Pair; +import org.apache.doris.common.util.MasterDaemon; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.statistics.util.StatisticsUtil; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class StatisticsJobAppender extends MasterDaemon { + + private static final Logger LOG = LogManager.getLogger(StatisticsJobAppender.class); + + public static final long INTERVAL = 1000; + public static final int JOB_MAP_SIZE = 1000; + public static final int TABLE_BATCH_SIZE = 100; + + private long currentDbId = 0; + private long currentTableId = 0; + private long lastRoundFinishTime = 0; + private final long lowJobIntervalMs = TimeUnit.MINUTES.toMillis(1); + + public StatisticsJobAppender() { + super("Statistics Job Appender", INTERVAL); + } + + @Override + protected void runAfterCatalogReady() { + if (!StatisticsUtil.enableAutoAnalyze()) { + return; + } + if (!Env.getCurrentEnv().isMaster()) { + return; + } + if (Env.isCheckpointThread()) { + return; + } + appendJobs(); + } + + protected void appendJobs() { + AnalysisManager manager = Env.getCurrentEnv().getAnalysisManager(); + appendColumnsToJobs(manager.highPriorityColumns, manager.highPriorityJobs); + appendColumnsToJobs(manager.midPriorityColumns, manager.midPriorityJobs); + if (StatisticsUtil.enableAutoAnalyzeInternalCatalog()) { + appendToLowJobs(manager.lowPriorityJobs); + } + } + + protected void appendColumnsToJobs(Queue columnQueue, Map>> jobs) { + int size = columnQueue.size(); + int processed = 0; + for (int i = 0; i < size; i++) { + QueryColumn column = columnQueue.poll(); + if (column == null) { + continue; + } + TableIf table; + try { + table = StatisticsUtil.findTable(column.catalogId, column.dbId, column.tblId); + } catch (Exception e) { + LOG.warn("Fail to find table {}.{}.{} for column {}", + column.catalogId, column.dbId, column.tblId, column.colName, e); + continue; + } + if (StatisticConstants.SYSTEM_DBS.contains(table.getDatabase().getFullName())) { + continue; + } + Column col = table.getColumn(column.colName); + if (col == null || !col.isVisible() || StatisticsUtil.isUnsupportedType(col.getType())) { + continue; + } + Set> columnIndexPairs = table.getColumnIndexPairs( + Collections.singleton(column.colName)).stream() + .filter(p -> StatisticsUtil.needAnalyzeColumn(table, p)) + .collect(Collectors.toSet()); + if (columnIndexPairs.isEmpty()) { + continue; + } + TableName tableName = new TableName(table.getDatabase().getCatalog().getName(), + table.getDatabase().getFullName(), table.getName()); + synchronized (jobs) { + // If job map reach the upper limit, stop putting new jobs. + if (!jobs.containsKey(tableName) && jobs.size() >= JOB_MAP_SIZE) { + LOG.info("High or mid job map full."); + break; + } + if (jobs.containsKey(tableName)) { + jobs.get(tableName).addAll(columnIndexPairs); + } else { + jobs.put(tableName, columnIndexPairs); + } + } + processed++; + } + if (size > 0 && LOG.isDebugEnabled()) { + LOG.debug("{} of {} columns append to jobs", processed, size); + } + } + + protected void appendToLowJobs(Map>> jobs) { + if (System.currentTimeMillis() - lastRoundFinishTime < lowJobIntervalMs) { + return; + } + InternalCatalog catalog = Env.getCurrentInternalCatalog(); + List sortedDbs = catalog.getDbIds().stream().sorted().collect(Collectors.toList()); + int processed = 0; + for (long dbId : sortedDbs) { + if (dbId < currentDbId || catalog.getDbNullable(dbId) == null + || StatisticConstants.SYSTEM_DBS.contains(catalog.getDbNullable(dbId).getFullName())) { + continue; + } + currentDbId = dbId; + Optional db = catalog.getDb(dbId); + if (!db.isPresent()) { + continue; + } + List tables = db.get().getTables().stream() + .sorted((t1, t2) -> (int) (t1.getId() - t2.getId())).collect(Collectors.toList()); + for (Table t : tables) { + if (!(t instanceof OlapTable) || t.getId() <= currentTableId) { + continue; + } + if (t.getBaseSchema().size() > StatisticsUtil.getAutoAnalyzeTableWidthThreshold()) { + continue; + } + Set> columnIndexPairs = t.getColumnIndexPairs( + t.getSchemaAllIndexes(false).stream() + .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) + .map(Column::getName).collect(Collectors.toSet())) + .stream().filter(p -> StatisticsUtil.needAnalyzeColumn(t, p)) + .collect(Collectors.toSet()); + if (columnIndexPairs.isEmpty()) { + continue; + } + TableName tableName = new TableName(t.getDatabase().getCatalog().getName(), + t.getDatabase().getFullName(), t.getName()); + synchronized (jobs) { + // If job map reach the upper limit, stop adding new jobs. + if (!jobs.containsKey(tableName) && jobs.size() >= JOB_MAP_SIZE) { + LOG.info("Low job map full."); + return; + } + if (jobs.containsKey(tableName)) { + jobs.get(tableName).addAll(columnIndexPairs); + } else { + jobs.put(tableName, columnIndexPairs); + } + } + currentTableId = t.getId(); + if (++processed >= TABLE_BATCH_SIZE) { + return; + } + } + } + // All tables have been processed once, reset for the next loop. + if (LOG.isDebugEnabled()) { + LOG.debug("All low priority internal tables are appended once."); + } + currentDbId = 0; + currentTableId = 0; + lastRoundFinishTime = System.currentTimeMillis(); + } + + // For unit test only. + public void setLastRoundFinishTime(long value) { + lastRoundFinishTime = value; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java index 87ce90c5300a2b..8ec5582af5c93f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java @@ -29,8 +29,8 @@ import org.apache.doris.statistics.util.DBObjects; import org.apache.doris.statistics.util.StatisticsUtil; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.commons.text.StringSubstitutor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -337,7 +337,7 @@ public static void alterColumnStatistics(AlterColumnStatsStmt alterColumnStatsSt AnalysisInfo mockedJobInfo = new AnalysisInfoBuilder() .setTblUpdateTime(System.currentTimeMillis()) .setColName("") - .setJobColumns(Lists.newArrayList()) + .setJobColumns(Sets.newHashSet()) .setUserInject(true) .setJobType(AnalysisInfo.JobType.MANUAL) .build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java index 3b9b1e2bead005..a5073a922143f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java @@ -25,7 +25,9 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.statistics.AnalysisInfo.AnalysisMethod; import org.apache.doris.statistics.AnalysisInfo.JobType; +import org.apache.doris.statistics.util.StatisticsUtil; import com.google.common.annotations.VisibleForTesting; import com.google.gson.annotations.SerializedName; @@ -129,30 +131,34 @@ public void update(AnalysisInfo analyzedJob, TableIf tableIf) { for (Pair colPair : analyzedJob.jobColumns) { ColStatsMeta colStatsMeta = colToColStatsMeta.get(colPair); if (colStatsMeta == null) { - colToColStatsMeta.put(colPair, new ColStatsMeta(updatedTime, - analyzedJob.analysisMethod, analyzedJob.analysisType, analyzedJob.jobType, 0)); + colToColStatsMeta.put(colPair, new ColStatsMeta(updatedTime, analyzedJob.analysisMethod, + analyzedJob.analysisType, analyzedJob.jobType, 0, analyzedJob.rowCount, + analyzedJob.updateRows)); } else { colStatsMeta.updatedTime = updatedTime; colStatsMeta.analysisType = analyzedJob.analysisType; colStatsMeta.analysisMethod = analyzedJob.analysisMethod; colStatsMeta.jobType = analyzedJob.jobType; + colStatsMeta.updatedRows = analyzedJob.updateRows; + colStatsMeta.rowCount = analyzedJob.rowCount; } } jobType = analyzedJob.jobType; if (tableIf != null) { if (tableIf instanceof OlapTable) { - rowCount = analyzedJob.emptyJob ? 0 : tableIf.getRowCount(); + rowCount = analyzedJob.rowCount; } - if (analyzedJob.emptyJob) { + if (rowCount == 0 && AnalysisMethod.SAMPLE.equals(analyzedJob.analysisMethod)) { return; } if (analyzedJob.jobColumns.containsAll( tableIf.getColumnIndexPairs( - tableIf.getSchemaAllIndexes(false).stream().map(Column::getName).collect(Collectors.toSet())))) { - updatedRows.set(0); + tableIf.getSchemaAllIndexes(false).stream() + .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) + .map(Column::getName).collect(Collectors.toSet())))) { newPartitionLoaded.set(false); - } - if (tableIf instanceof OlapTable) { + userInjected = false; + } else if (tableIf instanceof OlapTable) { PartitionInfo partitionInfo = ((OlapTable) tableIf).getPartitionInfo(); if (partitionInfo != null && analyzedJob.jobColumns .containsAll(tableIf.getColumnIndexPairs(partitionInfo.getPartitionColumns().stream() diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java index 44ff7dafe64cdf..12e4e8ff63abce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java @@ -56,6 +56,7 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.hive.HMSExternalTable.DLAType; import org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral; import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; import org.apache.doris.qe.AutoCloseConnectContext; @@ -65,11 +66,14 @@ import org.apache.doris.qe.StmtExecutor; import org.apache.doris.qe.VariableMgr; import org.apache.doris.statistics.AnalysisInfo; +import org.apache.doris.statistics.AnalysisManager; +import org.apache.doris.statistics.ColStatsMeta; import org.apache.doris.statistics.ColumnStatistic; import org.apache.doris.statistics.ColumnStatisticBuilder; import org.apache.doris.statistics.Histogram; import org.apache.doris.statistics.ResultRow; import org.apache.doris.statistics.StatisticConstants; +import org.apache.doris.statistics.TableStatsMeta; import org.apache.doris.system.Frontend; import com.google.common.base.Preconditions; @@ -790,6 +794,16 @@ public static boolean enableAutoAnalyze() { return false; } + public static boolean enableAutoAnalyzeInternalCatalog() { + try { + return findConfigFromGlobalSessionVar( + SessionVariable.ENABLE_AUTO_ANALYZE_INTERNAL_CATALOG).enableAutoAnalyzeInternalCatalog; + } catch (Exception e) { + LOG.warn("Fail to get value of enable auto analyze internal catalog, return false by default", e); + } + return true; + } + public static int getInsertMergeCount() { try { return findConfigFromGlobalSessionVar(SessionVariable.STATS_INSERT_MERGE_ITEM_COUNT) @@ -898,7 +912,7 @@ public static boolean isMvColumn(TableIf table, String columnName) { } public static boolean isEmptyTable(TableIf table, AnalysisInfo.AnalysisMethod method) { - int waitRowCountReportedTime = 90; + int waitRowCountReportedTime = 75; if (!(table instanceof OlapTable) || method.equals(AnalysisInfo.AnalysisMethod.FULL)) { return false; } @@ -921,4 +935,71 @@ public static boolean isEmptyTable(TableIf table, AnalysisInfo.AnalysisMethod me return true; } + public static boolean needAnalyzeColumn(TableIf table, Pair column) { + if (column == null) { + return false; + } + AnalysisManager manager = Env.getServingEnv().getAnalysisManager(); + TableStatsMeta tableStatsStatus = manager.findTableStatsStatus(table.getId()); + // Table never been analyzed, need analyze. + if (tableStatsStatus == null) { + return true; + } + // User injected column stats, don't do auto analyze, avoid overwrite user injected stats. + if (tableStatsStatus.userInjected) { + return false; + } + ColStatsMeta columnStatsMeta = tableStatsStatus.findColumnStatsMeta(column.first, column.second); + // Column never been analyzed, need analyze. + if (columnStatsMeta == null) { + return true; + } + if (table instanceof OlapTable) { + OlapTable olapTable = (OlapTable) table; + // 0. Check new partition first time loaded flag. + if (olapTable.isPartitionColumn(column.second) && tableStatsStatus.newPartitionLoaded.get()) { + return true; + } + // 1. Check row count. + // TODO: One conner case. Last analyze row count is 0, but actually it's not 0 because isEmptyTable waiting. + long currentRowCount = olapTable.getRowCount(); + long lastAnalyzeRowCount = columnStatsMeta.rowCount; + // 1.1 Empty table -> non-empty table. Need analyze. + if (currentRowCount != 0 && lastAnalyzeRowCount == 0) { + return true; + } + // 1.2 Non-empty table -> empty table. Need analyze; + if (currentRowCount == 0 && lastAnalyzeRowCount != 0) { + return true; + } + // 1.3 Table is still empty. Not need to analyze. lastAnalyzeRowCount == 0 is always true here. + if (currentRowCount == 0) { + return false; + } + // 1.4 If row count changed more than the threshold, need analyze. + // lastAnalyzeRowCount == 0 is always false here. + double changeRate = + ((double) Math.abs(currentRowCount - lastAnalyzeRowCount) / lastAnalyzeRowCount) * 100.0; + if (changeRate > StatisticsUtil.getTableStatsHealthThreshold()) { + return true; + } + // 2. Check update rows. + long currentUpdatedRows = tableStatsStatus.updatedRows.get(); + long lastAnalyzeUpdateRows = columnStatsMeta.updatedRows; + changeRate = ((double) Math.abs(currentUpdatedRows - lastAnalyzeUpdateRows) / lastAnalyzeRowCount) * 100.0; + return changeRate > StatisticsUtil.getTableStatsHealthThreshold(); + } else { + // Now, we only support Hive external table auto analyze. + if (!(table instanceof HMSExternalTable)) { + return false; + } + HMSExternalTable hmsTable = (HMSExternalTable) table; + if (!hmsTable.getDlaType().equals(DLAType.HIVE)) { + return false; + } + // External is hard to calculate change rate, use time interval to control analyze frequency. + return System.currentTimeMillis() + - tableStatsStatus.updatedTime > StatisticsUtil.getExternalTableAutoAnalyzeIntervalInMillis(); + } + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java index 1bf2041bb4f12c..8a163523eebabb 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java @@ -80,7 +80,7 @@ public void testAppendBufTest2(@Mocked AnalysisInfo analysisInfo, @Mocked OlapAn new MockUp() { @Mock - protected void writeBuf() { + protected void flushBuffer() { writeBufInvokeTimes.incrementAndGet(); } @@ -111,7 +111,7 @@ public void testAppendBufTest3(@Mocked AnalysisInfo analysisInfo, @Mocked OlapAn new MockUp() { @Mock - protected void writeBuf() { + protected void flushBuffer() { writeBufInvokeTimes.incrementAndGet(); } @@ -184,7 +184,7 @@ protected void executeWithExceptionOnFail(StmtExecutor stmtExecutor) throws Exce protected void syncLoadStats() { } }; - job.writeBuf(); + job.flushBuffer(); Assertions.assertEquals(0, job.queryFinished.size()); } @@ -210,7 +210,7 @@ protected void syncLoadStats() { job.buf.add(new ColStatsData()); job.queryFinished = new HashSet<>(); job.queryFinished.add(task2); - job.writeBuf(); + job.flushBuffer(); Assertions.assertEquals(0, job.queryFinished.size()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java index 674456b0b46891..9c6580ee5cc2af 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java @@ -21,20 +21,31 @@ import org.apache.doris.analysis.AnalyzeTblStmt; import org.apache.doris.analysis.PartitionNames; import org.apache.doris.analysis.ShowAnalyzeStmt; +import org.apache.doris.analysis.ShowAutoAnalyzeJobsStmt; +import org.apache.doris.analysis.StatementBase; import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.Table; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.Pair; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.types.IntegerType; import org.apache.doris.statistics.AnalysisInfo.AnalysisType; import org.apache.doris.statistics.AnalysisInfo.JobType; import org.apache.doris.statistics.AnalysisInfo.ScheduleType; import org.apache.doris.statistics.util.StatisticsUtil; +import org.apache.doris.thrift.TQueryColumn; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; import mockit.Expectations; import mockit.Injectable; import mockit.Mock; @@ -46,8 +57,10 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; // CHECKSTYLE OFF @@ -110,7 +123,7 @@ public String toString() { // test build sync job @Test public void testBuildAndAssignJob1() throws Exception { - AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setJobColumns(new ArrayList<>()).build(); + AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setJobColumns(new HashSet<>()).build(); new MockUp() { @Mock @@ -187,7 +200,7 @@ public void updateTableStats(AnalysisInfo jobInfo) { // test build async job @Test public void testBuildAndAssignJob2(@Injectable OlapAnalysisTask analysisTask) throws Exception { - AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setJobColumns(new ArrayList<>()) + AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setJobColumns(new HashSet<>()) .setScheduleType(ScheduleType.PERIOD) .build(); new MockUp() { @@ -261,69 +274,6 @@ public void logCreateAnalysisJob(AnalysisInfo analysisJob) { }; } - @Test - public void testReAnalyze() { - new MockUp() { - - final Column c = new Column("col1", PrimitiveType.INT); - @Mock - public List getBaseSchema() { - return Lists.newArrayList(c); - } - - @Mock - public List getColumns() { return Lists.newArrayList(c); } - - @Mock - public List> getColumnIndexPairs(Set columns) { - List> jobList = Lists.newArrayList(); - jobList.add(Pair.of("1", "1")); - jobList.add(Pair.of("2", "2")); - jobList.add(Pair.of("3", "3")); - return jobList; - } - }; - OlapTable olapTable = new OlapTable(); - List> jobList = Lists.newArrayList(); - jobList.add(Pair.of("1", "1")); - jobList.add(Pair.of("2", "2")); - TableStatsMeta stats0 = new TableStatsMeta( - 0, new AnalysisInfoBuilder().setJobColumns(jobList) - .setColName("col1").build(), olapTable); - Assertions.assertTrue(olapTable.needReAnalyzeTable(stats0)); - - new MockUp() { - int count = 0; - int[] rowCount = new int[]{100, 100, 200, 200, 1, 1}; - - @Mock - public long getRowCount() { - return rowCount[count++]; - } - @Mock - public List> getColumnIndexPairs(Set columns) { - List> jobList = Lists.newArrayList(); - return jobList; - } - }; - TableStatsMeta stats1 = new TableStatsMeta( - 50, new AnalysisInfoBuilder().setJobColumns(new ArrayList<>()) - .setColName("col1").build(), olapTable); - stats1.updatedRows.addAndGet(50); - - Assertions.assertTrue(olapTable.needReAnalyzeTable(stats1)); - TableStatsMeta stats2 = new TableStatsMeta( - 190, new AnalysisInfoBuilder() - .setJobColumns(new ArrayList<>()).setColName("col1").build(), olapTable); - stats2.updatedRows.addAndGet(20); - Assertions.assertFalse(olapTable.needReAnalyzeTable(stats2)); - - TableStatsMeta stats3 = new TableStatsMeta(0, new AnalysisInfoBuilder() - .setJobColumns(new ArrayList<>()).setEmptyJob(true).setColName("col1").build(), olapTable); - Assertions.assertTrue(olapTable.needReAnalyzeTable(stats3)); - - } - @Test public void testRecordLimit1() { Config.analyze_record_limit = 2; @@ -399,4 +349,293 @@ public void testShowAutoTasks(@Injectable ShowAnalyzeStmt stmt) { Assertions.assertEquals(AnalysisState.FINISHED, analysisInfos.get(1).getState()); Assertions.assertEquals(AnalysisState.FAILED, analysisInfos.get(2).getState()); } + + @Test + public void testAddQuerySlotToQueue() throws DdlException { + AnalysisManager analysisManager = new AnalysisManager(); + InternalCatalog testCatalog = new InternalCatalog(); + Database db = new Database(100, "testDb"); + testCatalog.unprotectCreateDb(db); + Column column1 = new Column("placeholder", PrimitiveType.INT); + Column column2 = new Column("placeholder", PrimitiveType.INT); + Column column3 = new Column("test", PrimitiveType.INT); + List schema = new ArrayList<>(); + schema.add(column1); + OlapTable table = new OlapTable(200, "testTable", schema, null, null, null); + db.createTableWithLock(table, true, false); + + new MockUp
() { + @Mock + public DatabaseIf getDatabase() { + return db; + } + }; + + new MockUp() { + @Mock + public CatalogIf getCatalog() { + return testCatalog; + } + }; + + SlotReference slot1 = new SlotReference(new ExprId(1), "slot1", IntegerType.INSTANCE, true, + new ArrayList<>(), table, column1, Optional.empty(), null); + SlotReference slot2 = new SlotReference(new ExprId(2), "slot2", IntegerType.INSTANCE, true, + new ArrayList<>(), table, column2, Optional.empty(), null); + SlotReference slot3 = new SlotReference(new ExprId(3), "slot3", IntegerType.INSTANCE, true, + new ArrayList<>(), table, column3, Optional.empty(), null); + Set set1 = new HashSet<>(); + set1.add(slot1); + set1.add(slot2); + analysisManager.updateHighPriorityColumn(set1); + Assertions.assertEquals(2, analysisManager.highPriorityColumns.size()); + QueryColumn result = analysisManager.highPriorityColumns.poll(); + Assertions.assertEquals("placeholder", result.colName); + Assertions.assertEquals(testCatalog.getId(), result.catalogId); + Assertions.assertEquals(db.getId(), result.dbId); + Assertions.assertEquals(table.getId(), result.tblId); + + result = analysisManager.highPriorityColumns.poll(); + Assertions.assertEquals("placeholder", result.colName); + Assertions.assertEquals(testCatalog.getId(), result.catalogId); + Assertions.assertEquals(db.getId(), result.dbId); + Assertions.assertEquals(table.getId(), result.tblId); + Assertions.assertEquals(0, analysisManager.highPriorityColumns.size()); + Set set2 = new HashSet<>(); + set2.add(slot3); + for (int i = 0; i < AnalysisManager.COLUMN_QUEUE_SIZE / 2 - 1; i++) { + analysisManager.updateHighPriorityColumn(set1); + } + Assertions.assertEquals(AnalysisManager.COLUMN_QUEUE_SIZE - 2, analysisManager.highPriorityColumns.size()); + analysisManager.updateHighPriorityColumn(set2); + Assertions.assertEquals(AnalysisManager.COLUMN_QUEUE_SIZE - 1, analysisManager.highPriorityColumns.size()); + analysisManager.updateHighPriorityColumn(set2); + Assertions.assertEquals(AnalysisManager.COLUMN_QUEUE_SIZE, analysisManager.highPriorityColumns.size()); + analysisManager.updateHighPriorityColumn(set2); + Assertions.assertEquals(AnalysisManager.COLUMN_QUEUE_SIZE, analysisManager.highPriorityColumns.size()); + + for (int i = 0; i < AnalysisManager.COLUMN_QUEUE_SIZE - 2; i++) { + result = analysisManager.highPriorityColumns.poll(); + Assertions.assertEquals("placeholder", result.colName); + Assertions.assertEquals(testCatalog.getId(), result.catalogId); + Assertions.assertEquals(db.getId(), result.dbId); + Assertions.assertEquals(table.getId(), result.tblId); + } + Assertions.assertEquals(2, analysisManager.highPriorityColumns.size()); + result = analysisManager.highPriorityColumns.poll(); + Assertions.assertEquals("test", result.colName); + Assertions.assertEquals(testCatalog.getId(), result.catalogId); + Assertions.assertEquals(db.getId(), result.dbId); + Assertions.assertEquals(table.getId(), result.tblId); + + Assertions.assertEquals(1, analysisManager.highPriorityColumns.size()); + result = analysisManager.highPriorityColumns.poll(); + Assertions.assertEquals("test", result.colName); + Assertions.assertEquals(testCatalog.getId(), result.catalogId); + Assertions.assertEquals(db.getId(), result.dbId); + Assertions.assertEquals(table.getId(), result.tblId); + + result = analysisManager.highPriorityColumns.poll(); + Assertions.assertNull(result); + } + + @Test + public void testMergeFollowerColumn() throws DdlException { + AnalysisManager analysisManager = new AnalysisManager(); + QueryColumn placeholder = new QueryColumn(1, 2, 3, "placeholder"); + QueryColumn high1 = new QueryColumn(10, 20, 30, "high1"); + QueryColumn high2 = new QueryColumn(11, 21, 31, "high2"); + QueryColumn mid1 = new QueryColumn(100, 200, 300, "mid1"); + QueryColumn mid2 = new QueryColumn(101, 201, 301, "mid2"); + List highColumns = new ArrayList<>(); + highColumns.add(high1.toThrift()); + highColumns.add(high2.toThrift()); + List midColumns = new ArrayList<>(); + midColumns.add(mid1.toThrift()); + midColumns.add(mid2.toThrift()); + for (int i = 0; i < AnalysisManager.COLUMN_QUEUE_SIZE - 1; i++) { + analysisManager.highPriorityColumns.offer(placeholder); + } + for (int i = 0; i < AnalysisManager.COLUMN_QUEUE_SIZE - 2; i++) { + analysisManager.midPriorityColumns.offer(placeholder); + } + Assertions.assertEquals(AnalysisManager.COLUMN_QUEUE_SIZE - 1, analysisManager.highPriorityColumns.size()); + Assertions.assertEquals(AnalysisManager.COLUMN_QUEUE_SIZE - 2, analysisManager.midPriorityColumns.size()); + analysisManager.mergeFollowerQueryColumns(highColumns, midColumns); + Assertions.assertEquals(AnalysisManager.COLUMN_QUEUE_SIZE, analysisManager.highPriorityColumns.size()); + Assertions.assertEquals(AnalysisManager.COLUMN_QUEUE_SIZE, analysisManager.midPriorityColumns.size()); + for (int i = 0; i < AnalysisManager.COLUMN_QUEUE_SIZE - 1; i++) { + QueryColumn poll = analysisManager.highPriorityColumns.poll(); + Assertions.assertEquals("placeholder", poll.colName); + Assertions.assertEquals(1, poll.catalogId); + Assertions.assertEquals(2, poll.dbId); + Assertions.assertEquals(3, poll.tblId); + } + QueryColumn poll = analysisManager.highPriorityColumns.poll(); + Assertions.assertEquals("high1", poll.colName); + Assertions.assertEquals(10, poll.catalogId); + Assertions.assertEquals(20, poll.dbId); + Assertions.assertEquals(30, poll.tblId); + Assertions.assertEquals(0, analysisManager.highPriorityColumns.size()); + + for (int i = 0; i < AnalysisManager.COLUMN_QUEUE_SIZE - 2; i++) { + QueryColumn pol2 = analysisManager.midPriorityColumns.poll(); + Assertions.assertEquals("placeholder", pol2.colName); + Assertions.assertEquals(1, pol2.catalogId); + Assertions.assertEquals(2, pol2.dbId); + Assertions.assertEquals(3, pol2.tblId); + } + QueryColumn pol2 = analysisManager.midPriorityColumns.poll(); + Assertions.assertEquals("mid1", pol2.colName); + Assertions.assertEquals(100, pol2.catalogId); + Assertions.assertEquals(200, pol2.dbId); + Assertions.assertEquals(300, pol2.tblId); + + pol2 = analysisManager.midPriorityColumns.poll(); + Assertions.assertEquals("mid2", pol2.colName); + Assertions.assertEquals(101, pol2.catalogId); + Assertions.assertEquals(201, pol2.dbId); + Assertions.assertEquals(301, pol2.tblId); + Assertions.assertEquals(0, analysisManager.midPriorityColumns.size()); + } + + @Test + public void testShowAutoJobs() { + AnalysisManager manager = new AnalysisManager(); + TableName high1 = new TableName("catalog1", "db1", "high1"); + TableName high2 = new TableName("catalog2", "db2", "high2"); + TableName mid1 = new TableName("catalog3", "db3", "mid1"); + TableName mid2 = new TableName("catalog4", "db4", "mid2"); + TableName low1 = new TableName("catalog5", "db5", "low1"); + + manager.highPriorityJobs.put(high1, new HashSet<>()); + manager.highPriorityJobs.get(high1).add(Pair.of("index1", "col1")); + manager.highPriorityJobs.get(high1).add(Pair.of("index2", "col2")); + manager.highPriorityJobs.put(high2, new HashSet<>()); + manager.highPriorityJobs.get(high2).add(Pair.of("index1", "col3")); + manager.midPriorityJobs.put(mid1, new HashSet<>()); + manager.midPriorityJobs.get(mid1).add(Pair.of("index1", "col4")); + manager.midPriorityJobs.put(mid2, new HashSet<>()); + manager.midPriorityJobs.get(mid2).add(Pair.of("index1", "col5")); + manager.lowPriorityJobs.put(low1, new HashSet<>()); + manager.lowPriorityJobs.get(low1).add(Pair.of("index1", "col6")); + manager.lowPriorityJobs.get(low1).add(Pair.of("index1", "col7")); + + new MockUp() { + @Mock + public boolean isAnalyzed() { + return true; + } + }; + ShowAutoAnalyzeJobsStmt stmt = new ShowAutoAnalyzeJobsStmt(null, null); + List autoAnalysisPendingJobs = manager.showAutoPendingJobs(stmt); + Assertions.assertEquals(5, autoAnalysisPendingJobs.size()); + AutoAnalysisPendingJob job = autoAnalysisPendingJobs.get(0); + Assertions.assertEquals("catalog1", job.catalogName); + Assertions.assertEquals("db1", job.dbName); + Assertions.assertEquals("high1", job.tableName); + Assertions.assertEquals(2, job.columns.size()); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col1"))); + Assertions.assertTrue(job.columns.contains(Pair.of("index2", "col2"))); + Assertions.assertEquals(JobPriority.HIGH, job.priority); + + job = autoAnalysisPendingJobs.get(1); + Assertions.assertEquals("catalog2", job.catalogName); + Assertions.assertEquals("db2", job.dbName); + Assertions.assertEquals("high2", job.tableName); + Assertions.assertEquals(1, job.columns.size()); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col3"))); + Assertions.assertEquals(JobPriority.HIGH, job.priority); + + job = autoAnalysisPendingJobs.get(2); + Assertions.assertEquals("catalog3", job.catalogName); + Assertions.assertEquals("db3", job.dbName); + Assertions.assertEquals("mid1", job.tableName); + Assertions.assertEquals(1, job.columns.size()); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col4"))); + Assertions.assertEquals(JobPriority.MID, job.priority); + + job = autoAnalysisPendingJobs.get(3); + Assertions.assertEquals("catalog4", job.catalogName); + Assertions.assertEquals("db4", job.dbName); + Assertions.assertEquals("mid2", job.tableName); + Assertions.assertEquals(1, job.columns.size()); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col5"))); + Assertions.assertEquals(JobPriority.MID, job.priority); + + job = autoAnalysisPendingJobs.get(4); + Assertions.assertEquals("catalog5", job.catalogName); + Assertions.assertEquals("db5", job.dbName); + Assertions.assertEquals("low1", job.tableName); + Assertions.assertEquals(2, job.columns.size()); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col6"))); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col7"))); + Assertions.assertEquals(JobPriority.LOW, job.priority); + + new MockUp() { + @Mock + public String getPriority() { + return JobPriority.HIGH.name().toUpperCase(); + } + }; + List highJobs = manager.showAutoPendingJobs(stmt); + Assertions.assertEquals(2, highJobs.size()); + job = highJobs.get(0); + Assertions.assertEquals("catalog1", job.catalogName); + Assertions.assertEquals("db1", job.dbName); + Assertions.assertEquals("high1", job.tableName); + Assertions.assertEquals(2, job.columns.size()); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col1"))); + Assertions.assertTrue(job.columns.contains(Pair.of("index2", "col2"))); + Assertions.assertEquals(JobPriority.HIGH, job.priority); + + job = highJobs.get(1); + Assertions.assertEquals("catalog2", job.catalogName); + Assertions.assertEquals("db2", job.dbName); + Assertions.assertEquals("high2", job.tableName); + Assertions.assertEquals(1, job.columns.size()); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col3"))); + Assertions.assertEquals(JobPriority.HIGH, job.priority); + + new MockUp() { + @Mock + public String getPriority() { + return JobPriority.MID.name().toUpperCase(); + } + }; + List midJobs = manager.showAutoPendingJobs(stmt); + Assertions.assertEquals(2, midJobs.size()); + job = midJobs.get(0); + Assertions.assertEquals("catalog3", job.catalogName); + Assertions.assertEquals("db3", job.dbName); + Assertions.assertEquals("mid1", job.tableName); + Assertions.assertEquals(1, job.columns.size()); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col4"))); + Assertions.assertEquals(JobPriority.MID, job.priority); + + job = midJobs.get(1); + Assertions.assertEquals("catalog4", job.catalogName); + Assertions.assertEquals("db4", job.dbName); + Assertions.assertEquals("mid2", job.tableName); + Assertions.assertEquals(1, job.columns.size()); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col5"))); + Assertions.assertEquals(JobPriority.MID, job.priority); + + new MockUp() { + @Mock + public String getPriority() { + return JobPriority.LOW.name().toUpperCase(); + } + }; + List lowJobs = manager.showAutoPendingJobs(stmt); + Assertions.assertEquals(1, lowJobs.size()); + job = lowJobs.get(0); + Assertions.assertEquals("catalog5", job.catalogName); + Assertions.assertEquals("db5", job.dbName); + Assertions.assertEquals("low1", job.tableName); + Assertions.assertEquals(2, job.columns.size()); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col6"))); + Assertions.assertTrue(job.columns.contains(Pair.of("index1", "col7"))); + Assertions.assertEquals(JobPriority.LOW, job.priority); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java index 5698f0e9b20e63..29e04b1ef4fd94 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java @@ -34,7 +34,7 @@ import org.apache.doris.statistics.util.StatisticsUtil; import org.apache.doris.utframe.TestWithFeService; -import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import mockit.Mock; import mockit.MockUp; import mockit.Mocked; @@ -44,6 +44,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; @@ -157,7 +158,7 @@ public void syncLoadColStats(long tableId, long idxId, String colName) { }; AnalysisTaskExecutor analysisTaskExecutor = new AnalysisTaskExecutor(1); - List> columns = Lists.newArrayList(); + Set> columns = Sets.newHashSet(); columns.add(Pair.of("col1", "t1")); AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setJobId(0).setTaskId(0) .setCatalogId(0).setDBId(0).setTblId(0) diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalyzeTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalyzeTest.java index bf6ce32e155f42..250a796cee4cd5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalyzeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalyzeTest.java @@ -36,7 +36,7 @@ import org.apache.doris.statistics.util.StatisticsUtil; import org.apache.doris.utframe.TestWithFeService; -import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import mockit.Expectations; import mockit.Mock; import mockit.MockUp; @@ -48,6 +48,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; public class AnalyzeTest extends TestWithFeService { @@ -159,7 +160,7 @@ public void execSQLs(List partitionAnalysisSQLs, Map par @Mock protected void runQuery(String sql) {} }; - List> colList = Lists.newArrayList(); + Set> colList = Sets.newHashSet(); colList.add(Pair.of("col1", "index1")); AnalysisInfo analysisJobInfo = new AnalysisInfoBuilder().setJobId(0).setTaskId(0) .setCatalogId(0) @@ -171,6 +172,7 @@ protected void runQuery(String sql) {} .setAnalysisType(AnalysisType.FUNDAMENTALS) .setJobColumns(colList) .setState(AnalysisState.RUNNING) + .setRowCount(10) .build(); new OlapAnalysisTask(analysisJobInfo).doExecute(); new Expectations() { diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/FollowerColumnSenderTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/FollowerColumnSenderTest.java new file mode 100644 index 00000000000000..2a5ae531d1e6dd --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/FollowerColumnSenderTest.java @@ -0,0 +1,88 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.statistics; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.Pair; +import org.apache.doris.statistics.util.StatisticsUtil; +import org.apache.doris.thrift.TQueryColumn; + +import mockit.Mock; +import mockit.MockUp; +import org.eclipse.jetty.util.BlockingArrayQueue; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Queue; +import java.util.Set; + +public class FollowerColumnSenderTest { + + @Test + public void testGetNeedAnalyzeColumns() { + new MockUp() { + @Mock + public Column getColumn(String name) { + return new Column("col", PrimitiveType.INT); + } + + @Mock + public Set> getColumnIndexPairs(Set columns) { + return Collections.singleton(Pair.of("mockIndex", "mockCol")); + } + }; + + new MockUp() { + boolean[] result = {false, true, false, true, true}; + int i = 0; + @Mock + public boolean needAnalyzeColumn(TableIf table, Pair column) { + return result[i++]; + } + + @Mock + public TableIf findTable(long catalogId, long dbId, long tblId) { + return new OlapTable(); + } + }; + QueryColumn column1 = new QueryColumn(1, 2, 3, "col1"); + QueryColumn column2 = new QueryColumn(1, 2, 3, "col2"); + QueryColumn column3 = new QueryColumn(1, 2, 3, "col3"); + QueryColumn column4 = new QueryColumn(1, 2, 3, "col4"); + Queue queue = new BlockingArrayQueue<>(); + queue.add(column1); + queue.add(column2); + queue.add(column3); + queue.add(column4); + queue.add(column4); + Assertions.assertEquals(5, queue.size()); + + FollowerColumnSender sender = new FollowerColumnSender(); + Set needAnalyzeColumns = sender.getNeedAnalyzeColumns(queue); + Assertions.assertEquals(2, needAnalyzeColumns.size()); + Assertions.assertFalse(needAnalyzeColumns.contains(column1.toThrift())); + Assertions.assertTrue(needAnalyzeColumns.contains(column2.toThrift())); + Assertions.assertFalse(needAnalyzeColumns.contains(column3.toThrift())); + Assertions.assertTrue(needAnalyzeColumns.contains(column4.toThrift())); + } + +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java index f7b75261cc54fa..6324624abac6a6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java @@ -17,457 +17,125 @@ package org.apache.doris.statistics; +import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; -import org.apache.doris.catalog.EnvFactory; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PrimitiveType; -import org.apache.doris.catalog.Table; -import org.apache.doris.catalog.TableIf; -import org.apache.doris.catalog.Type; -import org.apache.doris.catalog.View; -import org.apache.doris.common.Config; -import org.apache.doris.common.DdlException; -import org.apache.doris.common.FeConstants; import org.apache.doris.common.Pair; -import org.apache.doris.datasource.CatalogIf; -import org.apache.doris.datasource.InternalCatalog; -import org.apache.doris.statistics.util.StatisticsUtil; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.hive.HMSExternalTable.DLAType; +import org.apache.doris.datasource.jdbc.JdbcExternalTable; -import com.google.common.collect.Lists; -import mockit.Expectations; -import mockit.Injectable; import mockit.Mock; import mockit.MockUp; -import mockit.Mocked; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.time.LocalTime; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; +import java.util.HashSet; import java.util.List; -import java.util.Map; +import java.util.Map.Entry; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; public class StatisticsAutoCollectorTest { @Test - public void testAnalyzeAll(@Injectable AnalysisInfo analysisInfo) { - new MockUp() { - @Mock - public Collection getAllDbs() { - Database db1 = new Database(1, FeConstants.INTERNAL_DB_NAME); - Database db2 = new Database(2, "anyDB"); - List databaseIfs = new ArrayList<>(); - databaseIfs.add(db1); - databaseIfs.add(db2); - return databaseIfs; - } - }; - new MockUp() { - @Mock - public List constructAnalysisInfo(DatabaseIf db) { - return Arrays.asList(analysisInfo, analysisInfo); - } - - int count = 0; - - @Mock - public AnalysisInfo getReAnalyzeRequiredPart(AnalysisInfo jobInfo) { - return count++ == 0 ? null : jobInfo; - } - - @Mock - public void createSystemAnalysisJob(AnalysisInfo jobInfo) - throws DdlException { - - } - }; - - StatisticsAutoCollector saa = new StatisticsAutoCollector(); - saa.runAfterCatalogReady(); - new Expectations() { - { - try { - saa.createSystemAnalysisJob((AnalysisInfo) any); - times = 1; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - }; - } - - @Test - public void testConstructAnalysisInfo( - @Injectable OlapTable o2, @Injectable View v) { - new MockUp() { - @Mock - public List
getTables() { - List
tableIfs = new ArrayList<>(); - tableIfs.add(o2); - tableIfs.add(v); - return tableIfs; - } - - @Mock - public String getFullName() { - return "anyDb"; - } - }; - - new MockUp() { - @Mock - public String getName() { - return "anytable"; - } - - @Mock - public List getSchemaAllIndexes(boolean full) { - List columns = new ArrayList<>(); - columns.add(new Column("c1", PrimitiveType.INT)); - columns.add(new Column("c2", PrimitiveType.HLL)); - return columns; - } - }; - StatisticsAutoCollector saa = new StatisticsAutoCollector(); - List analysisInfoList = saa.constructAnalysisInfo(new Database(1, "anydb")); - Assertions.assertEquals(1, analysisInfoList.size()); - Assertions.assertNull(analysisInfoList.get(0).colName); - } - - @Test - public void testSkipWideTable() { - - TableIf tableIf = new OlapTable(); - - new MockUp() { - @Mock - public List getBaseSchema() { - return Lists.newArrayList(new Column("col1", Type.INT), new Column("col2", Type.INT)); - } - - @Mock - public List> getColumnIndexPairs(Set columns) { - ArrayList> list = Lists.newArrayList(); - list.add(Pair.of("1", "1")); - return list; - } - }; - - new MockUp() { - int count = 0; - int[] thresholds = {1, 10}; - - @Mock - public TableIf findTable(long catalogName, long dbName, long tblName) { - return tableIf; - } - - @Mock - public int getAutoAnalyzeTableWidthThreshold() { - return thresholds[count++]; - } - }; - - AnalysisInfo analysisInfo = new AnalysisInfoBuilder().build(); - StatisticsAutoCollector statisticsAutoCollector = new StatisticsAutoCollector(); - Assertions.assertNull(statisticsAutoCollector.getNeedAnalyzeColumns(analysisInfo)); - Assertions.assertNotNull(statisticsAutoCollector.getNeedAnalyzeColumns(analysisInfo)); - } - - @Test - public void testLoop() { - AtomicBoolean timeChecked = new AtomicBoolean(); - AtomicBoolean switchChecked = new AtomicBoolean(); - new MockUp() { - - @Mock - public boolean inAnalyzeTime(LocalTime now) { - timeChecked.set(true); - return true; - } - - @Mock - public boolean enableAutoAnalyze() { - switchChecked.set(true); - return true; - } - }; - StatisticsAutoCollector autoCollector = new StatisticsAutoCollector(); - autoCollector.collect(); - Assertions.assertTrue(timeChecked.get() && switchChecked.get()); - - } - - @Test - public void checkAvailableThread() { - StatisticsAutoCollector autoCollector = new StatisticsAutoCollector(); - Assertions.assertEquals(Config.auto_analyze_simultaneously_running_task_num, - autoCollector.analysisTaskExecutor.executors.getMaximumPoolSize()); - } - - @Test - public void testSkip(@Mocked OlapTable olapTable, @Mocked TableStatsMeta stats, @Mocked TableIf anyOtherTable) { - new MockUp() { - - @Mock - public long getDataSize(boolean singleReplica) { - return StatisticsUtil.getHugeTableLowerBoundSizeInBytes() * 5 + 1000000000; - } - }; - - new MockUp() { - - @Mock - public TableStatsMeta findTableStatsStatus(long tblId) { - return stats; - } - }; - // A very huge table has been updated recently, so we should skip it this time - stats.updatedTime = System.currentTimeMillis() - 1000; - stats.newPartitionLoaded = new AtomicBoolean(); - stats.newPartitionLoaded.set(true); - StatisticsAutoCollector autoCollector = new StatisticsAutoCollector(); - // Test new partition loaded data for the first time. Not skip. - Assertions.assertFalse(autoCollector.skip(olapTable)); - stats.newPartitionLoaded.set(false); - // Assertions.assertTrue(autoCollector.skip(olapTable)); - // The update of this huge table is long time ago, so we shouldn't skip it this time - stats.updatedTime = System.currentTimeMillis() - - StatisticsUtil.getHugeTableAutoAnalyzeIntervalInMillis() - 10000; - Assertions.assertFalse(autoCollector.skip(olapTable)); - new MockUp() { - - @Mock - public TableStatsMeta findTableStatsStatus(long tblId) { - return null; - } - }; - // can't find table stats meta, which means this table never get analyzed, so we shouldn't skip it this time - Assertions.assertFalse(autoCollector.skip(olapTable)); - new MockUp() { - - @Mock - public TableStatsMeta findTableStatsStatus(long tblId) { - return stats; - } - }; - stats.userInjected = true; - Assertions.assertTrue(autoCollector.skip(olapTable)); - // this is not olap table nor external table, so we should skip it this time - Assertions.assertTrue(autoCollector.skip(anyOtherTable)); + public void testFetchJob() { + AnalysisManager manager = new AnalysisManager(); + TableName high1 = new TableName("catalog", "db", "high1"); + TableName high2 = new TableName("catalog", "db", "high2"); + TableName mid1 = new TableName("catalog", "db", "mid1"); + TableName mid2 = new TableName("catalog", "db", "mid2"); + TableName low1 = new TableName("catalog", "db", "low1"); + + manager.highPriorityJobs.put(high1, new HashSet<>()); + manager.highPriorityJobs.get(high1).add(Pair.of("index1", "col1")); + manager.highPriorityJobs.get(high1).add(Pair.of("index1", "col2")); + manager.highPriorityJobs.put(high2, new HashSet<>()); + manager.highPriorityJobs.get(high2).add(Pair.of("index1", "col3")); + manager.midPriorityJobs.put(mid1, new HashSet<>()); + manager.midPriorityJobs.get(mid1).add(Pair.of("index1", "col4")); + manager.midPriorityJobs.put(mid2, new HashSet<>()); + manager.midPriorityJobs.get(mid2).add(Pair.of("index1", "col5")); + manager.lowPriorityJobs.put(low1, new HashSet<>()); + manager.lowPriorityJobs.get(low1).add(Pair.of("index1", "col6")); + manager.lowPriorityJobs.get(low1).add(Pair.of("index1", "col7")); + + + new MockUp() { + @Mock + public AnalysisManager getAnalysisManager() { + return manager; + } + }; + StatisticsAutoCollector collector = new StatisticsAutoCollector(); + Pair>>, JobPriority> job = collector.getJob(); + Assertions.assertEquals(high1, job.first.getKey()); + Assertions.assertEquals(2, job.first.getValue().size()); + Assertions.assertTrue(job.first.getValue().contains(Pair.of("index1", "col1"))); + Assertions.assertTrue(job.first.getValue().contains(Pair.of("index1", "col2"))); + Assertions.assertEquals(JobPriority.HIGH, job.second); + + job = collector.getJob(); + Assertions.assertEquals(high2, job.first.getKey()); + Assertions.assertEquals(1, job.first.getValue().size()); + Assertions.assertTrue(job.first.getValue().contains(Pair.of("index1", "col3"))); + Assertions.assertEquals(JobPriority.HIGH, job.second); + + job = collector.getJob(); + Assertions.assertEquals(mid1, job.first.getKey()); + Assertions.assertEquals(1, job.first.getValue().size()); + Assertions.assertTrue(job.first.getValue().contains(Pair.of("index1", "col4"))); + Assertions.assertEquals(JobPriority.MID, job.second); + + job = collector.getJob(); + Assertions.assertEquals(mid2, job.first.getKey()); + Assertions.assertEquals(1, job.first.getValue().size()); + Assertions.assertTrue(job.first.getValue().contains(Pair.of("index1", "col5"))); + Assertions.assertEquals(JobPriority.MID, job.second); + + job = collector.getJob(); + Assertions.assertEquals(low1, job.first.getKey()); + Assertions.assertEquals(2, job.first.getValue().size()); + Assertions.assertTrue(job.first.getValue().contains(Pair.of("index1", "col6"))); + Assertions.assertTrue(job.first.getValue().contains(Pair.of("index1", "col7"))); + Assertions.assertEquals(JobPriority.LOW, job.second); + + job = collector.getJob(); + Assertions.assertNull(job); } - // For small table, use full @Test - public void testCreateAnalyzeJobForTbl1( - @Injectable OlapTable t1, - @Injectable Database db - ) throws Exception { - new MockUp() { + public void testSupportAutoAnalyze() { + StatisticsAutoCollector collector = new StatisticsAutoCollector(); + Assertions.assertFalse(collector.supportAutoAnalyze(null)); + Column column1 = new Column("placeholder", PrimitiveType.INT); + List schema = new ArrayList<>(); + schema.add(column1); + OlapTable table1 = new OlapTable(200, "testTable", schema, null, null, null); + Assertions.assertTrue(collector.supportAutoAnalyze(table1)); - @Mock - public CatalogIf getCatalog() { - return Env.getCurrentInternalCatalog(); - } + ExternalTable externalTable = new JdbcExternalTable(1, "jdbctable", "jdbcdb", null); + Assertions.assertFalse(collector.supportAutoAnalyze(externalTable)); + new MockUp() { @Mock - public long getId() { - return 0; + public DLAType getDlaType() { + return DLAType.ICEBERG; } }; - new MockUp() { - - int count = 0; + ExternalTable icebergExternalTable = new HMSExternalTable(1, "hmsTable", "hmsDb", null); + Assertions.assertFalse(collector.supportAutoAnalyze(icebergExternalTable)); + new MockUp() { @Mock - public List getBaseSchema() { - return Lists.newArrayList(new Column("test", PrimitiveType.INT)); - } - - @Mock - public long getDataSize(boolean singleReplica) { - return StatisticsUtil.getHugeTableLowerBoundSizeInBytes() - 1; - } - - @Mock - public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { - return new OlapAnalysisTask(info); - } - - @Mock - public List getMvColumnIndexIds(String columnName) { - ArrayList objects = new ArrayList<>(); - objects.add(-1L); - return objects; + public DLAType getDlaType() { + return DLAType.HIVE; } }; - - new MockUp() { - @Mock - public TableIf findTable(long catalogId, long dbId, long tblId) { - return t1; - } - }; - - StatisticsAutoCollector sac = new StatisticsAutoCollector(); - List jobInfos = new ArrayList<>(); - sac.createAnalyzeJobForTbl(db, jobInfos, t1); - AnalysisInfo jobInfo = jobInfos.get(0); - List> columnNames = Lists.newArrayList(); - columnNames.add(Pair.of("test", "t1")); - jobInfo = new AnalysisInfoBuilder(jobInfo).setJobColumns(columnNames).build(); - Map analysisTasks = new HashMap<>(); - AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); - analysisManager.createTaskForEachColumns(jobInfo, analysisTasks, false); - Assertions.assertEquals(1, analysisTasks.size()); - for (BaseAnalysisTask task : analysisTasks.values()) { - Assertions.assertNull(task.getTableSample()); - } - } - - // for big table, use sample - @Test - public void testCreateAnalyzeJobForTbl2( - @Injectable OlapTable t1, - @Injectable Database db - ) throws Exception { - new MockUp() { - - @Mock - public CatalogIf getCatalog() { - return Env.getCurrentInternalCatalog(); - } - - @Mock - public long getId() { - return 0; - } - }; - new MockUp() { - - int count = 0; - - @Mock - public List getBaseSchema() { - return Lists.newArrayList(new Column("test", PrimitiveType.INT)); - } - - @Mock - public long getDataSize(boolean singleReplica) { - return StatisticsUtil.getHugeTableLowerBoundSizeInBytes() * 2; - } - - @Mock - public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { - return new OlapAnalysisTask(info); - } - - @Mock - public List getMvColumnIndexIds(String columnName) { - ArrayList objects = new ArrayList<>(); - objects.add(-1L); - return objects; - } - }; - - new MockUp() { - @Mock - public TableIf findTable(long catalogId, long dbId, long tblId) { - return t1; - } - }; - - StatisticsAutoCollector sac = new StatisticsAutoCollector(); - List jobInfos = new ArrayList<>(); - sac.createAnalyzeJobForTbl(db, jobInfos, t1); - AnalysisInfo jobInfo = jobInfos.get(0); - List> colNames = Lists.newArrayList(); - colNames.add(Pair.of("test", "1")); - jobInfo = new AnalysisInfoBuilder(jobInfo).setJobColumns(colNames).build(); - Map analysisTasks = new HashMap<>(); - AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); - analysisManager.createTaskForEachColumns(jobInfo, analysisTasks, false); - Assertions.assertEquals(1, analysisTasks.size()); - for (BaseAnalysisTask task : analysisTasks.values()) { - Assertions.assertNotNull(task.getTableSample()); - } - } - - @Test - public void testDisableAuto1() throws Exception { - InternalCatalog catalog1 = EnvFactory.getInstance().createInternalCatalog(); - List catalogs = Lists.newArrayList(); - catalogs.add(catalog1); - - new MockUp() { - @Mock - public List getCatalogsInOrder() { - return catalogs; - } - - @Mock - protected boolean canCollect() { - return false; - } - - }; - - StatisticsAutoCollector sac = new StatisticsAutoCollector(); - new Expectations(catalog1) {{ - catalog1.enableAutoAnalyze(); - times = 0; - }}; - - sac.analyzeAll(); - } - - @Test - public void testDisableAuto2() throws Exception { - InternalCatalog catalog1 = EnvFactory.getInstance().createInternalCatalog(); - List catalogs = Lists.newArrayList(); - catalogs.add(catalog1); - - Database db1 = new Database(); - List> dbs = Lists.newArrayList(); - dbs.add(db1); - - new MockUp() { - int count = 0; - boolean[] canCollectReturn = {true, false}; - @Mock - public List getCatalogsInOrder() { - return catalogs; - } - - @Mock - public List> getDatabasesInOrder(CatalogIf catalog) { - return dbs; - } - - @Mock - protected boolean canCollect() { - return canCollectReturn[count++]; - } - - }; - - StatisticsAutoCollector sac = new StatisticsAutoCollector(); - new Expectations(catalog1, db1) {{ - catalog1.enableAutoAnalyze(); - result = true; - times = 1; - db1.getFullName(); - times = 0; - }}; - - sac.analyzeAll(); + ExternalTable hiveExternalTable = new HMSExternalTable(1, "hmsTable", "hmsDb", null); + Assertions.assertTrue(collector.supportAutoAnalyze(hiveExternalTable)); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsJobAppenderTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsJobAppenderTest.java new file mode 100644 index 00000000000000..e3255ab23a0381 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsJobAppenderTest.java @@ -0,0 +1,281 @@ +// 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.statistics; + +import org.apache.doris.analysis.TableName; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.Type; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.Pair; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.statistics.util.StatisticsUtil; + +import com.google.common.collect.Lists; +import mockit.Mock; +import mockit.MockUp; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ArrayBlockingQueue; + +public class StatisticsJobAppenderTest { + + @Test + public void testAppendQueryColumnToHighAndMidJobMap() throws DdlException { + InternalCatalog testCatalog = new InternalCatalog(); + Database db = new Database(100, "testDb"); + testCatalog.unprotectCreateDb(db); + Column column1 = new Column("placeholder", PrimitiveType.INT); + List schema = new ArrayList<>(); + schema.add(column1); + OlapTable table1 = new OlapTable(200, "testTable", schema, null, null, null); + OlapTable table2 = new OlapTable(200, "testTable2", schema, null, null, null); + OlapTable table3 = new OlapTable(200, "testTable3", schema, null, null, null); + new MockUp() { + int i = 0; + Table[] tables = {table1, table2, table1, table3, table2}; + + @Mock + public boolean needAnalyzeColumn(TableIf table, Pair column) { + return true; + } + + @Mock + public TableIf findTable(long catalogId, long dbId, long tblId) { + return tables[i++]; + } + }; + + new MockUp
() { + @Mock + public DatabaseIf getDatabase() { + return db; + } + + @Mock + public Column getColumn(String name) { + return new Column("mockCol", Type.INT); + } + }; + + new MockUp() { + @Mock + public Set> getColumnIndexPairs(Set columns) { + String column = columns.iterator().next(); + return Collections.singleton(Pair.of("mockIndex", column)); + } + }; + + Queue testQueue = new ArrayBlockingQueue<>(100); + Map>> testMap = new HashMap<>(); + QueryColumn high1 = new QueryColumn(10, 20, 30, "high1"); + testQueue.add(high1); + + StatisticsJobAppender appender = new StatisticsJobAppender(); + appender.appendColumnsToJobs(testQueue, testMap); + Assertions.assertEquals(1, testMap.size()); + Assertions.assertEquals(1, testMap.values().size()); + Assertions.assertTrue(testMap.get(new TableName("internal", "testDb", "testTable")).contains(Pair.of("mockIndex", "high1"))); + + QueryColumn high2 = new QueryColumn(10, 20, 30, "high2"); + QueryColumn high3 = new QueryColumn(10, 20, 30, "high3"); + testQueue.add(high2); + testQueue.add(high3); + appender.appendColumnsToJobs(testQueue, testMap); + Assertions.assertEquals(2, testMap.size()); + + Set> table1Column = testMap.get(new TableName("internal", "testDb", "testTable")); + Assertions.assertEquals(2, table1Column.size()); + Assertions.assertTrue(table1Column.contains(Pair.of("mockIndex", "high1"))); + Assertions.assertTrue(table1Column.contains(Pair.of("mockIndex", "high3"))); + + Set> table2Column = testMap.get(new TableName("internal", "testDb", "testTable2")); + Assertions.assertEquals(1, table2Column.size()); + Assertions.assertTrue(table2Column.contains(Pair.of("mockIndex", "high2"))); + + for (int i = 0; i < StatisticsJobAppender.JOB_MAP_SIZE - 2; i++) { + testMap.put(new TableName("a", "b", UUID.randomUUID().toString()), new HashSet<>()); + } + Assertions.assertEquals(StatisticsJobAppender.JOB_MAP_SIZE, testMap.size()); + + QueryColumn high4 = new QueryColumn(10, 20, 30, "high4"); + testQueue.add(high4); + appender.appendColumnsToJobs(testQueue, testMap); + Assertions.assertEquals(StatisticsJobAppender.JOB_MAP_SIZE, testMap.size()); + + QueryColumn high5 = new QueryColumn(10, 20, 30, "high5"); + testQueue.add(high5); + appender.appendColumnsToJobs(testQueue, testMap); + table2Column = testMap.get(new TableName("internal", "testDb", "testTable2")); + Assertions.assertEquals(2, table2Column.size()); + Assertions.assertTrue(table2Column.contains(Pair.of("mockIndex", "high2"))); + Assertions.assertTrue(table2Column.contains(Pair.of("mockIndex", "high5"))); + } + + @Test + public void testAppendQueryColumnToLowJobMap() throws DdlException { + InternalCatalog testCatalog = new InternalCatalog(); + int id = 10; + for (int i = 0; i < 70; i++) { + Database db = new Database(id++, "testDb" + i); + testCatalog.unprotectCreateDb(db); + Column column1 = new Column("placeholder", PrimitiveType.INT); + List schema = new ArrayList<>(); + schema.add(column1); + OlapTable table1 = new OlapTable(id++, "testTable" + id + "_1", schema, null, null, null); + OlapTable table2 = new OlapTable(id++, "testTable" + id + "_1", schema, null, null, null); + db.createTableWithLock(table1, true, false); + db.createTableWithLock(table2, true, false); + } + + new MockUp() { + @Mock + public InternalCatalog getCurrentInternalCatalog() { + return testCatalog; + } + }; + + new MockUp() { + @Mock + public List getBaseSchema() { + return Lists.newArrayList(); + } + + @Mock + public Set> getColumnIndexPairs(Set columns) { + return Collections.singleton(Pair.of("mockIndex", "mockColumn")); + } + }; + + Map>> testMap = new HashMap<>(); + StatisticsJobAppender appender = new StatisticsJobAppender(); + appender.appendToLowJobs(testMap); + Assertions.assertEquals(100, testMap.size()); + testMap.clear(); + appender.appendToLowJobs(testMap); + Assertions.assertEquals(40, testMap.size()); + + for (int i = 0; i < StatisticsJobAppender.JOB_MAP_SIZE; i++) { + Database db = new Database(id++, "testDb" + i); + testCatalog.unprotectCreateDb(db); + Column column1 = new Column("placeholder", PrimitiveType.INT); + List schema = new ArrayList<>(); + schema.add(column1); + OlapTable table1 = new OlapTable(id++, "testTable" + id + "_1", schema, null, null, null); + OlapTable table2 = new OlapTable(id++, "testTable" + id + "_1", schema, null, null, null); + db.createTableWithLock(table1, true, false); + db.createTableWithLock(table2, true, false); + } + + testMap.clear(); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + Assertions.assertEquals(StatisticsJobAppender.JOB_MAP_SIZE, testMap.size()); + } + + @Test + public void testSkipWideTable() throws DdlException { + InternalCatalog testCatalog = new InternalCatalog(); + int id = 10; + Database db = new Database(id++, "testDb"); + testCatalog.unprotectCreateDb(db); + Column column1 = new Column("placeholder", PrimitiveType.INT); + List schema = new ArrayList<>(); + schema.add(column1); + OlapTable table1 = new OlapTable(id++, "testTable" + id + "_1", schema, null, null, null); + db.createTableWithLock(table1, true, false); + new MockUp() { + @Mock + public InternalCatalog getCurrentInternalCatalog() { + return testCatalog; + } + }; + new MockUp() { + @Mock + public List getBaseSchema() { + return Lists.newArrayList(new Column("col1", Type.INT), new Column("col2", Type.INT)); + } + + @Mock + public Set> getColumnIndexPairs(Set columns) { + return Collections.singleton(Pair.of("1", "1")); + } + }; + + new MockUp() { + int count = 0; + int[] thresholds = {1, 10}; + + @Mock + public int getAutoAnalyzeTableWidthThreshold() { + return thresholds[count++]; + } + }; + Map>> testMap = new HashMap<>(); + StatisticsJobAppender appender = new StatisticsJobAppender(); + appender.appendToLowJobs(testMap); + Assertions.assertEquals(0, testMap.size()); + appender.setLastRoundFinishTime(0); + appender.appendToLowJobs(testMap); + Assertions.assertEquals(1, testMap.size()); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/TableStatsMetaTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/TableStatsMetaTest.java index 94eab9e00cc501..10e1973aa3318d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/TableStatsMetaTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/TableStatsMetaTest.java @@ -19,27 +19,19 @@ import org.apache.doris.catalog.OlapTable; -import mockit.Mock; -import mockit.MockUp; import mockit.Mocked; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.util.ArrayList; +import java.util.HashSet; class TableStatsMetaTest { @Test void update(@Mocked OlapTable table) { - new MockUp() { - @Mock - public long getRowCount() { - return 4; - } - }; TableStatsMeta tableStatsMeta = new TableStatsMeta(); - AnalysisInfo jobInfo = new AnalysisInfoBuilder().setJobColumns(new ArrayList<>()) - .setColName("col1").build(); + AnalysisInfo jobInfo = new AnalysisInfoBuilder().setRowCount(4) + .setJobColumns(new HashSet<>()).setColName("col1").build(); tableStatsMeta.update(jobInfo, table); Assertions.assertEquals(4, tableStatsMeta.rowCount); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java index 724e0363833305..275471a66982b6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java @@ -17,10 +17,21 @@ package org.apache.doris.statistics.util; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Pair; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.hive.HMSExternalTable.DLAType; +import org.apache.doris.datasource.jdbc.JdbcExternalTable; import org.apache.doris.qe.SessionVariable; +import org.apache.doris.statistics.AnalysisManager; +import org.apache.doris.statistics.ColStatsMeta; import org.apache.doris.statistics.ResultRow; +import org.apache.doris.statistics.TableStatsMeta; import com.google.common.collect.Lists; import mockit.Mock; @@ -33,6 +44,7 @@ import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Base64; +import java.util.List; class StatisticsUtilTest { @Test @@ -150,4 +162,141 @@ void testEscape() { // \\''"" Assertions.assertEquals("\\\\''\"", StatisticsUtil.escapeSQL(origin)); } + + @Test + void testNeedAnalyzeColumn() { + Column column = new Column("testColumn", PrimitiveType.INT); + List schema = new ArrayList<>(); + schema.add(column); + OlapTable table = new OlapTable(200, "testTable", schema, null, null, null); + // Test table stats meta is null. + new MockUp() { + @Mock + public TableStatsMeta findTableStatsStatus(long tblId) { + return null; + } + }; + Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + // Test user injected flag is set. + TableStatsMeta tableMeta = new TableStatsMeta(); + tableMeta.userInjected = true; + new MockUp() { + @Mock + public TableStatsMeta findTableStatsStatus(long tblId) { + return tableMeta; + } + }; + Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + // Test column meta is null. + tableMeta.userInjected = false; + Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + new MockUp() { + @Mock + public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { + return new ColStatsMeta(0, null, null, null, 0, 0, 0); + } + }; + + // Test not supported external table type. + ExternalTable externalTable = new JdbcExternalTable(1, "jdbctable", "jdbcdb", null); + Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(externalTable, Pair.of("index", column.getName()))); + + // Test hms external table not hive type. + new MockUp() { + @Mock + public DLAType getDlaType() { + return DLAType.ICEBERG; + } + }; + ExternalTable hmsExternalTable = new HMSExternalTable(1, "hmsTable", "hmsDb", null); + Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(hmsExternalTable, Pair.of("index", column.getName()))); + + // Test partition first load. + new MockUp() { + @Mock + public boolean isPartitionColumn(String columnName) { + return true; + } + }; + tableMeta.newPartitionLoaded.set(true); + Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + // Test empty table to non-empty table. + new MockUp() { + @Mock + public long getRowCount() { + return 100; + } + }; + tableMeta.newPartitionLoaded.set(false); + Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + // Test non-empty table to empty table. + new MockUp() { + @Mock + public long getRowCount() { + return 0; + } + }; + new MockUp() { + @Mock + public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { + return new ColStatsMeta(0, null, null, null, 0, 100, 0); + } + }; + tableMeta.newPartitionLoaded.set(false); + Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + // Test table still empty. + new MockUp() { + @Mock + public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { + return new ColStatsMeta(0, null, null, null, 0, 0, 0); + } + }; + tableMeta.newPartitionLoaded.set(false); + Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + // Test row count changed more than threshold. + new MockUp() { + @Mock + public long getRowCount() { + return 1000; + } + }; + new MockUp() { + @Mock + public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { + return new ColStatsMeta(0, null, null, null, 0, 500, 0); + } + }; + tableMeta.newPartitionLoaded.set(false); + Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + // Test update rows changed more than threshold. + new MockUp() { + @Mock + public long getRowCount() { + return 120; + } + }; + new MockUp() { + @Mock + public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { + return new ColStatsMeta(0, null, null, null, 0, 100, 80); + } + }; + tableMeta.newPartitionLoaded.set(false); + tableMeta.updatedRows.set(200); + Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + // Test update rows changed less than threshold + tableMeta.newPartitionLoaded.set(false); + tableMeta.updatedRows.set(100); + Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + } } diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index f8e53af077dd85..6ed7c23ec3c2f8 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1468,6 +1468,18 @@ struct TReportCommitTxnResultRequest { 4: optional binary payload } +struct TQueryColumn { + 1: optional string catalogId + 2: optional string dbId + 3: optional string tblId + 4: optional string colName +} + +struct TSyncQueryColumns { + 1: optional list highPriorityColumns; + 2: optional list midPriorityColumns; +} + service FrontendService { TGetDbsResult getDbNames(1: TGetDbsParams params) TGetTablesResult getTableNames(1: TGetTablesParams params) @@ -1558,4 +1570,5 @@ service FrontendService { TShowProcessListResult showProcessList(1: TShowProcessListRequest request) Status.TStatus reportCommitTxnResult(1: TReportCommitTxnResultRequest request) TShowUserResult showUser(1: TShowUserRequest request) + Status.TStatus syncQueryColumns(1: TSyncQueryColumns request) } diff --git a/regression-test/suites/external_table_p2/hive/test_hive_statistic_auto.groovy b/regression-test/suites/external_table_p2/hive/test_hive_statistic_auto.groovy index 8a7591daeb1b1e..eddf0bd8e7c16c 100644 --- a/regression-test/suites/external_table_p2/hive/test_hive_statistic_auto.groovy +++ b/regression-test/suites/external_table_p2/hive/test_hive_statistic_auto.groovy @@ -32,7 +32,7 @@ suite("test_hive_statistic_auto", "p2,external,hive,external_remote,external_rem logger.info("catalog " + catalog_name + " created") // Test analyze table without init. - sql """analyze database ${catalog_name}.statistics PROPERTIES("use.auto.analyzer"="true")""" + sql """analyze table ${catalog_name}.statistics.statistics PROPERTIES("use.auto.analyzer"="true")""" sql """use ${catalog_name}.statistics""" for (int i = 0; i < 10; i++) { diff --git a/regression-test/suites/statistics/analyze_stats.groovy b/regression-test/suites/statistics/analyze_stats.groovy index 7c6231aa48fd02..bcbec86b1a77b9 100644 --- a/regression-test/suites/statistics/analyze_stats.groovy +++ b/regression-test/suites/statistics/analyze_stats.groovy @@ -2774,7 +2774,7 @@ PARTITION `p599` VALUES IN (599) // Test auto analyze with job type SYSTEM sql """drop stats trigger_test""" - sql """analyze database trigger PROPERTIES("use.auto.analyzer"="true")""" + sql """analyze table trigger_test PROPERTIES("use.auto.analyzer"="true")""" int i = 0; for (0; i < 10; i++) { result = sql """show column stats trigger_test"""