This is an automated email from the ASF dual-hosted git repository.
lijibing pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-3.0 by this push:
new e4bad85a604 [feature](statistics)Support enable/disable auto analyze
for table. (#39532) (#41836)
e4bad85a604 is described below
commit e4bad85a604be34fb940b9b8ec13d469cc991f68
Author: Jibing-Li <[email protected]>
AuthorDate: Tue Oct 15 16:23:40 2024 +0800
[feature](statistics)Support enable/disable auto analyze for table.
(#39532) (#41836)
backport: https://github.com/apache/doris/pull/39532
---
.../main/java/org/apache/doris/alter/Alter.java | 52 ++++++++++++---
.../apache/doris/alter/SchemaChangeHandler.java | 6 +-
.../org/apache/doris/analysis/AlterTableStmt.java | 3 -
.../analysis/ModifyTablePropertiesClause.java | 18 ++++++
.../apache/doris/analysis/ShowTableStatsStmt.java | 13 ++--
.../main/java/org/apache/doris/catalog/Env.java | 38 ++++++++++-
.../java/org/apache/doris/catalog/OlapTable.java | 15 +++++
.../main/java/org/apache/doris/catalog/Table.java | 5 ++
.../java/org/apache/doris/catalog/TableIf.java | 2 +
.../org/apache/doris/catalog/TableProperty.java | 10 +++
.../cloud/alter/CloudSchemaChangeHandler.java | 5 +-
.../apache/doris/common/util/PropertyAnalyzer.java | 5 ++
.../apache/doris/datasource/ExternalCatalog.java | 16 +++++
.../org/apache/doris/datasource/ExternalTable.java | 11 ++++
.../persist/ModifyTablePropertyOperationLog.java | 24 +++++++
.../java/org/apache/doris/qe/ShowExecutor.java | 6 +-
.../doris/statistics/util/StatisticsUtil.java | 7 +-
.../doris/statistics/util/StatisticsUtilTest.java | 62 ++++++++++--------
.../hive/test_hive_statistics_p0.groovy | 65 +++++++++++++++++++
.../test_auto_analyze_black_white_list.groovy | 75 ++++++++++++++++++++++
20 files changed, 384 insertions(+), 54 deletions(-)
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
index 1e1ebb40d9e..e6a2fe0229f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
@@ -44,6 +44,7 @@ import org.apache.doris.analysis.TableRenameClause;
import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.DataProperty;
import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.DatabaseIf;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.MTMV;
import org.apache.doris.catalog.MysqlTable;
@@ -53,6 +54,7 @@ import org.apache.doris.catalog.Partition;
import org.apache.doris.catalog.PartitionInfo;
import org.apache.doris.catalog.ReplicaAllocation;
import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.TableIf;
import org.apache.doris.catalog.TableIf.TableType;
import org.apache.doris.catalog.View;
import org.apache.doris.cloud.alter.CloudSchemaChangeHandler;
@@ -65,6 +67,7 @@ import org.apache.doris.common.util.DynamicPartitionUtil;
import org.apache.doris.common.util.MetaLockUtils;
import org.apache.doris.common.util.PropertyAnalyzer;
import org.apache.doris.common.util.PropertyAnalyzer.RewriteProperty;
+import org.apache.doris.datasource.ExternalTable;
import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo;
import org.apache.doris.persist.AlterMTMV;
import org.apache.doris.persist.AlterViewInfo;
@@ -72,6 +75,7 @@ import org.apache.doris.persist.BatchModifyPartitionsInfo;
import org.apache.doris.persist.ModifyCommentOperationLog;
import org.apache.doris.persist.ModifyPartitionInfo;
import org.apache.doris.persist.ModifyTableEngineOperationLog;
+import org.apache.doris.persist.ModifyTablePropertyOperationLog;
import org.apache.doris.persist.ReplaceTableOperationLog;
import org.apache.doris.policy.StoragePolicy;
import org.apache.doris.qe.ConnectContext;
@@ -303,6 +307,24 @@ public class Alter {
return needProcessOutsideTableLock;
}
+ private void setExternalTableAutoAnalyzePolicy(ExternalTable table,
List<AlterClause> alterClauses) {
+ Preconditions.checkState(alterClauses.size() == 1);
+ AlterClause alterClause = alterClauses.get(0);
+ Preconditions.checkState(alterClause instanceof
ModifyTablePropertiesClause);
+ Map<String, String> properties = alterClause.getProperties();
+ Preconditions.checkState(properties.size() == 1);
+
Preconditions.checkState(properties.containsKey(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY));
+ String value =
properties.get(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY);
+
Preconditions.checkState(PropertyAnalyzer.ENABLE_AUTO_ANALYZE_POLICY.equalsIgnoreCase(value)
+ ||
PropertyAnalyzer.DISABLE_AUTO_ANALYZE_POLICY.equalsIgnoreCase(value)
+ ||
PropertyAnalyzer.USE_CATALOG_AUTO_ANALYZE_POLICY.equalsIgnoreCase(value));
+ value =
value.equalsIgnoreCase(PropertyAnalyzer.USE_CATALOG_AUTO_ANALYZE_POLICY) ? null
: value;
+
table.getCatalog().setAutoAnalyzePolicy(table.getDatabase().getFullName(),
table.getName(), value);
+ ModifyTablePropertyOperationLog info = new
ModifyTablePropertyOperationLog(table.getCatalog().getName(),
+ table.getDatabase().getFullName(), table.getName(),
properties);
+ Env.getCurrentEnv().getEditLog().logModifyTableProperties(info);
+ }
+
private boolean needChangeMTMVState(List<AlterClause> alterClauses) {
for (AlterClause alterClause : alterClauses) {
if (alterClause.needChangeMTMVState()) {
@@ -467,31 +489,45 @@ public class Alter {
public void processAlterTable(AlterTableStmt stmt) throws UserException {
TableName dbTableName = stmt.getTbl();
+ String ctlName = dbTableName.getCtl();
String dbName = dbTableName.getDb();
String tableName = dbTableName.getTbl();
- Database db =
Env.getCurrentInternalCatalog().getDbOrDdlException(dbName);
- Table table = db.getTableOrDdlException(tableName);
+ DatabaseIf dbIf = Env.getCurrentEnv().getCatalogMgr()
+ .getCatalogOrException(ctlName, catalog -> new
DdlException("Unknown catalog " + catalog))
+ .getDbOrDdlException(dbName);
+ TableIf tableIf = dbIf.getTableOrDdlException(tableName);
List<AlterClause> alterClauses = Lists.newArrayList();
// some operations will take long time to process, need to be done
outside the table lock
boolean needProcessOutsideTableLock = false;
- switch (table.getType()) {
+ switch (tableIf.getType()) {
case MATERIALIZED_VIEW:
case OLAP:
- OlapTable olapTable = (OlapTable) table;
- needProcessOutsideTableLock = processAlterOlapTable(stmt,
olapTable, alterClauses, db);
+ OlapTable olapTable = (OlapTable) tableIf;
+ needProcessOutsideTableLock = processAlterOlapTable(stmt,
olapTable, alterClauses, (Database) dbIf);
break;
case ODBC:
case JDBC:
case HIVE:
case MYSQL:
case ELASTICSEARCH:
- processAlterExternalTable(stmt, table, db);
+ processAlterExternalTable(stmt, (Table) tableIf, (Database)
dbIf);
+ return;
+ case HMS_EXTERNAL_TABLE:
+ case JDBC_EXTERNAL_TABLE:
+ case ICEBERG_EXTERNAL_TABLE:
+ case PAIMON_EXTERNAL_TABLE:
+ case MAX_COMPUTE_EXTERNAL_TABLE:
+ case HUDI_EXTERNAL_TABLE:
+ case TRINO_CONNECTOR_EXTERNAL_TABLE:
+ alterClauses.addAll(stmt.getOps());
+ setExternalTableAutoAnalyzePolicy((ExternalTable) tableIf,
alterClauses);
return;
default:
throw new DdlException("Do not support alter "
- + table.getType().toString() + " table[" + tableName +
"]");
+ + tableIf.getType().toString() + " table[" + tableName
+ "]");
}
+ Database db = (Database) dbIf;
// the following ops should done outside table lock. because it
contain synchronized create operation
if (needProcessOutsideTableLock) {
Preconditions.checkState(alterClauses.size() == 1);
@@ -514,7 +550,7 @@ public class Alter {
List<String> partitionNames = clause.getPartitionNames();
((SchemaChangeHandler)
schemaChangeHandler).updatePartitionsProperties(
db, tableName, partitionNames, properties);
- OlapTable olapTable = (OlapTable) table;
+ OlapTable olapTable = (OlapTable) tableIf;
olapTable.writeLockOrDdlException();
try {
modifyPartitionsProperty(db, olapTable, partitionNames,
properties, clause.isTempPartition());
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
index 99258bd205e..30fa6516b43 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
@@ -2280,7 +2280,8 @@ public class SchemaChangeHandler extends AlterHandler {
||
properties.containsKey(PropertyAnalyzer.PROPERTIES_DISABLE_AUTO_COMPACTION)
||
properties.containsKey(PropertyAnalyzer.PROPERTIES_SKIP_WRITE_INDEX_ON_LOAD)
||
properties.containsKey(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_EMPTY_ROWSETS_THRESHOLD)
- ||
properties.containsKey(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_LEVEL_THRESHOLD));
+ ||
properties.containsKey(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_LEVEL_THRESHOLD)
+ ||
properties.containsKey(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY));
Env.getCurrentEnv().getAlterInstance().checkNoForceProperty(properties);
List<Partition> partitions = Lists.newArrayList();
@@ -2356,7 +2357,8 @@ public class SchemaChangeHandler extends AlterHandler {
&&
!properties.containsKey(PropertyAnalyzer.PROPERTIES_DISABLE_AUTO_COMPACTION)
&&
!properties.containsKey(PropertyAnalyzer.PROPERTIES_GROUP_COMMIT_INTERVAL_MS)
&&
!properties.containsKey(PropertyAnalyzer.PROPERTIES_GROUP_COMMIT_DATA_BYTES)
- &&
!properties.containsKey(PropertyAnalyzer.PROPERTIES_SKIP_WRITE_INDEX_ON_LOAD)) {
+ &&
!properties.containsKey(PropertyAnalyzer.PROPERTIES_SKIP_WRITE_INDEX_ON_LOAD)
+ &&
!properties.containsKey(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY)) {
LOG.info("Properties already up-to-date");
return;
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java
b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java
index a9e9c78d418..db51dfffd9f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java
@@ -29,7 +29,6 @@ import org.apache.doris.common.ErrorReport;
import org.apache.doris.common.UserException;
import org.apache.doris.common.util.InternalDatabaseUtil;
import org.apache.doris.common.util.PropertyAnalyzer;
-import org.apache.doris.common.util.Util;
import org.apache.doris.mysql.privilege.PrivPredicate;
import org.apache.doris.qe.ConnectContext;
@@ -66,8 +65,6 @@ public class AlterTableStmt extends DdlStmt implements
NotFallbackInParser {
ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_TABLES_USED);
}
tbl.analyze(analyzer);
- // disallow external catalog
- Util.prohibitExternalCatalog(tbl.getCtl(),
this.getClass().getSimpleName());
InternalDatabaseUtil.checkDatabase(tbl.getDb(), ConnectContext.get());
if (!Env.getCurrentEnv().getAccessManager()
.checkTblPriv(ConnectContext.get(), tbl.getCtl(), tbl.getDb(),
tbl.getTbl(),
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java
b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java
index c7ea5c2ecca..2d3bc4ae05f 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java
@@ -29,6 +29,7 @@ import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.util.DynamicPartitionUtil;
import org.apache.doris.common.util.PrintableMap;
import org.apache.doris.common.util.PropertyAnalyzer;
+import org.apache.doris.datasource.InternalCatalog;
import com.google.common.base.Strings;
@@ -350,6 +351,19 @@ public class ModifyTablePropertiesClause extends
AlterTableClause {
// do nothing, will be analyzed when creating alter job
} else if
(properties.containsKey(PropertyAnalyzer.PROPERTIES_ROW_STORE_COLUMNS)) {
// do nothing, will be analyzed when creating alter job
+ } else if
(properties.containsKey(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY)) {
+ String analyzePolicy =
properties.getOrDefault(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY, "");
+ if (analyzePolicy != null
+ &&
!analyzePolicy.equals(PropertyAnalyzer.ENABLE_AUTO_ANALYZE_POLICY)
+ &&
!analyzePolicy.equals(PropertyAnalyzer.DISABLE_AUTO_ANALYZE_POLICY)
+ &&
!analyzePolicy.equals(PropertyAnalyzer.USE_CATALOG_AUTO_ANALYZE_POLICY)) {
+ throw new AnalysisException(
+ "Table auto analyze policy only support for " +
PropertyAnalyzer.ENABLE_AUTO_ANALYZE_POLICY
+ + " or " + PropertyAnalyzer.DISABLE_AUTO_ANALYZE_POLICY
+ + " or " +
PropertyAnalyzer.USE_CATALOG_AUTO_ANALYZE_POLICY);
+ }
+ this.needTableStable = false;
+ this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC;
} else {
throw new AnalysisException("Unknown table property: " +
properties.keySet());
}
@@ -358,6 +372,10 @@ public class ModifyTablePropertiesClause extends
AlterTableClause {
private void analyzeForMTMV() throws AnalysisException {
if (tableName != null) {
+ // Skip external catalog.
+ if
(!(InternalCatalog.INTERNAL_CATALOG_NAME.equals(tableName.getCtl()))) {
+ return;
+ }
Table table =
Env.getCurrentInternalCatalog().getDbOrAnalysisException(tableName.getDb())
.getTableOrAnalysisException(tableName.getTbl());
if (!(table instanceof MTMV)) {
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java
b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java
index 2e3f9e07cc0..54c8f652e12 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java
@@ -61,6 +61,7 @@ public class ShowTableStatsStmt extends ShowStmt implements
NotFallbackInParser
.add("trigger")
.add("new_partition")
.add("user_inject")
+ .add("enable_auto_analyze")
.build();
private static final ImmutableList<String> PARTITION_TITLE_NAMES =
@@ -197,12 +198,12 @@ public class ShowTableStatsStmt extends ShowStmt
implements NotFallbackInParser
return tableId;
}
- public ShowResultSet constructResultSet(TableStatsMeta tableStatistic) {
+ public ShowResultSet constructResultSet(TableStatsMeta tableStatistic,
TableIf table) {
if (indexName != null) {
return constructIndexResultSet(tableStatistic);
}
if (partitionNames == null) {
- return constructTableResultSet(tableStatistic);
+ return constructTableResultSet(tableStatistic, table);
}
if (columnNames == null) {
return constructPartitionResultSet(tableStatistic);
@@ -215,7 +216,7 @@ public class ShowTableStatsStmt extends ShowStmt implements
NotFallbackInParser
return new ShowResultSet(getMetaData(), new ArrayList<>());
}
- public ShowResultSet constructResultSet(long rowCount) {
+ public ShowResultSet constructResultSet(TableIf table) {
List<List<String>> result = Lists.newArrayList();
if (partitionNames != null) {
// For partition, return empty result if table stats not exist.
@@ -224,17 +225,18 @@ public class ShowTableStatsStmt extends ShowStmt
implements NotFallbackInParser
List<String> row = Lists.newArrayList();
row.add("");
row.add("");
- row.add(String.valueOf(rowCount));
+ row.add(String.valueOf(table.getCachedRowCount()));
row.add("");
row.add("");
row.add("");
row.add("");
row.add("");
+ row.add(String.valueOf(table.autoAnalyzeEnabled()));
result.add(row);
return new ShowResultSet(getMetaData(), result);
}
- public ShowResultSet constructTableResultSet(TableStatsMeta
tableStatistic) {
+ public ShowResultSet constructTableResultSet(TableStatsMeta
tableStatistic, TableIf table) {
DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd
HH:mm:ss");
if (tableStatistic == null) {
return new ShowResultSet(getMetaData(), new ArrayList<>());
@@ -253,6 +255,7 @@ public class ShowTableStatsStmt extends ShowStmt implements
NotFallbackInParser
row.add(tableStatistic.jobType.toString());
row.add(String.valueOf(tableStatistic.partitionChanged.get()));
row.add(String.valueOf(tableStatistic.userInjected));
+ row.add(table == null ? "N/A" :
String.valueOf(table.autoAnalyzeEnabled()));
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 b08ef17aac5..33e1f84980b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
@@ -132,6 +132,7 @@ import org.apache.doris.consistency.ConsistencyChecker;
import org.apache.doris.cooldown.CooldownConfHandler;
import org.apache.doris.datasource.CatalogIf;
import org.apache.doris.datasource.CatalogMgr;
+import org.apache.doris.datasource.ExternalCatalog;
import org.apache.doris.datasource.ExternalMetaCacheMgr;
import org.apache.doris.datasource.ExternalMetaIdMgr;
import org.apache.doris.datasource.InternalCatalog;
@@ -5335,7 +5336,8 @@ public class Env {
.buildEnableSingleReplicaCompaction()
.buildTimeSeriesCompactionEmptyRowsetsThreshold()
.buildTimeSeriesCompactionLevelThreshold()
- .buildTTLSeconds();
+ .buildTTLSeconds()
+ .buildAutoAnalyzeProperty();
// need to update partition info meta
for (Partition partition : table.getPartitions()) {
@@ -5362,10 +5364,17 @@ public class Env {
public void replayModifyTableProperty(short opCode,
ModifyTablePropertyOperationLog info)
throws MetaNotFoundException {
+ String ctlName = info.getCtlName();
long dbId = info.getDbId();
long tableId = info.getTableId();
Map<String, String> properties = info.getProperties();
+ // Handle HMSExternalTable set auto analyze policy.
+ if (ctlName != null &&
!(InternalCatalog.INTERNAL_CATALOG_NAME.equalsIgnoreCase(ctlName))) {
+ setExternalTableAutoAnalyze(properties, info);
+ return;
+ }
+
Database db = getInternalCatalog().getDbOrMetaException(dbId);
OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId,
TableType.OLAP);
olapTable.writeLock();
@@ -5405,6 +5414,33 @@ public class Env {
}
}
+ private void setExternalTableAutoAnalyze(Map<String, String> properties,
ModifyTablePropertyOperationLog info) {
+ if (properties.size() != 1) {
+ LOG.warn("External table property should contain exactly 1
entry.");
+ return;
+ }
+ if
(!properties.containsKey(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY)) {
+ LOG.warn("External table property should only contain
auto_analyze_policy");
+ return;
+ }
+ String value =
properties.get(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY);
+ if
(!PropertyAnalyzer.ENABLE_AUTO_ANALYZE_POLICY.equalsIgnoreCase(value)
+ &&
!PropertyAnalyzer.DISABLE_AUTO_ANALYZE_POLICY.equalsIgnoreCase(value)
+ &&
!PropertyAnalyzer.USE_CATALOG_AUTO_ANALYZE_POLICY.equalsIgnoreCase(value)) {
+ LOG.warn("External table property should be 'enable', 'disable' or
'base_on_catalog'");
+ return;
+ }
+ try {
+ CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr()
+ .getCatalogOrException(info.getCtlName(),
+ ctlName -> new DdlException("Unknown catalog " +
ctlName));
+ value =
value.equalsIgnoreCase(PropertyAnalyzer.USE_CATALOG_AUTO_ANALYZE_POLICY) ? null
: value;
+ ((ExternalCatalog) catalog).setAutoAnalyzePolicy(info.getDbName(),
info.getTableName(), value);
+ } catch (Exception e) {
+ LOG.warn("Failed to replay external table set property.", e);
+ }
+ }
+
public void modifyDefaultDistributionBucketNum(Database db, OlapTable
olapTable,
ModifyDistributionClause
modifyDistributionClause)
throws DdlException {
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 090c58fcbed..4dfdff2468d 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
@@ -3216,4 +3216,19 @@ public class OlapTable extends Table implements
MTMVRelatedTableIf, GsonPostProc
return false;
}
}
+
+ @Override
+ public boolean autoAnalyzeEnabled() {
+ if (tableProperty == null) {
+ return super.autoAnalyzeEnabled();
+ }
+ Map<String, String> properties = tableProperty.getProperties();
+ if (properties == null ||
!properties.containsKey(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY)
+ ||
properties.get(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY)
+
.equalsIgnoreCase(PropertyAnalyzer.USE_CATALOG_AUTO_ANALYZE_POLICY)) {
+ return super.autoAnalyzeEnabled();
+ }
+ return properties.get(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY)
+ .equalsIgnoreCase(PropertyAnalyzer.ENABLE_AUTO_ANALYZE_POLICY);
+ }
}
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 906a710f369..e9392deae4a 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
@@ -634,4 +634,9 @@ public abstract class Table extends MetaObject implements
Writable, TableIf, Gso
public long getCachedRowCount() {
return getRowCount();
}
+
+ @Override
+ public boolean autoAnalyzeEnabled() {
+ return true;
+ }
}
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 1ddd083cff4..ed40840239a 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
@@ -559,4 +559,6 @@ public interface TableIf {
default boolean isPartitionedTable() {
return false;
}
+
+ boolean autoAnalyzeEnabled();
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
index 4b7d5e6a88b..1c1d7e35880 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
@@ -124,6 +124,9 @@ public class TableProperty implements Writable,
GsonPostProcessable {
private long timeSeriesCompactionLevelThreshold
=
PropertyAnalyzer.TIME_SERIES_COMPACTION_LEVEL_THRESHOLD_DEFAULT_VALUE;
+ private String autoAnalyzePolicy =
PropertyAnalyzer.ENABLE_AUTO_ANALYZE_POLICY;
+
+
private DataSortInfo dataSortInfo = new DataSortInfo();
public TableProperty(Map<String, String> properties) {
@@ -163,6 +166,7 @@ public class TableProperty implements Writable,
GsonPostProcessable {
buildTimeSeriesCompactionEmptyRowsetsThreshold();
buildTimeSeriesCompactionLevelThreshold();
buildTTLSeconds();
+ buildAutoAnalyzeProperty();
break;
default:
break;
@@ -256,6 +260,12 @@ public class TableProperty implements Writable,
GsonPostProcessable {
return this;
}
+ public TableProperty buildAutoAnalyzeProperty() {
+ autoAnalyzePolicy =
properties.getOrDefault(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY,
+ PropertyAnalyzer.ENABLE_AUTO_ANALYZE_POLICY);
+ return this;
+ }
+
public boolean disableAutoCompaction() {
return disableAutoCompaction;
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/cloud/alter/CloudSchemaChangeHandler.java
b/fe/fe-core/src/main/java/org/apache/doris/cloud/alter/CloudSchemaChangeHandler.java
index e48b8a745a3..fa3bb96014c 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/cloud/alter/CloudSchemaChangeHandler.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/cloud/alter/CloudSchemaChangeHandler.java
@@ -100,7 +100,8 @@ public class CloudSchemaChangeHandler extends
SchemaChangeHandler {
||
properties.containsKey(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_EMPTY_ROWSETS_THRESHOLD)
||
properties.containsKey(PropertyAnalyzer.PROPERTIES_TIME_SERIES_COMPACTION_LEVEL_THRESHOLD)
||
properties.containsKey(PropertyAnalyzer.PROPERTIES_DISABLE_AUTO_COMPACTION)
- ||
properties.containsKey(PropertyAnalyzer.PROPERTIES_ENABLE_MOW_LIGHT_DELETE));
+ ||
properties.containsKey(PropertyAnalyzer.PROPERTIES_ENABLE_MOW_LIGHT_DELETE)
+ ||
properties.containsKey(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY));
if (properties.size() != 1) {
throw new UserException("Can only set one table property at a
time");
@@ -316,6 +317,8 @@ public class CloudSchemaChangeHandler extends
SchemaChangeHandler {
}
param.enableMowLightDelete = enableMowLightDelete;
param.type =
UpdatePartitionMetaParam.TabletMetaType.ENABLE_MOW_LIGHT_DELETE;
+ } else if
(properties.containsKey(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY)) {
+ // Do nothing.
} else {
LOG.warn("invalid properties:{}", properties);
throw new UserException("invalid properties");
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
index 08e6f906222..43eda6cc189 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
@@ -223,6 +223,11 @@ public class PropertyAnalyzer {
public static final boolean
PROPERTIES_ENABLE_MOW_LIGHT_DELETE_DEFAULT_VALUE
= Config.enable_mow_light_delete;
+ public static final String PROPERTIES_AUTO_ANALYZE_POLICY =
"auto_analyze_policy";
+ public static final String ENABLE_AUTO_ANALYZE_POLICY = "enable";
+ public static final String DISABLE_AUTO_ANALYZE_POLICY = "disable";
+ public static final String USE_CATALOG_AUTO_ANALYZE_POLICY =
"base_on_catalog";
+
// compaction policy
public static final String SIZE_BASED_COMPACTION_POLICY = "size_based";
public static final String TIME_SERIES_COMPACTION_POLICY = "time_series";
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
index 0d854a9ecdc..37e44c63569 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
@@ -33,6 +33,7 @@ import org.apache.doris.catalog.TableIf;
import org.apache.doris.cluster.ClusterNamespace;
import org.apache.doris.common.Config;
import org.apache.doris.common.DdlException;
+import org.apache.doris.common.Pair;
import org.apache.doris.common.UserException;
import org.apache.doris.common.Version;
import org.apache.doris.common.io.Text;
@@ -127,6 +128,9 @@ public abstract class ExternalCatalog
protected Map<Long, ExternalDatabase<? extends ExternalTable>> idToDb =
Maps.newConcurrentMap();
@SerializedName(value = "lastUpdateTime")
protected long lastUpdateTime;
+ // <db name, table name> to tableAutoAnalyzePolicy
+ @SerializedName(value = "taap")
+ protected Map<Pair<String, String>, String> tableAutoAnalyzePolicy =
Maps.newHashMap();
// db name does not contains "default_cluster"
protected Map<String, Long> dbNameToId = Maps.newConcurrentMap();
private boolean objectCreated = false;
@@ -723,6 +727,9 @@ public abstract class ExternalCatalog
this.propLock = new byte[0];
this.initialized = false;
setDefaultPropsIfMissing(true);
+ if (tableAutoAnalyzePolicy == null) {
+ tableAutoAnalyzePolicy = Maps.newHashMap();
+ }
}
public void addDatabaseForTest(ExternalDatabase<? extends ExternalTable>
db) {
@@ -884,4 +891,13 @@ public abstract class ExternalCatalog
public String getQualifiedName(String dbName) {
return String.join(".", name, dbName);
}
+
+ public void setAutoAnalyzePolicy(String dbName, String tableName, String
policy) {
+ Pair<String, String> key = Pair.of(dbName, tableName);
+ if (policy == null) {
+ tableAutoAnalyzePolicy.remove(key);
+ } else {
+ tableAutoAnalyzePolicy.put(key, policy);
+ }
+ }
}
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 2d5689a5468..eedbe4e20da 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
@@ -27,6 +27,7 @@ import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.Pair;
import org.apache.doris.common.io.Text;
import org.apache.doris.common.io.Writable;
+import org.apache.doris.common.util.PropertyAnalyzer;
import org.apache.doris.common.util.Util;
import org.apache.doris.persist.gson.GsonPostProcessable;
import org.apache.doris.persist.gson.GsonUtils;
@@ -282,6 +283,16 @@ public class ExternalTable implements TableIf, Writable,
GsonPostProcessable {
return getFullSchema();
}
+ @Override
+ public boolean autoAnalyzeEnabled() {
+ makeSureInitialized();
+ String policy =
catalog.getTableAutoAnalyzePolicy().get(Pair.of(dbName, name));
+ if (policy == null) {
+ return catalog.enableAutoAnalyze();
+ }
+ return
policy.equalsIgnoreCase(PropertyAnalyzer.ENABLE_AUTO_ANALYZE_POLICY);
+ }
+
@Override
public Optional<ColumnStatistic> getColumnStatistic(String colName) {
return Optional.empty();
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyTablePropertyOperationLog.java
b/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyTablePropertyOperationLog.java
index a782db9f9c0..50b49074ce1 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyTablePropertyOperationLog.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyTablePropertyOperationLog.java
@@ -35,6 +35,10 @@ public class ModifyTablePropertyOperationLog implements
Writable {
private long dbId;
@SerializedName(value = "tableId")
private long tableId;
+ @SerializedName(value = "ctlName")
+ private String ctlName;
+ @SerializedName(value = "dbName")
+ private String dbName;
@SerializedName(value = "tableName")
private String tableName;
@SerializedName(value = "properties")
@@ -58,6 +62,14 @@ public class ModifyTablePropertyOperationLog implements
Writable {
this.sql = sb.toString();
}
+ public ModifyTablePropertyOperationLog(String ctlName, String dbName,
String tableName,
+ Map<String, String> properties) {
+ this.ctlName = ctlName;
+ this.dbName = dbName;
+ this.tableName = tableName;
+ this.properties = properties;
+ }
+
public long getDbId() {
return dbId;
}
@@ -66,6 +78,18 @@ public class ModifyTablePropertyOperationLog implements
Writable {
return tableId;
}
+ public String getCtlName() {
+ return ctlName;
+ }
+
+ public String getDbName() {
+ return dbName;
+ }
+
+ public String getTableName() {
+ return tableName;
+ }
+
public Map<String, String> getProperties() {
return properties;
}
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 cc0577b54d3..65a42730ebe 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
@@ -2733,7 +2733,7 @@ public class ShowExecutor {
if (tableStats == null) {
resultSet = showTableStatsStmt.constructEmptyResultSet();
} else {
- resultSet = showTableStatsStmt.constructResultSet(tableStats);
+ resultSet = showTableStatsStmt.constructResultSet(tableStats,
tableIf);
}
return;
}
@@ -2742,9 +2742,9 @@ public class ShowExecutor {
tableStats == null means it's not analyzed, in this case show the
estimated row count.
*/
if (tableStats == null) {
- resultSet =
showTableStatsStmt.constructResultSet(tableIf.getCachedRowCount());
+ resultSet = showTableStatsStmt.constructResultSet(tableIf);
} else {
- resultSet = showTableStatsStmt.constructResultSet(tableStats);
+ resultSet = showTableStatsStmt.constructResultSet(tableStats,
tableIf);
}
}
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 ee79ec9e791..b4ba12a3f0d 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
@@ -998,12 +998,7 @@ public class StatisticsUtil {
if (column == null) {
return false;
}
- try {
- if (!table.getDatabase().getCatalog().enableAutoAnalyze()) {
- return false;
- }
- } catch (Throwable t) {
- LOG.warn("Failed to get catalog property. {}", t.getMessage());
+ if (!table.autoAnalyzeEnabled()) {
return false;
}
AnalysisManager manager = Env.getServingEnv().getAnalysisManager();
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 6dccd3af46e..32521882939 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
@@ -18,21 +18,21 @@
package org.apache.doris.statistics.util;
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.TableProperty;
import org.apache.doris.catalog.Type;
import org.apache.doris.common.AnalysisException;
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.common.util.PropertyAnalyzer;
import org.apache.doris.datasource.ExternalCatalog;
import org.apache.doris.datasource.ExternalTable;
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.jdbc.JdbcExternalCatalog;
import org.apache.doris.datasource.jdbc.JdbcExternalTable;
import org.apache.doris.qe.SessionVariable;
import org.apache.doris.statistics.AnalysisManager;
@@ -51,7 +51,9 @@ import java.time.LocalTime;
import java.time.format.DateTimeFormatter;
import java.util.ArrayList;
import java.util.Base64;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
class StatisticsUtilTest {
@Test
@@ -176,42 +178,46 @@ class StatisticsUtilTest {
List<Column> schema = new ArrayList<>();
schema.add(column);
OlapTable table = new OlapTable(200, "testTable", schema, null, null,
null);
- Database db = new Database(111, "TestDb");
- ExternalCatalog externalCatalog = new HMSExternalCatalog();
+ HMSExternalCatalog externalCatalog = new HMSExternalCatalog();
- // Test get database/catalog exception
- new MockUp<OlapTable>() {
- @Mock
- public DatabaseIf getDatabase() {
- throw new RuntimeException();
- }
- };
+ // Test olap table auto analyze disabled.
+ Map<String, String> properties = new HashMap<>();
+ properties.put(PropertyAnalyzer.PROPERTIES_AUTO_ANALYZE_POLICY,
"disable");
+ table.setTableProperty(new TableProperty(properties));
Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(table,
Pair.of("index", column.getName())));
+ table.setTableProperty(null);
- // Test auto analyze disabled.
- new MockUp<OlapTable>() {
+ new MockUp<HMSExternalTable>() {
@Mock
- public DatabaseIf getDatabase() {
- return db;
+ protected synchronized void makeSureInitialized() {
}
};
- new MockUp<Database>() {
+
+ // Test auto analyze catalog disabled.
+ HMSExternalTable hmsTable = new HMSExternalTable(1, "name", "dbName",
externalCatalog);
+ Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(hmsTable,
Pair.of("index", column.getName())));
+
+ // Test catalog auto analyze enabled.
+ new MockUp<AnalysisManager>() {
@Mock
- public CatalogIf getCatalog() {
- return externalCatalog;
+ public TableStatsMeta findTableStatsStatus(long tblId) {
+ return null;
}
};
- Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(table,
Pair.of("index", column.getName())));
+
externalCatalog.getCatalogProperty().addProperty(ExternalCatalog.ENABLE_AUTO_ANALYZE,
"true");
+ Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table,
Pair.of("index", column.getName())));
- // Test auto analyze enabled.
+ // Test external table auto analyze enabled.
new MockUp<AnalysisManager>() {
@Mock
public TableStatsMeta findTableStatsStatus(long tblId) {
return null;
}
};
-
externalCatalog.getCatalogProperty().addProperty(ExternalCatalog.ENABLE_AUTO_ANALYZE,
"true");
- Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table,
Pair.of("index", column.getName())));
+
externalCatalog.getCatalogProperty().addProperty(ExternalCatalog.ENABLE_AUTO_ANALYZE,
"false");
+ HMSExternalTable hmsTable1 = new HMSExternalTable(1, "name", "dbName",
externalCatalog);
+ externalCatalog.setAutoAnalyzePolicy("dbName", "name", "enable");
+ Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(hmsTable1,
Pair.of("index", column.getName())));
// Test table stats meta is null.
@@ -256,8 +262,14 @@ class StatisticsUtilTest {
}
};
+ new MockUp<JdbcExternalTable>() {
+ @Mock
+ protected synchronized void makeSureInitialized() {
+ }
+ };
// Test not supported external table type.
- ExternalTable externalTable = new JdbcExternalTable(1, "jdbctable",
"jdbcdb", null);
+ ExternalTable externalTable = new JdbcExternalTable(1, "jdbctable",
"jdbcdb",
+ new JdbcExternalCatalog(1, "name", "resource", new
HashMap<>(), ""));
Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(externalTable,
Pair.of("index", column.getName())));
// Test hms external table not hive type.
@@ -267,7 +279,7 @@ class StatisticsUtilTest {
return DLAType.ICEBERG;
}
};
- ExternalTable hmsExternalTable = new HMSExternalTable(1, "hmsTable",
"hmsDb", null);
+ ExternalTable hmsExternalTable = new HMSExternalTable(1, "hmsTable",
"hmsDb", externalCatalog);
Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(hmsExternalTable,
Pair.of("index", column.getName())));
// Test partition first load.
diff --git
a/regression-test/suites/external_table_p0/hive/test_hive_statistics_p0.groovy
b/regression-test/suites/external_table_p0/hive/test_hive_statistics_p0.groovy
index 71a43d2aec9..c11ff176d95 100644
---
a/regression-test/suites/external_table_p0/hive/test_hive_statistics_p0.groovy
+++
b/regression-test/suites/external_table_p0/hive/test_hive_statistics_p0.groovy
@@ -232,6 +232,71 @@ suite("test_hive_statistics_p0",
"all_types,p0,external,hive,external_docker,ext
assertEquals("N/A", result[0][7])
assertEquals("N/A", result[0][8])
+
+ // Test auto analyze policy
+ sql """drop stats stats_test1"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("false", result[0][8])
+
+ sql """ALTER CATALOG `${catalog_name}` SET PROPERTIES
('enable.auto.analyze'='true')"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("true", result[0][8])
+
+ sql """ALTER CATALOG `${catalog_name}` SET PROPERTIES
('enable.auto.analyze'='false')"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("false", result[0][8])
+
+ sql """analyze table stats_test1
PROPERTIES("use.auto.analyzer"="true")"""
+ result = sql """show auto analyze stats_test1"""
+ assertEquals(0, result.size())
+
+ sql """ALTER TABLE stats_test1 SET ("auto_analyze_policy" =
"enable");"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("true", result[0][8])
+
+ sql """analyze table stats_test1
PROPERTIES("use.auto.analyzer"="true")"""
+ result = sql """show auto analyze stats_test1"""
+ assertEquals(1, result.size())
+
+ sql """ALTER CATALOG `${catalog_name}` SET PROPERTIES
('enable.auto.analyze'='true')"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("true", result[0][8])
+
+ sql """ALTER CATALOG `${catalog_name}` SET PROPERTIES
('enable.auto.analyze'='false')"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("true", result[0][8])
+
+ sql """ALTER TABLE stats_test1 SET ("auto_analyze_policy" =
"disable");"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("false", result[0][8])
+
+ sql """ALTER CATALOG `${catalog_name}` SET PROPERTIES
('enable.auto.analyze'='true')"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("false", result[0][8])
+
+ sql """ALTER CATALOG `${catalog_name}` SET PROPERTIES
('enable.auto.analyze'='false')"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("false", result[0][8])
+
+ sql """ALTER TABLE stats_test1 SET ("auto_analyze_policy" =
"base_on_catalog");"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("false", result[0][8])
+
+ sql """ALTER CATALOG `${catalog_name}` SET PROPERTIES
('enable.auto.analyze'='true')"""
+ result = sql """show table stats stats_test1"""
+ assertEquals(1, result.size())
+ assertEquals("true", result[0][8])
+
sql """drop catalog if exists ${catalog_name}"""
} finally {
diff --git
a/regression-test/suites/statistics/test_auto_analyze_black_white_list.groovy
b/regression-test/suites/statistics/test_auto_analyze_black_white_list.groovy
new file mode 100644
index 00000000000..8ba453b81f5
--- /dev/null
+++
b/regression-test/suites/statistics/test_auto_analyze_black_white_list.groovy
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_auto_analyze_black_white_list") {
+
+ sql """drop database if exists test_auto_analyze_black_white_list"""
+ sql """create database test_auto_analyze_black_white_list"""
+ sql """use test_auto_analyze_black_white_list"""
+ sql """set global force_sample_analyze=false"""
+ sql """set global enable_auto_analyze=false"""
+
+ sql """CREATE TABLE test_bw (
+ key1 bigint NOT NULL,
+ key2 bigint NOT NULL,
+ value1 int NOT NULL,
+ value2 int NOT NULL,
+ value3 int NOT NULL
+ )ENGINE=OLAP
+ DUPLICATE KEY(`key1`, `key2`)
+ COMMENT "OLAP"
+ DISTRIBUTED BY HASH(`key1`) BUCKETS 2
+ PROPERTIES (
+ "replication_num" = "1"
+ )
+ """
+
+ // Test show index row count
+ def result = sql """show table stats test_bw"""
+ assertEquals(1, result.size())
+ assertEquals("true", result[0][8])
+
+ sql """ALTER TABLE test_bw SET ("auto_analyze_policy" = "disable");"""
+ result = sql """show table stats test_bw"""
+ assertEquals(1, result.size())
+ assertEquals("false", result[0][8])
+
+ sql """analyze table test_bw PROPERTIES("use.auto.analyzer"="true")"""
+ result = sql """show auto analyze test_bw"""
+ assertEquals(0, result.size())
+
+ sql """ALTER TABLE test_bw SET ("auto_analyze_policy" = "enable");"""
+ result = sql """show table stats test_bw"""
+ assertEquals(1, result.size())
+ assertEquals("true", result[0][8])
+
+ sql """analyze table test_bw PROPERTIES("use.auto.analyzer"="true")"""
+ result = sql """show auto analyze test_bw"""
+ assertEquals(1, result.size())
+
+ sql """ALTER TABLE test_bw SET ("auto_analyze_policy" = "disable");"""
+ result = sql """show table stats test_bw"""
+ assertEquals(1, result.size())
+ assertEquals("false", result[0][8])
+ sql """ALTER TABLE test_bw SET ("auto_analyze_policy" =
"base_on_catalog");"""
+ result = sql """show table stats test_bw"""
+ assertEquals(1, result.size())
+ assertEquals("true", result[0][8])
+
+ sql """drop database if exists test_auto_analyze_black_white_list"""
+}
+
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]