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

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


The following commit(s) were added to refs/heads/master by this push:
     new 677e34428 [Improvement]: Controller dosenot depend on TableService 
(#3425)
677e34428 is described below

commit 677e34428822a62b275d16e06f4ed07da9e7afe0
Author: baiyangtx <[email protected]>
AuthorDate: Thu Feb 6 10:57:07 2025 +0800

    [Improvement]: Controller dosenot depend on TableService (#3425)
    
    * table optimizing info
    
    * table runtime meta
    
    * Controller donot depend on tableService
    
    * ci
    
    * ci
    
    * ci
    
    * ci
    
    ---------
    
    Co-authored-by: zhangyongxiang.alpha <[email protected]>
    Co-authored-by: ZhouJinsong <[email protected]>
---
 .../apache/amoro/server/AmoroServiceContainer.java |   7 +-
 .../amoro/server/DefaultOptimizingService.java     |  24 +
 .../amoro/server/dashboard/DashboardServer.java    |  10 +-
 .../controller/OptimizerGroupController.java       |  26 +-
 .../dashboard/controller/TableController.java      |  48 +-
 .../server/dashboard/utils/OptimizingUtil.java     |  78 +-
 .../amoro/server/optimizing/TaskRuntime.java       |  23 +-
 .../extension/InListExtendedLanguageDriver.java    | 107 +++
 .../persistence/mapper/OptimizingMapper.java       |  91 +-
 .../server/persistence/mapper/TableMetaMapper.java | 181 ++--
 .../amoro/server/table/DefaultTableManager.java    |  73 +-
 .../amoro/server/table/MaintainedTableManager.java |   6 +-
 .../amoro/server/TestDefaultOptimizingService.java |  23 +-
 amoro-common/pom.xml                               |   2 +-
 .../org/apache/amoro/api/OptimizingService.java    | 913 ++++++++++++++++++++-
 .../main/thrift/amoro_optimizing_service.thrift    |   2 +
 .../apache/amoro/MockAmoroManagementServer.java    |   5 +
 17 files changed, 1383 insertions(+), 236 deletions(-)

diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java 
b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java
index a751eeea1..208f70d3e 100644
--- a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java
+++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java
@@ -253,12 +253,7 @@ public class AmoroServiceContainer {
   private void initHttpService() {
     DashboardServer dashboardServer =
         new DashboardServer(
-            serviceConfig,
-            catalogManager,
-            tableManager,
-            optimizingService,
-            terminalManager,
-            tableService);
+            serviceConfig, catalogManager, tableManager, optimizingService, 
terminalManager);
     RestCatalogService restCatalogService = new 
RestCatalogService(catalogManager, tableManager);
 
     httpServer =
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/DefaultOptimizingService.java 
b/amoro-ams/src/main/java/org/apache/amoro/server/DefaultOptimizingService.java
index 9e940bf3d..49fad3491 100644
--- 
a/amoro-ams/src/main/java/org/apache/amoro/server/DefaultOptimizingService.java
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/DefaultOptimizingService.java
@@ -38,11 +38,14 @@ import org.apache.amoro.properties.CatalogMetaProperties;
 import org.apache.amoro.resource.Resource;
 import org.apache.amoro.resource.ResourceGroup;
 import org.apache.amoro.server.catalog.CatalogManager;
+import org.apache.amoro.server.optimizing.OptimizingProcess;
+import org.apache.amoro.server.optimizing.OptimizingProcessMeta;
 import org.apache.amoro.server.optimizing.OptimizingQueue;
 import org.apache.amoro.server.optimizing.OptimizingStatus;
 import org.apache.amoro.server.optimizing.TaskRuntime;
 import org.apache.amoro.server.persistence.StatedPersistentBase;
 import org.apache.amoro.server.persistence.mapper.OptimizerMapper;
+import org.apache.amoro.server.persistence.mapper.OptimizingMapper;
 import org.apache.amoro.server.persistence.mapper.ResourceMapper;
 import org.apache.amoro.server.resource.OptimizerInstance;
 import org.apache.amoro.server.resource.OptimizerManager;
@@ -55,6 +58,7 @@ import org.apache.amoro.server.table.TableService;
 import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions;
 import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableList;
 import 
org.apache.amoro.shade.guava32.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.amoro.shade.thrift.org.apache.thrift.TException;
 import org.apache.amoro.table.TableProperties;
 import org.apache.commons.lang3.StringUtils;
 import org.jetbrains.annotations.NotNull;
@@ -269,6 +273,26 @@ public class DefaultOptimizingService extends 
StatedPersistentBase
     return optimizer.getToken();
   }
 
+  @Override
+  public boolean cancelProcess(long processId) throws TException {
+    OptimizingProcessMeta processMeta =
+        getAs(OptimizingMapper.class, m -> m.getOptimizingProcess(processId));
+    if (processMeta == null) {
+      return false;
+    }
+    long tableId = processMeta.getTableId();
+    TableRuntime tableRuntime = tableService.getRuntime(tableId);
+    if (tableRuntime == null) {
+      return false;
+    }
+    OptimizingProcess process = tableRuntime.getOptimizingProcess();
+    if (process == null || process.getProcessId() != processId) {
+      return false;
+    }
+    process.close();
+    return true;
+  }
+
   /**
    * Get optimizing queue.
    *
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/DashboardServer.java
 
b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/DashboardServer.java
index 4fb925c33..d21c83813 100644
--- 
a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/DashboardServer.java
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/DashboardServer.java
@@ -52,7 +52,6 @@ import 
org.apache.amoro.server.dashboard.controller.VersionController;
 import org.apache.amoro.server.dashboard.response.ErrorResponse;
 import org.apache.amoro.server.dashboard.utils.ParamSignatureCalculator;
 import org.apache.amoro.server.table.TableManager;
-import org.apache.amoro.server.table.TableService;
 import org.apache.amoro.server.terminal.TerminalManager;
 import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions;
 import org.apache.commons.lang3.StringUtils;
@@ -101,15 +100,13 @@ public class DashboardServer {
       CatalogManager catalogManager,
       TableManager tableManager,
       DefaultOptimizingService optimizerManager,
-      TerminalManager terminalManager,
-      TableService tableService) {
+      TerminalManager terminalManager) {
     PlatformFileManager platformFileManager = new PlatformFileManager();
     this.catalogController = new CatalogController(catalogManager, 
platformFileManager);
     this.healthCheckController = new HealthCheckController();
     this.loginController = new LoginController(serviceConfig);
     // TODO: remove table service from OptimizerGroupController
-    this.optimizerGroupController =
-        new OptimizerGroupController(tableManager, tableService, 
optimizerManager);
+    this.optimizerGroupController = new OptimizerGroupController(tableManager, 
optimizerManager);
     this.optimizerController = new OptimizerController(optimizerManager);
     this.platformFileInfoController = new 
PlatformFileInfoController(platformFileManager);
     this.settingController = new SettingController(serviceConfig, 
optimizerManager);
@@ -117,8 +114,7 @@ public class DashboardServer {
         new ServerTableDescriptor(catalogManager, tableManager, serviceConfig);
     // TODO: remove table service from TableController
     this.tableController =
-        new TableController(
-            catalogManager, tableManager, tableService, tableDescriptor, 
serviceConfig);
+        new TableController(catalogManager, tableManager, tableDescriptor, 
serviceConfig);
     this.terminalController = new TerminalController(terminalManager);
     this.versionController = new VersionController();
     this.overviewController = new OverviewController();
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerGroupController.java
 
b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerGroupController.java
index ddaca1399..e8b3a6d5a 100644
--- 
a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerGroupController.java
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerGroupController.java
@@ -28,15 +28,11 @@ import 
org.apache.amoro.server.dashboard.model.OptimizerResourceInfo;
 import org.apache.amoro.server.dashboard.model.TableOptimizingInfo;
 import org.apache.amoro.server.dashboard.response.OkResponse;
 import org.apache.amoro.server.dashboard.response.PageResult;
-import org.apache.amoro.server.dashboard.utils.OptimizingUtil;
 import org.apache.amoro.server.optimizing.OptimizingStatus;
-import org.apache.amoro.server.persistence.TableRuntimeMeta;
 import org.apache.amoro.server.resource.ContainerMetadata;
 import org.apache.amoro.server.resource.OptimizerInstance;
 import org.apache.amoro.server.resource.ResourceContainers;
 import org.apache.amoro.server.table.TableManager;
-import org.apache.amoro.server.table.TableRuntime;
-import org.apache.amoro.server.table.TableService;
 import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions;
 import org.apache.commons.lang3.tuple.Pair;
 import org.slf4j.Logger;
@@ -60,15 +56,11 @@ public class OptimizerGroupController {
 
   private static final String ALL_GROUP = "all";
   private final TableManager tableManager;
-  private final TableService tableService;
   private final DefaultOptimizingService optimizerManager;
 
   public OptimizerGroupController(
-      TableManager tableManager,
-      TableService tableService,
-      DefaultOptimizingService optimizerManager) {
+      TableManager tableManager, DefaultOptimizingService optimizerManager) {
     this.tableManager = tableManager;
-    this.tableService = tableService;
     this.optimizerManager = optimizerManager;
   }
 
@@ -106,26 +98,16 @@ public class OptimizerGroupController {
     if (statusCodes.isEmpty()) {
       statusCodes = null;
     }
-    Pair<List<TableRuntimeMeta>, Integer> tableRuntimeBeans =
-        tableManager.queryTableRuntimeMetas(
+    Pair<List<TableOptimizingInfo>, Integer> tableRuntimeBeans =
+        tableManager.queryTableOptimizingInfo(
             optimizerGroupUsedInDbFilter,
             dbFilterStr,
             tableFilterStr,
             statusCodes,
             pageSize,
             offset);
-
-    List<TableRuntime> tableRuntimes =
-        tableRuntimeBeans.getLeft().stream()
-            .map(meta -> tableService.getRuntime(meta.getTableId()))
-            .collect(Collectors.toList());
-
     PageResult<TableOptimizingInfo> amsPageResult =
-        PageResult.of(
-            tableRuntimes.stream()
-                .map(OptimizingUtil::buildTableOptimizeInfo)
-                .collect(Collectors.toList()),
-            tableRuntimeBeans.getRight());
+        PageResult.of(tableRuntimeBeans.getLeft(), 
tableRuntimeBeans.getRight());
     ctx.json(OkResponse.of(amsPageResult));
   }
 
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/TableController.java
 
b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/TableController.java
index f6c19de32..de8b1fcbc 100644
--- 
a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/TableController.java
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/TableController.java
@@ -25,6 +25,8 @@ import org.apache.amoro.Constants;
 import org.apache.amoro.ServerTableIdentifier;
 import org.apache.amoro.TableFormat;
 import org.apache.amoro.api.CatalogMeta;
+import org.apache.amoro.api.OptimizingService;
+import org.apache.amoro.client.OptimizingClientPools;
 import org.apache.amoro.config.Configurations;
 import org.apache.amoro.hive.CachedHiveClientPool;
 import org.apache.amoro.hive.HMSClientPool;
@@ -51,12 +53,12 @@ import 
org.apache.amoro.server.dashboard.response.PageResult;
 import org.apache.amoro.server.dashboard.utils.AmsUtil;
 import org.apache.amoro.server.dashboard.utils.CommonUtil;
 import org.apache.amoro.server.optimizing.OptimizingStatus;
+import org.apache.amoro.server.persistence.TableRuntimeMeta;
 import org.apache.amoro.server.table.TableManager;
-import org.apache.amoro.server.table.TableRuntime;
-import org.apache.amoro.server.table.TableService;
 import org.apache.amoro.shade.guava32.com.google.common.base.Function;
 import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions;
 import 
org.apache.amoro.shade.guava32.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.amoro.shade.thrift.org.apache.thrift.TException;
 import org.apache.amoro.table.TableIdentifier;
 import org.apache.amoro.table.TableMetaStore;
 import org.apache.amoro.table.TableProperties;
@@ -103,7 +105,6 @@ public class TableController {
 
   private final CatalogManager catalogManager;
   private final TableManager tableManager;
-  private final TableService tableService;
   private final ServerTableDescriptor tableDescriptor;
   private final Configurations serviceConfig;
   private final ConcurrentHashMap<TableIdentifier, UpgradeRunningInfo> 
upgradeRunningInfo =
@@ -113,12 +114,10 @@ public class TableController {
   public TableController(
       CatalogManager catalogManager,
       TableManager tableManager,
-      TableService tableService,
       ServerTableDescriptor tableDescriptor,
       Configurations serviceConfig) {
     this.catalogManager = catalogManager;
     this.tableManager = tableManager;
-    this.tableService = tableService;
     this.tableDescriptor = tableDescriptor;
     this.serviceConfig = serviceConfig;
     this.tableUpgradeExecutor =
@@ -157,11 +156,12 @@ public class TableController {
             tableManager.getServerTableIdentifier(
                 TableIdentifier.of(catalog, database, 
tableName).buildTableIdentifier()));
     if (serverTableIdentifier.isPresent()) {
-      TableRuntime tableRuntime = 
tableService.getRuntime(serverTableIdentifier.get().getId());
-      if (tableRuntime != null) {
-        
tableSummary.setOptimizingStatus(tableRuntime.getOptimizingStatus().name());
+      TableRuntimeMeta tableRuntimeMeta =
+          tableManager.getTableRuntimeMata(serverTableIdentifier.get());
+      if (tableRuntimeMeta != null) {
+        
tableSummary.setOptimizingStatus(tableRuntimeMeta.getTableStatus().name());
         AbstractOptimizingEvaluator.PendingInput tableRuntimeSummary =
-            tableRuntime.getTableSummary();
+            tableRuntimeMeta.getTableSummary();
         if (tableRuntimeSummary != null) {
           tableSummary.setHealthScore(tableRuntimeSummary.getHealthScore());
         }
@@ -674,31 +674,31 @@ public class TableController {
     String catalog = ctx.pathParam("catalog");
     String db = ctx.pathParam("db");
     String table = ctx.pathParam("table");
-    String processId = ctx.pathParam("processId");
+    String processIds = ctx.pathParam("processId");
     Preconditions.checkArgument(
         StringUtils.isNotBlank(catalog)
             && StringUtils.isNotBlank(db)
             && StringUtils.isNotBlank(table),
         "catalog.database.tableName can not be empty in any element");
     Preconditions.checkState(catalogManager.catalogExist(catalog), "invalid 
catalog!");
-
+    long processId = Long.parseLong(processIds);
     ServerTableIdentifier serverTableIdentifier =
         tableManager.getServerTableIdentifier(
             TableIdentifier.of(catalog, db, table).buildTableIdentifier());
-    TableRuntime tableRuntime =
-        serverTableIdentifier != null
-            ? tableService.getRuntime(serverTableIdentifier.getId())
-            : null;
+    TableRuntimeMeta meta = 
tableManager.getTableRuntimeMata(serverTableIdentifier);
+    if (meta == null || meta.getOptimizingProcessId() != processId) {
+      throw new IllegalArgumentException(
+          String.format("Can't cancel optimizing process %s", processId));
+    }
 
-    Preconditions.checkArgument(
-        tableRuntime != null
-            && tableRuntime.getOptimizingProcess() != null
-            && Objects.equals(
-                tableRuntime.getOptimizingProcess().getProcessId(), 
Long.parseLong(processId)),
-        "Can't cancel optimizing process %s",
-        processId);
-
-    tableRuntime.getOptimizingProcess().close();
+    OptimizingService.Iface client =
+        OptimizingClientPools.getClient(
+            AmsUtil.getAMSThriftAddress(serviceConfig, 
Constants.THRIFT_OPTIMIZING_SERVICE_NAME));
+    try {
+      client.cancelProcess(processId);
+    } catch (TException e) {
+      throw new IllegalStateException("Failed to cancel optimizing process:" + 
e.getMessage());
+    }
     ctx.json(OkResponse.ok());
   }
 
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/utils/OptimizingUtil.java
 
b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/utils/OptimizingUtil.java
index d618eb789..a3b042a01 100644
--- 
a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/utils/OptimizingUtil.java
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/utils/OptimizingUtil.java
@@ -18,15 +18,21 @@
 
 package org.apache.amoro.server.dashboard.utils;
 
+import org.apache.amoro.ServerTableIdentifier;
+import org.apache.amoro.config.OptimizingConfig;
 import org.apache.amoro.optimizing.MetricsSummary;
 import org.apache.amoro.optimizing.plan.AbstractOptimizingEvaluator;
 import org.apache.amoro.server.dashboard.model.TableOptimizingInfo;
-import org.apache.amoro.server.optimizing.OptimizingProcess;
 import org.apache.amoro.server.optimizing.OptimizingStatus;
-import org.apache.amoro.server.table.TableRuntime;
+import org.apache.amoro.server.optimizing.OptimizingTaskMeta;
+import org.apache.amoro.server.optimizing.TaskRuntime;
+import org.apache.amoro.server.persistence.TableRuntimeMeta;
 import org.apache.amoro.table.descriptor.FilesStatistics;
 import org.apache.iceberg.ContentFile;
 
+import java.util.List;
+import java.util.stream.Collectors;
+
 public class OptimizingUtil {
 
   /**
@@ -34,20 +40,44 @@ public class OptimizingUtil {
    *
    * @return TableOptimizeInfo
    */
-  public static TableOptimizingInfo buildTableOptimizeInfo(TableRuntime 
optimizingTableRuntime) {
-    OptimizingProcess process = optimizingTableRuntime.getOptimizingProcess();
-    TableOptimizingInfo tableOptimizeInfo =
-        new TableOptimizingInfo(optimizingTableRuntime.getTableIdentifier());
-    tableOptimizeInfo.setOptimizeStatus(
-        optimizingTableRuntime.getOptimizingStatus().displayValue());
+  public static TableOptimizingInfo buildTableOptimizeInfo(
+      TableRuntimeMeta optimizingTableRuntime,
+      List<OptimizingTaskMeta> processTasks,
+      List<TaskRuntime.TaskQuota> quotas) {
+    ServerTableIdentifier identifier =
+        ServerTableIdentifier.of(
+            optimizingTableRuntime.getTableId(),
+            optimizingTableRuntime.getCatalogName(),
+            optimizingTableRuntime.getDbName(),
+            optimizingTableRuntime.getTableName(),
+            optimizingTableRuntime.getFormat());
+    TableOptimizingInfo tableOptimizeInfo = new 
TableOptimizingInfo(identifier);
+    OptimizingStatus optimizingStatus = 
optimizingTableRuntime.getTableStatus();
+    tableOptimizeInfo.setOptimizeStatus(optimizingStatus.displayValue());
     tableOptimizeInfo.setDuration(
         System.currentTimeMillis() - 
optimizingTableRuntime.getCurrentStatusStartTime());
-    tableOptimizeInfo.setQuota(optimizingTableRuntime.getTargetQuota());
-    
tableOptimizeInfo.setQuotaOccupation(optimizingTableRuntime.calculateQuotaOccupy());
+    OptimizingConfig optimizingConfig =
+        optimizingTableRuntime.getTableConfig().getOptimizingConfig();
+    tableOptimizeInfo.setQuota(optimizingConfig.getTargetQuota());
+    double quotaOccupy =
+        calculateQuotaOccupy(
+            processTasks,
+            quotas,
+            optimizingTableRuntime.getCurrentStatusStartTime(),
+            System.currentTimeMillis());
+    tableOptimizeInfo.setQuotaOccupation(quotaOccupy);
     FilesStatistics optimizeFileInfo;
-    if (optimizingTableRuntime.getOptimizingStatus().isProcessing()) {
-      optimizeFileInfo = collectOptimizingFileInfo(process == null ? null : 
process.getSummary());
-    } else if (optimizingTableRuntime.getOptimizingStatus() == 
OptimizingStatus.PENDING) {
+    if (optimizingStatus.isProcessing()) {
+      MetricsSummary summary = null;
+      if (processTasks != null && !processTasks.isEmpty()) {
+        List<MetricsSummary> taskSummary =
+            processTasks.stream()
+                .map(OptimizingTaskMeta::getMetricsSummary)
+                .collect(Collectors.toList());
+        summary = new MetricsSummary(taskSummary);
+      }
+      optimizeFileInfo = collectOptimizingFileInfo(summary);
+    } else if (optimizingStatus == OptimizingStatus.PENDING) {
       optimizeFileInfo = 
collectPendingFileInfo(optimizingTableRuntime.getPendingInput());
     } else {
       optimizeFileInfo = null;
@@ -60,6 +90,28 @@ public class OptimizingUtil {
     return tableOptimizeInfo;
   }
 
+  private static double calculateQuotaOccupy(
+      List<OptimizingTaskMeta> processTasks,
+      List<TaskRuntime.TaskQuota> quotas,
+      long startTime,
+      long endTime) {
+    double finishedOccupy = 0;
+    if (quotas != null) {
+      finishedOccupy = quotas.stream().mapToDouble(q -> 
q.getQuotaTime(startTime)).sum();
+    }
+    double runningOccupy = 0;
+    if (processTasks != null) {
+      runningOccupy =
+          processTasks.stream()
+              .mapToDouble(
+                  t ->
+                      TaskRuntime.taskRunningQuotaTime(
+                          startTime, endTime, t.getStartTime(), 
t.getCostTime()))
+              .sum();
+    }
+    return finishedOccupy + runningOccupy;
+  }
+
   private static FilesStatistics collectPendingFileInfo(
       AbstractOptimizingEvaluator.PendingInput pendingInput) {
     if (pendingInput == null) {
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/TaskRuntime.java 
b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/TaskRuntime.java
index 59339dece..9f7071c91 100644
--- 
a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/TaskRuntime.java
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/TaskRuntime.java
@@ -52,6 +52,20 @@ public class TaskRuntime<T extends StagedTaskDescriptor<?, 
?, ?>> extends Stated
   @StateField private int threadId = -1;
   @StateField private String failReason;
 
+  public static long taskRunningQuotaTime(
+      long calculatingStartTime, long calculatingEndTime, long taskStartTime, 
long taskCostTime) {
+    if (taskStartTime == AmoroServiceConstants.INVALID_TIME) {
+      return 0;
+    }
+    calculatingStartTime = Math.max(taskStartTime, calculatingStartTime);
+    calculatingEndTime =
+        taskCostTime == AmoroServiceConstants.INVALID_TIME
+            ? calculatingEndTime
+            : taskCostTime + taskStartTime;
+    long lastingTime = calculatingEndTime - calculatingStartTime;
+    return Math.max(0, lastingTime);
+  }
+
   private TaskRuntime() {}
 
   public TaskRuntime(OptimizingTaskId taskId, T taskDescriptor) {
@@ -204,14 +218,7 @@ public class TaskRuntime<T extends StagedTaskDescriptor<?, 
?, ?>> extends Stated
   }
 
   public long getQuotaTime(long calculatingStartTime, long calculatingEndTime) 
{
-    if (startTime == AmoroServiceConstants.INVALID_TIME) {
-      return 0;
-    }
-    calculatingStartTime = Math.max(startTime, calculatingStartTime);
-    calculatingEndTime =
-        costTime == AmoroServiceConstants.INVALID_TIME ? calculatingEndTime : 
costTime + startTime;
-    long lastingTime = calculatingEndTime - calculatingStartTime;
-    return Math.max(0, lastingTime);
+    return taskRunningQuotaTime(calculatingStartTime, calculatingEndTime, 
startTime, costTime);
   }
 
   public void setStatus(Status status) {
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/extension/InListExtendedLanguageDriver.java
 
b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/extension/InListExtendedLanguageDriver.java
new file mode 100644
index 000000000..8879af25e
--- /dev/null
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/extension/InListExtendedLanguageDriver.java
@@ -0,0 +1,107 @@
+/*
+ * 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.amoro.server.persistence.extension;
+
+import org.apache.ibatis.mapping.SqlSource;
+import org.apache.ibatis.scripting.LanguageDriver;
+import org.apache.ibatis.scripting.xmltags.XMLLanguageDriver;
+import org.apache.ibatis.session.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Mybatis language extensions, this diver will replace sql script like
+ *
+ * <pre>
+ *  select * from tbl where id in (#{ids::number[]})
+ * </pre>
+ *
+ * to
+ *
+ * <pre>
+ * <script>
+ *   select * from tbl where id in
+ *    <foreach collection="items" item="item" separator="," open="(" close=")" 
>
+ *      #{items}
+ *    </foreach>
+ * </script>
+ * </pre>
+ *
+ * <p>or replace sql script like
+ *
+ * <pre>
+ *  select * from tbl where id in (#{ids::string[]})
+ * </pre>
+ *
+ * to
+ *
+ * <pre>
+ * <script>
+ *   select * from tbl where id in
+ *    <foreach collection="items" item="item" separator="," open="(" close=")" 
>
+ *      #{items}
+ *    </foreach>
+ * </script>
+ * </pre>
+ */
+public class InListExtendedLanguageDriver extends XMLLanguageDriver implements 
LanguageDriver {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(InListExtendedLanguageDriver.class);
+
+  private final Pattern inNumberPattern = 
Pattern.compile("\\(#\\{(\\w+)::number\\[]}\\)");
+
+  private final Pattern inStringPattern = 
Pattern.compile("\\(#\\{(\\w+)::string\\[]}\\)");
+
+  @Override
+  public SqlSource createSqlSource(
+      Configuration configuration, String script, Class<?> parameterType) {
+    Matcher matcher = inNumberPattern.matcher(script);
+    if (matcher.find()) {
+      String newScript = inNumber(matcher);
+      LOG.info("REPLACE IN_NUMBER_LIST original script:{}, new script:{}", 
script, newScript);
+      return super.createSqlSource(configuration, newScript, parameterType);
+    }
+
+    matcher = inStringPattern.matcher(script);
+    if (matcher.find()) {
+      String newScript = inString(matcher);
+      LOG.info("REPLACE IN_NUMBER_LIST original script:{}, new script:{}", 
script, newScript);
+      return super.createSqlSource(configuration, newScript, parameterType);
+    }
+
+    return super.createSqlSource(configuration, script, parameterType);
+  }
+
+  private String inNumber(Matcher matcher) {
+    final String replacement =
+        "(<foreach collection=\"$1\" item=\"__item\" separator=\",\" 
>#{__item}</foreach>)";
+    String script = matcher.replaceAll(replacement);
+    return "<script>" + script + "</script>";
+  }
+
+  private String inString(Matcher matcher) {
+    final String replacement =
+        "(<foreach collection=\"$1\" item=\"__item\" separator=\",\" 
>\"#{__item}\"</foreach>)";
+    String script = matcher.replaceAll(replacement);
+    return "<script>" + script + "</script>";
+  }
+}
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/OptimizingMapper.java
 
b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/OptimizingMapper.java
index 99c5a2645..81f22b15e 100644
--- 
a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/OptimizingMapper.java
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/OptimizingMapper.java
@@ -34,15 +34,19 @@ import 
org.apache.amoro.server.persistence.converter.Map2StringConverter;
 import org.apache.amoro.server.persistence.converter.MapLong2StringConverter;
 import org.apache.amoro.server.persistence.converter.Object2ByteArrayConvert;
 import 
org.apache.amoro.server.persistence.converter.TaskDescriptorTypeConverter;
+import 
org.apache.amoro.server.persistence.extension.InListExtendedLanguageDriver;
 import org.apache.ibatis.annotations.Delete;
 import org.apache.ibatis.annotations.Insert;
+import org.apache.ibatis.annotations.Lang;
 import org.apache.ibatis.annotations.Param;
 import org.apache.ibatis.annotations.Result;
+import org.apache.ibatis.annotations.ResultMap;
 import org.apache.ibatis.annotations.Results;
 import org.apache.ibatis.annotations.Select;
 import org.apache.ibatis.annotations.Update;
 import org.apache.ibatis.type.JdbcType;
 
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
@@ -102,29 +106,31 @@ public interface OptimizingMapper {
           + " <if test='optimizingStatus != null'> AND a.status = 
#{optimizingStatus}</if>"
           + " ORDER BY process_id desc"
           + "</script>")
-  @Results({
-    @Result(property = "processId", column = "process_id"),
-    @Result(property = "tableId", column = "table_id"),
-    @Result(property = "catalogName", column = "catalog_name"),
-    @Result(property = "dbName", column = "db_name"),
-    @Result(property = "tableName", column = "table_name"),
-    @Result(property = "targetSnapshotId", column = "target_snapshot_id"),
-    @Result(property = "targetChangeSnapshotId", column = 
"target_change_snapshot_id"),
-    @Result(property = "status", column = "status"),
-    @Result(property = "optimizingType", column = "optimizing_type"),
-    @Result(property = "planTime", column = "plan_time", typeHandler = 
Long2TsConverter.class),
-    @Result(property = "endTime", column = "end_time", typeHandler = 
Long2TsConverter.class),
-    @Result(property = "failReason", column = "fail_reason"),
-    @Result(property = "summary", column = "summary", typeHandler = 
JsonObjectConverter.class),
-    @Result(
-        property = "fromSequence",
-        column = "from_sequence",
-        typeHandler = MapLong2StringConverter.class),
-    @Result(
-        property = "toSequence",
-        column = "to_sequence",
-        typeHandler = MapLong2StringConverter.class)
-  })
+  @Results(
+      id = "processMeta",
+      value = {
+        @Result(property = "processId", column = "process_id"),
+        @Result(property = "tableId", column = "table_id"),
+        @Result(property = "catalogName", column = "catalog_name"),
+        @Result(property = "dbName", column = "db_name"),
+        @Result(property = "tableName", column = "table_name"),
+        @Result(property = "targetSnapshotId", column = "target_snapshot_id"),
+        @Result(property = "targetChangeSnapshotId", column = 
"target_change_snapshot_id"),
+        @Result(property = "status", column = "status"),
+        @Result(property = "optimizingType", column = "optimizing_type"),
+        @Result(property = "planTime", column = "plan_time", typeHandler = 
Long2TsConverter.class),
+        @Result(property = "endTime", column = "end_time", typeHandler = 
Long2TsConverter.class),
+        @Result(property = "failReason", column = "fail_reason"),
+        @Result(property = "summary", column = "summary", typeHandler = 
JsonObjectConverter.class),
+        @Result(
+            property = "fromSequence",
+            column = "from_sequence",
+            typeHandler = MapLong2StringConverter.class),
+        @Result(
+            property = "toSequence",
+            column = "to_sequence",
+            typeHandler = MapLong2StringConverter.class)
+      })
   List<OptimizingProcessMeta> selectOptimizingProcesses(
       @Param("catalogName") String catalogName,
       @Param("dbName") String dbName,
@@ -132,6 +138,14 @@ public interface OptimizingMapper {
       @Param("optimizingType") String optimizingType,
       @Param("optimizingStatus") ProcessStatus optimizingStatus);
 
+  @Select(
+      "SELECT a.process_id, a.table_id, a.catalog_name, a.db_name, 
a.table_name, a.target_snapshot_id,"
+          + " a.target_change_snapshot_id, a.status, a.optimizing_type, 
a.plan_time, a.end_time,"
+          + " a.fail_reason, a.summary, a.from_sequence, a.to_sequence FROM 
table_optimizing_process a "
+          + " WHERE a.process_id = #{processId}")
+  @ResultMap("processMeta")
+  OptimizingProcessMeta getOptimizingProcess(@Param("processId") long 
processId);
+
   /** Optimizing TaskRuntime operation below */
   @Insert({
     "<script>",
@@ -261,18 +275,31 @@ public interface OptimizingMapper {
   @Select(
       "SELECT process_id, task_id, retry_num, table_id, start_time, end_time, 
fail_reason "
           + "FROM optimizing_task_quota WHERE table_id = #{tableId} AND 
process_id >= #{startTime}")
-  @Results({
-    @Result(property = "processId", column = "process_id"),
-    @Result(property = "taskId", column = "task_id"),
-    @Result(property = "retryNum", column = "retry_num"),
-    @Result(property = "tableId", column = "table_id"),
-    @Result(property = "startTime", column = "start_time", typeHandler = 
Long2TsConverter.class),
-    @Result(property = "endTime", column = "end_time", typeHandler = 
Long2TsConverter.class),
-    @Result(property = "failReason", column = "fail_reason")
-  })
+  @Results(
+      id = "taskQuota",
+      value = {
+        @Result(property = "processId", column = "process_id"),
+        @Result(property = "taskId", column = "task_id"),
+        @Result(property = "retryNum", column = "retry_num"),
+        @Result(property = "tableId", column = "table_id"),
+        @Result(
+            property = "startTime",
+            column = "start_time",
+            typeHandler = Long2TsConverter.class),
+        @Result(property = "endTime", column = "end_time", typeHandler = 
Long2TsConverter.class),
+        @Result(property = "failReason", column = "fail_reason")
+      })
   List<TaskRuntime.TaskQuota> selectTaskQuotasByTime(
       @Param("tableId") long tableId, @Param("startTime") long startTime);
 
+  @Select(
+      "SELECT process_id, task_id, retry_num, table_id, start_time, end_time, 
fail_reason "
+          + "FROM optimizing_task_quota WHERE table_id in 
(#{tables::number[]}) AND process_id >= #{startTime}")
+  @Lang(InListExtendedLanguageDriver.class)
+  @ResultMap("taskQuota")
+  List<TaskRuntime.TaskQuota> selectTableQuotas(
+      @Param("tables") Collection<Long> tables, @Param("startTime") long 
startTime);
+
   @Insert(
       "INSERT INTO optimizing_task_quota (process_id, task_id, retry_num, 
table_id, start_time, end_time,"
           + " fail_reason) VALUES (#{taskQuota.processId}, 
#{taskQuota.taskId}, #{taskQuota.retryNum},"
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableMetaMapper.java
 
b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableMetaMapper.java
index e91d88b98..fd6d0ff7f 100644
--- 
a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableMetaMapper.java
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableMetaMapper.java
@@ -33,6 +33,7 @@ import org.apache.ibatis.annotations.Insert;
 import org.apache.ibatis.annotations.Options;
 import org.apache.ibatis.annotations.Param;
 import org.apache.ibatis.annotations.Result;
+import org.apache.ibatis.annotations.ResultMap;
 import org.apache.ibatis.annotations.Results;
 import org.apache.ibatis.annotations.Select;
 import org.apache.ibatis.annotations.Update;
@@ -391,67 +392,82 @@ public interface TableMetaMapper {
           + " b.target_change_snapshot_id, b.plan_time, b.from_sequence, 
b.to_sequence FROM table_runtime a"
           + " INNER JOIN table_identifier i ON a.table_id = i.table_id "
           + " LEFT JOIN table_optimizing_process b ON a.optimizing_process_id 
= b.process_id")
-  @Results({
-    @Result(property = "tableId", column = "table_id"),
-    @Result(property = "catalogName", column = "catalog_name"),
-    @Result(property = "dbName", column = "db_name"),
-    @Result(property = "tableName", column = "table_name"),
-    @Result(property = "format", column = "format", typeHandler = 
TableFormatConverter.class),
-    @Result(property = "currentSnapshotId", column = "current_snapshot_id"),
-    @Result(property = "currentChangeSnapshotId", column = 
"current_change_snapshotId"),
-    @Result(property = "lastOptimizedSnapshotId", column = 
"last_optimized_snapshotId"),
-    @Result(
-        property = "lastOptimizedChangeSnapshotId",
-        column = "last_optimized_change_snapshotId"),
-    @Result(
-        property = "lastMajorOptimizingTime",
-        column = "last_major_optimizing_time",
-        typeHandler = Long2TsConverter.class),
-    @Result(
-        property = "lastMinorOptimizingTime",
-        column = "last_minor_optimizing_time",
-        typeHandler = Long2TsConverter.class),
-    @Result(
-        property = "lastFullOptimizingTime",
-        column = "last_full_optimizing_time",
-        typeHandler = Long2TsConverter.class),
-    @Result(
-        property = "tableStatus",
-        column = "optimizing_status_code",
-        typeHandler = OptimizingStatusConverter.class),
-    @Result(
-        property = "currentStatusStartTime",
-        column = "optimizing_status_start_time",
-        typeHandler = Long2TsConverter.class),
-    @Result(property = "optimizingProcessId", column = 
"optimizing_process_id"),
-    @Result(property = "optimizerGroup", column = "optimizer_group"),
-    @Result(
-        property = "tableConfig",
-        column = "table_config",
-        typeHandler = JsonObjectConverter.class),
-    @Result(
-        property = "pendingInput",
-        column = "pending_input",
-        typeHandler = JsonObjectConverter.class),
-    @Result(
-        property = "tableSummary",
-        column = "table_summary",
-        typeHandler = JsonObjectConverter.class),
-    @Result(property = "optimizingType", column = "optimizing_type"),
-    @Result(property = "targetSnapshotId", column = "target_snapshot_id"),
-    @Result(property = "targetChangeSnapshotId", column = 
"target_change_snapshot_id"),
-    @Result(property = "planTime", column = "plan_time", typeHandler = 
Long2TsConverter.class),
-    @Result(
-        property = "fromSequence",
-        column = "from_sequence",
-        typeHandler = MapLong2StringConverter.class),
-    @Result(
-        property = "toSequence",
-        column = "to_sequence",
-        typeHandler = MapLong2StringConverter.class)
-  })
+  @Results(
+      id = "tableRuntimeMeta",
+      value = {
+        @Result(property = "tableId", column = "table_id"),
+        @Result(property = "catalogName", column = "catalog_name"),
+        @Result(property = "dbName", column = "db_name"),
+        @Result(property = "tableName", column = "table_name"),
+        @Result(property = "format", column = "format", typeHandler = 
TableFormatConverter.class),
+        @Result(property = "currentSnapshotId", column = 
"current_snapshot_id"),
+        @Result(property = "currentChangeSnapshotId", column = 
"current_change_snapshotId"),
+        @Result(property = "lastOptimizedSnapshotId", column = 
"last_optimized_snapshotId"),
+        @Result(
+            property = "lastOptimizedChangeSnapshotId",
+            column = "last_optimized_change_snapshotId"),
+        @Result(
+            property = "lastMajorOptimizingTime",
+            column = "last_major_optimizing_time",
+            typeHandler = Long2TsConverter.class),
+        @Result(
+            property = "lastMinorOptimizingTime",
+            column = "last_minor_optimizing_time",
+            typeHandler = Long2TsConverter.class),
+        @Result(
+            property = "lastFullOptimizingTime",
+            column = "last_full_optimizing_time",
+            typeHandler = Long2TsConverter.class),
+        @Result(
+            property = "tableStatus",
+            column = "optimizing_status_code",
+            typeHandler = OptimizingStatusConverter.class),
+        @Result(
+            property = "currentStatusStartTime",
+            column = "optimizing_status_start_time",
+            typeHandler = Long2TsConverter.class),
+        @Result(property = "optimizingProcessId", column = 
"optimizing_process_id"),
+        @Result(property = "optimizerGroup", column = "optimizer_group"),
+        @Result(
+            property = "tableConfig",
+            column = "table_config",
+            typeHandler = JsonObjectConverter.class),
+        @Result(
+            property = "pendingInput",
+            column = "pending_input",
+            typeHandler = JsonObjectConverter.class),
+        @Result(
+            property = "tableSummary",
+            column = "table_summary",
+            typeHandler = JsonObjectConverter.class),
+        @Result(property = "optimizingType", column = "optimizing_type"),
+        @Result(property = "targetSnapshotId", column = "target_snapshot_id"),
+        @Result(property = "targetChangeSnapshotId", column = 
"target_change_snapshot_id"),
+        @Result(property = "planTime", column = "plan_time", typeHandler = 
Long2TsConverter.class),
+        @Result(
+            property = "fromSequence",
+            column = "from_sequence",
+            typeHandler = MapLong2StringConverter.class),
+        @Result(
+            property = "toSequence",
+            column = "to_sequence",
+            typeHandler = MapLong2StringConverter.class)
+      })
   List<TableRuntimeMeta> selectTableRuntimeMetas();
 
+  @Select(
+      "SELECT a.table_id, a.catalog_name, a.db_name, a.table_name, i.format, 
a.current_snapshot_id,"
+          + " a.current_change_snapshotId, a.last_optimized_snapshotId, 
a.last_optimized_change_snapshotId,"
+          + " a.last_major_optimizing_time, a.last_minor_optimizing_time, 
a.last_full_optimizing_time,"
+          + " a.optimizing_status_code, a.optimizing_status_start_time, 
a.optimizing_process_id,"
+          + " a.optimizer_group, a.table_config, a.pending_input, 
a.table_summary, b.optimizing_type, b.target_snapshot_id,"
+          + " b.target_change_snapshot_id, b.plan_time, b.from_sequence, 
b.to_sequence FROM table_runtime a"
+          + " INNER JOIN table_identifier i ON a.table_id = i.table_id "
+          + " LEFT JOIN table_optimizing_process b ON a.optimizing_process_id 
= b.process_id "
+          + " WHERE a.table_id = #{tableId}")
+  @ResultMap("tableRuntimeMeta")
+  TableRuntimeMeta getTableRuntimeMeta(@Param("tableId") long tableId);
+
   @Select(
       "<script>"
           + "<bind name=\"isMySQL\" value=\"_databaseId == 'mysql'\" />"
@@ -476,52 +492,7 @@ public interface TableMetaMapper {
           + "</foreach> ) </if>"
           + "ORDER BY optimizing_status_code, optimizing_status_start_time 
DESC "
           + "</script>")
-  @Results({
-    @Result(property = "tableId", column = "table_id"),
-    @Result(property = "catalogName", column = "catalog_name"),
-    @Result(property = "dbName", column = "db_name"),
-    @Result(property = "tableName", column = "table_name"),
-    @Result(property = "currentSnapshotId", column = "current_snapshot_id"),
-    @Result(property = "currentChangeSnapshotId", column = 
"current_change_snapshotId"),
-    @Result(property = "lastOptimizedSnapshotId", column = 
"last_optimized_snapshotId"),
-    @Result(
-        property = "lastOptimizedChangeSnapshotId",
-        column = "last_optimized_change_snapshotId"),
-    @Result(
-        property = "lastMajorOptimizingTime",
-        column = "last_major_optimizing_time",
-        typeHandler = Long2TsConverter.class),
-    @Result(
-        property = "lastMinorOptimizingTime",
-        column = "last_minor_optimizing_time",
-        typeHandler = Long2TsConverter.class),
-    @Result(
-        property = "lastFullOptimizingTime",
-        column = "last_full_optimizing_time",
-        typeHandler = Long2TsConverter.class),
-    @Result(
-        property = "tableStatus",
-        column = "optimizing_status_code",
-        typeHandler = OptimizingStatusConverter.class),
-    @Result(
-        property = "currentStatusStartTime",
-        column = "optimizing_status_start_time",
-        typeHandler = Long2TsConverter.class),
-    @Result(property = "optimizingProcessId", column = 
"optimizing_process_id"),
-    @Result(property = "optimizerGroup", column = "optimizer_group"),
-    @Result(
-        property = "pendingInput",
-        column = "pending_input",
-        typeHandler = JsonObjectConverter.class),
-    @Result(
-        property = "tableSummary",
-        column = "table_summary",
-        typeHandler = JsonObjectConverter.class),
-    @Result(
-        property = "tableConfig",
-        column = "table_config",
-        typeHandler = JsonObjectConverter.class),
-  })
+  @ResultMap("tableRuntimeMeta")
   List<TableRuntimeMeta> selectTableRuntimesForOptimizerGroup(
       @Param("optimizerGroup") String optimizerGroup,
       @Param("fuzzyDbName") String fuzzyDbName,
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableManager.java
 
b/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableManager.java
index be63b0406..3051e56d1 100644
--- 
a/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableManager.java
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableManager.java
@@ -32,14 +32,22 @@ import org.apache.amoro.exception.IllegalMetadataException;
 import org.apache.amoro.exception.ObjectNotExistsException;
 import org.apache.amoro.exception.PersistenceException;
 import org.apache.amoro.server.AmoroManagementConf;
+import org.apache.amoro.server.AmoroServiceConstants;
 import org.apache.amoro.server.catalog.CatalogManager;
 import org.apache.amoro.server.catalog.InternalCatalog;
+import org.apache.amoro.server.dashboard.model.TableOptimizingInfo;
+import org.apache.amoro.server.dashboard.utils.OptimizingUtil;
+import org.apache.amoro.server.optimizing.OptimizingTaskMeta;
+import org.apache.amoro.server.optimizing.TaskRuntime;
 import org.apache.amoro.server.persistence.PersistentBase;
 import org.apache.amoro.server.persistence.TableRuntimeMeta;
+import org.apache.amoro.server.persistence.mapper.OptimizingMapper;
 import org.apache.amoro.server.persistence.mapper.TableBlockerMapper;
 import org.apache.amoro.server.persistence.mapper.TableMetaMapper;
 import org.apache.amoro.server.table.blocker.TableBlocker;
 import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions;
+import org.apache.amoro.shade.guava32.com.google.common.collect.Lists;
+import org.apache.amoro.shade.guava32.com.google.common.collect.Maps;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.slf4j.Logger;
@@ -222,7 +230,12 @@ public class DefaultTableManager extends PersistentBase 
implements TableManager
   }
 
   @Override
-  public Pair<List<TableRuntimeMeta>, Integer> queryTableRuntimeMetas(
+  public TableRuntimeMeta getTableRuntimeMata(ServerTableIdentifier id) {
+    return getAs(TableMetaMapper.class, mapper -> 
mapper.getTableRuntimeMeta(id.getId()));
+  }
+
+  @Override
+  public Pair<List<TableOptimizingInfo>, Integer> queryTableOptimizingInfo(
       String optimizerGroup,
       @Nullable String fuzzyDbName,
       @Nullable String fuzzyTableName,
@@ -232,10 +245,10 @@ public class DefaultTableManager extends PersistentBase 
implements TableManager
 
     // page helper is 1-based
     int pageNumber = (offset / limit) + 1;
-
+    int total = 0;
+    List<TableRuntimeMeta> ret;
     try (Page<?> ignore = PageHelper.startPage(pageNumber, limit, true)) {
-      int total = 0;
-      List<TableRuntimeMeta> ret =
+      ret =
           getAs(
               TableMetaMapper.class,
               mapper ->
@@ -243,7 +256,57 @@ public class DefaultTableManager extends PersistentBase 
implements TableManager
                       optimizerGroup, fuzzyDbName, fuzzyTableName, 
statusCodeFilters));
       PageInfo<TableRuntimeMeta> pageInfo = new PageInfo<>(ret);
       total = (int) pageInfo.getTotal();
-      return Pair.of(ret, total);
     }
+    List<Long> processIds =
+        ret.stream()
+            .map(TableRuntimeMeta::getOptimizingProcessId)
+            .filter(i -> i != -1)
+            .collect(Collectors.toList());
+    List<Long> tableIds =
+        
ret.stream().map(TableRuntimeMeta::getTableId).collect(Collectors.toList());
+
+    List<OptimizingTaskMeta> taskMetas =
+        getAs(
+            OptimizingMapper.class,
+            m -> {
+              if (processIds.isEmpty()) {
+                return Lists.newArrayList();
+              } else {
+                return m.selectOptimizeTaskMetas(processIds);
+              }
+            });
+    Map<Long, List<OptimizingTaskMeta>> tableTaskMetaMap =
+        taskMetas.stream()
+            .collect(Collectors.groupingBy(OptimizingTaskMeta::getTableId, 
Collectors.toList()));
+
+    // load quota info
+    Map<Long, List<TaskRuntime.TaskQuota>> tableQuotaMap = 
getQuotaTime(tableIds);
+
+    List<TableOptimizingInfo> infos =
+        ret.stream()
+            .map(
+                meta -> {
+                  List<OptimizingTaskMeta> tasks = 
tableTaskMetaMap.get(meta.getTableId());
+                  List<TaskRuntime.TaskQuota> quotas = 
tableQuotaMap.get(meta.getTableId());
+                  return OptimizingUtil.buildTableOptimizeInfo(meta, tasks, 
quotas);
+                })
+            .collect(Collectors.toList());
+    return Pair.of(infos, total);
+  }
+
+  private Map<Long, List<TaskRuntime.TaskQuota>> getQuotaTime(List<Long> 
tableIds) {
+    if (tableIds == null || tableIds.isEmpty()) {
+      return Maps.newHashMap();
+    }
+    long calculatingEndTime = System.currentTimeMillis();
+    long calculatingStartTime = calculatingEndTime - 
AmoroServiceConstants.QUOTA_LOOK_BACK_TIME;
+
+    List<TaskRuntime.TaskQuota> quotas =
+        getAs(
+            OptimizingMapper.class,
+            mapper -> mapper.selectTableQuotas(tableIds, 
calculatingStartTime));
+
+    return quotas.stream()
+        .collect(Collectors.groupingBy(TaskRuntime.TaskQuota::getTableId, 
Collectors.toList()));
   }
 }
diff --git 
a/amoro-ams/src/main/java/org/apache/amoro/server/table/MaintainedTableManager.java
 
b/amoro-ams/src/main/java/org/apache/amoro/server/table/MaintainedTableManager.java
index f4acbcfc2..53df0a827 100644
--- 
a/amoro-ams/src/main/java/org/apache/amoro/server/table/MaintainedTableManager.java
+++ 
b/amoro-ams/src/main/java/org/apache/amoro/server/table/MaintainedTableManager.java
@@ -20,6 +20,7 @@ package org.apache.amoro.server.table;
 
 import org.apache.amoro.ServerTableIdentifier;
 import org.apache.amoro.api.TableIdentifier;
+import org.apache.amoro.server.dashboard.model.TableOptimizingInfo;
 import org.apache.amoro.server.persistence.TableRuntimeMeta;
 import org.apache.commons.lang3.tuple.Pair;
 
@@ -44,6 +45,9 @@ public interface MaintainedTableManager {
    */
   ServerTableIdentifier getServerTableIdentifier(TableIdentifier id);
 
+  /** Get the table runtime meta. */
+  TableRuntimeMeta getTableRuntimeMata(ServerTableIdentifier id);
+
   /**
    * Get the table info from database for given parameters.
    *
@@ -59,7 +63,7 @@ public interface MaintainedTableManager {
    * @return A pair with the first entry is the actual list under the filters 
with the offset and
    *     limit, and second value will be the number of total entries under the 
filters.
    */
-  Pair<List<TableRuntimeMeta>, Integer> queryTableRuntimeMetas(
+  Pair<List<TableOptimizingInfo>, Integer> queryTableOptimizingInfo(
       String optimizerGroup,
       @Nullable String fuzzyDbName,
       @Nullable String fuzzyTableName,
diff --git 
a/amoro-ams/src/test/java/org/apache/amoro/server/TestDefaultOptimizingService.java
 
b/amoro-ams/src/test/java/org/apache/amoro/server/TestDefaultOptimizingService.java
index b68660a9c..a97738ae8 100644
--- 
a/amoro-ams/src/test/java/org/apache/amoro/server/TestDefaultOptimizingService.java
+++ 
b/amoro-ams/src/test/java/org/apache/amoro/server/TestDefaultOptimizingService.java
@@ -40,10 +40,10 @@ import org.apache.amoro.optimizing.RewriteFilesOutput;
 import org.apache.amoro.optimizing.TableOptimizing;
 import org.apache.amoro.optimizing.plan.AbstractOptimizingEvaluator;
 import org.apache.amoro.process.ProcessStatus;
+import org.apache.amoro.server.dashboard.model.TableOptimizingInfo;
 import org.apache.amoro.server.optimizing.OptimizingProcess;
 import org.apache.amoro.server.optimizing.OptimizingStatus;
 import org.apache.amoro.server.optimizing.TaskRuntime;
-import org.apache.amoro.server.persistence.TableRuntimeMeta;
 import org.apache.amoro.server.resource.OptimizerInstance;
 import org.apache.amoro.server.table.AMSTableTestBase;
 import org.apache.amoro.server.table.TableManager;
@@ -363,7 +363,7 @@ public class TestDefaultOptimizingService extends 
AMSTableTestBase {
   }
 
   /**
-   * Test the logic for {@link TableManager#queryTableRuntimeMetas(String, 
String, String, List,
+   * Test the logic for {@link TableManager#queryTableOptimizingInfo(String, 
String, String, List,
    * int, int)}.
    */
   @Test
@@ -562,9 +562,9 @@ public class TestDefaultOptimizingService extends 
AMSTableTestBase {
 
     // 2 test and assert the result
     // 2.1 only optimize group filter set
-    Pair<List<TableRuntimeMeta>, Integer> res =
+    Pair<List<TableOptimizingInfo>, Integer> res =
         tableManager()
-            .queryTableRuntimeMetas(optimizerGroup1, null, null, 
Collections.emptyList(), 10, 0);
+            .queryTableOptimizingInfo(optimizerGroup1, null, null, 
Collections.emptyList(), 10, 0);
     Integer expectedTotalinGroup1 = 14;
     Assert.assertEquals(expectedTotalinGroup1, res.getRight());
     Assert.assertEquals(10, res.getLeft().size());
@@ -572,7 +572,7 @@ public class TestDefaultOptimizingService extends 
AMSTableTestBase {
     // 2.2 set optimize group and db filter
     res =
         tableManager()
-            .queryTableRuntimeMetas(optimizerGroup1, db1, null, 
Collections.emptyList(), 5, 0);
+            .queryTableOptimizingInfo(optimizerGroup1, db1, null, 
Collections.emptyList(), 5, 0);
     // there are 8 tables in db1 in optimizerGroup1
     Integer expectedTotalGroup1Db1 = 8;
     Assert.assertEquals(expectedTotalGroup1Db1, res.getRight());
@@ -583,14 +583,14 @@ public class TestDefaultOptimizingService extends 
AMSTableTestBase {
     String fuzzyDbName = "InOtherGroup";
     res =
         tableManager()
-            .queryTableRuntimeMetas(opGroup2, null, fuzzyDbName, 
Collections.emptyList(), 2, 0);
+            .queryTableOptimizingInfo(opGroup2, null, fuzzyDbName, 
Collections.emptyList(), 2, 0);
     Integer expectedTotalWithFuzzyDbName = 3;
     Assert.assertEquals(expectedTotalWithFuzzyDbName, res.getRight());
     Assert.assertEquals(2, res.getLeft().size());
 
     res =
         tableManager()
-            .queryTableRuntimeMetas(opGroup2, null, fuzzyDbName, 
Collections.emptyList(), 5, 0);
+            .queryTableOptimizingInfo(opGroup2, null, fuzzyDbName, 
Collections.emptyList(), 5, 0);
     Assert.assertEquals(expectedTotalWithFuzzyDbName, res.getRight());
     // there are only 3 tables with the suffix in opGroup2
     Assert.assertEquals(3, res.getLeft().size());
@@ -598,7 +598,7 @@ public class TestDefaultOptimizingService extends 
AMSTableTestBase {
     // 2.4 set optimize group and status filter, with only one status
     List<Integer> statusCode = new ArrayList<>();
     statusCode.add(OptimizingStatus.MAJOR_OPTIMIZING.getCode());
-    res = tableManager().queryTableRuntimeMetas(optimizerGroup1, null, null, 
statusCode, 10, 0);
+    res = tableManager().queryTableOptimizingInfo(optimizerGroup1, null, null, 
statusCode, 10, 0);
     Integer expectedTotalInGroup1WithMajorStatus = 2;
     Assert.assertEquals(expectedTotalInGroup1WithMajorStatus, res.getRight());
     Assert.assertEquals(2, res.getLeft().size());
@@ -607,7 +607,7 @@ public class TestDefaultOptimizingService extends 
AMSTableTestBase {
     statusCode.clear();
     statusCode.add(OptimizingStatus.MINOR_OPTIMIZING.getCode());
     statusCode.add(OptimizingStatus.MAJOR_OPTIMIZING.getCode());
-    res = tableManager().queryTableRuntimeMetas(optimizerGroup1, null, null, 
statusCode, 3, 0);
+    res = tableManager().queryTableOptimizingInfo(optimizerGroup1, null, null, 
statusCode, 3, 0);
     Integer expectedTotalInGroup1WithMinorMajorStatus = 4;
     Assert.assertEquals(expectedTotalInGroup1WithMinorMajorStatus, 
res.getRight());
     Assert.assertEquals(3, res.getLeft().size());
@@ -618,7 +618,8 @@ public class TestDefaultOptimizingService extends 
AMSTableTestBase {
     statusCode.add(OptimizingStatus.FULL_OPTIMIZING.getCode());
     String tableFilter = "pending";
     res =
-        tableManager().queryTableRuntimeMetas(optimizerGroup1, db1, 
tableFilter, statusCode, 10, 0);
+        tableManager()
+            .queryTableOptimizingInfo(optimizerGroup1, db1, tableFilter, 
statusCode, 10, 0);
     Integer expectedTotalInGroup1InDb1WithTableFilterAndStatus = 2;
     Assert.assertEquals(expectedTotalInGroup1InDb1WithTableFilterAndStatus, 
res.getRight());
     Assert.assertEquals(2, res.getLeft().size());
@@ -630,7 +631,7 @@ public class TestDefaultOptimizingService extends 
AMSTableTestBase {
     String wrongTableFilter2 = "noTableWithName";
     res =
         tableManager()
-            .queryTableRuntimeMetas(optimizerGroup1, db1, wrongTableFilter2, 
statusCode, 10, 0);
+            .queryTableOptimizingInfo(optimizerGroup1, db1, wrongTableFilter2, 
statusCode, 10, 0);
     Assert.assertEquals(0, (int) res.getRight());
     Assert.assertTrue(res.getLeft().isEmpty());
   }
diff --git a/amoro-common/pom.xml b/amoro-common/pom.xml
index e65c42415..0f50d55ca 100644
--- a/amoro-common/pom.xml
+++ b/amoro-common/pom.xml
@@ -166,7 +166,7 @@
                 </executions>
             </plugin>
 
-            <!--Compile thrift api with commend: mvn thrift:compile -pl 
amoro-core-->
+            <!--Compile thrift api with commend: mvn thrift:compile -pl 
amoro-common-->
             <plugin>
                 <groupId>org.apache.thrift.tools</groupId>
                 <artifactId>maven-thrift-plugin</artifactId>
diff --git 
a/amoro-common/src/main/gen-java/org/apache/amoro/api/OptimizingService.java 
b/amoro-common/src/main/gen-java/org/apache/amoro/api/OptimizingService.java
index d2bbdb766..331e69388 100644
--- a/amoro-common/src/main/gen-java/org/apache/amoro/api/OptimizingService.java
+++ b/amoro-common/src/main/gen-java/org/apache/amoro/api/OptimizingService.java
@@ -6,7 +6,7 @@
  */
 package org.apache.amoro.api;
 
[email protected](value = "Autogenerated by Thrift Compiler 
(0.20.0)", date = "2024-06-11")
[email protected](value = "Autogenerated by Thrift Compiler 
(0.20.0)", date = "2025-01-23")
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class OptimizingService {
 
@@ -24,6 +24,8 @@ public class OptimizingService {
 
     public java.lang.String authenticate(OptimizerRegisterInfo registerInfo) 
throws org.apache.amoro.api.AmoroException, 
org.apache.amoro.shade.thrift.org.apache.thrift.TException;
 
+    public boolean cancelProcess(long processId) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -40,6 +42,8 @@ public class OptimizingService {
 
     public void authenticate(OptimizerRegisterInfo registerInfo, 
org.apache.amoro.shade.thrift.org.apache.thrift.async.AsyncMethodCallback<java.lang.String>
 resultHandler) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException;
 
+    public void cancelProcess(long processId, 
org.apache.amoro.shade.thrift.org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>
 resultHandler) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException;
+
   }
 
   public static class Client extends 
org.apache.amoro.shade.thrift.org.apache.thrift.TServiceClient implements Iface 
{
@@ -214,6 +218,30 @@ public class OptimizingService {
       throw new 
org.apache.amoro.shade.thrift.org.apache.thrift.TApplicationException(org.apache.amoro.shade.thrift.org.apache.thrift.TApplicationException.MISSING_RESULT,
 "authenticate failed: unknown result");
     }
 
+    @Override
+    public boolean cancelProcess(long processId) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException
+    {
+      send_cancelProcess(processId);
+      return recv_cancelProcess();
+    }
+
+    public void send_cancelProcess(long processId) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException
+    {
+      cancelProcess_args args = new cancelProcess_args();
+      args.setProcessId(processId);
+      sendBase("cancelProcess", args);
+    }
+
+    public boolean recv_cancelProcess() throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException
+    {
+      cancelProcess_result result = new cancelProcess_result();
+      receiveBase(result, "cancelProcess");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new 
org.apache.amoro.shade.thrift.org.apache.thrift.TApplicationException(org.apache.amoro.shade.thrift.org.apache.thrift.TApplicationException.MISSING_RESULT,
 "cancelProcess failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends 
org.apache.amoro.shade.thrift.org.apache.thrift.async.TAsyncClient implements 
AsyncIface {
     public static class Factory implements 
org.apache.amoro.shade.thrift.org.apache.thrift.async.TAsyncClientFactory<AsyncClient>
 {
@@ -456,6 +484,41 @@ public class OptimizingService {
       }
     }
 
+    @Override
+    public void cancelProcess(long processId, 
org.apache.amoro.shade.thrift.org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>
 resultHandler) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+      checkReady();
+      cancelProcess_call method_call = new cancelProcess_call(processId, 
resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class cancelProcess_call extends 
org.apache.amoro.shade.thrift.org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean>
 {
+      private long processId;
+      public cancelProcess_call(long processId, 
org.apache.amoro.shade.thrift.org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>
 resultHandler, 
org.apache.amoro.shade.thrift.org.apache.thrift.async.TAsyncClient client, 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocolFactory 
protocolFactory, 
org.apache.amoro.shade.thrift.org.apache.thrift.transport.TNonblockingTransport 
transport) throws org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.processId = processId;
+      }
+
+      @Override
+      public void 
write_args(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol 
prot) throws org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        prot.writeMessageBegin(new 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TMessage("cancelProcess",
 org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TMessageType.CALL, 
0));
+        cancelProcess_args args = new cancelProcess_args();
+        args.setProcessId(processId);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public java.lang.Boolean getResult() throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        if (getState() != 
org.apache.amoro.shade.thrift.org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ)
 {
+          throw new java.lang.IllegalStateException("Method call not 
finished!");
+        }
+        
org.apache.amoro.shade.thrift.org.apache.thrift.transport.TMemoryInputTransport 
memoryTransport = new 
org.apache.amoro.shade.thrift.org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol 
prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_cancelProcess();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends 
org.apache.amoro.shade.thrift.org.apache.thrift.TBaseProcessor<I> implements 
org.apache.amoro.shade.thrift.org.apache.thrift.TProcessor {
@@ -475,6 +538,7 @@ public class OptimizingService {
       processMap.put("ackTask", new ackTask());
       processMap.put("completeTask", new completeTask());
       processMap.put("authenticate", new authenticate());
+      processMap.put("cancelProcess", new cancelProcess());
       return processMap;
     }
 
@@ -666,6 +730,35 @@ public class OptimizingService {
       }
     }
 
+    public static class cancelProcess<I extends Iface> extends 
org.apache.amoro.shade.thrift.org.apache.thrift.ProcessFunction<I, 
cancelProcess_args> {
+      public cancelProcess() {
+        super("cancelProcess");
+      }
+
+      @Override
+      public cancelProcess_args getEmptyArgsInstance() {
+        return new cancelProcess_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public cancelProcess_result getResult(I iface, cancelProcess_args args) 
throws org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        cancelProcess_result result = new cancelProcess_result();
+        result.success = iface.cancelProcess(args.processId);
+        result.setSuccessIsSet(true);
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends 
org.apache.amoro.shade.thrift.org.apache.thrift.TBaseAsyncProcessor<I> {
@@ -685,6 +778,7 @@ public class OptimizingService {
       processMap.put("ackTask", new ackTask());
       processMap.put("completeTask", new completeTask());
       processMap.put("authenticate", new authenticate());
+      processMap.put("cancelProcess", new cancelProcess());
       return processMap;
     }
 
@@ -1106,6 +1200,74 @@ public class OptimizingService {
       }
     }
 
+    public static class cancelProcess<I extends AsyncIface> extends 
org.apache.amoro.shade.thrift.org.apache.thrift.AsyncProcessFunction<I, 
cancelProcess_args, java.lang.Boolean> {
+      public cancelProcess() {
+        super("cancelProcess");
+      }
+
+      @Override
+      public cancelProcess_args getEmptyArgsInstance() {
+        return new cancelProcess_args();
+      }
+
+      @Override
+      public 
org.apache.amoro.shade.thrift.org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>
 getResultHandler(final 
org.apache.amoro.shade.thrift.org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer
 fb, final int seqid) {
+        final 
org.apache.amoro.shade.thrift.org.apache.thrift.AsyncProcessFunction fcall = 
this;
+        return new 
org.apache.amoro.shade.thrift.org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>()
 { 
+          @Override
+          public void onComplete(java.lang.Boolean o) {
+            cancelProcess_result result = new cancelProcess_result();
+            result.success = o;
+            result.setSuccessIsSet(true);
+            try {
+              fcall.sendResponse(fb, result, 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch 
(org.apache.amoro.shade.thrift.org.apache.thrift.transport.TTransportException 
e) {
+              _LOGGER.error("TTransportException writing to internal frame 
buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            byte msgType = 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.amoro.shade.thrift.org.apache.thrift.TSerializable msg;
+            cancelProcess_result result = new cancelProcess_result();
+            if (e instanceof 
org.apache.amoro.shade.thrift.org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof 
org.apache.amoro.shade.thrift.org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = 
(org.apache.amoro.shade.thrift.org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new 
org.apache.amoro.shade.thrift.org.apache.thrift.TApplicationException(org.apache.amoro.shade.thrift.org.apache.thrift.TApplicationException.INTERNAL_ERROR,
 e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      public void start(I iface, cancelProcess_args args, 
org.apache.amoro.shade.thrift.org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>
 resultHandler) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        iface.cancelProcess(args.processId,resultHandler);
+      }
+    }
+
   }
 
   @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
@@ -6074,4 +6236,753 @@ public class OptimizingService {
     }
   }
 
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class cancelProcess_args implements 
org.apache.amoro.shade.thrift.org.apache.thrift.TBase<cancelProcess_args, 
cancelProcess_args._Fields>, java.io.Serializable, Cloneable, 
Comparable<cancelProcess_args>   {
+    private static final 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TStruct STRUCT_DESC = 
new 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TStruct("cancelProcess_args");
+
+    private static final 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TField 
PROCESS_ID_FIELD_DESC = new 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TField("processId", 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TType.I64, (short)1);
+
+    private static final 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.SchemeFactory 
STANDARD_SCHEME_FACTORY = new cancelProcess_argsStandardSchemeFactory();
+    private static final 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.SchemeFactory 
TUPLE_SCHEME_FACTORY = new cancelProcess_argsTupleSchemeFactory();
+
+    public long processId; // required
+
+    /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
+    public enum _Fields implements 
org.apache.amoro.shade.thrift.org.apache.thrift.TFieldIdEnum {
+      PROCESS_ID((short)1, "processId");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = 
new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not 
found.
+       */
+      @org.apache.amoro.shade.thrift.org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // PROCESS_ID
+            return PROCESS_ID;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new 
java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.amoro.shade.thrift.org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __PROCESSID_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, 
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldMetaData> 
metaDataMap;
+    static {
+      java.util.Map<_Fields, 
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldMetaData> tmpMap 
= new java.util.EnumMap<_Fields, 
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.PROCESS_ID, new 
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldMetaData("processId",
 org.apache.amoro.shade.thrift.org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new 
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldValueMetaData(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TType.I64)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelProcess_args.class,
 metaDataMap);
+    }
+
+    public cancelProcess_args() {
+    }
+
+    public cancelProcess_args(
+      long processId)
+    {
+      this();
+      this.processId = processId;
+      setProcessIdIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public cancelProcess_args(cancelProcess_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.processId = other.processId;
+    }
+
+    @Override
+    public cancelProcess_args deepCopy() {
+      return new cancelProcess_args(this);
+    }
+
+    @Override
+    public void clear() {
+      setProcessIdIsSet(false);
+      this.processId = 0;
+    }
+
+    public long getProcessId() {
+      return this.processId;
+    }
+
+    public cancelProcess_args setProcessId(long processId) {
+      this.processId = processId;
+      setProcessIdIsSet(true);
+      return this;
+    }
+
+    public void unsetProcessId() {
+      __isset_bitfield = 
org.apache.amoro.shade.thrift.org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield,
 __PROCESSID_ISSET_ID);
+    }
+
+    /** Returns true if field processId is set (has been assigned a value) and 
false otherwise */
+    public boolean isSetProcessId() {
+      return 
org.apache.amoro.shade.thrift.org.apache.thrift.EncodingUtils.testBit(__isset_bitfield,
 __PROCESSID_ISSET_ID);
+    }
+
+    public void setProcessIdIsSet(boolean value) {
+      __isset_bitfield = 
org.apache.amoro.shade.thrift.org.apache.thrift.EncodingUtils.setBit(__isset_bitfield,
 __PROCESSID_ISSET_ID, value);
+    }
+
+    @Override
+    public void setFieldValue(_Fields field, 
@org.apache.amoro.shade.thrift.org.apache.thrift.annotation.Nullable 
java.lang.Object value) {
+      switch (field) {
+      case PROCESS_ID:
+        if (value == null) {
+          unsetProcessId();
+        } else {
+          setProcessId((java.lang.Long)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.amoro.shade.thrift.org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case PROCESS_ID:
+        return getProcessId();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been 
assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case PROCESS_ID:
+        return isSetProcessId();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof cancelProcess_args)
+        return this.equals((cancelProcess_args)that);
+      return false;
+    }
+
+    public boolean equals(cancelProcess_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_processId = true;
+      boolean that_present_processId = true;
+      if (this_present_processId || that_present_processId) {
+        if (!(this_present_processId && that_present_processId))
+          return false;
+        if (this.processId != that.processId)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + 
org.apache.amoro.shade.thrift.org.apache.thrift.TBaseHelper.hashCode(processId);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(cancelProcess_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetProcessId(), 
other.isSetProcessId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetProcessId()) {
+        lastComparison = 
org.apache.amoro.shade.thrift.org.apache.thrift.TBaseHelper.compareTo(this.processId,
 other.processId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.amoro.shade.thrift.org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void 
read(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol iprot) 
throws org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    public void 
write(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol oprot) 
throws org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new 
java.lang.StringBuilder("cancelProcess_args(");
+      boolean first = true;
+
+      sb.append("processId:");
+      sb.append(this.processId);
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws 
java.io.IOException {
+      try {
+        write(new 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.amoro.shade.thrift.org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.amoro.shade.thrift.org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws 
java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java 
serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.amoro.shade.thrift.org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.amoro.shade.thrift.org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class cancelProcess_argsStandardSchemeFactory implements 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public cancelProcess_argsStandardScheme getScheme() {
+        return new cancelProcess_argsStandardScheme();
+      }
+    }
+
+    private static class cancelProcess_argsStandardScheme extends 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.StandardScheme<cancelProcess_args>
 {
+
+      @Override
+      public void 
read(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol iprot, 
cancelProcess_args struct) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TField 
schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // PROCESS_ID
+              if (schemeField.type == 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TType.I64) {
+                struct.processId = iprot.readI64();
+                struct.setProcessIdIsSet(true);
+              } else { 
+                
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocolUtil.skip(iprot,
 schemeField.type);
+              }
+              break;
+            default:
+              
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocolUtil.skip(iprot,
 schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked 
in the validate method
+        struct.validate();
+      }
+
+      @Override
+      public void 
write(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol oprot, 
cancelProcess_args struct) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldBegin(PROCESS_ID_FIELD_DESC);
+        oprot.writeI64(struct.processId);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class cancelProcess_argsTupleSchemeFactory implements 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public cancelProcess_argsTupleScheme getScheme() {
+        return new cancelProcess_argsTupleScheme();
+      }
+    }
+
+    private static class cancelProcess_argsTupleScheme extends 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.TupleScheme<cancelProcess_args>
 {
+
+      @Override
+      public void 
write(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol prot, 
cancelProcess_args struct) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TTupleProtocol oprot = 
(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetProcessId()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetProcessId()) {
+          oprot.writeI64(struct.processId);
+        }
+      }
+
+      @Override
+      public void 
read(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol prot, 
cancelProcess_args struct) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TTupleProtocol iprot = 
(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.processId = iprot.readI64();
+          struct.setProcessIdIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.IScheme> S 
scheme(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol 
proto) {
+      return 
(org.apache.amoro.shade.thrift.org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme())
 ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class cancelProcess_result implements 
org.apache.amoro.shade.thrift.org.apache.thrift.TBase<cancelProcess_result, 
cancelProcess_result._Fields>, java.io.Serializable, Cloneable, 
Comparable<cancelProcess_result>   {
+    private static final 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TStruct STRUCT_DESC = 
new 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TStruct("cancelProcess_result");
+
+    private static final 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TField 
SUCCESS_FIELD_DESC = new 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TField("success", 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TType.BOOL, (short)0);
+
+    private static final 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.SchemeFactory 
STANDARD_SCHEME_FACTORY = new cancelProcess_resultStandardSchemeFactory();
+    private static final 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.SchemeFactory 
TUPLE_SCHEME_FACTORY = new cancelProcess_resultTupleSchemeFactory();
+
+    public boolean success; // required
+
+    /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
+    public enum _Fields implements 
org.apache.amoro.shade.thrift.org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = 
new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not 
found.
+       */
+      @org.apache.amoro.shade.thrift.org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new 
java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.amoro.shade.thrift.org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __SUCCESS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, 
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldMetaData> 
metaDataMap;
+    static {
+      java.util.Map<_Fields, 
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldMetaData> tmpMap 
= new java.util.EnumMap<_Fields, 
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new 
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldMetaData("success",
 org.apache.amoro.shade.thrift.org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new 
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldValueMetaData(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TType.BOOL)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      
org.apache.amoro.shade.thrift.org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelProcess_result.class,
 metaDataMap);
+    }
+
+    public cancelProcess_result() {
+    }
+
+    public cancelProcess_result(
+      boolean success)
+    {
+      this();
+      this.success = success;
+      setSuccessIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public cancelProcess_result(cancelProcess_result other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.success = other.success;
+    }
+
+    @Override
+    public cancelProcess_result deepCopy() {
+      return new cancelProcess_result(this);
+    }
+
+    @Override
+    public void clear() {
+      setSuccessIsSet(false);
+      this.success = false;
+    }
+
+    public boolean isSuccess() {
+      return this.success;
+    }
+
+    public cancelProcess_result setSuccess(boolean success) {
+      this.success = success;
+      setSuccessIsSet(true);
+      return this;
+    }
+
+    public void unsetSuccess() {
+      __isset_bitfield = 
org.apache.amoro.shade.thrift.org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield,
 __SUCCESS_ISSET_ID);
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and 
false otherwise */
+    public boolean isSetSuccess() {
+      return 
org.apache.amoro.shade.thrift.org.apache.thrift.EncodingUtils.testBit(__isset_bitfield,
 __SUCCESS_ISSET_ID);
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      __isset_bitfield = 
org.apache.amoro.shade.thrift.org.apache.thrift.EncodingUtils.setBit(__isset_bitfield,
 __SUCCESS_ISSET_ID, value);
+    }
+
+    @Override
+    public void setFieldValue(_Fields field, 
@org.apache.amoro.shade.thrift.org.apache.thrift.annotation.Nullable 
java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.lang.Boolean)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.amoro.shade.thrift.org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return isSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been 
assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof cancelProcess_result)
+        return this.equals((cancelProcess_result)that);
+      return false;
+    }
+
+    public boolean equals(cancelProcess_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((success) ? 131071 : 524287);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(cancelProcess_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetSuccess(), 
other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = 
org.apache.amoro.shade.thrift.org.apache.thrift.TBaseHelper.compareTo(this.success,
 other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.amoro.shade.thrift.org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void 
read(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol iprot) 
throws org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void 
write(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol oprot) 
throws org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new 
java.lang.StringBuilder("cancelProcess_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      sb.append(this.success);
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws 
java.io.IOException {
+      try {
+        write(new 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.amoro.shade.thrift.org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.amoro.shade.thrift.org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws 
java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java 
serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.amoro.shade.thrift.org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.amoro.shade.thrift.org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class cancelProcess_resultStandardSchemeFactory implements 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public cancelProcess_resultStandardScheme getScheme() {
+        return new cancelProcess_resultStandardScheme();
+      }
+    }
+
+    private static class cancelProcess_resultStandardScheme extends 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.StandardScheme<cancelProcess_result>
 {
+
+      @Override
+      public void 
read(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol iprot, 
cancelProcess_result struct) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TField 
schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == 
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TType.BOOL) {
+                struct.success = iprot.readBool();
+                struct.setSuccessIsSet(true);
+              } else { 
+                
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocolUtil.skip(iprot,
 schemeField.type);
+              }
+              break;
+            default:
+              
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocolUtil.skip(iprot,
 schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked 
in the validate method
+        struct.validate();
+      }
+
+      @Override
+      public void 
write(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol oprot, 
cancelProcess_result struct) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.isSetSuccess()) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeBool(struct.success);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class cancelProcess_resultTupleSchemeFactory implements 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public cancelProcess_resultTupleScheme getScheme() {
+        return new cancelProcess_resultTupleScheme();
+      }
+    }
+
+    private static class cancelProcess_resultTupleScheme extends 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.TupleScheme<cancelProcess_result>
 {
+
+      @Override
+      public void 
write(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol prot, 
cancelProcess_result struct) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TTupleProtocol oprot = 
(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          oprot.writeBool(struct.success);
+        }
+      }
+
+      @Override
+      public void 
read(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol prot, 
cancelProcess_result struct) throws 
org.apache.amoro.shade.thrift.org.apache.thrift.TException {
+        
org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TTupleProtocol iprot = 
(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = iprot.readBool();
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends 
org.apache.amoro.shade.thrift.org.apache.thrift.scheme.IScheme> S 
scheme(org.apache.amoro.shade.thrift.org.apache.thrift.protocol.TProtocol 
proto) {
+      return 
(org.apache.amoro.shade.thrift.org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme())
 ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
 }
diff --git a/amoro-common/src/main/thrift/amoro_optimizing_service.thrift 
b/amoro-common/src/main/thrift/amoro_optimizing_service.thrift
index 360ab6b6b..77b6b7dce 100644
--- a/amoro-common/src/main/thrift/amoro_optimizing_service.thrift
+++ b/amoro-common/src/main/thrift/amoro_optimizing_service.thrift
@@ -65,4 +65,6 @@ service OptimizingService {
 
     string authenticate(1: OptimizerRegisterInfo registerInfo)
             throws (1: amoro_commons.AmoroException e1)
+
+    bool cancelProcess(1:i64 processId)
 }
diff --git 
a/amoro-common/src/test/java/org/apache/amoro/MockAmoroManagementServer.java 
b/amoro-common/src/test/java/org/apache/amoro/MockAmoroManagementServer.java
index 26eec2408..c4775b548 100644
--- a/amoro-common/src/test/java/org/apache/amoro/MockAmoroManagementServer.java
+++ b/amoro-common/src/test/java/org/apache/amoro/MockAmoroManagementServer.java
@@ -464,6 +464,11 @@ public class MockAmoroManagementServer implements Runnable 
{
       return token;
     }
 
+    @Override
+    public boolean cancelProcess(long processId) throws TException {
+      return false;
+    }
+
     public Map<String, OptimizerRegisterInfo> getRegisteredOptimizers() {
       return registeredOptimizers;
     }

Reply via email to