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 73059c35a031b7c..f25c32d0c24225d 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 @@ -1603,7 +1603,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 d6a8ffd8ac478bc..0da35bca91a621b 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -4424,6 +4424,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/ShowAnalyzeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeStmt.java index efcfc5170246833..734073901fe7f66 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 000000000000000..560387fa5bc11cb --- /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 a4216f55661e16e..04ecd7a5849e3c2 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; @@ -161,6 +163,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 e8c609b3c246bf9..c4d9f06c653cceb 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 @@ -242,9 +242,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; @@ -520,6 +522,10 @@ public class Env { private StatisticsAutoCollector statisticsAutoCollector; + private StatisticsJobAppender statisticsJobAppender; + + private FollowerColumnSender followerColumnSender; + private HiveTransactionMgr hiveTransactionMgr; private TopicPublisherThread topicPublisherThread; @@ -749,6 +755,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(); @@ -1044,13 +1051,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(); TopicPublisher wgPublisher = new WorkloadGroupPublisher(this); @@ -1700,6 +1700,11 @@ protected void startMasterOnlyDaemonThreads() { binlogGcer.start(); columnIdFlusher.start(); insertOverwriteManager.start(); + + // auto analyze related threads. + statisticsCleaner.start(); + statisticsAutoCollector.start(); + statisticsJobAppender.start(); } // start threads that should run on all FE @@ -1757,6 +1762,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. @@ -6070,6 +6080,10 @@ public StatisticsAutoCollector getStatisticsAutoCollector() { return statisticsAutoCollector; } + 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 ba8305a69876fa9..5bbf10c4acd796e 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; @@ -100,6 +99,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -776,9 +776,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; @@ -1297,29 +1308,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) { @@ -1328,7 +1319,7 @@ public List> getColumnIndexPairs(Set columns) { if (col == null || StatisticsUtil.isUnsupportedType(col.getType())) { continue; } - ret.add(Pair.of(getIndexNameById(meta.getIndexId()), column)); + ret.add(Pair.of(getIndexNameById(meta.getIndexId()), column.toLowerCase(Locale.ROOT))); } } return ret; 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 52655fa064943d7..d927d8055ccc6a0 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; @@ -386,11 +386,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; @@ -643,11 +638,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"); @@ -659,7 +649,7 @@ public long fetchRowCount() { } @Override - public List> getColumnIndexPairs(Set columns) { - return Lists.newArrayList(); + public Set> getColumnIndexPairs(Set columns) { + return Sets.newHashSet(); } } 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 f7c8b4b83252bfc..20a0f4e3a73d6eb 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 { @@ -183,13 +186,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 82390b916560a86..bb5b5089a2f9961 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; @@ -48,10 +47,10 @@ import java.io.DataOutput; import java.io.IOException; import java.util.List; +import java.util.Locale; 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 +150,6 @@ public List getBaseSchema() { return getFullSchema(); } - @Override - public List getSchemaAllIndexes(boolean full) { - return getBaseSchema(); - } - @Override public List getBaseSchema(boolean full) { return getFullSchema(); @@ -318,32 +312,15 @@ 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())) { continue; } // External table put table name as index name. - ret.add(Pair.of(String.valueOf(name), column)); + ret.add(Pair.of(String.valueOf(name), column.toLowerCase(Locale.ROOT))); } return ret; } 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 fb5ea1a869f41e1..269383416b487a7 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 @@ -3162,7 +3162,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. @@ -3171,6 +3170,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. @@ -3331,10 +3331,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 a68c7510965b265..bc35330e748afff 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; @@ -409,7 +410,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 aa38af04ec50c42..8f9de063b9a2585 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 @@ -307,6 +307,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 000000000000000..ebf361de1d3a9a7 --- /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 b5a9d3752e95c6c..20e81e959ec4c15 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 @@ -477,6 +477,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"; @@ -1523,6 +1525,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 7513963f1164a95..2c8b2d7c49de87f 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; @@ -200,6 +201,7 @@ import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.mysql.privilege.Privilege; 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; @@ -437,6 +439,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) { @@ -2783,6 +2787,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: {}", @@ -2793,6 +2798,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 4dc88507bd8e592..087010730e10710 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 @@ -223,6 +223,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; @@ -3754,4 +3755,11 @@ public TShowProcessListResult showProcessList(TShowProcessListRequest request) { return result; } + @Override + 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 c167db2228d8cc2..e0fd91d1100354f 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 00cf9f7b1bc5607..83da112d33a3661 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 5fd5e43be53f2b3..0bc0a437898c71f 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 985f98ad90b95b4..e3391756bc483ff 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,18 @@ 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.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 +111,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 +169,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 +188,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 +217,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 (stmt.getAnalyzeProperties().getProperties().containsKey("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; @@ -295,7 +311,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 +345,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 +361,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 +370,11 @@ 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)); + long rowCount = table.getRowCount(); + infoBuilder.setRowCount(rowCount); + TableStatsMeta tableStatsStatus = findTableStatsStatus(table.getId()); + infoBuilder.setUpdateRows(tableStatsStatus == null ? 0 : tableStatsStatus.updatedRows.get()); + infoBuilder.setPriority(JobPriority.MANUAL); return infoBuilder.build(); } @@ -372,7 +390,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 +523,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 +547,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(analysisJobInfoMap.values(), stmt); } @@ -555,7 +606,7 @@ protected List findShowAnalyzeResult(Collection anal 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 +725,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 +803,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); } } @@ -1088,4 +1140,56 @@ public boolean canSample(TableIf table) { } return false; } + + + 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; + } + } + } } 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 000000000000000..e349e4fcb3f2e8f --- /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 f871e8761a5e557..d19ab296b156b91 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 @@ -48,7 +48,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`, " 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 445641b25056106..7e317d67bd740fb 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 287941be526635f..16e741f4563f1d2 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()) { 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 000000000000000..b5b80741a2b8630 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/FollowerColumnSender.java @@ -0,0 +1,150 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.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; + for (int i = 0; i < columnQueue.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/JobPriority.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/JobPriority.java new file mode 100644 index 000000000000000..c3656b929279e6b --- /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 60bfcab61573776..08d770454bc1364 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 @@ -65,7 +65,7 @@ public OlapAnalysisTask(AnalysisInfo info) { } public void doExecute() throws Exception { - List> columnList = info.jobColumns; + Set> columnList = info.jobColumns; if (StatisticsUtil.isEmptyTable(tbl, info.analysisMethod) || columnList == null || columnList.isEmpty()) { StatsId statsId = new StatsId(concatColumnStatsId(), info.catalogId, info.dbId, info.tblId, info.indexId, info.colName, null); @@ -228,7 +228,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); + String collectColStats = stringSubstitutor.replace(FULL_ANALYZE_TEMPLATE); runQuery(collectColStats); } 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 000000000000000..df91ea7f4c05820 --- /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 74c7bd7c9db127a..314cf6648bde9b1 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 @@ -95,7 +95,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 9ca971845b7e645..57f3f494573cb5d 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,186 @@ 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.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(tableNameSetEntry -> Pair.of(tableNameSetEntry, 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; } - return analysisInfos; + AnalysisInfo analyzeJob = createAnalyzeJobForTbl(table, columns, priority); + LOG.debug("Auto analyze job : {}", analyzeJob.toString()); + executeSystemAnalysisJob(analyzeJob); } - // 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 = 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 { + 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); } + Env.getCurrentEnv().getAnalysisManager().constructJob(jobInfo, analysisTasks.values()); + Env.getCurrentEnv().getAnalysisManager().registerSysJob(jobInfo, analysisTasks); + analysisTasks.values().forEach(analysisTaskExecutor::submitTask); + } - if (needRunColumns == null || needRunColumns.isEmpty()) { - return null; - } - StringJoiner stringJoiner = new StringJoiner(",", "[", "]"); - for (Pair pair : needRunColumns) { - stringJoiner.add(pair.toString()); - } - return new AnalysisInfoBuilder(jobInfo) - .setColName(stringJoiner.toString()).setJobColumns(needRunColumns).build(); + protected AnalysisInfo getNeedAnalyzeColumns(AnalysisInfo jobInfo) { + return jobInfo; } } 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 ec187fe893af49a..000000000000000 --- 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 000000000000000..be5b933ce830526 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsJobAppender.java @@ -0,0 +1,201 @@ +// 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; + } + 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 5caa5bd9751c1e4..cd6be1c0af733a2 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; @@ -321,7 +321,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 3b9b1e2bead0051..900606276468c23 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 && analyzedJob.analysisMethod.equals(AnalysisMethod.SAMPLE)) { 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 f56aa0db607a3c1..bf458018fda1722 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 @@ -59,6 +59,7 @@ import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.hive.HMSExternalTable.DLAType; import org.apache.doris.datasource.hive.HiveMetaStoreCache; import org.apache.doris.datasource.hive.HivePartition; import org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral; @@ -70,11 +71,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; @@ -898,6 +902,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) @@ -1029,4 +1043,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 1bf2041bb4f12c1..cb2637d5cf685aa 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 @@ -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 674456b0b468913..529e008951192dc 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 @@ -23,18 +23,27 @@ import org.apache.doris.analysis.ShowAnalyzeStmt; 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 +55,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 +121,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 +198,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 +272,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 +347,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("col1"); + // manager.highPriorityJobs.get(high1).add("col2"); + // manager.highPriorityJobs.put(high2, new HashSet()); + // manager.highPriorityJobs.get(high2).add("col3"); + // manager.midPriorityJobs.put(mid1, new HashSet()); + // manager.midPriorityJobs.get(mid1).add("col4"); + // manager.midPriorityJobs.put(mid2, new HashSet()); + // manager.midPriorityJobs.get(mid2).add("col5"); + // manager.lowPriorityJobs.put(low1, new HashSet()); + // manager.lowPriorityJobs.get(low1).add("col6"); + // manager.lowPriorityJobs.get(low1).add("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.columnNames.size()); + // Assertions.assertTrue(job.columnNames.contains("col1")); + // Assertions.assertTrue(job.columnNames.contains("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.columnNames.size()); + // Assertions.assertTrue(job.columnNames.contains("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.columnNames.size()); + // Assertions.assertTrue(job.columnNames.contains("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.columnNames.size()); + // Assertions.assertTrue(job.columnNames.contains("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.columnNames.size()); + // Assertions.assertTrue(job.columnNames.contains("col6")); + // Assertions.assertTrue(job.columnNames.contains("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.columnNames.size()); + // Assertions.assertTrue(job.columnNames.contains("col1")); + // Assertions.assertTrue(job.columnNames.contains("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.columnNames.size()); + // Assertions.assertTrue(job.columnNames.contains("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.columnNames.size()); + // Assertions.assertTrue(job.columnNames.contains("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.columnNames.size()); + // Assertions.assertTrue(job.columnNames.contains("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.columnNames.size()); + // Assertions.assertTrue(job.columnNames.contains("col6")); + // Assertions.assertTrue(job.columnNames.contains("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 5698f0e9b20e634..29e04b1ef4fd942 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 bf6ce32e155f421..f70b2d416c7d32e 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) 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 000000000000000..7cbad7539942586 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/FollowerColumnSenderTest.java @@ -0,0 +1,65 @@ +// 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.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.Queue; +import java.util.Set; + +public class FollowerColumnSenderTest { + + @Test + public void testGetNeedAnalyzeColumns() { + new MockUp() { + boolean[] result = {false, true, false, true, true}; + int i = 0; + @Mock + public boolean needAnalyzeColumn(QueryColumn column) { + return result[i++]; + } + }; + 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 f7b75261cc54fa7..eef5832c81695b1 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 @@ -18,128 +18,125 @@ package org.apache.doris.statistics; 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.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.statistics.util.StatisticsUtil; 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.List; -import java.util.Map; 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 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("col1"); + // manager.highPriorityJobs.get(high1).add("col2"); + // manager.highPriorityJobs.put(high2, new HashSet()); + // manager.highPriorityJobs.get(high2).add("col3"); + // manager.midPriorityJobs.put(mid1, new HashSet()); + // manager.midPriorityJobs.get(mid1).add("col4"); + // manager.midPriorityJobs.put(mid2, new HashSet()); + // manager.midPriorityJobs.get(mid2).add("col5"); + // manager.lowPriorityJobs.put(low1, new HashSet()); + // manager.lowPriorityJobs.get(low1).add("col6"); + // manager.lowPriorityJobs.get(low1).add("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("col1")); + // Assertions.assertTrue(job.first.getValue().contains("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("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("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("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("col6")); + // Assertions.assertTrue(job.first.getValue().contains("col7")); + // Assertions.assertEquals(JobPriority.LOW, job.second); + // + // job = collector.getJob(); + // Assertions.assertNull(job); + // } @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; - } + 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 String getFullName() { - return "anyDb"; - } - }; + ExternalTable externalTable = new JdbcExternalTable(1, "jdbctable", "jdbcdb", null); + Assertions.assertFalse(collector.supportAutoAnalyze(externalTable)); - new MockUp() { + new MockUp() { @Mock - public String getName() { - return "anytable"; + public DLAType getDlaType() { + return DLAType.ICEBERG; } + }; + ExternalTable icebergExternalTable = new HMSExternalTable(1, "hmsTable", "hmsDb", null); + Assertions.assertFalse(collector.supportAutoAnalyze(icebergExternalTable)); + new MockUp() { @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; + public DLAType getDlaType() { + return DLAType.HIVE; } }; - StatisticsAutoCollector saa = new StatisticsAutoCollector(); - List analysisInfoList = saa.constructAnalysisInfo(new Database(1, "anydb")); - Assertions.assertEquals(1, analysisInfoList.size()); - Assertions.assertNull(analysisInfoList.get(0).colName); + ExternalTable hiveExternalTable = new HMSExternalTable(1, "hmsTable", "hmsDb", null); + Assertions.assertTrue(collector.supportAutoAnalyze(hiveExternalTable)); } @Test @@ -181,293 +178,4 @@ public int getAutoAnalyzeTableWidthThreshold() { 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)); - } - - // For small table, use full - @Test - public void testCreateAnalyzeJobForTbl1( - @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() - 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; - } - }; - - 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(); - } } 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 000000000000000..6bf2539e9a754f8 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsJobAppenderTest.java @@ -0,0 +1,206 @@ +// 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.common.DdlException; +// import org.apache.doris.common.Pair; +// import org.apache.doris.datasource.InternalCatalog; +// import org.apache.doris.statistics.util.StatisticsUtil; +// +// 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.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(QueryColumn column) { + // return true; + // } + // + // @Mock + // public TableIf findTable(long catalogId, long dbId, long tblId) { + // return tables[i++]; + // } + // }; + // + // new MockUp
() { + // @Mock + // public DatabaseIf getDatabase() { + // return db; + // } + // }; + // + // 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("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("high1")); + // Assertions.assertTrue(table1Column.contains("high3")); + // + // Set table2Column = testMap.get(new TableName("internal", "testDb", "testTable2")); + // Assertions.assertEquals(1, table2Column.size()); + // Assertions.assertTrue(table2Column.contains("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("high2")); + // Assertions.assertTrue(table2Column.contains("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; + // } + // }; + // + // 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()); + // } +} 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 94eab9e00cc501a..b03ca72b6d0ae9f 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 @@ -25,7 +25,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.util.ArrayList; +import java.util.HashSet; class TableStatsMetaTest { @@ -38,7 +38,7 @@ public long getRowCount() { } }; TableStatsMeta tableStatsMeta = new TableStatsMeta(); - AnalysisInfo jobInfo = new AnalysisInfoBuilder().setJobColumns(new ArrayList<>()) + AnalysisInfo jobInfo = new AnalysisInfoBuilder().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 724e0363833305f..3be11b4d931f9ae 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 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 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 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 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 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 ef90e97b41b5ec1..1df088f9c77f926 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1441,6 +1441,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) @@ -1530,4 +1542,5 @@ service FrontendService { TShowProcessListResult showProcessList(1: TShowProcessListRequest request) Status.TStatus reportCommitTxnResult(1: TReportCommitTxnResultRequest request) + Status.TStatus syncQueryColumns(1: TSyncQueryColumns request) }