zhoujinsong commented on code in PR #3457:
URL: https://github.com/apache/amoro/pull/3457#discussion_r1990499317


##########
amoro-ams/src/main/java/org/apache/amoro/server/resource/DefaultOptimizerManager.java:
##########
@@ -81,11 +97,19 @@ public void createResourceGroup(ResourceGroup 
resourceGroup) {
 
   @Override
   public void deleteResourceGroup(String groupName) {
-    doAs(ResourceMapper.class, mapper -> 
mapper.deleteResourceGroup(groupName));
+    if (canDeleteResourceGroup(groupName)) {
+      doAs(ResourceMapper.class, mapper -> 
mapper.deleteResourceGroup(groupName));
+    } else {
+      throw new RuntimeException(
+          String.format(
+              "The resource group %s cannot be deleted because it is currently 
in " + "use.",

Review Comment:
   ```suggestion
                 "The resource group %s cannot be deleted because it is 
currently in use.",
   ```



##########
amoro-ams/src/main/java/org/apache/amoro/server/DefaultOptimizingService.java:
##########
@@ -571,4 +531,67 @@ private Predicate<TaskRuntime<?>> 
buildSuspendingPredication(Set<String> activeT
                   && task.getStartTime() + taskAckTimeout < 
System.currentTimeMillis();
     }
   }
+
+  private class OptimizingConfigWatcher implements Runnable {

Review Comment:
   OptimizingConfigWatcher may need some improvements:
   
   - The lock is not needed as only one thread will call the sync method at one 
time.
   - We do not need to sync optimizer information as the service will find that 
when the optimizer stops touching it.
   - The sync group codes can be improved.
   
   Here is a example you can reference:
   
   ```
   private class OptimizingConfigWatcher implements Runnable {
       private final ScheduledExecutorService scheduler =
           Executors.newSingleThreadScheduledExecutor(
               new 
ThreadFactoryBuilder().setNameFormat("resource-group-watcher-%d").build());
   
       void start() {
         run();
         scheduler.scheduleAtFixedRate(
             this, queueRefreshInterval, queueRefreshInterval, 
TimeUnit.MILLISECONDS);
       }
   
       @Override
       public void run() {
         syncGroups();
       }
   
       private void syncGroups() {
         try {
           List<ResourceGroup> resourceGroups = 
optimizerManager.listResourceGroups();
           Set<String> groupNames =
               
resourceGroups.stream().map(ResourceGroup::getName).collect(Collectors.toSet());
           Sets.difference(optimizingQueueByGroup.keySet(), groupNames)
               .forEach(DefaultOptimizingService.this::deleteResourceGroup);
           resourceGroups.forEach(
               resourceGroup -> {
                 if (optimizingQueueByGroup.containsKey(resourceGroup.getName())
                     && !optimizingQueueByGroup
                         .get(resourceGroup.getName())
                         .getOptimizerGroup()
                         .equals(resourceGroup)) {
                   updateResourceGroup(resourceGroup);
                 } else {
                   createResourceGroup(resourceGroup);
                 }
               });
         } catch (Throwable t) {
           LOG.error("Sync optimizer groups failed, will retry later.", t);
         }
       }
   
       void dispose() {
         scheduler.shutdown();
       }
     }
   ```



##########
amoro-ams/src/main/java/org/apache/amoro/server/resource/DefaultOptimizerManager.java:
##########
@@ -125,4 +149,35 @@ public List<Resource> listResourcesByGroup(String 
groupName) {
   public Resource getResource(String resourceId) {
     return getAs(ResourceMapper.class, mapper -> 
mapper.selectResource(resourceId));
   }
+
+  @Override
+  public boolean canDeleteResourceGroup(String name) {
+    for (CatalogMeta catalogMeta : catalogManager.listCatalogMetas()) {
+      if (catalogMeta.getCatalogProperties() != null
+          && catalogMeta
+              .getCatalogProperties()
+              .getOrDefault(
+                  CatalogMetaProperties.TABLE_PROPERTIES_PREFIX
+                      + TableProperties.SELF_OPTIMIZING_GROUP,
+                  TableProperties.SELF_OPTIMIZING_GROUP_DEFAULT)
+              .equals(name)) {
+        return false;
+      }
+    }
+    for (OptimizerInstance optimizer : listOptimizers()) {
+      if (optimizer.getGroupName().equals(name)) {
+        return false;
+      }
+    }
+    List<String> tableRuntimeMetas =

Review Comment:
   I am not sure if we should add a new method in the mapper or just reuse the 
`selectTableRuntimeForOptimizerGroup` method.
   
   I believe the key point is whether the old methods are still acceptable. 
While the new methods can improve efficiency, they also increase management 
costs.



##########
amoro-ams/src/main/java/org/apache/amoro/server/resource/DefaultOptimizerManager.java:
##########
@@ -125,4 +149,35 @@ public List<Resource> listResourcesByGroup(String 
groupName) {
   public Resource getResource(String resourceId) {
     return getAs(ResourceMapper.class, mapper -> 
mapper.selectResource(resourceId));
   }
+
+  @Override
+  public boolean canDeleteResourceGroup(String name) {
+    for (CatalogMeta catalogMeta : catalogManager.listCatalogMetas()) {
+      if (catalogMeta.getCatalogProperties() != null
+          && catalogMeta
+              .getCatalogProperties()
+              .getOrDefault(
+                  CatalogMetaProperties.TABLE_PROPERTIES_PREFIX
+                      + TableProperties.SELF_OPTIMIZING_GROUP,
+                  TableProperties.SELF_OPTIMIZING_GROUP_DEFAULT)
+              .equals(name)) {
+        return false;
+      }
+    }
+    for (OptimizerInstance optimizer : listOptimizers()) {
+      if (optimizer.getGroupName().equals(name)) {
+        return false;
+      }
+    }
+    List<String> tableRuntimeMetas =
+        getAs(
+            TableMetaMapper.class,
+            mapper -> mapper.selectTableRuntimeNamesForOptimizerGroup(name, 
null, null, null));
+    for (ServerTableIdentifier identifier : tableManager.listManagedTables()) {

Review Comment:
   Maybe checking if the tableRuntimeMetas is empty is enough?



##########
amoro-ams/src/main/java/org/apache/amoro/server/DefaultOptimizingService.java:
##########
@@ -178,7 +176,6 @@ private void registerOptimizer(OptimizerInstance optimizer, 
boolean needPersiste
   }
 
   private void unregisterOptimizer(String token) {
-    doAs(OptimizerMapper.class, mapper -> mapper.deleteOptimizer(token));

Review Comment:
   We may keep the code here because the optimizing service needs to delete the 
optimizer instance when the optimizer is lost.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to