You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/04/06 13:46:58 UTC

[GitHub] [incubator-doris] morrySnow commented on a diff in pull request #8860: [feature-wip](statistics) step3: schedule the statistics tasks and update relevant info

morrySnow commented on code in PR #8860:
URL: https://github.com/apache/incubator-doris/pull/8860#discussion_r843966913


##########
fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsManager.java:
##########
@@ -133,7 +139,25 @@ private Table validateTableName(TableName dbTableName) throws AnalysisException
         String tableName = dbTableName.getTbl();
 
         Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbName);
-        Table table = db.getTableOrAnalysisException(tableName);
-        return table;
+        return db.getTableOrAnalysisException(tableName);
+    }
+
+    public void alterTableStatistics(StatisticsTaskResult taskResult) throws AnalysisException {
+        StatsGranularityDesc desc = taskResult.getGranularityDesc();
+        long tableId = desc.getTableId();
+        Map<String, String> statsNameToValue = taskResult.getStatsNameToValue();
+        this.statistics.updateTableStats(tableId, statsNameToValue);

Review Comment:
   should we drop table's statistics if we dropped table?
   if that, could be a problem that we don't check whether table exists?



##########
fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsManager.java:
##########
@@ -133,7 +139,25 @@ private Table validateTableName(TableName dbTableName) throws AnalysisException
         String tableName = dbTableName.getTbl();
 
         Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbName);
-        Table table = db.getTableOrAnalysisException(tableName);
-        return table;
+        return db.getTableOrAnalysisException(tableName);
+    }
+
+    public void alterTableStatistics(StatisticsTaskResult taskResult) throws AnalysisException {
+        StatsGranularityDesc desc = taskResult.getGranularityDesc();
+        long tableId = desc.getTableId();
+        Map<String, String> statsNameToValue = taskResult.getStatsNameToValue();
+        this.statistics.updateTableStats(tableId, statsNameToValue);
+    }
+
+    public void alterColumnStatistics(StatisticsTaskResult taskResult) throws AnalysisException {
+        StatsCategoryDesc categoryDesc = taskResult.getCategoryDesc();
+        long dbId = categoryDesc.getDbId();
+        long tableId = categoryDesc.getTableId();
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbId);
+        Table table = db.getTableOrAnalysisException(tableId);
+        String columnName = categoryDesc.getColumnName();
+        Type columnType = table.getColumn(columnName).getType();
+        Map<String, String> statsNameToValue = taskResult.getStatsNameToValue();
+        this.statistics.updateColumnStats(tableId, columnName, columnType, statsNameToValue);

Review Comment:
   what will happen, if schema changed when we before this function and after statistics job started? 



##########
fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsTask.java:
##########
@@ -33,23 +37,88 @@
  * @granularityDesc: StatsGranularity=partition
  */
 public class StatisticsTask implements Callable<StatisticsTaskResult> {
-    protected long id = Catalog.getCurrentCatalog().getNextId();;
+    protected static final Logger LOG = LogManager.getLogger(StatisticsTask.class);
+
+    public enum TaskState {
+        CREATED,
+        RUNNING,
+        FINISHED,
+        FAILED
+    }
+
+    protected long id = Catalog.getCurrentCatalog().getNextId();
+
     protected long jobId;
     protected StatsGranularityDesc granularityDesc;
     protected StatsCategoryDesc categoryDesc;
     protected List<StatsType> statsTypeList;
+    protected TaskState taskState = TaskState.CREATED;
+
+    protected final long createTime = System.currentTimeMillis();
+    protected long scheduleTime;
+    protected long finishTime;
 
-    public StatisticsTask(long jobId, StatsGranularityDesc granularityDesc,
-                          StatsCategoryDesc categoryDesc, List<StatsType> statsTypeList) {
+    public StatisticsTask(long jobId,
+                          StatsGranularityDesc granularityDesc,
+                          StatsCategoryDesc categoryDesc,
+                          List<StatsType> statsTypeList) {
         this.jobId = jobId;
         this.granularityDesc = granularityDesc;
         this.categoryDesc = categoryDesc;
         this.statsTypeList = statsTypeList;
     }
 
+    public long getId() {
+        return this.id;
+    }
+
+    public long getJobId() {
+        return this.jobId;
+    }
+
+    public StatsGranularityDesc getGranularityDesc() {
+        return this.granularityDesc;
+    }
+
+    public StatsCategoryDesc getCategoryDesc() {
+        return this.categoryDesc;
+    }
+
+    public List<StatsType> getStatsTypeList() {
+        return this.statsTypeList;
+    }
+
+    public TaskState getTaskState() {
+        return this.taskState;
+    }
+
+    public void setTaskState(TaskState taskState) {
+        this.taskState = taskState;
+    }
+
+    public long getCreateTime() {
+        return this.createTime;
+    }
+
+    public long getScheduleTime() {
+        return this.scheduleTime;
+    }
+
+    public void setScheduleTime(long scheduleTime) {
+        this.scheduleTime = scheduleTime;
+    }
+
+    public long getFinishTime() {
+        return this.finishTime;
+    }
+
+    public void setFinishTime(long finishTime) {
+        this.finishTime = finishTime;
+    }
+
     @Override
     public StatisticsTaskResult call() throws Exception {

Review Comment:
   should we just implement this class as a abstract class, and turn this method to an interface ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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