This is an automated email from the ASF dual-hosted git repository.

lijibing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 8517b8f71fe Support show partition stats. (#35114)
8517b8f71fe is described below

commit 8517b8f71fe1a4813e2ff7f1ee2085804ca239b7
Author: Jibing-Li <64681310+jibing...@users.noreply.github.com>
AuthorDate: Wed May 22 19:29:55 2024 +0800

    Support show partition stats. (#35114)
---
 .../apache/doris/analysis/ShowColumnStatsStmt.java | 48 +++++++++--
 .../apache/doris/analysis/ShowTableStatsStmt.java  | 94 +++++++++++++++-------
 .../java/org/apache/doris/qe/ShowExecutor.java     | 45 +++++------
 .../doris/statistics/StatisticsRepository.java     | 58 ++++++-------
 .../statistics/test_show_partition_stats.groovy    | 92 +++++++++++++++++++++
 5 files changed, 246 insertions(+), 91 deletions(-)

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 cfe2d426f7b..01612af9f9f 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
@@ -22,7 +22,6 @@ import org.apache.doris.catalog.DatabaseIf;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.TableIf;
-import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.Pair;
@@ -35,6 +34,7 @@ import org.apache.doris.qe.ShowResultSetMetaData;
 import org.apache.doris.statistics.AnalysisManager;
 import org.apache.doris.statistics.ColStatsMeta;
 import org.apache.doris.statistics.ColumnStatistic;
+import org.apache.doris.statistics.ResultRow;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -46,7 +46,7 @@ import java.util.stream.Collectors;
 
 public class ShowColumnStatsStmt extends ShowStmt {
 
-    private static final ImmutableList<String> TITLE_NAMES =
+    private static final ImmutableList<String> TABLE_COLUMN_TITLE_NAMES =
             new ImmutableList.Builder<String>()
                     .add("column_name")
                     .add("index_name")
@@ -66,6 +66,22 @@ public class ShowColumnStatsStmt extends ShowStmt {
                     .add("last_analyze_row_count")
                     .build();
 
+    private static final ImmutableList<String> PARTITION_COLUMN_TITLE_NAMES =
+            new ImmutableList.Builder<String>()
+                    .add("column_name")
+                    .add("partition_name")
+                    .add("index_name")
+                    .add("count")
+                    .add("ndv")
+                    .add("num_null")
+                    .add("min")
+                    .add("max")
+                    .add("data_size")
+                    .add("updated_time")
+                    .add("update_rows")
+                    .add("trigger")
+                    .build();
+
     private final TableName tableName;
 
     private final List<String> columnNames;
@@ -92,9 +108,6 @@ public class ShowColumnStatsStmt extends ShowStmt {
         tableName.analyze(analyzer);
         if (partitionNames != null) {
             partitionNames.analyze(analyzer);
-            if (partitionNames.getPartitionNames().size() > 1) {
-                throw new AnalysisException("Only one partition name could be 
specified");
-            }
         }
         CatalogIf<DatabaseIf> catalog = 
Env.getCurrentEnv().getCatalogMgr().getCatalog(tableName.getCtl());
         if (catalog == null) {
@@ -129,8 +142,8 @@ public class ShowColumnStatsStmt extends ShowStmt {
     @Override
     public ShowResultSetMetaData getMetaData() {
         ShowResultSetMetaData.Builder builder = 
ShowResultSetMetaData.builder();
-
-        for (String title : TITLE_NAMES) {
+        ImmutableList<String> titles = partitionNames == null ? 
TABLE_COLUMN_TITLE_NAMES : PARTITION_COLUMN_TITLE_NAMES;
+        for (String title : titles) {
             builder.addColumn(new Column(title, ScalarType.createVarchar(30)));
         }
         return builder.build();
@@ -171,6 +184,27 @@ public class ShowColumnStatsStmt extends ShowStmt {
         return new ShowResultSet(getMetaData(), result);
     }
 
+    public ShowResultSet constructPartitionResultSet(List<ResultRow> 
resultRows, TableIf tableIf) {
+        List<List<String>> result = Lists.newArrayList();
+        resultRows.forEach(r -> {
+            List<String> row = Lists.newArrayList();
+            row.add(r.get(0));
+            row.add(r.get(1));
+            row.add(r.get(2)); // TODO: Get index name.
+            row.add(r.get(3));
+            row.add(r.get(4));
+            row.add(r.get(5));
+            row.add(r.get(6));
+            row.add(r.get(7));
+            row.add(r.get(8));
+            row.add(r.get(9));
+            row.add("N/A");
+            row.add("Manual");
+            result.add(row);
+        });
+        return new ShowResultSet(getMetaData(), result);
+    }
+
     public PartitionNames getPartitionNames() {
         return partitionNames;
     }
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 8d8cd32c454..3b647c8e9fa 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
@@ -20,10 +20,10 @@ package org.apache.doris.analysis;
 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.ScalarType;
 import org.apache.doris.catalog.TableIf;
-import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.UserException;
@@ -41,12 +41,12 @@ import java.time.Instant;
 import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 public class ShowTableStatsStmt extends ShowStmt {
 
-    // TODO add more columns
-    private static final ImmutableList<String> TITLE_NAMES =
+    private static final ImmutableList<String> TABLE_TITLE_NAMES =
             new ImmutableList.Builder<String>()
                     .add("updated_rows")
                     .add("query_times")
@@ -58,6 +58,13 @@ public class ShowTableStatsStmt extends ShowStmt {
                     .add("user_inject")
                     .build();
 
+    private static final ImmutableList<String> PARTITION_TITLE_NAMES =
+            new ImmutableList.Builder<String>()
+                    .add("partition_name")
+                    .add("updated_rows")
+                    .add("row_count")
+                    .build();
+
     private final TableName tableName;
 
     private final PartitionNames partitionNames;
@@ -81,9 +88,6 @@ public class ShowTableStatsStmt extends ShowStmt {
         tableName.analyze(analyzer);
         if (partitionNames != null) {
             partitionNames.analyze(analyzer);
-            if (partitionNames.getPartitionNames().size() > 1) {
-                throw new AnalysisException("Only one partition name could be 
specified");
-            }
         }
         CatalogIf<DatabaseIf> catalog = 
Env.getCurrentEnv().getCatalogMgr().getCatalog(tableName.getCtl());
         if (catalog == null) {
@@ -97,11 +101,12 @@ public class ShowTableStatsStmt extends ShowStmt {
         if (table == null) {
             ErrorReport.reportAnalysisException(String.format("Table: %s not 
exists", tableName.getTbl()));
         }
-        if (partitionNames != null) {
-            String partitionName = partitionNames.getPartitionNames().get(0);
-            Partition partition = table.getPartition(partitionName);
-            if (partition == null) {
-                ErrorReport.reportAnalysisException(String.format("Partition: 
%s not exists", partitionName));
+        if (partitionNames != null && !partitionNames.isStar()) {
+            for (String partitionName : partitionNames.getPartitionNames()) {
+                Partition partition = table.getPartition(partitionName);
+                if (partition == null) {
+                    
ErrorReport.reportAnalysisException(String.format("Partition: %s not exists", 
partitionName));
+                }
             }
         }
         if (!Env.getCurrentEnv().getAccessManager()
@@ -117,7 +122,8 @@ public class ShowTableStatsStmt extends ShowStmt {
     public ShowResultSetMetaData getMetaData() {
         ShowResultSetMetaData.Builder builder = 
ShowResultSetMetaData.builder();
 
-        for (String title : TITLE_NAMES) {
+        ImmutableList<String> titles = partitionNames == null ? 
TABLE_TITLE_NAMES : PARTITION_TITLE_NAMES;
+        for (String title : titles) {
             builder.addColumn(new Column(title, ScalarType.createVarchar(30)));
         }
         return builder.build();
@@ -127,15 +133,34 @@ public class ShowTableStatsStmt extends ShowStmt {
         return table;
     }
 
-    public long getPartitionId() {
+    public ShowResultSet constructResultSet(TableStatsMeta tableStatistic) {
         if (partitionNames == null) {
-            return 0;
+            return constructTableResultSet(tableStatistic);
+        } else {
+            return constructPartitionResultSet(tableStatistic);
         }
-        String partitionName = partitionNames.getPartitionNames().get(0);
-        return table.getPartition(partitionName).getId();
     }
 
-    public ShowResultSet constructResultSet(TableStatsMeta tableStatistic) {
+    public ShowResultSet constructResultSet(long rowCount) {
+        List<List<String>> result = Lists.newArrayList();
+        if (partitionNames != null) {
+            // For partition, return empty result if table stats not exist.
+            return new ShowResultSet(getMetaData(), result);
+        }
+        List<String> row = Lists.newArrayList();
+        row.add("");
+        row.add("");
+        row.add(String.valueOf(rowCount));
+        row.add("");
+        row.add("");
+        row.add("");
+        row.add("");
+        row.add("");
+        result.add(row);
+        return new ShowResultSet(getMetaData(), result);
+    }
+
+    public ShowResultSet constructTableResultSet(TableStatsMeta 
tableStatistic) {
         DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd 
HH:mm:ss");
         if (tableStatistic == null) {
             return new ShowResultSet(getMetaData(), new ArrayList<>());
@@ -147,7 +172,7 @@ public class ShowTableStatsStmt extends ShowStmt {
         row.add(String.valueOf(tableStatistic.rowCount));
         LocalDateTime dateTime =
                 
LocalDateTime.ofInstant(Instant.ofEpochMilli(tableStatistic.updatedTime),
-                        java.time.ZoneId.systemDefault());
+                java.time.ZoneId.systemDefault());
         String formattedDateTime = dateTime.format(formatter);
         row.add(formattedDateTime);
         row.add(tableStatistic.analyzeColumns().toString());
@@ -158,18 +183,29 @@ public class ShowTableStatsStmt extends ShowStmt {
         return new ShowResultSet(getMetaData(), result);
     }
 
-    public ShowResultSet constructResultSet(long rowCount) {
+    public ShowResultSet constructPartitionResultSet(TableStatsMeta 
tableStatistic) {
         List<List<String>> result = Lists.newArrayList();
-        List<String> row = Lists.newArrayList();
-        row.add("");
-        row.add("");
-        row.add(String.valueOf(rowCount));
-        row.add("");
-        row.add("");
-        row.add("");
-        row.add("");
-        row.add("");
-        result.add(row);
+        if (!(table instanceof OlapTable)) {
+            return new ShowResultSet(getMetaData(), result);
+        }
+        Collection<String> partitions = partitionNames.isStar()
+                ? table.getPartitionNames()
+                : partitionNames.getPartitionNames();
+        for (String part : partitions) {
+            Partition partition = table.getPartition(part);
+            if (partition == null) {
+                continue;
+            }
+            Long updateRows = 
tableStatistic.partitionUpdateRows.get(partition.getId());
+            if (updateRows == null) {
+                continue;
+            }
+            List<String> row = Lists.newArrayList();
+            row.add(part);
+            row.add(String.valueOf(updateRows.longValue()));
+            row.add(String.valueOf(partition.getBaseIndex().getRowCount()));
+            result.add(row);
+        }
         return new ShowResultSet(getMetaData(), result);
     }
 
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 6d3f73d4264..757b24583c6 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
@@ -115,7 +115,6 @@ import org.apache.doris.analysis.ShowUserPropertyStmt;
 import org.apache.doris.analysis.ShowVariablesStmt;
 import org.apache.doris.analysis.ShowViewStmt;
 import org.apache.doris.analysis.ShowWorkloadGroupsStmt;
-import org.apache.doris.analysis.TableName;
 import org.apache.doris.analysis.UserIdentity;
 import org.apache.doris.backup.AbstractJob;
 import org.apache.doris.backup.BackupJob;
@@ -2627,19 +2626,27 @@ public class ShowExecutor {
 
     private void handleShowColumnStats() throws AnalysisException {
         ShowColumnStatsStmt showColumnStatsStmt = (ShowColumnStatsStmt) stmt;
-        TableName tableName = showColumnStatsStmt.getTableName();
         TableIf tableIf = showColumnStatsStmt.getTable();
         List<Pair<Pair<String, String>, ColumnStatistic>> columnStatistics = 
new ArrayList<>();
         Set<String> columnNames = showColumnStatsStmt.getColumnNames();
         PartitionNames partitionNames = 
showColumnStatsStmt.getPartitionNames();
         boolean showCache = showColumnStatsStmt.isCached();
         boolean isAllColumns = showColumnStatsStmt.isAllColumns();
-        if (isAllColumns && !showCache && partitionNames == null) {
-            getStatsForAllColumns(columnStatistics, tableIf);
+        if (partitionNames != null) {
+            List<String> partNames = partitionNames.getPartitionNames() == null
+                    ? new ArrayList<>(tableIf.getPartitionNames())
+                    : partitionNames.getPartitionNames();
+            List<ResultRow> partitionColumnStats =
+                    
StatisticsRepository.queryColumnStatisticsByPartitions(tableIf, columnNames, 
partNames);
+            resultSet = 
showColumnStatsStmt.constructPartitionResultSet(partitionColumnStats, tableIf);
         } else {
-            getStatsForSpecifiedColumns(columnStatistics, columnNames, 
tableIf, showCache, tableName, partitionNames);
+            if (isAllColumns && !showCache) {
+                getStatsForAllColumns(columnStatistics, tableIf);
+            } else {
+                getStatsForSpecifiedColumns(columnStatistics, columnNames, 
tableIf, showCache);
+            }
+            resultSet = 
showColumnStatsStmt.constructResultSet(columnStatistics);
         }
-        resultSet = showColumnStatsStmt.constructResultSet(columnStatistics);
     }
 
     private void getStatsForAllColumns(List<Pair<Pair<String, String>, 
ColumnStatistic>> columnStatistics,
@@ -2662,8 +2669,7 @@ public class ShowExecutor {
     }
 
     private void getStatsForSpecifiedColumns(List<Pair<Pair<String, String>, 
ColumnStatistic>> columnStatistics,
-            Set<String> columnNames, TableIf tableIf, boolean showCache,
-            TableName tableName, PartitionNames partitionNames)
+            Set<String> columnNames, TableIf tableIf, boolean showCache)
             throws AnalysisException {
         for (String colName : columnNames) {
             // Olap base index use -1 as index id.
@@ -2683,24 +2689,17 @@ public class ShowExecutor {
                     continue;
                 }
                 // Show column statistics in columnStatisticsCache.
+                ColumnStatistic columnStatistic;
                 if (showCache) {
-                    ColumnStatistic columnStatistic = 
Env.getCurrentEnv().getStatisticsCache().getColumnStatistics(
-                            tableIf.getDatabase().getCatalog().getId(),
-                            tableIf.getDatabase().getId(), tableIf.getId(), 
indexId, colName);
-                    columnStatistics.add(Pair.of(Pair.of(indexName, colName), 
columnStatistic));
-                } else if (partitionNames == null) {
-                    ColumnStatistic columnStatistic =
-                            StatisticsRepository.queryColumnStatisticsByName(
-                                    tableIf.getDatabase().getCatalog().getId(),
-                                    tableIf.getDatabase().getId(), 
tableIf.getId(), indexId, colName);
-                    columnStatistics.add(Pair.of(Pair.of(indexName, colName), 
columnStatistic));
+                    columnStatistic = 
Env.getCurrentEnv().getStatisticsCache().getColumnStatistics(
+                        tableIf.getDatabase().getCatalog().getId(),
+                        tableIf.getDatabase().getId(), tableIf.getId(), 
indexId, colName);
                 } else {
-                    String finalIndexName = indexName;
-                    
columnStatistics.addAll(StatisticsRepository.queryColumnStatisticsByPartitions(tableName,
-                                    colName, 
partitionNames.getPartitionNames())
-                            .stream().map(s -> Pair.of(Pair.of(finalIndexName, 
colName), s))
-                            .collect(Collectors.toList()));
+                    columnStatistic = 
StatisticsRepository.queryColumnStatisticsByName(
+                        tableIf.getDatabase().getCatalog().getId(),
+                        tableIf.getDatabase().getId(), tableIf.getId(), 
indexId, colName);
                 }
+                columnStatistics.add(Pair.of(Pair.of(indexName, colName), 
columnStatistic));
             }
         }
     }
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 7e253985f5b..a17b85002d2 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
@@ -21,8 +21,7 @@ import org.apache.doris.analysis.AlterColumnStatsStmt;
 import org.apache.doris.analysis.TableName;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Env;
-import org.apache.doris.catalog.Partition;
-import org.apache.doris.common.AnalysisException;
+import org.apache.doris.catalog.TableIf;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
@@ -44,7 +43,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.StringJoiner;
-import java.util.stream.Collectors;
 
 /**
  * All the logic that interacts with internal statistics table should be 
placed here.
@@ -58,6 +56,9 @@ public class StatisticsRepository {
     private static final String FULL_QUALIFIED_COLUMN_STATISTICS_NAME = 
FULL_QUALIFIED_DB_NAME + "."
             + "`" + StatisticConstants.TABLE_STATISTIC_TBL_NAME + "`";
 
+    private static final String FULL_QUALIFIED_PARTITION_STATISTICS_NAME = 
FULL_QUALIFIED_DB_NAME + "."
+            + "`" + StatisticConstants.PARTITION_STATISTIC_TBL_NAME + "`";
+
     private static final String FULL_QUALIFIED_COLUMN_HISTOGRAM_NAME = 
FULL_QUALIFIED_DB_NAME + "."
             + "`" + StatisticConstants.HISTOGRAM_TBL_NAME + "`";
 
@@ -65,9 +66,11 @@ public class StatisticsRepository {
             + FULL_QUALIFIED_COLUMN_STATISTICS_NAME
             + " WHERE `id` = '${id}' AND `catalog_id` = '${catalogId}' AND 
`db_id` = '${dbId}'";
 
-    private static final String FETCH_PARTITIONS_STATISTIC_TEMPLATE = "SELECT 
* FROM "
-            + FULL_QUALIFIED_COLUMN_STATISTICS_NAME
-            + " WHERE `id` IN (${idList}) AND `catalog_id` = '${catalogId}' 
AND `db_id` = '${dbId}'";
+    private static final String FETCH_PARTITIONS_STATISTIC_TEMPLATE = "SELECT 
col_id, part_id, idx_id, count, "
+            + "hll_cardinality(ndv) as ndv, null_count, min, max, 
data_size_in_bytes, update_time FROM "
+            + FULL_QUALIFIED_PARTITION_STATISTICS_NAME
+            + " WHERE `catalog_id` = '${catalogId}' AND `db_id` = '${dbId}' 
AND `tbl_id` = ${tableId}"
+            + " AND `part_id` in (${partitionInfo}) AND `col_id` in 
(${columnInfo})";
 
     private static final String FETCH_COLUMN_HISTOGRAM_TEMPLATE = "SELECT * 
FROM "
             + FULL_QUALIFIED_COLUMN_HISTOGRAM_NAME
@@ -117,20 +120,24 @@ public class StatisticsRepository {
         return ColumnStatistic.fromResultRow(resultRow);
     }
 
-    public static List<ColumnStatistic> 
queryColumnStatisticsByPartitions(TableName tableName, String colName,
-            List<String> partitionNames) throws AnalysisException {
-        DBObjects dbObjects = 
StatisticsUtil.convertTableNameToObjects(tableName);
-        Set<Long> partitionIds = new HashSet<>();
-        for (String partitionName : partitionNames) {
-            Partition partition = dbObjects.table.getPartition(partitionName);
-            if (partition == null) {
-                throw new AnalysisException(String.format("partition:%s not 
exists", partitionName));
-            }
-            partitionIds.add(partition.getId());
+    public static List<ResultRow> queryColumnStatisticsByPartitions(TableIf 
table, Set<String> columnNames,
+            List<String> partitionNames) {
+        long ctlId = table.getDatabase().getCatalog().getId();
+        long dbId = table.getDatabase().getId();
+        Map<String, String> params = new HashMap<>();
+        generateCtlDbIdParams(ctlId, dbId, params);
+        params.put("tableId", String.valueOf(table.getId()));
+        StringJoiner sj = new StringJoiner(",");
+        for (String colName : columnNames) {
+            sj.add("'" + colName + "'");
+        }
+        params.put("columnInfo", sj.toString());
+        sj = new StringJoiner(",");
+        for (String part : partitionNames) {
+            sj.add("'" + part + "'");
         }
-        return queryPartitionStatistics(dbObjects.catalog.getId(), 
dbObjects.db.getId(), dbObjects.table.getId(),
-                colName, 
partitionIds).stream().map(ColumnStatistic::fromResultRow).collect(
-                Collectors.toList());
+        params.put("partitionInfo", sj.toString());
+        return 
StatisticsUtil.executeQuery(FETCH_PARTITIONS_STATISTIC_TEMPLATE, params);
     }
 
     public static List<ResultRow> queryColumnStatisticsForTable(long ctlId, 
long dbId, long tableId) {
@@ -166,19 +173,6 @@ public class StatisticsRepository {
         return size == 0 ? null : rows.get(0);
     }
 
-    private static List<ResultRow> queryPartitionStatistics(
-            long ctlId, long dbId, long tblId, String colName, Set<Long> 
partIds) {
-        StringJoiner sj = new StringJoiner(",");
-        for (Long partId : partIds) {
-            sj.add("'" + constructId(tblId, -1, colName, partId) + "'");
-        }
-        Map<String, String> params = new HashMap<>();
-        params.put("idList", sj.toString());
-        generateCtlDbIdParams(ctlId, dbId, params);
-        List<ResultRow> rows = 
StatisticsUtil.executeQuery(FETCH_PARTITIONS_STATISTIC_TEMPLATE, params);
-        return rows == null ? Collections.emptyList() : rows;
-    }
-
     private static Histogram queryColumnHistogramByName(
             long ctlId, long dbId, long tableId, long indexId, String colName) 
{
         ResultRow resultRow = queryColumnHistogramById(ctlId, dbId, tableId, 
indexId, colName);
diff --git a/regression-test/suites/statistics/test_show_partition_stats.groovy 
b/regression-test/suites/statistics/test_show_partition_stats.groovy
new file mode 100644
index 00000000000..45253d43a6c
--- /dev/null
+++ b/regression-test/suites/statistics/test_show_partition_stats.groovy
@@ -0,0 +1,92 @@
+// 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_show_partition_stats") {
+
+    def wait_row_count_reported = { db, table, row, column, expected ->
+        def result = sql """show frontends;"""
+        logger.info("show frontends result origin: " + result)
+        def host
+        def port
+        for (int i = 0; i < result.size(); i++) {
+            if (result[i][8] == "true") {
+                host = result[i][1]
+                port = result[i][4]
+            }
+        }
+        def tokens = context.config.jdbcUrl.split('/')
+        def url=tokens[0] + "//" + host + ":" + port
+        logger.info("Master url is " + url)
+        connect(user = context.config.jdbcUser, password = 
context.config.jdbcPassword, url) {
+            sql """use ${db}"""
+            result = sql """show frontends;"""
+            logger.info("show frontends result master: " + result)
+            for (int i = 0; i < 120; i++) {
+                Thread.sleep(5000)
+                result = sql """SHOW DATA FROM ${table};"""
+                logger.info("result " + result)
+                if (result[row][column] == expected) {
+                    return;
+                }
+            }
+            throw new Exception("Row count report timeout.")
+        }
+
+    }
+
+    sql """drop database if exists test_show_partition_stats"""
+    sql """create database test_show_partition_stats"""
+    sql """use test_show_partition_stats"""
+    def enable = sql """show variables like "%enable_partition_analyze%" """
+    if (enable[0][1].equalsIgnoreCase("false")) {
+        logger.info("partition analyze disabled. " + enable)
+        return;
+    }
+
+    sql """CREATE TABLE `part` (
+        `id` INT NULL,
+        `colint` INT NULL,
+        `coltinyint` tinyint NULL,
+        `colsmallint` smallINT NULL,
+        `colbigint` bigINT NULL,
+        `collargeint` largeINT NULL,
+        `colfloat` float NULL,
+        `coldouble` double NULL,
+        `coldecimal` decimal(27, 9) NULL
+    ) ENGINE=OLAP
+    DUPLICATE KEY(`id`)
+    COMMENT 'OLAP'
+    PARTITION BY RANGE(`id`)
+    (
+        PARTITION p1 VALUES [("-2147483648"), ("10000")),
+        PARTITION p2 VALUES [("10000"), ("20000")),
+        PARTITION p3 VALUES [("20000"), ("30000"))
+    )
+    DISTRIBUTED BY HASH(`id`) BUCKETS 3
+    PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1"
+    )   
+    """
+
+    sql """analyze table part with sync;"""
+    sql """Insert into part values (1, 1, 1, 1, 1, 1, 1.1, 1.1, 1.1), (2, 2, 
2, 2, 2, 2, 2.2, 2.2, 2.2), (3, 3, 3, 3, 3, 3, 3.3, 3.3, 3.3),(4, 4, 4, 4, 4, 
4, 4.4, 4.4, 4.4),(5, 5, 5, 5, 5, 5, 5.5, 5.5, 5.5),(6, 6, 6, 6, 6, 6, 6.6, 
6.6, 6.6),(10001, 10001, 10001, 10001, 10001, 10001, 10001.10001, 10001.10001, 
10001.10001),(10002, 10002, 10002, 10002, 10002, 10002, 10002.10002, 
10002.10002, 10002.10002),(10003, 10003, 10003, 10003, 10003, 10003, 
10003.10003, 10003.10003, 10003.10003),(1000 [...]
+
+    def result = sql """show table stats part"""
+    assertEquals(1, result.size())
+    sql """drop database if exists test_show_partition_stats"""
+}
+


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org
For additional commands, e-mail: commits-h...@doris.apache.org

Reply via email to